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/07/27 16:43:40 UTC

[1/6] nifi-minifi-cpp git commit: MINIFI-249: Update prov repo to better abstract deser.

Repository: nifi-minifi-cpp
Updated Branches:
  refs/heads/master 20622f6d1 -> 7c16d1bbb


http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/test/unit/YamlConfigurationTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/YamlConfigurationTests.cpp b/libminifi/test/unit/YamlConfigurationTests.cpp
index ba73a34..fc11d04 100644
--- a/libminifi/test/unit/YamlConfigurationTests.cpp
+++ b/libminifi/test/unit/YamlConfigurationTests.cpp
@@ -20,6 +20,7 @@
 #include <map>
 #include <memory>
 #include <string>
+#include "core/repository/VolatileContentRepository.h"
 #include <core/RepositoryFactory.h>
 #include "core/yaml/YamlConfiguration.h"
 #include "../TestBase.h"
@@ -30,8 +31,9 @@ TEST_CASE("Test YAML Config Processing", "[YamlConfiguration]") {
   std::shared_ptr<core::Repository> testProvRepo = core::createRepository("provenancerepository", true);
   std::shared_ptr<core::Repository> testFlowFileRepo = core::createRepository("flowfilerepository", true);
   std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
-  std::shared_ptr<minifi::io::StreamFactory> streamFactory = std::make_shared<minifi::io::StreamFactory>(configuration);
-  core::YamlConfiguration *yamlConfig = new core::YamlConfiguration(testProvRepo, testFlowFileRepo, streamFactory, configuration);
+  std::shared_ptr<minifi::io::StreamFactory> streamFactory = std::make_shared < minifi::io::StreamFactory > (configuration);
+  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
+  core::YamlConfiguration *yamlConfig = new core::YamlConfiguration(testProvRepo, testFlowFileRepo, content_repo, streamFactory, configuration);
 
   SECTION("loading YAML without optional component IDs works") {
   static const std::string CONFIG_YAML_WITHOUT_IDS = ""

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/main/MiNiFiMain.cpp
----------------------------------------------------------------------
diff --git a/main/MiNiFiMain.cpp b/main/MiNiFiMain.cpp
index ba59623..3afa06e 100644
--- a/main/MiNiFiMain.cpp
+++ b/main/MiNiFiMain.cpp
@@ -90,6 +90,7 @@ int main(int argc, char **argv) {
   std::string prov_repo_class = "provenancerepository";
   std::string flow_repo_class = "flowfilerepository";
   std::string nifi_configuration_class_name = "yamlconfiguration";
+  std::string content_repo_class = "filesystemrepository";
 
   running = sem_open("MiNiFiMain", O_CREAT, 0644, 0);
   if (running == SEM_FAILED || running == 0) {
@@ -108,14 +109,11 @@ int main(int argc, char **argv) {
     char full_path[PATH_MAX];
     path = realpath(argv[0], full_path);
     std::string minifiHomePath(path);
-    minifiHomePath = minifiHomePath.substr(0,
-                                           minifiHomePath.find_last_of("/\\"));  //Remove /minifi from path
+    minifiHomePath = minifiHomePath.substr(0, minifiHomePath.find_last_of("/\\"));  //Remove /minifi from path
     minifiHome = minifiHomePath.substr(0, minifiHomePath.find_last_of("/\\"));	//Remove /bin from path
   }
 
-  if (signal(SIGINT, sigHandler) == SIG_ERR
-      || signal(SIGTERM, sigHandler) == SIG_ERR
-      || signal(SIGPIPE, SIG_IGN) == SIG_ERR) {
+  if (signal(SIGINT, sigHandler) == SIG_ERR || signal(SIGTERM, sigHandler) == SIG_ERR || signal(SIGPIPE, SIG_IGN) == SIG_ERR) {
     logger->log_error("Can not install signal handler");
     return -1;
   }
@@ -132,56 +130,49 @@ int main(int argc, char **argv) {
 
   // Make a record of minifi home in the configured log file.
   logger->log_info("MINIFI_HOME=%s", minifiHome);
-  
+
   std::shared_ptr<minifi::Configure> configure = std::make_shared<minifi::Configure>();
   configure->setHome(minifiHome);
   configure->loadConfigureFile(DEFAULT_NIFI_PROPERTIES_FILE);
 
-  if (configure->get(minifi::Configure::nifi_graceful_shutdown_seconds,
-                     graceful_shutdown_seconds)) {
+  if (configure->get(minifi::Configure::nifi_graceful_shutdown_seconds, graceful_shutdown_seconds)) {
     try {
       stop_wait_time = std::stoi(graceful_shutdown_seconds);
     } catch (const std::out_of_range &e) {
-      logger->log_error("%s is out of range. %s",
-                        minifi::Configure::nifi_graceful_shutdown_seconds,
-                        e.what());
+      logger->log_error("%s is out of range. %s", minifi::Configure::nifi_graceful_shutdown_seconds, e.what());
     } catch (const std::invalid_argument &e) {
-      logger->log_error("%s contains an invalid argument set. %s",
-                        minifi::Configure::nifi_graceful_shutdown_seconds,
-                        e.what());
+      logger->log_error("%s contains an invalid argument set. %s", minifi::Configure::nifi_graceful_shutdown_seconds, e.what());
     }
   } else {
-    logger->log_debug("%s not set, defaulting to %d",
-                      minifi::Configure::nifi_graceful_shutdown_seconds,
-                      STOP_WAIT_TIME_MS);
+    logger->log_debug("%s not set, defaulting to %d", minifi::Configure::nifi_graceful_shutdown_seconds,
+    STOP_WAIT_TIME_MS);
   }
 
-  configure->get(minifi::Configure::nifi_provenance_repository_class_name,
-                 prov_repo_class);
+  configure->get(minifi::Configure::nifi_provenance_repository_class_name, prov_repo_class);
   // Create repos for flow record and provenance
-  std::shared_ptr<core::Repository> prov_repo = core::createRepository(
-      prov_repo_class, true,"provenance");
+  std::shared_ptr<core::Repository> prov_repo = core::createRepository(prov_repo_class, true, "provenance");
   prov_repo->initialize(configure);
 
-  configure->get(minifi::Configure::nifi_flow_repository_class_name,
-                 flow_repo_class);
+  configure->get(minifi::Configure::nifi_flow_repository_class_name, flow_repo_class);
 
-  std::shared_ptr<core::Repository> flow_repo = core::createRepository(
-      flow_repo_class, true, "flowfile");
+  std::shared_ptr<core::Repository> flow_repo = core::createRepository(flow_repo_class, true, "flowfile");
 
   flow_repo->initialize(configure);
 
-  configure->get(minifi::Configure::nifi_configuration_class_name,
-                 nifi_configuration_class_name);
+  configure->get(minifi::Configure::nifi_content_repository_class_name, content_repo_class);
+
+  std::shared_ptr<core::ContentRepository> content_repo = core::createContentRepository(content_repo_class, true, "content");
+
+  content_repo->initialize(configure);
+
+  configure->get(minifi::Configure::nifi_configuration_class_name, nifi_configuration_class_name);
+
   std::shared_ptr<minifi::io::StreamFactory> stream_factory = std::make_shared<minifi::io::StreamFactory>(configure);
 
-  std::unique_ptr<core::FlowConfiguration> flow_configuration = std::move(
-      core::createFlowConfiguration(prov_repo, flow_repo, configure, stream_factory,
-                                   nifi_configuration_class_name));
+  std::unique_ptr<core::FlowConfiguration> flow_configuration = std::move(core::createFlowConfiguration(prov_repo, flow_repo, content_repo, configure, stream_factory, nifi_configuration_class_name));
 
   std::shared_ptr<minifi::FlowController> controller = std::unique_ptr<minifi::FlowController>(
-      new minifi::FlowController(prov_repo, flow_repo, configure,
-                                 std::move(flow_configuration)));
+      new minifi::FlowController(prov_repo, flow_repo, configure, std::move(flow_configuration), content_repo));
 
   logger->log_info("Loading FlowController");
   // Load flow from specified configuration file
@@ -205,9 +196,9 @@ int main(int argc, char **argv) {
    * Trigger unload -- wait stop_wait_time
    */
   controller->waitUnload(stop_wait_time);
-  
+
   flow_repo = nullptr;
-  
+
   prov_repo = nullptr;
 
   logger->log_info("MiNiFi exit");


[2/6] nifi-minifi-cpp git commit: MINIFI-249: Update prov repo to better abstract deser.

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/test/integration/HttpGetIntegrationTest.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/integration/HttpGetIntegrationTest.cpp b/libminifi/test/integration/HttpGetIntegrationTest.cpp
index ae60dc1..a235759 100644
--- a/libminifi/test/integration/HttpGetIntegrationTest.cpp
+++ b/libminifi/test/integration/HttpGetIntegrationTest.cpp
@@ -26,6 +26,7 @@
 #include <thread>
 #include <type_traits>
 #include <vector>
+#include "../TestServer.h"
 #include "../TestBase.h"
 #include "utils/StringUtils.h"
 #include "core/Core.h"
@@ -41,9 +42,23 @@ void waitToVerifyProcessor() {
   std::this_thread::sleep_for(std::chrono::seconds(10));
 }
 
+int log_message(const struct mg_connection *conn, const char *message) {
+  puts(message);
+  return 1;
+}
+
+int ssl_enable(void *ssl_context, void *user_data) {
+  struct ssl_ctx_st *ctx = (struct ssl_ctx_st *) ssl_context;
+  return 0;
+}
+
 int main(int argc, char **argv) {
-  LogTestController::getInstance().setInfo<minifi::processors::InvokeHTTP>();
-  LogTestController::getInstance().setInfo<minifi::processors::LogAttribute>();
+  init_webserver();
+  LogTestController::getInstance().setDebug<core::Processor>();
+  LogTestController::getInstance().setDebug<core::ProcessSession>();
+  LogTestController::getInstance().setDebug<core::repository::VolatileContentRepository>();
+  LogTestController::getInstance().setDebug<minifi::processors::InvokeHTTP>();
+  LogTestController::getInstance().setDebug<minifi::processors::LogAttribute>();
   std::string key_dir, test_file_location;
   if (argc > 1) {
     test_file_location = argv[1];
@@ -59,27 +74,61 @@ int main(int argc, char **argv) {
   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<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
+
+  content_repo->initialize(configuration);
+
+  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<core::YamlConfiguration>(
+      new core::YamlConfiguration(test_repo, test_repo, content_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);
+  std::shared_ptr<minifi::FlowController> controller = std::make_shared<minifi::FlowController>(test_repo, test_flow_repo, configuration, std::move(yaml_ptr),
+                                                                                                content_repo,
+                                                                                                DEFAULT_ROOT_GROUP_NAME,
+                                                                                                true);
 
-  core::YamlConfiguration yaml_config(test_repo, test_repo, stream_factory, configuration, test_file_location);
+  core::YamlConfiguration yaml_config(test_repo, test_repo, content_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::Processor> proc = ptr->findProcessor("invoke");
+  assert(proc != nullptr);
 
+  std::shared_ptr<minifi::processors::InvokeHTTP> inv = std::dynamic_pointer_cast<minifi::processors::InvokeHTTP>(proc);
+
+  assert(inv != nullptr);
+  std::string url = "";
+  inv->getProperty(minifi::processors::InvokeHTTP::URL.getName(), url);
+  ptr.release();
+  std::string port, scheme, path;
+  parse_http_components(url, port, scheme, path);
+  struct mg_callbacks callback;
+  if (url.find("localhost") != std::string::npos) {
+    if (scheme == "https") {
+      std::string cert = "";
+      cert = key_dir + "nifi-cert.pem";
+      memset(&callback, 0, sizeof(callback));
+      callback.init_ssl = ssl_enable;
+      callback.log_message = log_message;
+      std::cout << cert << std::endl;
+      start_webserver(port, path, "hi this is a get test", &callback, cert);
+    } else {
+      start_webserver(port, path, "hi this is a get test");
+    }
+  }
   controller->load();
   controller->start();
   waitToVerifyProcessor();
 
   controller->waitUnload(60000);
+  if (url.find("localhost") != std::string::npos) {
+    stop_webserver();
+  }
   std::string logs = LogTestController::getInstance().log_output.str();
+
   assert(logs.find("key:filename value:") != std::string::npos);
-  assert(logs.find("key:invokehttp.request.url value:https://raw.githubusercontent.com/curl/curl/master/docs/examples/httpput.c") != std::string::npos);
-  assert(logs.find("Size:3734 Offset:0") != std::string::npos);
+  assert(logs.find("key:invokehttp.request.url value:" + url) != std::string::npos);
   assert(logs.find("key:invokehttp.status.code value:200") != std::string::npos);
   std::string stringtofind = "Resource Claim created ./content_repository/";
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/test/integration/HttpPostIntegrationTest.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/integration/HttpPostIntegrationTest.cpp b/libminifi/test/integration/HttpPostIntegrationTest.cpp
index dfa284f..9a46574 100644
--- a/libminifi/test/integration/HttpPostIntegrationTest.cpp
+++ b/libminifi/test/integration/HttpPostIntegrationTest.cpp
@@ -28,6 +28,7 @@
 #include <vector>
 #include "utils/StringUtils.h"
 #include "core/Core.h"
+#include "../TestServer.h"
 #include "../include/core/logging/Logger.h"
 #include "core/ProcessGroup.h"
 #include "core/yaml/YamlConfiguration.h"
@@ -42,6 +43,7 @@ void waitToVerifyProcessor() {
 }
 
 int main(int argc, char **argv) {
+  init_webserver();
   LogTestController::getInstance().setDebug<processors::InvokeHTTP>();
   LogTestController::getInstance().setDebug<minifi::core::ProcessSession>();
   std::string test_file_location;
@@ -51,7 +53,6 @@ int main(int argc, char **argv) {
   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);
 
@@ -62,31 +63,44 @@ int main(int argc, char **argv) {
 
   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<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
+  content_repo->initialize(configuration);
+  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<core::YamlConfiguration>(
+      new core::YamlConfiguration(test_repo, test_repo, content_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);
+  std::shared_ptr<minifi::FlowController> controller = std::make_shared<minifi::FlowController>(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), content_repo, DEFAULT_ROOT_GROUP_NAME,
+  true);
 
-  core::YamlConfiguration yaml_config(test_repo, test_repo, stream_factory, configuration, test_file_location);
+  core::YamlConfiguration yaml_config(test_repo, test_repo, content_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::Processor> proc = ptr->findProcessor("OhJeez");
+  assert(proc != nullptr);
+
+  std::shared_ptr<minifi::processors::InvokeHTTP> inv = std::dynamic_pointer_cast<minifi::processors::InvokeHTTP>(proc);
 
+  assert(inv != nullptr);
+  std::string url = "";
+  inv->getProperty(minifi::processors::InvokeHTTP::URL.getName(), url);
+  ptr.release();
+  std::string port, scheme, path;
+  parse_http_components(url, port, scheme, path);
+  start_webserver(port, path, "hi this is a post test");
   controller->load();
   controller->start();
   waitToVerifyProcessor();
 
   controller->waitUnload(60000);
+  std::string logs = LogTestController::getInstance().log_output.str();
+  // stop webserver
+  stop_webserver();
   assert(LogTestController::getInstance().contains("curl performed") == true);
-  assert(LogTestController::getInstance().contains("Import offset 0 length 12") == true);
+  assert(LogTestController::getInstance().contains("Import offset 0 length 22") == true);
 
   std::string stringtofind = "Resource Claim created ./content_repository/";
 
-  std::string logs = LogTestController::getInstance().log_output.str();
   size_t loc = logs.find(stringtofind);
   while (loc > 0 && loc != std::string::npos) {
     std::string id = logs.substr(loc + stringtofind.size(), 36);

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/test/integration/ProvenanceReportingTest.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/integration/ProvenanceReportingTest.cpp b/libminifi/test/integration/ProvenanceReportingTest.cpp
index a7bcc2b..a6dc377 100644
--- a/libminifi/test/integration/ProvenanceReportingTest.cpp
+++ b/libminifi/test/integration/ProvenanceReportingTest.cpp
@@ -53,21 +53,20 @@ int main(int argc, char **argv) {
   LogTestController::getInstance().setDebug<core::ProcessGroup>();
 
   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<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
+  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<core::YamlConfiguration>(
+      new core::YamlConfiguration(test_repo, test_repo, content_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,
+  std::shared_ptr<minifi::FlowController> controller = std::make_shared<minifi::FlowController>(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), content_repo, DEFAULT_ROOT_GROUP_NAME,
                                                                                                 true);
 
-  core::YamlConfiguration yaml_config(test_repo, test_repo, stream_factory, configuration, test_file_location);
+  core::YamlConfiguration yaml_config(test_repo, test_repo, content_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());

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/test/integration/Site2SiteRestTest.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/integration/Site2SiteRestTest.cpp b/libminifi/test/integration/Site2SiteRestTest.cpp
index 01aa7a8..1773cdb 100644
--- a/libminifi/test/integration/Site2SiteRestTest.cpp
+++ b/libminifi/test/integration/Site2SiteRestTest.cpp
@@ -45,22 +45,22 @@ void waitToVerifyProcessor() {
   std::this_thread::sleep_for(std::chrono::seconds(10));
 }
 
-class ConfigHandler: public CivetHandler {
+class ConfigHandler : public CivetHandler {
  public:
   bool handleGet(CivetServer *server, struct mg_connection *conn) {
     static const std::string site2site_rest_resp = "{"
-           "\"revision\": {"
-           "\"clientId\": \"483d53eb-53ec-4e93-b4d4-1fc3d23dae6f\""
-           "},"
-           "\"controller\": {"
-           "\"id\": \"fe4a3a42-53b6-4af1-a80d-6fdfe60de97f\","
-           "\"name\": \"NiFi Flow\","
-           "\"remoteSiteListeningPort\": 10001,"
-           "\"siteToSiteSecure\": false"
-           "}}";
+        "\"revision\": {"
+        "\"clientId\": \"483d53eb-53ec-4e93-b4d4-1fc3d23dae6f\""
+        "},"
+        "\"controller\": {"
+        "\"id\": \"fe4a3a42-53b6-4af1-a80d-6fdfe60de97f\","
+        "\"name\": \"NiFi Flow\","
+        "\"remoteSiteListeningPort\": 10001,"
+        "\"siteToSiteSecure\": false"
+        "}}";
     mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-        "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-        site2site_rest_resp.length());
+              "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
+              site2site_rest_resp.length());
     mg_printf(conn, "%s", site2site_rest_resp.c_str());
     return true;
   }
@@ -71,7 +71,7 @@ int main(int argc, char **argv) {
   LogTestController::getInstance().setInfo<minifi::FlowController>();
 
   const char *options[] = { "document_root", ".", "listening_ports", "8082", 0 };
-  std::vector < std::string > cpp_options;
+  std::vector<std::string> cpp_options;
   for (int i = 0; i < (sizeof(options) / sizeof(options[0]) - 1); i++) {
     cpp_options.push_back(options[i]);
   }
@@ -106,28 +106,31 @@ int main(int argc, char **argv) {
       TestFlowRepository>();
 
   configuration->set(minifi::Configure::nifi_flow_configuration_file,
-      test_file_location);
+                     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);
+      <minifi::io::StreamFactory>(configuration);
+  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
 
-  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);
+  content_repo->initialize(configuration);
 
-  core::YamlConfiguration yaml_config(test_repo, test_repo, stream_factory,
-      configuration, test_file_location);
+  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<core::YamlConfiguration>(
+      new core::YamlConfiguration(test_repo, test_repo, content_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),
+                                                                                                content_repo,
+                                                                                                DEFAULT_ROOT_GROUP_NAME,
+                                                                                                true);
+
+  core::YamlConfiguration yaml_config(test_repo, test_repo, content_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());
+                                                                test_file_location);
+  std::shared_ptr<core::ProcessGroup> pg = std::shared_ptr<core::ProcessGroup
+      >(ptr.get());
   ptr.release();
 
   controller->load();

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/test/integration/TestExecuteProcess.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/integration/TestExecuteProcess.cpp b/libminifi/test/integration/TestExecuteProcess.cpp
index ef0d113..5506c32 100644
--- a/libminifi/test/integration/TestExecuteProcess.cpp
+++ b/libminifi/test/integration/TestExecuteProcess.cpp
@@ -27,7 +27,7 @@
 #include <memory>
 #include <vector>
 #include <fstream>
-
+#include "core/repository/VolatileContentRepository.h"
 #include "../unit/ProvenanceTestHelper.h"
 #include "FlowController.h"
 #include "processors/GetFile.h"
@@ -47,15 +47,17 @@ int main(int argc, char **argv) {
   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);
+  std::shared_ptr<core::Repository> test_repo =
+      std::make_shared<TestRepository>();
+  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
+  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, content_repo);
 
   uuid_t processoruuid;
   assert(true == processor->getUUID(processoruuid));
-
-  std::shared_ptr<minifi::Connection> connection = std::make_shared<minifi::Connection>(test_repo, "executeProcessConnection");
+  std::shared_ptr<minifi::Connection> connection = std::make_shared<minifi::Connection>(test_repo, content_repo, "executeProcessConnection");
   connection->setRelationship(core::Relationship("success", "description"));
 
   // link the connections so that we can test results at the end for this
@@ -79,7 +81,7 @@ int main(int argc, char **argv) {
 
   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);
+  std::shared_ptr<core::ProcessContext> contextset = std::make_shared<core::ProcessContext>(node2, controller_services_provider, test_repo, test_repo);
   core::ProcessSessionFactory factory(contextset.get());
   processor->onSchedule(contextset.get(), &factory);
 
@@ -87,7 +89,7 @@ int main(int argc, char **argv) {
     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);
+      std::shared_ptr<core::ProcessContext> context = std::make_shared<core::ProcessContext>(node, controller_services_provider, test_repo, test_repo);
       context->setProperty(org::apache::nifi::minifi::processors::ExecuteProcess::Command, "sleep 0.5");
       std::shared_ptr<core::ProcessSession> session = std::make_shared<core::ProcessSession>(context.get());
       while (!is_ready.load(std::memory_order_relaxed)) {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/test/resources/TestHTTPGet.yml
----------------------------------------------------------------------
diff --git a/libminifi/test/resources/TestHTTPGet.yml b/libminifi/test/resources/TestHTTPGet.yml
index 2f64f2a..58f95d9 100644
--- a/libminifi/test/resources/TestHTTPGet.yml
+++ b/libminifi/test/resources/TestHTTPGet.yml
@@ -32,7 +32,7 @@ Processors:
       auto-terminated relationships list:
       Properties:
           HTTP Method: GET
-          Remote URL: https://raw.githubusercontent.com/curl/curl/master/docs/examples/httpput.c
+          Remote URL: http://localhost:10003/geturl
     - name: OhJeez
       id: 2438e3c8-015a-1000-79ca-83af40ec1992
       class: org.apache.nifi.processors.standard.LogAttribute

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/test/resources/TestHTTPGetSecure.yml
----------------------------------------------------------------------
diff --git a/libminifi/test/resources/TestHTTPGetSecure.yml b/libminifi/test/resources/TestHTTPGetSecure.yml
index f3a23e5..9d19632 100644
--- a/libminifi/test/resources/TestHTTPGetSecure.yml
+++ b/libminifi/test/resources/TestHTTPGetSecure.yml
@@ -33,7 +33,7 @@ Processors:
       Properties:
           SSL Context Service: SSLContextService
           HTTP Method: GET
-          Remote URL: https://raw.githubusercontent.com/curl/curl/master/docs/examples/httpput.c
+          Remote URL: https://raw.githubusercontent.com/apache/nifi-minifi-cpp/master/docs/minifi-logo.png
     - name: OhJeez
       id: 2438e3c8-015a-1000-79ca-83af40ec1992
       class: org.apache.nifi.processors.standard.LogAttribute

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/test/resources/TestHTTPPost.yml
----------------------------------------------------------------------
diff --git a/libminifi/test/resources/TestHTTPPost.yml b/libminifi/test/resources/TestHTTPPost.yml
index 837194d..c76069a 100644
--- a/libminifi/test/resources/TestHTTPPost.yml
+++ b/libminifi/test/resources/TestHTTPPost.yml
@@ -46,7 +46,7 @@ Processors:
       auto-terminated relationships list: response
       Properties:
           HTTP Method: POST
-          Remote URL: http://requestb.in/u8ax9uu8
+          Remote URL: http://localhost:10003/urlofchampions
           
     - name: Loggit
       id: 2438e3c8-015a-1000-79ca-83af40ec1993

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/test/resources/cn.ckey.pem
----------------------------------------------------------------------
diff --git a/libminifi/test/resources/cn.ckey.pem b/libminifi/test/resources/cn.ckey.pem
index 23017fa..fc42f06 100644
--- a/libminifi/test/resources/cn.ckey.pem
+++ b/libminifi/test/resources/cn.ckey.pem
@@ -1,5 +1,4 @@
 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-----

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/test/resources/cn.crt.pem
----------------------------------------------------------------------
diff --git a/libminifi/test/resources/cn.crt.pem b/libminifi/test/resources/cn.crt.pem
index 3a786db..60a38ac 100644
--- a/libminifi/test/resources/cn.crt.pem
+++ b/libminifi/test/resources/cn.crt.pem
@@ -1,5 +1,4 @@
 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

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/test/resources/nifi-cert.pem
----------------------------------------------------------------------
diff --git a/libminifi/test/resources/nifi-cert.pem b/libminifi/test/resources/nifi-cert.pem
index 4e404cd..0c3b7da 100644
--- a/libminifi/test/resources/nifi-cert.pem
+++ b/libminifi/test/resources/nifi-cert.pem
@@ -18,3 +18,30 @@ lvrRtWOqyGHiRoaRE5+VUjyO+0ToEgj9E+3rV8JL66BT7SWQusLGqbX1OoANCMTj
 BRYeqB0g0PrXU+6chh6StpNSnYzkQdoxLUIDYYZx2XGsbkjDh/k6ni6bgJEKEOCu
 T3Z2tyvGpc+PjLRXW/WyXCpg/xfr3+GSVKI6ark=
 -----END CERTIFICATE-----
+-----BEGIN RSA PRIVATE KEY-----
+MIIEpAIBAAKCAQEAwCF6Tchue7tR66BPg886WOYNPgSwNaq1KJQSuGcEHK2wlAEu
+YfiYz9LbjFLZRLRY2CF9mIGb683byrnvOMcq6a+YdXDaOHZnkKBSsI/xTzScXTv3
+EKSueZ0sMuD7L0y/2Cs2lf8heBUEUqmNe15J9yvEQ1GpJ0j7iCCneKYjjezFWglR
+Sv/9suvqVCxIxr4j9gXODgyU3wdwIxkQUBJXk4GtDp03Rxcx6Ch0VBwjcGkYHhcs
+GHRzg6dcr795tLfOQNA/Vlje0+RtH/KU/WXgzl9nKtxD7XUwZyhoElzNcehN0WmK
+DgAmASncvy7+YYzKU69H14Q+2n/apdoqx/kTQQIDAQABAoIBAQCz7eY69+y4BXo3
+nz84Ipby8CcQoJVg/QiBAwLxHNCWBvdp9B069PQvFLo1FNWSaQ8XAW48p4yc7YHb
+vftRgfwnMyIlQdWrsP9WSz6FSZhkY9HX4rODK6aWD+J3l4jFCCxVxkpteKwgaBZP
+T6hHE8tTJfK8VLqEJu4g0uvjqjt7ydJT69lThdyf3VE0v6ZeSjsya5qqw+9RK+uC
+q5T/8FxeFZgpfR6UXXnoLAmAkfcMZNIBo6cOJWi/BQHjZdpCOVXUBtu0/lC8bffa
+4/ESaxRS8kOp+WEb64pT7u6F7yhD/kve6ZnJj/SX1EvN+RzB3zoVG42WUs/+/SwN
+dU1ERz+tAoGBAPbgZPDnWuKxW7Cam/Aqmvux624C1lNfhfXEGURhyc+wHWjjhWRe
+2vEPJOVxG5pN/FAo+lFoGiLe3QsLRLPlQrGfT/92W28QEcRrRSutjRZOL3wKezQA
+DkAPU9HX3lACR5yQD6+a0HHgMr1MqeNFPi9MPPjywGywTyWzHd4WQqvTAoGBAMc7
+J4fpr5uPVq9mKemK67i7meJ8AxjjU7oNe8EN+2XfCYcQUmgIo+dLzV9+DTrYkoTz
+iqjA6Ph2DNs6YHI/JNwsdSbAz6KVDteimt3t+uyNpiMGuyLmfOgpYEMJcHp+q6I6
+7PGKVS4c5iPFiYuIo23Is9ZMxOVQp76+UOy09rwbAoGBAOM5Za7VQjGkTGAf7ab/
+j+ZZu/dlZR8XrJSoCRmHZ9hgoLEJuJzJMXruFWeY028SmEivbrW+u0+dEJY5qOJr
+ARe7KkZXCZEPmUrP8Lpi4pjFHa9tdjhGVNdhRCTAKz442vCfJ9DZDUHCuPDCvxsP
+gEzIPtZjl/hxzmdElRj0JClBAoGAaXmfzAyjs6+HLQThW4r4kKyBI66T1TFEulM5
+GVPVrHEQEjlJ51nrrCAtckjBqE3QBCMLXZwDusaEt+uH8/QKB6Zhv0qEooZXfUHQ
+y32aQnIbap+9oxRzPFXraJIuwisdop2fo6Cgx/D0xitmTkDghNaknue1tdGlfQ40
+uZx0o9ECgYBeKeNbMnWoO46ZOrhaz8On+fIY7xtboV2bALy7lvUbWd9B41ntqYUm
+NHlYXDDU+Izs5wnNJnNnx4vECuUzYbpeY82dvMewlQwfl5aiyKrjo7VxLm//2U/K
+hlID6DU5wi9O+TAQ319DhxT7Ja+AQxO/OFS/mfrtwJEevxXqJLu55Q==
+-----END RSA PRIVATE KEY-----

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/test/unit/FileStreamTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/FileStreamTests.cpp b/libminifi/test/unit/FileStreamTests.cpp
new file mode 100644
index 0000000..5c86f19
--- /dev/null
+++ b/libminifi/test/unit/FileStreamTests.cpp
@@ -0,0 +1,210 @@
+/**
+ *
+ * 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 "io/FileStream.h"
+#include <string>
+#include <vector>
+#include <uuid/uuid.h>
+#include "../TestBase.h"
+
+TEST_CASE("TestFileOverWrite", "[TestFiles]") {
+  TestController testController;
+  char format[] = "/tmp/gt.XXXXXX";
+  char *dir = testController.createTempDirectory(format);
+
+  std::fstream file;
+  std::stringstream ss;
+  ss << dir << "/" << "tstFile.ext";
+  std::string path = ss.str();
+  file.open(path, std::ios::out);
+  file << "tempFile";
+  file.close();
+
+  minifi::io::FileStream stream(path, 0, true);
+  std::vector<uint8_t> readBuffer;
+  REQUIRE(stream.readData(readBuffer, stream.getSize()) == stream.getSize());
+
+  uint8_t* data = readBuffer.data();
+
+  REQUIRE(std::string(reinterpret_cast<char*>(data), readBuffer.size()) == "tempFile");
+
+  stream.seek(4);
+
+  stream.write(reinterpret_cast<uint8_t*>(const_cast<char*>("file")), 4);
+
+  stream.seek(0);
+
+  std::vector<uint8_t> verifybuffer;
+
+  REQUIRE(stream.readData(verifybuffer, stream.getSize()) == stream.getSize());
+
+  data = verifybuffer.data();
+
+  REQUIRE(std::string(reinterpret_cast<char*>(data), verifybuffer.size()) == "tempfile");
+
+  unlink(ss.str().c_str());
+}
+
+TEST_CASE("TestFileBadArgumentNoChange", "[TestLoader]") {
+  TestController testController;
+  char format[] = "/tmp/gt.XXXXXX";
+  char *dir = testController.createTempDirectory(format);
+
+  std::fstream file;
+  std::stringstream ss;
+  ss << dir << "/" << "tstFile.ext";
+  std::string path = ss.str();
+  file.open(path, std::ios::out);
+  file << "tempFile";
+  file.close();
+
+  minifi::io::FileStream stream(path, 0, true);
+  std::vector<uint8_t> readBuffer;
+  REQUIRE(stream.readData(readBuffer, stream.getSize()) == stream.getSize());
+
+  uint8_t* data = readBuffer.data();
+
+  REQUIRE(std::string(reinterpret_cast<char*>(data), readBuffer.size()) == "tempFile");
+
+  stream.seek(4);
+
+  stream.write(reinterpret_cast<uint8_t*>(const_cast<char*>("file")), 0);
+
+  stream.seek(0);
+
+  std::vector<uint8_t> verifybuffer;
+
+  REQUIRE(stream.readData(verifybuffer, stream.getSize()) == stream.getSize());
+
+  data = verifybuffer.data();
+
+  REQUIRE(std::string(reinterpret_cast<char*>(data), verifybuffer.size()) == "tempFile");
+
+  unlink(ss.str().c_str());
+}
+
+TEST_CASE("TestFileBadArgumentNoChange2", "[TestLoader]") {
+  TestController testController;
+  char format[] = "/tmp/gt.XXXXXX";
+  char *dir = testController.createTempDirectory(format);
+
+  std::fstream file;
+  std::stringstream ss;
+  ss << dir << "/" << "tstFile.ext";
+  std::string path = ss.str();
+  file.open(path, std::ios::out);
+  file << "tempFile";
+  file.close();
+
+  minifi::io::FileStream stream(path, 0, true);
+  std::vector<uint8_t> readBuffer;
+  REQUIRE(stream.readData(readBuffer, stream.getSize()) == stream.getSize());
+
+  uint8_t* data = readBuffer.data();
+
+  REQUIRE(std::string(reinterpret_cast<char*>(data), readBuffer.size()) == "tempFile");
+
+  stream.seek(4);
+
+  stream.write(nullptr, 0);
+
+  stream.seek(0);
+
+  std::vector<uint8_t> verifybuffer;
+
+  REQUIRE(stream.readData(verifybuffer, stream.getSize()) == stream.getSize());
+
+  data = verifybuffer.data();
+
+  REQUIRE(std::string(reinterpret_cast<char*>(data), verifybuffer.size()) == "tempFile");
+
+  unlink(ss.str().c_str());
+}
+
+TEST_CASE("TestFileBadArgumentNoChange3", "[TestLoader]") {
+  TestController testController;
+  char format[] = "/tmp/gt.XXXXXX";
+  char *dir = testController.createTempDirectory(format);
+
+  std::fstream file;
+  std::stringstream ss;
+  ss << dir << "/" << "tstFile.ext";
+  std::string path = ss.str();
+  file.open(path, std::ios::out);
+  file << "tempFile";
+  file.close();
+
+  minifi::io::FileStream stream(path, 0, true);
+  std::vector<uint8_t> readBuffer;
+  REQUIRE(stream.readData(readBuffer, stream.getSize()) == stream.getSize());
+
+  uint8_t* data = readBuffer.data();
+
+  REQUIRE(std::string(reinterpret_cast<char*>(data), readBuffer.size()) == "tempFile");
+
+  stream.seek(4);
+
+  stream.write(nullptr, 0);
+
+  stream.seek(0);
+
+  std::vector<uint8_t> verifybuffer;
+
+  REQUIRE(stream.readData(nullptr, stream.getSize()) == -1);
+
+  data = verifybuffer.data();
+
+  REQUIRE(std::string(reinterpret_cast<char*>(data), verifybuffer.size()) == "");
+
+  unlink(ss.str().c_str());
+}
+
+
+TEST_CASE("TestFileBeyondEnd3", "[TestLoader]") {
+  TestController testController;
+  char format[] = "/tmp/gt.XXXXXX";
+  char *dir = testController.createTempDirectory(format);
+
+  std::fstream file;
+  std::stringstream ss;
+  ss << dir << "/" << "tstFile.ext";
+  std::string path = ss.str();
+  file.open(path, std::ios::out);
+  file << "tempFile";
+  file.close();
+
+  minifi::io::FileStream stream(path, 0, true);
+  std::vector<uint8_t> readBuffer;
+  REQUIRE(stream.readData(readBuffer, stream.getSize()) == stream.getSize());
+
+  uint8_t* data = readBuffer.data();
+
+  REQUIRE(std::string(reinterpret_cast<char*>(data), readBuffer.size()) == "tempFile");
+
+  stream.seek(0);
+
+  std::vector<uint8_t> verifybuffer;
+
+  REQUIRE(stream.readData(verifybuffer, 8192) == 8);
+
+  data = verifybuffer.data();
+
+  REQUIRE(std::string(reinterpret_cast<char*>(data), verifybuffer.size()) == "tempFile");
+
+  unlink(ss.str().c_str());
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/test/unit/InvokeHTTPTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/InvokeHTTPTests.cpp b/libminifi/test/unit/InvokeHTTPTests.cpp
index 705ac84..2ef3c17 100644
--- a/libminifi/test/unit/InvokeHTTPTests.cpp
+++ b/libminifi/test/unit/InvokeHTTPTests.cpp
@@ -25,6 +25,7 @@
 #include <string>
 #include <set>
 #include "FlowController.h"
+#include "io/BaseStream.h"
 #include "../TestBase.h"
 #include "processors/GetFile.h"
 #include "core/Core.h"
@@ -35,105 +36,9 @@
 #include "core/ProcessSession.h"
 #include "core/ProcessorNode.h"
 
-TEST_CASE("HTTPTestsPostNoResourceClaim", "[httptest1]") {
-  TestController testController;
-  LogTestController::getInstance().setInfo<org::apache::nifi::minifi::processors::InvokeHTTP>();
-
-  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");
-
-  std::shared_ptr<core::Processor> invokehttp = std::make_shared<org::apache::nifi::minifi::processors::InvokeHTTP>("invokehttp");
-  uuid_t processoruuid;
-  REQUIRE(true == processor->getUUID(processoruuid));
-
-  uuid_t invokehttp_uuid;
-  REQUIRE(true == invokehttp->getUUID(invokehttp_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, "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);
-
-  processor->addConnection(connection);
-  invokehttp->addConnection(connection);
-  invokehttp->addConnection(connection2);
-
-  core::ProcessorNode node(processor);
-  core::ProcessorNode node2(invokehttp);
-
-  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");
-
-  context2.setProperty(org::apache::nifi::minifi::processors::InvokeHTTP::Method, "POST");
-  context2.setProperty(org::apache::nifi::minifi::processors::InvokeHTTP::URL, "http://localhost:8685/testytesttest");
-  core::ProcessSession session(&context);
-  core::ProcessSession session2(&context2);
-
-  REQUIRE(processor->getName() == "listenhttp");
-
-  core::ProcessSessionFactory factory(&context);
-
-  std::shared_ptr<core::FlowFile> record;
-  processor->setScheduledState(core::ScheduledState::RUNNING);
-  processor->onSchedule(&context, &factory);
-  processor->onTrigger(&context, &session);
-
-  invokehttp->incrementActiveTasks();
-  invokehttp->setScheduledState(core::ScheduledState::RUNNING);
-  core::ProcessSessionFactory factory2(&context2);
-  invokehttp->onSchedule(&context2, &factory2);
-  invokehttp->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);
-
-  processor->incrementActiveTasks();
-  processor->setScheduledState(core::ScheduledState::RUNNING);
-  processor->onTrigger(&context, &session);
-
-  reporter = session.getProvenanceReporter();
-
-  records = reporter->getEvents();
-  session.commit();
-
-  invokehttp->incrementActiveTasks();
-  invokehttp->setScheduledState(core::ScheduledState::RUNNING);
-  invokehttp->onTrigger(&context2, &session2);
-
-  session2.commit();
-  records = reporter->getEvents();
-
-  for (provenance::ProvenanceEventRecord *provEventRecord : records) {
-    REQUIRE(provEventRecord->getComponentType() == processor->getName());
-  }
-  std::shared_ptr<core::FlowFile> ffr = session2.get();
-  REQUIRE(true == LogTestController::getInstance().contains("exiting because method is POST"));
-  LogTestController::getInstance().reset();
-}
-
 TEST_CASE("HTTPTestsWithNoResourceClaimPOST", "[httptest1]") {
   TestController testController;
+  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
   LogTestController::getInstance().setInfo<org::apache::nifi::minifi::processors::InvokeHTTP>();
 
   std::shared_ptr<TestRepository> repo = std::make_shared<TestRepository>();
@@ -154,16 +59,16 @@ 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");
+  std::shared_ptr<minifi::Connection> gcConnection = std::make_shared<minifi::Connection>(repo, content_repo, "getfileCreate2Connection");
   gcConnection->setRelationship(core::Relationship("success", "description"));
 
-  std::shared_ptr<minifi::Connection> laConnection = std::make_shared<minifi::Connection>(repo, "logattribute");
+  std::shared_ptr<minifi::Connection> laConnection = std::make_shared<minifi::Connection>(repo, content_repo, "logattribute");
   laConnection->setRelationship(core::Relationship("success", "description"));
 
-  std::shared_ptr<minifi::Connection> connection = std::make_shared<minifi::Connection>(repo, "getfileCreate2Connection");
+  std::shared_ptr<minifi::Connection> connection = std::make_shared<minifi::Connection>(repo, content_repo, "getfileCreate2Connection");
   connection->setRelationship(core::Relationship("success", "description"));
 
-  std::shared_ptr<minifi::Connection> connection2 = std::make_shared<minifi::Connection>(repo, "listenhttp");
+  std::shared_ptr<minifi::Connection> connection2 = std::make_shared<minifi::Connection>(repo, content_repo, "listenhttp");
 
   connection2->setRelationship(core::Relationship("No Retry", "description"));
 
@@ -181,8 +86,8 @@ TEST_CASE("HTTPTestsWithNoResourceClaimPOST", "[httptest1]") {
   core::ProcessorNode node(listenhttp);
   core::ProcessorNode node2(invokehttp);
   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);
+  core::ProcessContext context(node, controller_services_provider, repo, repo, content_repo);
+  core::ProcessContext context2(node2, controller_services_provider, repo, repo, content_repo);
   context.setProperty(org::apache::nifi::minifi::processors::ListenHTTP::Port, "8686");
   context.setProperty(org::apache::nifi::minifi::processors::ListenHTTP::BasePath, "/testytesttest");
 
@@ -242,9 +147,10 @@ class CallBack : public minifi::OutputStreamCallback {
   }
   virtual ~CallBack() {
   }
-  virtual void process(std::ofstream *stream) {
+  virtual int64_t process(std::shared_ptr<minifi::io::BaseStream> stream) {
+    // leaving the typo for posterity sake
     std::string st = "we're gnna write some test stuff";
-    stream->write(st.c_str(), st.length());
+    return stream->write(reinterpret_cast<uint8_t*>(const_cast<char*>(st.c_str())), st.length());
   }
 };
 
@@ -270,16 +176,18 @@ 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");
+  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
+
+  std::shared_ptr<minifi::Connection> gcConnection = std::make_shared<minifi::Connection>(repo, content_repo, "getfileCreate2Connection");
   gcConnection->setRelationship(core::Relationship("success", "description"));
 
-  std::shared_ptr<minifi::Connection> laConnection = std::make_shared<minifi::Connection>(repo, "logattribute");
+  std::shared_ptr<minifi::Connection> laConnection = std::make_shared<minifi::Connection>(repo, content_repo, "logattribute");
   laConnection->setRelationship(core::Relationship("success", "description"));
 
-  std::shared_ptr<minifi::Connection> connection = std::make_shared<minifi::Connection>(repo, "getfileCreate2Connection");
+  std::shared_ptr<minifi::Connection> connection = std::make_shared<minifi::Connection>(repo, content_repo, "getfileCreate2Connection");
   connection->setRelationship(core::Relationship("success", "description"));
 
-  std::shared_ptr<minifi::Connection> connection2 = std::make_shared<minifi::Connection>(repo, "listenhttp");
+  std::shared_ptr<minifi::Connection> connection2 = std::make_shared<minifi::Connection>(repo, content_repo, "listenhttp");
 
   connection2->setRelationship(core::Relationship("No Retry", "description"));
 
@@ -299,8 +207,8 @@ TEST_CASE("HTTPTestsWithResourceClaimPOST", "[httptest1]") {
   core::ProcessorNode node(invokehttp);
   core::ProcessorNode node2(listenhttp);
   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);
+  core::ProcessContext context(node, controller_services_provider, repo, repo, content_repo);
+  core::ProcessContext context2(node2, controller_services_provider, repo, repo,  content_repo);
   context.setProperty(org::apache::nifi::minifi::processors::ListenHTTP::Port, "8680");
   context.setProperty(org::apache::nifi::minifi::processors::ListenHTTP::BasePath, "/testytesttest");
 
@@ -317,14 +225,9 @@ TEST_CASE("HTTPTestsWithResourceClaimPOST", "[httptest1]") {
 
   CallBack callback;
 
-  /*
-   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;
   attributes["testy"] = "test";
-  std::shared_ptr<minifi::FlowFileRecord> flow = std::make_shared<minifi::FlowFileRecord>(repo, attributes);
+  std::shared_ptr<minifi::FlowFileRecord> flow = std::make_shared<minifi::FlowFileRecord>(repo, content_repo, attributes);
   session2.write(flow, &callback);
 
   invokehttp->incrementActiveTasks();
@@ -368,3 +271,39 @@ TEST_CASE("HTTPTestsWithResourceClaimPOST", "[httptest1]") {
   LogTestController::getInstance().reset();
 }
 
+TEST_CASE("HTTPTestsPostNoResourceClaim", "[httptest1]") {
+  TestController testController;
+  LogTestController::getInstance().setInfo<org::apache::nifi::minifi::processors::InvokeHTTP>();
+  LogTestController::getInstance().setInfo<org::apache::nifi::minifi::processors::ListenHTTP>();
+  LogTestController::getInstance().setInfo<core::Processor>();
+
+  std::shared_ptr<TestPlan> plan = testController.createPlan();
+  std::shared_ptr<core::Processor> processor = plan->addProcessor("ListenHTTP", "listenhttp", core::Relationship("No Retry", "description"), false);
+  std::shared_ptr<core::Processor> invokehttp = plan->addProcessor("InvokeHTTP", "invokehttp", core::Relationship("success", "description"), true);
+
+  REQUIRE(true == plan->setProperty(processor, org::apache::nifi::minifi::processors::ListenHTTP::Port.getName(), "8685"));
+  REQUIRE(true == plan->setProperty(processor, org::apache::nifi::minifi::processors::ListenHTTP::BasePath.getName(), "/testytesttest"));
+
+  REQUIRE(true == plan->setProperty(invokehttp, org::apache::nifi::minifi::processors::InvokeHTTP::Method.getName(), "POST"));
+  REQUIRE(true == plan->setProperty(invokehttp, org::apache::nifi::minifi::processors::InvokeHTTP::URL.getName(), "http://localhost:8685/testytesttest"));
+  plan->reset();
+  testController.runSession(plan, true);
+
+  std::set<provenance::ProvenanceEventRecord*> records = plan->getProvenanceRecords();
+  std::shared_ptr<core::FlowFile> record = plan->getCurrentFlowFile();
+  REQUIRE(record == nullptr);
+  REQUIRE(records.size() == 0);
+
+  plan->reset();
+  testController.runSession(plan, true);
+
+  records = plan->getProvenanceRecords();
+  record = plan->getCurrentFlowFile();
+
+  for (provenance::ProvenanceEventRecord *provEventRecord : records) {
+    REQUIRE(provEventRecord->getComponentType() == processor->getName());
+  }
+  std::shared_ptr<core::FlowFile> ffr = plan->getCurrentFlowFile();
+  REQUIRE(true == LogTestController::getInstance().contains("exiting because method is POST"));
+  LogTestController::getInstance().reset();
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/test/unit/ProcessorTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/ProcessorTests.cpp b/libminifi/test/unit/ProcessorTests.cpp
index 9e2d50c..7f34ba4 100644
--- a/libminifi/test/unit/ProcessorTests.cpp
+++ b/libminifi/test/unit/ProcessorTests.cpp
@@ -23,11 +23,12 @@
 #include <vector>
 #include <set>
 #include <fstream>
-#include "../unit/ProvenanceTestHelper.h"
+
 #include "../TestBase.h"
 #include "processors/ListenHTTP.h"
 #include "processors/LogAttribute.h"
 #include "processors/GetFile.h"
+#include "../unit/ProvenanceTestHelper.h"
 #include "core/Core.h"
 #include "core/FlowFile.h"
 #include "core/Processor.h"
@@ -42,131 +43,12 @@ TEST_CASE("Test Creation of GetFile", "[getfileCreate]") {
   REQUIRE(processor->getName() == "processorname");
 }
 
-TEST_CASE("Test Find file", "[getfileCreate2]") {
-  TestController testController;
-
-  std::shared_ptr<core::Processor> processor = std::make_shared<org::apache::nifi::minifi::processors::GetFile>("getfileCreate2");
-  std::shared_ptr<org::apache::nifi::minifi::Configure> configure = std::make_shared<org::apache::nifi::minifi::Configure>();
-
-  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>(configure), 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(reinterpret_cast<uint8_t*>(const_cast<char*>(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;
-
+  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
   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";
@@ -175,7 +57,8 @@ TEST_CASE("Test GetFileLikeIt'sThreaded", "[getfileCreate3]") {
   uuid_t processoruuid;
   REQUIRE(true == processor->getUUID(processoruuid));
 
-  std::shared_ptr<minifi::Connection> connection = std::make_shared<minifi::Connection>(test_repo, "getfileCreate2Connection");
+  std::shared_ptr<minifi::Connection> connection = std::make_shared<minifi::Connection>(test_repo, content_repo, "getfileCreate2Connection");
+
   connection->setRelationship(core::Relationship("success", "description"));
 
   // link the connections so that we can test results at the end for this
@@ -190,7 +73,7 @@ TEST_CASE("Test GetFileLikeIt'sThreaded", "[getfileCreate3]") {
 
   core::ProcessorNode node(processor);
   std::shared_ptr<core::controller::ControllerServiceProvider> controller_services_provider = nullptr;
-  core::ProcessContext context(node, controller_services_provider, test_repo);
+  core::ProcessContext context(node, controller_services_provider, test_repo, test_repo);
   core::ProcessSessionFactory factory(&context);
   context.setProperty(org::apache::nifi::minifi::processors::GetFile::Directory, dir);
   // replicate 10 threads
@@ -245,71 +128,59 @@ TEST_CASE("LogAttributeTest", "[getfileCreate3]") {
   TestController testController;
   LogTestController::getInstance().setDebug<minifi::processors::LogAttribute>();
 
-  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<TestPlan> plan = testController.createPlan();
+  std::shared_ptr<core::Processor> getfile = plan->addProcessor("GetFile", "getfileCreate2");
 
-  std::shared_ptr<core::Processor> logAttribute = std::make_shared<org::apache::nifi::minifi::processors::LogAttribute>("logattribute");
+  plan->addProcessor("LogAttribute", "logattribute", core::Relationship("success", "description"), true);
 
   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);
+  plan->setProperty(getfile, org::apache::nifi::minifi::processors::GetFile::Directory.getName(), dir);
+  testController.runSession(plan, false);
+  std::set<provenance::ProvenanceEventRecord*> records = plan->getProvenanceRecords();
+  std::shared_ptr<core::FlowFile> record = plan->getCurrentFlowFile();
+  REQUIRE(record == nullptr);
+  REQUIRE(records.size() == 0);
 
-  processor->addConnection(connection);
-  logAttribute->addConnection(connection);
-  logAttribute->addConnection(connection2);
-  REQUIRE(dir != NULL);
+  std::fstream file;
+  std::stringstream ss;
+  ss << dir << "/" << "tstFile.ext";
+  file.open(ss.str(), std::ios::out);
+  file << "tempFile";
+  file.close();
+  plan->reset();
+  testController.runSession(plan, false);
 
-  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);
+  unlink(ss.str().c_str());
 
-  REQUIRE(processor->getName() == "getfileCreate2");
+  records = plan->getProvenanceRecords();
+  record = plan->getCurrentFlowFile();
+  testController.runSession(plan, false);
 
-  std::shared_ptr<core::FlowFile> record;
-  processor->setScheduledState(core::ScheduledState::RUNNING);
+  records = plan->getProvenanceRecords();
+  record = plan->getCurrentFlowFile();
 
-  core::ProcessSessionFactory factory(&context);
-  processor->onSchedule(&context, &factory);
-  processor->onTrigger(&context, &session);
+  REQUIRE(true == LogTestController::getInstance().contains("key:absolute.path value:" + ss.str()));
+  REQUIRE(true == LogTestController::getInstance().contains("Size:8 Offset:0"));
+  REQUIRE(true == LogTestController::getInstance().contains("key:path value:" + std::string(dir)));
+  LogTestController::getInstance().reset();
+}
 
-  logAttribute->incrementActiveTasks();
-  logAttribute->setScheduledState(core::ScheduledState::RUNNING);
-  core::ProcessSessionFactory factory2(&context2);
-  logAttribute->onSchedule(&context2, &factory2);
-  logAttribute->onTrigger(&context2, &session2);
+TEST_CASE("Test Find file", "[getfileCreate3]") {
+  TestController testController;
+  std::shared_ptr<TestPlan> plan = testController.createPlan();
+  std::shared_ptr<core::Processor> processor = plan->addProcessor("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::make_shared<org::apache::nifi::minifi::Configure>());
+  plan->addProcessor(processorReport, "reporter", core::Relationship("success", "description"), false);
+  char format[] = "/tmp/gt.XXXXXX";
+  char *dir = testController.createTempDirectory(format);
 
-  provenance::ProvenanceReporter *reporter = session.getProvenanceReporter();
-  std::set<provenance::ProvenanceEventRecord*> records = reporter->getEvents();
-  record = session.get();
+  plan->setProperty(processor, org::apache::nifi::minifi::processors::GetFile::Directory.getName(), dir);
+  testController.runSession(plan, false);
+  std::set<provenance::ProvenanceEventRecord*> records = plan->getProvenanceRecords();
+  std::shared_ptr<core::FlowFile> record = plan->getCurrentFlowFile();
   REQUIRE(record == nullptr);
   REQUIRE(records.size() == 0);
 
@@ -319,26 +190,58 @@ TEST_CASE("LogAttributeTest", "[getfileCreate3]") {
   file.open(ss.str(), std::ios::out);
   file << "tempFile";
   file.close();
+  plan->reset();
+  testController.runSession(plan, false);
 
-  processor->incrementActiveTasks();
-  processor->setScheduledState(core::ScheduledState::RUNNING);
-  processor->onTrigger(&context, &session);
-  unlink(ss.str().c_str());
-  reporter = session.getProvenanceReporter();
-
-  records = reporter->getEvents();
-  session.commit();
-
-  logAttribute->incrementActiveTasks();
-  logAttribute->setScheduledState(core::ScheduledState::RUNNING);
-  logAttribute->onTrigger(&context2, &session2);
+  records = plan->getProvenanceRecords();
+  record = plan->getCurrentFlowFile();
+  for (provenance::ProvenanceEventRecord *provEventRecord : records) {
+    REQUIRE(provEventRecord->getComponentType() == processor->getName());
+  }
+  std::shared_ptr<core::FlowFile> ffr = plan->getCurrentFlowFile();
+  REQUIRE(ffr != nullptr);
+  ffr->getResourceClaim()->decreaseFlowFileRecordOwnedCount();
+  auto repo = std::static_pointer_cast<TestRepository>(plan->getProvenanceRepo());
+  REQUIRE(2 == repo->getRepoMap().size());
 
-  records = reporter->getEvents();
+  for (auto entry : repo->getRepoMap()) {
+    provenance::ProvenanceEventRecord newRecord;
+    newRecord.DeSerialize(reinterpret_cast<uint8_t*>(const_cast<char*>(entry.second.data())), entry.second.length());
 
-  REQUIRE(true == LogTestController::getInstance().contains("key:absolute.path value:" + ss.str()));
-  REQUIRE(true == LogTestController::getInstance().contains("Size:8 Offset:0"));
-  REQUIRE(true == LogTestController::getInstance().contains("key:path value:" + std::string(dir)));
-  LogTestController::getInstance().reset();
+    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");
+    }
+  }
+  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<core::SerializableComponent>> recordsReport;
+  recordsReport.push_back(std::make_shared<provenance::ProvenanceEventRecord>());
+  processorReport->incrementActiveTasks();
+  processorReport->setScheduledState(core::ScheduledState::RUNNING);
+  std::string jsonStr;
+  std::size_t deserialized = 0;
+  repo->DeSerialize(recordsReport, deserialized);
+  std::function<void(core::ProcessContext*, core::ProcessSession*)> verifyReporter = [&](core::ProcessContext *context, core::ProcessSession *session) {
+    taskReport->getJsonReport(context, session, 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);
+  };
+
+  testController.runSession(plan, false, verifyReporter);
 }
 
 int fileSize(const char *add) {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/test/unit/ProvenanceTestHelper.h
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/ProvenanceTestHelper.h b/libminifi/test/unit/ProvenanceTestHelper.h
index 17e6078..1b39700 100644
--- a/libminifi/test/unit/ProvenanceTestHelper.h
+++ b/libminifi/test/unit/ProvenanceTestHelper.h
@@ -18,11 +18,22 @@
 #ifndef LIBMINIFI_TEST_UNIT_PROVENANCETESTHELPER_H_
 #define LIBMINIFI_TEST_UNIT_PROVENANCETESTHELPER_H_
 
-#include "provenance/Provenance.h"
-#include "FlowController.h"
-#include "core/Repository.h"
-#include "core/repository/FlowFileRepository.h"
-#include "core/Core.h"
+#include <atomic>
+#include <cstdint>
+#include <iostream>
+#include <map>
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+#include "core/repository/VolatileContentRepository.h"
+#include "../../include/core/Processor.h"
+#include "../../include/core/repository/FlowFileRepository.h"
+#include "../../include/Connection.h"
+#include "../../include/FlowController.h"
+#include "../../include/properties/Configure.h"
+#include "../../include/provenance/Provenance.h"
+
 /**
  * Test repository
  */
@@ -41,17 +52,22 @@ class TestRepository : public core::Repository {
 
   }
 
-  bool Put(std::string key, uint8_t *buf, int bufLen) {
+  bool Put(std::string key, const uint8_t *buf, size_t bufLen) {
     repositoryResults.insert(std::pair<std::string, std::string>(key, std::string((const char*) buf, bufLen)));
     return true;
   }
+
+  virtual bool Serialize(const std::string &key, const uint8_t *buffer, const size_t bufferSize) {
+    return Put(key, buffer, bufferSize);
+  }
+
   // Delete
   bool Delete(std::string key) {
     repositoryResults.erase(key);
     return true;
   }
   // Get
-  bool Get(std::string key, std::string &value) {
+  bool Get(const std::string &key, std::string &value) {
     auto result = repositoryResults.find(key);
     if (result != repositoryResults.end()) {
       value = result->second;
@@ -61,6 +77,39 @@ class TestRepository : public core::Repository {
     }
   }
 
+  virtual bool Serialize(std::vector<std::shared_ptr<core::SerializableComponent>> &store, size_t max_size) {
+    return false;
+  }
+
+  virtual bool DeSerialize(std::vector<std::shared_ptr<core::SerializableComponent>> &store, size_t &max_size) {
+    max_size = 0;
+    for (auto entry : repositoryResults) {
+      std::shared_ptr<core::SerializableComponent> eventRead = store.at(max_size);
+
+      if (eventRead->DeSerialize((uint8_t*) entry.second.data(), entry.second.length())) {
+      }
+      if (+max_size >= store.size()) {
+        break;
+      }
+    }
+    return true;
+  }
+
+  virtual bool Serialize(const std::shared_ptr<core::SerializableComponent> &store) {
+    return false;
+  }
+
+  virtual bool DeSerialize(const std::shared_ptr<core::SerializableComponent> &store) {
+    std::string value;
+    Get(store->getUUIDStr(), value);
+    store->DeSerialize(reinterpret_cast<uint8_t*>(const_cast<char*>(value.c_str())), value.size());
+    return true;
+  }
+
+  virtual bool DeSerialize(const uint8_t *buffer, const size_t bufferSize) {
+    return false;
+  }
+
   const std::map<std::string, std::string> &getRepoMap() const {
     return repositoryResults;
   }
@@ -134,6 +183,9 @@ class TestFlowRepository : public core::repository::FlowFileRepository {
       }
     }
   }
+  
+  void loadComponent(const std::shared_ptr<core::ContentRepository> &content_repo) {
+  }
 
   void run() {
     // do nothing
@@ -145,8 +197,8 @@ class TestFlowRepository : public core::repository::FlowFileRepository {
 class TestFlowController : public minifi::FlowController {
 
  public:
-  TestFlowController(std::shared_ptr<core::Repository> repo, std::shared_ptr<core::Repository> flow_file_repo)
-      : minifi::FlowController(repo, flow_file_repo, std::make_shared<minifi::Configure>(), nullptr, "", true) {
+  TestFlowController(std::shared_ptr<core::Repository> repo, std::shared_ptr<core::Repository> flow_file_repo, std::shared_ptr<core::ContentRepository> content_repo)
+      : minifi::FlowController(repo, flow_file_repo,std::make_shared<minifi::Configure>(), nullptr, std::make_shared<core::repository::VolatileContentRepository>(), "", true) {
   }
   ~TestFlowController() {
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/test/unit/ProvenanceTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/ProvenanceTests.cpp b/libminifi/test/unit/ProvenanceTests.cpp
index 6a58597..97cb646 100644
--- a/libminifi/test/unit/ProvenanceTests.cpp
+++ b/libminifi/test/unit/ProvenanceTests.cpp
@@ -26,8 +26,8 @@
 #include "provenance/Provenance.h"
 #include "FlowFileRecord.h"
 #include "core/Core.h"
-#include "core/repository/FlowFileRepository.h"
-#include "core/repository/VolatileRepository.h"
+#include "core/repository/AtomicRepoEntries.h"
+#include "core/repository/VolatileProvenanceRepository.h"
 
 TEST_CASE("Test Provenance record create", "[Testprovenance::ProvenanceEventRecord]") {
   provenance::ProvenanceEventRecord record1(provenance::ProvenanceEventRecord::ProvenanceEventType::CREATE, "blah", "blahblah");
@@ -49,7 +49,8 @@ TEST_CASE("Test Provenance record serialization", "[Testprovenance::ProvenanceEv
 
   record1.Serialize(testRepository);
   provenance::ProvenanceEventRecord record2;
-  REQUIRE(record2.DeSerialize(testRepository, eventId) == true);
+  record2.setEventId(eventId);
+  REQUIRE(record2.DeSerialize(testRepository) == true);
   REQUIRE(record2.getEventId() == record1.getEventId());
   REQUIRE(record2.getComponentId() == record1.getComponentId());
   REQUIRE(record2.getComponentType() == record1.getComponentType());
@@ -60,12 +61,13 @@ TEST_CASE("Test Provenance record serialization", "[Testprovenance::ProvenanceEv
 
 TEST_CASE("Test Flowfile record added to provenance", "[TestFlowAndProv1]") {
   provenance::ProvenanceEventRecord record1(provenance::ProvenanceEventRecord::ProvenanceEventType::CLONE, "componentid", "componenttype");
+  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
   std::string eventId = record1.getEventId();
   std::map<std::string, std::string> attributes;
   attributes.insert(std::pair<std::string, std::string>("potato", "potatoe"));
   attributes.insert(std::pair<std::string, std::string>("tomato", "tomatoe"));
   std::shared_ptr<core::repository::FlowFileRepository> frepo = std::make_shared<core::repository::FlowFileRepository>("ff", "./content_repository", 0, 0, 0);
-  std::shared_ptr<minifi::FlowFileRecord> ffr1 = std::make_shared<minifi::FlowFileRecord>(frepo, attributes);
+  std::shared_ptr<minifi::FlowFileRecord> ffr1 = std::make_shared<minifi::FlowFileRecord>(frepo, content_repo, attributes);
 
   record1.addChildFlowFile(ffr1);
 
@@ -75,7 +77,8 @@ TEST_CASE("Test Flowfile record added to provenance", "[TestFlowAndProv1]") {
 
   record1.Serialize(testRepository);
   provenance::ProvenanceEventRecord record2;
-  REQUIRE(record2.DeSerialize(testRepository, eventId) == true);
+  record2.setEventId(eventId);
+  REQUIRE(record2.DeSerialize(testRepository) == true);
   REQUIRE(record1.getChildrenUuids().size() == 1);
   REQUIRE(record2.getChildrenUuids().size() == 1);
   std::string childId = record2.getChildrenUuids().at(0);
@@ -94,13 +97,14 @@ TEST_CASE("Test Provenance record serialization Volatile", "[Testprovenance::Pro
 
   uint64_t sample = 65555;
 
-  std::shared_ptr<core::Repository> testRepository = std::make_shared<core::repository::VolatileRepository>();
+  std::shared_ptr<core::Repository> testRepository = std::make_shared<core::repository::VolatileProvenanceRepository>();
   testRepository->initialize(0);
   record1.setEventDuration(sample);
 
   record1.Serialize(testRepository);
   provenance::ProvenanceEventRecord record2;
-  REQUIRE(record2.DeSerialize(testRepository, eventId) == true);
+  record2.setEventId(eventId);
+  REQUIRE(record2.DeSerialize(testRepository) == true);
   REQUIRE(record2.getEventId() == record1.getEventId());
   REQUIRE(record2.getComponentId() == record1.getComponentId());
   REQUIRE(record2.getComponentType() == record1.getComponentType());
@@ -111,24 +115,26 @@ TEST_CASE("Test Provenance record serialization Volatile", "[Testprovenance::Pro
 
 TEST_CASE("Test Flowfile record added to provenance using Volatile Repo", "[TestFlowAndProv1]") {
   provenance::ProvenanceEventRecord record1(provenance::ProvenanceEventRecord::ProvenanceEventType::CLONE, "componentid", "componenttype");
+  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
   std::string eventId = record1.getEventId();
   std::map<std::string, std::string> attributes;
   attributes.insert(std::pair<std::string, std::string>("potato", "potatoe"));
   attributes.insert(std::pair<std::string, std::string>("tomato", "tomatoe"));
-  std::shared_ptr<core::Repository> frepo = std::make_shared<core::repository::VolatileRepository>();
+  std::shared_ptr<core::Repository> frepo = std::make_shared<core::repository::VolatileProvenanceRepository>();
   frepo->initialize(0);
-  std::shared_ptr<minifi::FlowFileRecord> ffr1 = std::make_shared<minifi::FlowFileRecord>(frepo, attributes);
+  std::shared_ptr<minifi::FlowFileRecord> ffr1 = std::make_shared<minifi::FlowFileRecord>(frepo, content_repo, attributes);
 
   record1.addChildFlowFile(ffr1);
 
   uint64_t sample = 65555;
-  std::shared_ptr<core::Repository> testRepository = std::make_shared<core::repository::VolatileRepository>();
+  std::shared_ptr<core::Repository> testRepository = std::make_shared<core::repository::VolatileProvenanceRepository>();
   testRepository->initialize(0);
   record1.setEventDuration(sample);
 
   record1.Serialize(testRepository);
   provenance::ProvenanceEventRecord record2;
-  REQUIRE(record2.DeSerialize(testRepository, eventId) == true);
+  record2.setEventId(eventId);
+  REQUIRE(record2.DeSerialize(testRepository) == true);
   REQUIRE(record1.getChildrenUuids().size() == 1);
   REQUIRE(record2.getChildrenUuids().size() == 1);
   std::string childId = record2.getChildrenUuids().at(0);
@@ -151,7 +157,8 @@ TEST_CASE("Test Provenance record serialization NoOp", "[Testprovenance::Provena
   testRepository->initialize(0);
   record1.setEventDuration(sample);
 
-  record1.Serialize(testRepository);
+  REQUIRE(record1.Serialize(testRepository) == true);
   provenance::ProvenanceEventRecord record2;
-  REQUIRE(record2.DeSerialize(testRepository, eventId) == false);
+  record2.setEventId(eventId);
+  REQUIRE(record2.DeSerialize(testRepository) == false);
 }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/test/unit/RepoTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/RepoTests.cpp b/libminifi/test/unit/RepoTests.cpp
index 4424a93..3b18310 100644
--- a/libminifi/test/unit/RepoTests.cpp
+++ b/libminifi/test/unit/RepoTests.cpp
@@ -23,7 +23,7 @@
 #include "provenance/Provenance.h"
 #include "FlowFileRecord.h"
 #include "core/Core.h"
-#include "core/repository/FlowFileRepository.h"
+#include "../../include/core/repository/AtomicRepoEntries.h"
 #include "properties/Configure.h"
 
 TEST_CASE("Test Repo Empty Value Attribute", "[TestFFR1]") {
@@ -34,7 +34,8 @@ TEST_CASE("Test Repo Empty Value Attribute", "[TestFFR1]") {
 
   repository->initialize(std::make_shared<minifi::Configure>());
 
-  minifi::FlowFileRecord record(repository);
+  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
+  minifi::FlowFileRecord record(repository, content_repo);
 
   record.addAttribute("keyA", "");
 
@@ -50,8 +51,8 @@ TEST_CASE("Test Repo Empty Key Attribute ", "[TestFFR2]") {
   std::shared_ptr<core::repository::FlowFileRepository> repository = std::make_shared<core::repository::FlowFileRepository>("ff", dir, 0, 0, 1);
 
   repository->initialize(std::make_shared<minifi::Configure>());
-
-  minifi::FlowFileRecord record(repository);
+  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
+  minifi::FlowFileRecord record(repository, content_repo);
 
   record.addAttribute("keyA", "hasdgasdgjsdgasgdsgsadaskgasd");
 
@@ -70,9 +71,10 @@ TEST_CASE("Test Repo Key Attribute Verify ", "[TestFFR3]") {
 
   repository->initialize(std::make_shared<org::apache::nifi::minifi::Configure>());
 
-  minifi::FlowFileRecord record(repository);
+  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
+  minifi::FlowFileRecord record(repository, content_repo);
 
-  minifi::FlowFileRecord record2(repository);
+  minifi::FlowFileRecord record2(repository, content_repo);
 
   std::string uuid = record.getUUIDStr();
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/test/unit/TailFileTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/TailFileTests.cpp b/libminifi/test/unit/TailFileTests.cpp
index e800b4c..eb33f8c 100644
--- a/libminifi/test/unit/TailFileTests.cpp
+++ b/libminifi/test/unit/TailFileTests.cpp
@@ -42,130 +42,137 @@ static const char *TMP_FILE = "/tmp/minifi-tmpfile.txt";
 static const char *STATE_FILE = "/tmp/minifi-state-file.txt";
 
 TEST_CASE("TailFileWithDelimiter", "[tailfiletest1]") {
-    try {
-        // Create and write to the test file
-        std::ofstream tmpfile;
-        tmpfile.open(TMP_FILE);
-        tmpfile << NEWLINE_FILE;
-        tmpfile.close();
+  try {
+    // Create and write to the test file
+    std::ofstream tmpfile;
+    tmpfile.open(TMP_FILE);
+    tmpfile << NEWLINE_FILE;
+    tmpfile.close();
 
-        TestController testController;
-        LogTestController::getInstance().setInfo<org::apache::nifi::minifi::processors::TailFile>();
+    TestController testController;
+    LogTestController::getInstance().setDebug<org::apache::nifi::minifi::processors::TailFile>();
+    LogTestController::getInstance().setDebug<core::ProcessSession>();
+    LogTestController::getInstance().setDebug<core::repository::VolatileContentRepository>();
 
-        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::TailFile>("tailfile");
-        std::shared_ptr<core::Processor> logAttributeProcessor = std::make_shared<org::apache::nifi::minifi::processors::LogAttribute>("logattribute");
+    std::shared_ptr<core::Processor> processor = std::make_shared<org::apache::nifi::minifi::processors::TailFile>("tailfile");
+    std::shared_ptr<core::Processor> logAttributeProcessor = std::make_shared<org::apache::nifi::minifi::processors::LogAttribute>("logattribute");
 
-        uuid_t processoruuid;
-        REQUIRE(true == processor->getUUID(processoruuid));
-        uuid_t logAttributeuuid;
-        REQUIRE(true == logAttributeProcessor->getUUID(logAttributeuuid));
+    uuid_t processoruuid;
+    REQUIRE(true == processor->getUUID(processoruuid));
+    uuid_t logAttributeuuid;
+    REQUIRE(true == logAttributeProcessor->getUUID(logAttributeuuid));
 
-        std::shared_ptr<minifi::Connection> connection = std::make_shared<minifi::Connection>(repo, "logattributeconnection");
-        connection->setRelationship(core::Relationship("success", "TailFile successful output"));
+    std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
+    content_repo->initialize(std::make_shared<org::apache::nifi::minifi::Configure>());
+    std::shared_ptr<minifi::Connection> connection = std::make_shared<minifi::Connection>(repo, content_repo, "logattributeconnection");
+    connection->setRelationship(core::Relationship("success", "TailFile successful output"));
 
-        // link the connections so that we can test results at the end for this
-        connection->setDestination(connection);
+    // link the connections so that we can test results at the end for this
+    connection->setDestination(connection);
 
-        connection->setSourceUUID(processoruuid);
+    connection->setSourceUUID(processoruuid);
 
-        processor->addConnection(connection);
+    processor->addConnection(connection);
 
-        core::ProcessorNode node(processor);
+    core::ProcessorNode node(processor);
 
-        std::shared_ptr<core::controller::ControllerServiceProvider> controller_services_provider = nullptr;
-        core::ProcessContext context(node, controller_services_provider, repo);
-        context.setProperty(org::apache::nifi::minifi::processors::TailFile::Delimiter, "\n");
-        context.setProperty(org::apache::nifi::minifi::processors::TailFile::FileName, TMP_FILE);
-        context.setProperty(org::apache::nifi::minifi::processors::TailFile::StateFile, STATE_FILE);
+    std::shared_ptr<core::controller::ControllerServiceProvider> controller_services_provider = nullptr;
+    core::ProcessContext context(node, controller_services_provider, repo, repo, content_repo);
+    context.setProperty(org::apache::nifi::minifi::processors::TailFile::Delimiter, "\n");
+    context.setProperty(org::apache::nifi::minifi::processors::TailFile::FileName, TMP_FILE);
+    context.setProperty(org::apache::nifi::minifi::processors::TailFile::StateFile, STATE_FILE);
 
-        core::ProcessSession session(&context);
+    core::ProcessSession session(&context);
 
-        REQUIRE(processor->getName() == "tailfile");
+    REQUIRE(processor->getName() == "tailfile");
 
-        core::ProcessSessionFactory factory(&context);
+    core::ProcessSessionFactory factory(&context);
 
-        std::shared_ptr<core::FlowFile> record;
-        processor->setScheduledState(core::ScheduledState::RUNNING);
-        processor->onSchedule(&context, &factory);
-        processor->onTrigger(&context, &session);
+    std::shared_ptr<core::FlowFile> record;
+    processor->setScheduledState(core::ScheduledState::RUNNING);
+    processor->onSchedule(&context, &factory);
+    processor->onTrigger(&context, &session);
 
-        provenance::ProvenanceReporter *reporter = session.getProvenanceReporter();
-        std::set<provenance::ProvenanceEventRecord*> provRecords = reporter->getEvents();
-        record = session.get();
-        REQUIRE(record == nullptr);
-        std::shared_ptr<core::FlowFile> ff = session.get();
-        REQUIRE(provRecords.size() == 4);   // 2 creates and 2 modifies for flowfiles
+    provenance::ProvenanceReporter *reporter = session.getProvenanceReporter();
+    std::set<provenance::ProvenanceEventRecord*> provRecords = reporter->getEvents();
+    record = session.get();
+    REQUIRE(record == nullptr);
+    std::shared_ptr<core::FlowFile> ff = session.get();
+    REQUIRE(provRecords.size() == 4);   // 2 creates and 2 modifies for flowfiles
 
-        LogTestController::getInstance().reset();
-    } catch (...) { }
+    LogTestController::getInstance().reset();
+  } catch (...) {
+  }
 
-    // Delete the test and state file.
-    std::remove(TMP_FILE);
-    std::remove(STATE_FILE);
+  // Delete the test and state file.
+  std::remove(TMP_FILE);
+  std::remove(STATE_FILE);
 }
 
-
 TEST_CASE("TailFileWithoutDelimiter", "[tailfiletest2]") {
-    try {
-        // Create and write to the test file
-        std::ofstream tmpfile;
-        tmpfile.open(TMP_FILE);
-        tmpfile << NEWLINE_FILE;
-        tmpfile.close();
+  try {
+    // Create and write to the test file
+    std::ofstream tmpfile;
+    tmpfile.open(TMP_FILE);
+    tmpfile << NEWLINE_FILE;
+    tmpfile.close();
 
-        TestController testController;
-        LogTestController::getInstance().setInfo<org::apache::nifi::minifi::processors::TailFile>();
+    TestController testController;
+    LogTestController::getInstance().setInfo<org::apache::nifi::minifi::processors::TailFile>();
 
-        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::TailFile>("tailfile");
-        std::shared_ptr<core::Processor> logAttributeProcessor = std::make_shared<org::apache::nifi::minifi::processors::LogAttribute>("logattribute");
+    std::shared_ptr<core::Processor> processor = std::make_shared<org::apache::nifi::minifi::processors::TailFile>("tailfile");
+    std::shared_ptr<core::Processor> logAttributeProcessor = std::make_shared<org::apache::nifi::minifi::processors::LogAttribute>("logattribute");
 
-        uuid_t processoruuid;
-        REQUIRE(true == processor->getUUID(processoruuid));
-        uuid_t logAttributeuuid;
-        REQUIRE(true == logAttributeProcessor->getUUID(logAttributeuuid));
+    uuid_t processoruuid;
+    REQUIRE(true == processor->getUUID(processoruuid));
+    uuid_t logAttributeuuid;
+    REQUIRE(true == logAttributeProcessor->getUUID(logAttributeuuid));
 
-        std::shared_ptr<minifi::Connection> connection = std::make_shared<minifi::Connection>(repo, "logattributeconnection");
-        connection->setRelationship(core::Relationship("success", "TailFile successful output"));
+    std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
+    content_repo->initialize(std::make_shared<org::apache::nifi::minifi::Configure>());
+    std::shared_ptr<minifi::Connection> connection = std::make_shared<minifi::Connection>(repo, content_repo, "logattributeconnection");
+    connection->setRelationship(core::Relationship("success", "TailFile successful output"));
 
-        // link the connections so that we can test results at the end for this
-        connection->setDestination(connection);
-        connection->setSourceUUID(processoruuid);
+    // link the connections so that we can test results at the end for this
+    connection->setDestination(connection);
+    connection->setSourceUUID(processoruuid);
 
-        processor->addConnection(connection);
+    processor->addConnection(connection);
 
-        core::ProcessorNode node(processor);
+    core::ProcessorNode node(processor);
 
-        std::shared_ptr<core::controller::ControllerServiceProvider> controller_services_provider = nullptr;
-        core::ProcessContext context(node, controller_services_provider, repo);
-        context.setProperty(org::apache::nifi::minifi::processors::TailFile::FileName, TMP_FILE);
-        context.setProperty(org::apache::nifi::minifi::processors::TailFile::StateFile, STATE_FILE);
+    std::shared_ptr<core::controller::ControllerServiceProvider> controller_services_provider = nullptr;
+    core::ProcessContext context(node, controller_services_provider, repo, repo, content_repo);
+    context.setProperty(org::apache::nifi::minifi::processors::TailFile::FileName, TMP_FILE);
+    context.setProperty(org::apache::nifi::minifi::processors::TailFile::StateFile, STATE_FILE);
 
-        core::ProcessSession session(&context);
+    core::ProcessSession session(&context);
 
-        REQUIRE(processor->getName() == "tailfile");
+    REQUIRE(processor->getName() == "tailfile");
 
-        core::ProcessSessionFactory factory(&context);
+    core::ProcessSessionFactory factory(&context);
 
-        std::shared_ptr<core::FlowFile> record;
-        processor->setScheduledState(core::ScheduledState::RUNNING);
-        processor->onSchedule(&context, &factory);
-        processor->onTrigger(&context, &session);
+    std::shared_ptr<core::FlowFile> record;
+    processor->setScheduledState(core::ScheduledState::RUNNING);
+    processor->onSchedule(&context, &factory);
+    processor->onTrigger(&context, &session);
 
-        provenance::ProvenanceReporter *reporter = session.getProvenanceReporter();
-        std::set<provenance::ProvenanceEventRecord*> provRecords = reporter->getEvents();
-        record = session.get();
-        REQUIRE(record == nullptr);
-        std::shared_ptr<core::FlowFile> ff = session.get();
-        REQUIRE(provRecords.size() == 2);
+    provenance::ProvenanceReporter *reporter = session.getProvenanceReporter();
+    std::set<provenance::ProvenanceEventRecord*> provRecords = reporter->getEvents();
+    record = session.get();
+    REQUIRE(record == nullptr);
+    std::shared_ptr<core::FlowFile> ff = session.get();
+    REQUIRE(provRecords.size() == 2);
 
-        LogTestController::getInstance().reset();
-    } catch (...) { }
+    LogTestController::getInstance().reset();
+  } catch (...) {
+  }
 
-    // Delete the test and state file.
-    std::remove(TMP_FILE);
-    std::remove(STATE_FILE);
+  // Delete the test and state file.
+  std::remove(TMP_FILE);
+  std::remove(STATE_FILE);
 }


[5/6] nifi-minifi-cpp git commit: MINIFI-249: Update prov repo to better abstract deser.

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/core/repository/VolatileProvenanceRepository.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/repository/VolatileProvenanceRepository.h b/libminifi/include/core/repository/VolatileProvenanceRepository.h
new file mode 100644
index 0000000..7397751
--- /dev/null
+++ b/libminifi/include/core/repository/VolatileProvenanceRepository.h
@@ -0,0 +1,60 @@
+/**
+ *
+ * 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_REPOSITORY_VOLATILEPROVENANCEREPOSITORY_H_
+#define LIBMINIFI_INCLUDE_CORE_REPOSITORY_VOLATILEPROVENANCEREPOSITORY_H_
+
+#include "VolatileRepository.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+namespace repository {
+
+/**
+ * Volatile provenance repository.
+ */
+class VolatileProvenanceRepository : public VolatileRepository<std::string>
+{
+
+ public:
+  explicit VolatileProvenanceRepository(std::string repo_name = "", std::string dir = REPOSITORY_DIRECTORY, int64_t maxPartitionMillis = MAX_REPOSITORY_ENTRY_LIFE_TIME, int64_t maxPartitionBytes =
+  MAX_REPOSITORY_STORAGE_SIZE,
+                                        uint64_t purgePeriod = REPOSITORY_PURGE_PERIOD)
+      : VolatileRepository(repo_name.length() > 0 ? repo_name : core::getClassName<VolatileRepository>(), "", maxPartitionMillis, maxPartitionBytes, purgePeriod)
+
+  {
+    purge_required_ = false;
+  }
+
+  virtual void run() {
+    repo_full_ = false;
+  }
+ private:
+
+};
+
+} /* namespace repository */
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_REPOSITORY_VOLATILEPROVENANCEREPOSITORY_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/core/repository/VolatileRepository.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/repository/VolatileRepository.h b/libminifi/include/core/repository/VolatileRepository.h
index 870a1f5..958d91a 100644
--- a/libminifi/include/core/repository/VolatileRepository.h
+++ b/libminifi/include/core/repository/VolatileRepository.h
@@ -22,9 +22,11 @@
 #include <chrono>
 #include <vector>
 #include <map>
+#include "core/SerializableComponent.h"
 #include "core/Core.h"
 #include "Connection.h"
 #include "utils/StringUtils.h"
+#include "AtomicRepoEntries.h"
 
 namespace org {
 namespace apache {
@@ -33,290 +35,89 @@ namespace minifi {
 namespace core {
 namespace repository {
 
-static uint16_t accounting_size = sizeof(std::vector<uint8_t>) + sizeof(std::string) + sizeof(size_t);
-
-class RepoValue {
- public:
-
-  explicit RepoValue() {
-  }
-
-  explicit RepoValue(std::string key, uint8_t *ptr, size_t size)
-      : key_(key) {
-    buffer_.resize(size);
-    std::memcpy(buffer_.data(), ptr, size);
-    fast_size_ = key.size() + size;
-  }
-
-  explicit RepoValue(RepoValue &&other)
-noexcept      : key_(std::move(other.key_)),
-      buffer_(std::move(other.buffer_)),
-      fast_size_(other.fast_size_) {
-      }
-
-      ~RepoValue()
-      {
-      }
-
-      std::string &getKey() {
-        return key_;
-      }
-
-      /**
-       * Return the size of the memory within the key
-       * buffer, the size of timestamp, and the general
-       * system word size
-       */
-      uint64_t size() {
-        return fast_size_;
-      }
-
-      size_t bufferSize() {
-        return buffer_.size();
-      }
-
-      void emplace(std::string &str) {
-        str.insert(0, reinterpret_cast<const char*>(buffer_.data()), buffer_.size());
-      }
-
-      RepoValue &operator=(RepoValue &&other) noexcept {
-        key_ = std::move(other.key_);
-        buffer_ = std::move(other.buffer_);
-        other.buffer_.clear();
-        return *this;
-      }
-
-    private:
-      size_t fast_size_;
-      std::string key_;
-      std::vector<uint8_t> buffer_;
-    };
-
-    /**
-     * Purpose: Atomic Entry allows us to create a statically
-     * sized ring buffer, with the ability to create
-     **/
-class AtomicEntry {
-
- public:
-  AtomicEntry()
-      : write_pending_(false),
-        has_value_(false) {
-
-  }
-
-  bool setRepoValue(RepoValue &new_value, size_t &prev_size) {
-    // delete the underlying pointer
-    bool lock = false;
-    if (!write_pending_.compare_exchange_weak(lock, true) && !lock)
-      return false;
-    if (has_value_) {
-      prev_size = value_.size();
-    }
-    value_ = std::move(new_value);
-    has_value_ = true;
-    try_unlock();
-    return true;
-  }
-
-  bool getValue(RepoValue &value) {
-    try_lock();
-    if (!has_value_) {
-      try_unlock();
-      return false;
-    }
-    value = std::move(value_);
-    has_value_ = false;
-    try_unlock();
-    return true;
-  }
-
-  bool getValue(const std::string &key, RepoValue &value) {
-    try_lock();
-    if (!has_value_) {
-      try_unlock();
-      return false;
-    }
-    if (value_.getKey() != key) {
-      try_unlock();
-      return false;
-    }
-    value = std::move(value_);
-    has_value_ = false;
-    try_unlock();
-    return true;
-  }
-
- private:
-
-  inline void try_lock() {
-    bool lock = false;
-    while (!write_pending_.compare_exchange_weak(lock, true) && !lock) {
-      // attempt again
-    }
-  }
-
-  inline void try_unlock() {
-    bool lock = true;
-    while (!write_pending_.compare_exchange_weak(lock, false) && lock) {
-      // attempt again
-    }
-  }
-
-  std::atomic<bool> write_pending_;
-  std::atomic<bool> has_value_;
-  RepoValue value_;
-};
-
 /**
  * Flow File repository
  * Design: Extends Repository and implements the run function, using LevelDB as the primary substrate.
  */
-class VolatileRepository : public core::Repository, public std::enable_shared_from_this<VolatileRepository> {
+template<typename T>
+class VolatileRepository : public core::Repository, public std::enable_shared_from_this<VolatileRepository<T>> {
  public:
 
   static const char *volatile_repo_max_count;
+  static const char *volatile_repo_max_bytes;
   // Constructor
 
-  VolatileRepository(std::string repo_name = "", std::string dir = REPOSITORY_DIRECTORY, int64_t maxPartitionMillis = MAX_REPOSITORY_ENTRY_LIFE_TIME, int64_t maxPartitionBytes =
+  explicit VolatileRepository(std::string repo_name = "", std::string dir = REPOSITORY_DIRECTORY, int64_t maxPartitionMillis = MAX_REPOSITORY_ENTRY_LIFE_TIME, int64_t maxPartitionBytes =
   MAX_REPOSITORY_STORAGE_SIZE,
-                     uint64_t purgePeriod = REPOSITORY_PURGE_PERIOD)
+                              uint64_t purgePeriod = REPOSITORY_PURGE_PERIOD)
       : Repository(repo_name.length() > 0 ? repo_name : core::getClassName<VolatileRepository>(), "", maxPartitionMillis, maxPartitionBytes, purgePeriod),
         max_size_(maxPartitionBytes * 0.75),
         current_index_(0),
         max_count_(10000),
+        current_size_(0),
         logger_(logging::LoggerFactory<VolatileRepository>::getLogger())
 
   {
-
+    purge_required_ = false;
   }
 
   // Destructor
-  ~VolatileRepository() {
-    for (auto ent : value_vector_) {
-      delete ent;
-    }
-  }
+  virtual ~VolatileRepository();
 
   /**
    * Initialize thevolatile repsitory
    **/
-  virtual bool initialize(const std::shared_ptr<Configure> &configure) {
-    std::string value = "";
-
-    if (configure != nullptr) {
-      int64_t max_cnt = 0;
-      std::stringstream strstream;
-      strstream << Configure::nifi_volatile_repository_options << getName() << "." << volatile_repo_max_count;
-      if (configure->get(strstream.str(), value)) {
-        if (core::Property::StringToInt(value, max_cnt)) {
-          max_count_ = max_cnt;
-        }
 
-      }
-    }
+  virtual bool initialize(const std::shared_ptr<Configure> &configure);
 
-    logger_->log_debug("Resizing value_vector_ for %s count is %d", getName(), max_count_);
-    value_vector_.reserve(max_count_);
-    for (int i = 0; i < max_count_; i++) {
-      value_vector_.emplace_back(new AtomicEntry());
-    }
-    return true;
-  }
-
-  virtual void run();
+  virtual void run() = 0;
 
   /**
    * Places a new object into the volatile memory area
    * @param key key to add to the repository
    * @param buf buffer 
    **/
-  virtual bool Put(std::string key, uint8_t *buf, int bufLen) {
-    RepoValue new_value(key, buf, bufLen);
-
-    const size_t size = new_value.size();
-    bool updated = false;
-    size_t reclaimed_size = 0;
-    do {
-
-      int private_index = current_index_.fetch_add(1);
-      // round robin through the beginning
-      if (private_index >= max_count_) {
-        uint16_t new_index = 0;
-        if (current_index_.compare_exchange_weak(new_index, 0)) {
-          private_index = 0;
-        } else {
-          continue;
-        }
-      }
-      logger_->log_info("Set repo value at %d out of %d", private_index, max_count_);
-      updated = value_vector_.at(private_index)->setRepoValue(new_value, reclaimed_size);
+  virtual bool Put(T key, const uint8_t *buf, size_t bufLen);
 
-      if (reclaimed_size > 0) {
-        current_size_ -= reclaimed_size;
-      }
-
-    } while (!updated);
-    current_size_ += size;
-
-    logger_->log_info("VolatileRepository -- put %s %d %d", key, current_size_.load(), current_index_.load());
-    return true;
-  }
   /**
-   *c
    * Deletes the key
    * @return status of the delete operation
    */
-  virtual bool Delete(std::string key) {
-
-    logger_->log_info("VolatileRepository -- delete %s", key);
-    for (auto ent : value_vector_) {
-      // let the destructor do the cleanup
-      RepoValue value;
-      if (ent->getValue(key, value)) {
-        current_size_ -= value.size();
-        return true;
-      }
+  virtual bool Delete(T key);
 
-    }
-    return false;
-  }
   /**
    * Sets the value from the provided key. Once the item is retrieved
    * it may not be retrieved again.
    * @return status of the get operation.
    */
-  virtual bool Get(std::string key, std::string &value) {
-    for (auto ent : value_vector_) {
-      // let the destructor do the cleanup
-      RepoValue repo_value;
-
-      if (ent->getValue(key, repo_value)) {
-        current_size_ -= value.size();
-        repo_value.emplace(value);
-        logger_->log_info("VolatileRepository -- get %s %d", key, current_size_.load());
-        return true;
-      }
+  virtual bool Get(const T &key, std::string &value);
+  /**
+   * Deserializes objects into store
+   * @param store vector in which we will store newly created objects.
+   * @param max_size size of objects deserialized
+   */
+  virtual bool DeSerialize(std::vector<std::shared_ptr<core::SerializableComponent>> &store, size_t &max_size, std::function<std::shared_ptr<core::SerializableComponent>()> lambda);
 
-    }
-    return false;
-  }
+  /**
+   * Deserializes objects into a store that contains a fixed number of objects in which
+   * we will deserialize from this repo
+   * @param store precreated object vector
+   * @param max_size size of objects deserialized
+   */
+  virtual bool DeSerialize(std::vector<std::shared_ptr<core::SerializableComponent>> &store, size_t &max_size);
 
-  void setConnectionMap(std::map<std::string, std::shared_ptr<minifi::Connection>> &connectionMap) {
-    this->connectionMap = connectionMap;
-  }
-  void loadComponent();
-
-  void start() {
-    if (this->purge_period_ <= 0)
-      return;
-    if (running_)
-      return;
-    thread_ = std::thread(&VolatileRepository::run, shared_from_this());
-    thread_.detach();
-    running_ = true;
-    logger_->log_info("%s Repository Monitor Thread Start", name_.c_str());
-  }
+  /**
+   * Set the connection map
+   * @param connectionMap map of all connections through this repo.
+   */
+  void setConnectionMap(std::map<std::string, std::shared_ptr<minifi::Connection>> &connectionMap);
+
+  /**
+   * Function to load this component.
+   */
+  virtual void loadComponent(const std::shared_ptr<core::ContentRepository> &content_repo);
+
+  virtual void start();
 
  protected:
 
@@ -331,22 +132,240 @@ class VolatileRepository : public core::Repository, public std::enable_shared_fr
     else
       return false;
   }
-  /**
-   * Purges the volatile repository.
-   */
-  void purge();
 
- private:
   std::map<std::string, std::shared_ptr<minifi::Connection>> connectionMap;
-
-  std::atomic<uint32_t> current_size_;
+  // current size of the volatile repo.
+  std::atomic<size_t> current_size_;
+  // current index.
   std::atomic<uint16_t> current_index_;
-  std::vector<AtomicEntry*> value_vector_;
+  // value vector that exists for non blocking iteration over
+  // objects that store data for this repo instance.
+  std::vector<AtomicEntry<T>*> value_vector_;
+
+  // max count we are allowed to store.
   uint32_t max_count_;
-  uint32_t max_size_;
+  // maximum estimated size
+  size_t max_size_;
+
+  bool purge_required_;
+
+  std::mutex purge_mutex_;
+  // purge list
+  std::vector<T> purge_list_;
+
+ private:
   std::shared_ptr<logging::Logger> logger_;
+
+};
+
+template<typename T>
+const char *VolatileRepository<T>::volatile_repo_max_count = "max.count";
+template<typename T>
+const char *VolatileRepository<T>::volatile_repo_max_bytes = "max.bytes";
+
+template<typename T>
+void VolatileRepository<T>::loadComponent(const std::shared_ptr<core::ContentRepository> &content_repo) {
+}
+
+// Destructor
+template<typename T>
+VolatileRepository<T>::~VolatileRepository() {
+  for (auto ent : value_vector_) {
+    delete ent;
+  }
+}
+
+/**
+ * Initialize the volatile repsitory
+ **/
+template<typename T>
+bool VolatileRepository<T>::initialize(const std::shared_ptr<Configure> &configure) {
+  std::string value = "";
+
+  if (configure != nullptr) {
+    int64_t max_cnt = 0;
+    std::stringstream strstream;
+    strstream << Configure::nifi_volatile_repository_options << getName() << "." << volatile_repo_max_count;
+    if (configure->get(strstream.str(), value)) {
+      if (core::Property::StringToInt(value, max_cnt)) {
+        max_count_ = max_cnt;
+      }
+    }
+
+    strstream.str("");
+    strstream.clear();
+    int64_t max_bytes = 0;
+    strstream << Configure::nifi_volatile_repository_options << getName() << "." << volatile_repo_max_bytes;
+    if (configure->get(strstream.str(), value)) {
+      if (core::Property::StringToInt(value, max_bytes)) {
+        if (max_bytes <= 0) {
+          max_size_ = std::numeric_limits<uint32_t>::max();
+        } else {
+          max_size_ = max_bytes;
+        }
+      }
+    }
+  }
+
+  logger_->log_info("Resizing value_vector_ for %s count is %d", getName(), max_count_);
+  logger_->log_info("Using a maximum size of %u", max_size_);
+  value_vector_.reserve(max_count_);
+  for (int i = 0; i < max_count_; i++) {
+    value_vector_.emplace_back(new AtomicEntry<T>(&current_size_, &max_size_));
+  }
+  return true;
+}
+
+/**
+ * Places a new object into the volatile memory area
+ * @param key key to add to the repository
+ * @param buf buffer
+ **/
+template<typename T>
+bool VolatileRepository<T>::Put(T key, const uint8_t *buf, size_t bufLen) {
+  RepoValue<T> new_value(key, buf, bufLen);
+
+  const size_t size = new_value.size();
+  bool updated = false;
+  size_t reclaimed_size = 0;
+  RepoValue<T> old_value;
+  do {
+    int private_index = current_index_.fetch_add(1);
+    // round robin through the beginning
+    if (private_index >= max_count_) {
+      uint16_t new_index = 0;
+      if (current_index_.compare_exchange_weak(new_index, 0)) {
+        private_index = 0;
+      } else {
+        continue;
+      }
+    }
+    
+    updated = value_vector_.at(private_index)->setRepoValue(new_value, old_value, reclaimed_size);
+    logger_->log_debug("Set repo value at %d out of %d updated %d current_size %d, adding %d to  %d", private_index, max_count_,updated==true,reclaimed_size,size, current_size_.load());
+    if (updated && reclaimed_size > 0)
+    {
+      std::lock_guard<std::mutex> lock(mutex_);
+      purge_list_.push_back(old_value.getKey());
+    }
+    if (reclaimed_size > 0) {
+      /**
+       * this is okay since current_size_ is really an estimate.
+       * we don't need precise counts.
+       */
+      if (current_size_ < reclaimed_size) {
+        current_size_ = 0;
+      } else {
+        current_size_ -= reclaimed_size;
+      }
+    }
+  } while (!updated);
+  current_size_ += size;
+
+  logger_->log_debug("VolatileRepository -- put %d %d", current_size_.load(), current_index_.load());
+  return true;
+}
+/**
+ * Deletes the key
+ * @return status of the delete operation
+ */
+template<typename T>
+bool VolatileRepository<T>::Delete(T key) {
+  for (auto ent : value_vector_) {
+    // let the destructor do the cleanup
+    RepoValue<T> value;
+    if (ent->getValue(key, value)) {
+      current_size_ -= value.size();
+      return true;
+    }
+  }
+  return false;
+}
+/**
+ * Sets the value from the provided key. Once the item is retrieved
+ * it may not be retrieved again.
+ * @return status of the get operation.
+ */
+template<typename T>
+bool VolatileRepository<T>::Get(const T &key, std::string &value) {
+
+  for (auto ent : value_vector_) {
+    // let the destructor do the cleanup
+    RepoValue<T> repo_value;
+    if (ent->getValue(key, repo_value)) {
+      current_size_ -= value.size();
+      repo_value.emplace(value);
+      return true;
+    }
+  }
+  return false;
+}
+
+template<typename T>
+bool VolatileRepository<T>::DeSerialize(std::vector<std::shared_ptr<core::SerializableComponent>> &store, size_t &max_size, std::function<std::shared_ptr<core::SerializableComponent>()> lambda) {
+  size_t requested_batch = max_size;
+  max_size = 0;
+  for (auto ent : value_vector_) {
+    // let the destructor do the cleanup
+    RepoValue<T> repo_value;
+
+    if (ent->getValue(repo_value)) {
+      std::shared_ptr<core::SerializableComponent> newComponent = lambda();
+      // we've taken ownership of this repo value
+      newComponent->DeSerialize(repo_value.getBuffer(), repo_value.getBufferSize());
+
+      store.push_back(newComponent);
+
+      if (max_size++ >= requested_batch) {
+        break;
+      }
+    }
+  }
+  if (max_size > 0) {
+    return true;
+  } else {
+    return false;
+  }
+}
+
+template<typename T>
+bool VolatileRepository<T>::DeSerialize(std::vector<std::shared_ptr<core::SerializableComponent>> &store, size_t &max_size) {
+  logger_->log_debug("VolatileRepository -- DeSerialize %d", current_size_.load());
+  max_size = 0;
+  for (auto ent : value_vector_) {
+    // let the destructor do the cleanup
+    RepoValue<T> repo_value;
+
+    if (ent->getValue(repo_value)) {
+      // we've taken ownership of this repo value
+      store.at(max_size)->DeSerialize(repo_value.getBuffer(), repo_value.getBufferSize());
+      if (max_size++ >= store.size()) {
+        break;
+      }
+    }
+  }
+  if (max_size > 0) {
+    return true;
+  } else {
+    return false;
+  }
+}
+
+template<typename T>
+void VolatileRepository<T>::setConnectionMap(std::map<std::string, std::shared_ptr<minifi::Connection>> &connectionMap) {
+  this->connectionMap = connectionMap;
+}
+
+template<typename T>
+void VolatileRepository<T>::start() {
+  if (this->purge_period_ <= 0)
+    return;
+  if (running_)
+    return;
+  running_ = true;
+  thread_ = std::thread(&VolatileRepository<T>::run, std::enable_shared_from_this<VolatileRepository<T>>::shared_from_this());
+  logger_->log_info("%s Repository Monitor Thread Start", name_);
 }
-;
 
 } /* namespace repository */
 } /* namespace core */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/core/yaml/YamlConfiguration.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/yaml/YamlConfiguration.h b/libminifi/include/core/yaml/YamlConfiguration.h
index e03c794..17b060f 100644
--- a/libminifi/include/core/yaml/YamlConfiguration.h
+++ b/libminifi/include/core/yaml/YamlConfiguration.h
@@ -21,7 +21,7 @@
 #include "core/ProcessorConfig.h"
 #include "yaml-cpp/yaml.h"
 #include "processors/LoadProcessors.h"
-#include "../FlowConfiguration.h"
+#include "core/FlowConfiguration.h"
 #include "Site2SiteClientProtocol.h"
 #include <string>
 #include "io/validation.h"
@@ -46,13 +46,12 @@ namespace core {
 class YamlConfiguration : public FlowConfiguration {
 
  public:
-  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, configuration, path),
+  explicit YamlConfiguration(std::shared_ptr<core::Repository> repo, std::shared_ptr<core::Repository> flow_file_repo, std::shared_ptr<core::ContentRepository> content_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, content_repo, stream_factory, configuration, path),
         logger_(logging::LoggerFactory<YamlConfiguration>::getLogger()) {
     stream_factory_ = stream_factory;
-    if (IsNullOrEmpty(config_path_)) {
+    if (IsNullOrEmpty (config_path_)) {
       config_path_ = DEFAULT_FLOW_YAML_FILE_NAME;
     }
   }
@@ -93,20 +92,20 @@ class YamlConfiguration : public FlowConfiguration {
   }
 
   /**
-    * Returns a shared pointer to a ProcessGroup object containing the
-    * flow configuration. The yamlConfigPayload argument must be
-    * a payload for the raw YAML configuration.
-    *
-    * @param yamlConfigPayload an input payload for the raw YAML configuration
-    *                           to be parsed and loaded into the flow
-    *                           configuration tree
-    * @return                 the root ProcessGroup node of the flow
-    *                           configuration tree
-    */
-   std::unique_ptr<core::ProcessGroup> getRootFromPayload(std::string &yamlConfigPayload) {
-     YAML::Node rootYamlNode = YAML::Load(yamlConfigPayload);
-     return getRoot(&rootYamlNode);
-   }
+   * Returns a shared pointer to a ProcessGroup object containing the
+   * flow configuration. The yamlConfigPayload argument must be
+   * a payload for the raw YAML configuration.
+   *
+   * @param yamlConfigPayload an input payload for the raw YAML configuration
+   *                           to be parsed and loaded into the flow
+   *                           configuration tree
+   * @return                 the root ProcessGroup node of the flow
+   *                           configuration tree
+   */
+  std::unique_ptr<core::ProcessGroup> getRootFromPayload(std::string &yamlConfigPayload) {
+    YAML::Node rootYamlNode = YAML::Load(yamlConfigPayload);
+    return getRoot(&rootYamlNode);
+  }
 
  protected:
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/io/AtomicEntryStream.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/AtomicEntryStream.h b/libminifi/include/io/AtomicEntryStream.h
new file mode 100644
index 0000000..5f200f0
--- /dev/null
+++ b/libminifi/include/io/AtomicEntryStream.h
@@ -0,0 +1,205 @@
+/**
+ *
+ * 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_ATOMICENTRYSTREAM_H_
+#define LIBMINIFI_INCLUDE_IO_ATOMICENTRYSTREAM_H_
+
+#include <mutex>
+#include <cstring>
+#include "BaseStream.h"
+#include "core/repository/AtomicRepoEntries.h"
+#include "Exception.h"
+#include "core/logging/LoggerConfiguration.h"
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace io {
+
+template<typename T>
+class AtomicEntryStream : public BaseStream {
+ public:
+  AtomicEntryStream(const T key, core::repository::AtomicEntry<T> *entry)
+      : key_(key),
+        entry_(entry),
+        offset_(0),
+        length_(0),
+        logger_(logging::LoggerFactory<AtomicEntryStream()>::getLogger()) {
+    core::repository::RepoValue<T> *value;
+    if (entry_->getValue(key, &value)) {
+      length_ = value->getBufferSize();
+      entry_->decrementOwnership();
+      invalid_stream_ = false;
+    } else {
+      invalid_stream_ = true;
+    }
+  }
+  
+  virtual ~AtomicEntryStream();
+
+  virtual void closeStream() {
+
+  }
+
+  /**
+   * Skip to the specified offset.
+   * @param offset offset to which we will skip
+   */
+  void seek(uint64_t offset);
+
+  virtual const uint32_t getSize() const {
+    return length_;
+  }
+
+  // data stream extensions
+  /**
+   * Reads data and places it into buf
+   * @param buf buffer in which we extract data
+   * @param buflen
+   */
+  virtual int readData(std::vector<uint8_t> &buf, int buflen);
+  /**
+   * 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
+   *
+   */
+  virtual int writeData(std::vector<uint8_t> &buf, int buflen);
+
+  /**
+   * writes value to stream
+   * @param value value to write
+   * @param size size of value
+   */
+  virtual int writeData(uint8_t *value, int size);
+
+  /**
+   * Returns the underlying buffer
+   * @return vector's array
+   **/
+  const uint8_t *getBuffer() const {
+    throw std::runtime_error("Stream does not support this operation");
+  }
+
+ protected:
+  size_t length_;
+  size_t offset_;
+  T key_;
+  core::repository::AtomicEntry<T> *entry_;
+  std::atomic<bool> invalid_stream_;
+  std::recursive_mutex entry_lock_;
+
+  // Logger
+  std::shared_ptr<logging::Logger> logger_;
+
+};
+
+template<typename T>
+AtomicEntryStream<T>::~AtomicEntryStream(){
+  logger_->log_debug("Decrementing");
+    entry_->decrementOwnership();
+}
+
+template<typename T>
+void AtomicEntryStream<T>::seek(uint64_t offset) {
+  std::lock_guard<std::recursive_mutex> lock(entry_lock_);
+  offset_ = offset;
+}
+
+template<typename T>
+int AtomicEntryStream<T>::writeData(std::vector<uint8_t> &buf, int buflen) {
+  if (buf.capacity() < buflen || invalid_stream_)
+    return -1;
+  return writeData(reinterpret_cast<uint8_t *>(&buf[0]), buflen);
+}
+
+// data stream overrides
+template<typename T>
+int AtomicEntryStream<T>::writeData(uint8_t *value, int size) {
+  if (nullptr != value && !invalid_stream_) {
+    std::lock_guard<std::recursive_mutex> lock(entry_lock_);
+    if (entry_->insert(key_, value, size)) {
+      offset_ += size;
+      if (offset_ > length_)
+          {
+        length_ = offset_;
+      }
+      return size;
+    }
+    else {
+      logger_->log_debug("Cannot insert %d bytes due to insufficient space in atomic entry", size);
+    }
+
+  }
+  return -1;
+
+}
+
+template<typename T>
+int AtomicEntryStream<T>::readData(std::vector<uint8_t> &buf, int buflen) {
+  if (invalid_stream_){
+    return -1;
+  }
+  if (buf.capacity() < buflen) {
+    buf.resize(buflen);
+  }
+  int ret = readData(reinterpret_cast<uint8_t*>(&buf[0]), buflen);
+
+  if (ret < buflen) {
+    buf.resize(ret);
+  }
+  return ret;
+}
+
+template<typename T>
+int AtomicEntryStream<T>::readData(uint8_t *buf, int buflen) {
+  if (nullptr != buf && !invalid_stream_) {
+    std::lock_guard<std::recursive_mutex> lock(entry_lock_);
+    int len = buflen;
+    core::repository::RepoValue<T> *value;
+    if (entry_->getValue(key_, &value)) {
+      if (offset_ + len > value->getBufferSize()) {
+        len = value->getBufferSize() - offset_;
+        if (len <= 0) {
+	  entry_->decrementOwnership();
+          return 0;
+        }
+      }
+      std::memcpy(buf, reinterpret_cast<uint8_t*>(const_cast<uint8_t*>(value->getBuffer()) + offset_), len);
+      offset_ += len;
+    entry_->decrementOwnership();
+      return len;
+    }
+
+  }
+  return -1;
+}
+
+} /* namespace io */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_IO_ATOMICENTRYSTREAM_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/io/BaseStream.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/BaseStream.h b/libminifi/include/io/BaseStream.h
index cae8a43..cd982bb 100644
--- a/libminifi/include/io/BaseStream.h
+++ b/libminifi/include/io/BaseStream.h
@@ -30,6 +30,11 @@ namespace nifi {
 namespace minifi {
 namespace io {
 
+/**
+ * Base Stream. Not intended to be thread safe as it is not intended to be shared
+ *
+ * Extensions may be thread safe and thus shareable, but that is up to the implementation.
+ */
 class BaseStream : public DataStream, public Serializable {
 
  public:
@@ -55,6 +60,14 @@ class BaseStream : public DataStream, public Serializable {
 
   int writeData(uint8_t *value, int size);
 
+  virtual void seek(uint32_t offset) {
+    if (composable_stream_ != this) {
+      composable_stream_->seek(offset);
+    } else {
+      DataStream::seek(offset);
+    }
+  }
+
   /**
    * write 2 bytes to stream
    * @param base_value non encoded value

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/io/ClientSocket.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/ClientSocket.h b/libminifi/include/io/ClientSocket.h
index c7db7f1..cd8a4fc 100644
--- a/libminifi/include/io/ClientSocket.h
+++ b/libminifi/include/io/ClientSocket.h
@@ -84,7 +84,7 @@ class Socket : public BaseStream {
    * Static function to return the current machine's host name
    */
   static std::string getMyHostName() {
-    static char *HOSTNAME = init_hostname();
+    static std::string HOSTNAME = init_hostname();
     return HOSTNAME;
   }
 
@@ -239,12 +239,12 @@ class Socket : public BaseStream {
 
  private:
   std::shared_ptr<logging::Logger> logger_;
-  static char* init_hostname() {
+  static std::string init_hostname() {
     char hostname[1024];
     gethostname(hostname, 1024);
     Socket mySock(nullptr, hostname, 0);
     mySock.initialize();
-    return const_cast<char*>(mySock.getHostname().c_str());
+    return mySock.getHostname();
   }
 };
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/io/DataStream.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/DataStream.h b/libminifi/include/io/DataStream.h
index 460930d..2ebc9a4 100644
--- a/libminifi/include/io/DataStream.h
+++ b/libminifi/include/io/DataStream.h
@@ -30,6 +30,8 @@ namespace io {
 /**
  * DataStream defines the mechanism through which
  * binary data will be written to a sink
+ *
+ * This object is not intended to be thread safe.
  */
 class DataStream {
  public:
@@ -58,6 +60,10 @@ class DataStream {
     return 0;
   }
 
+  virtual void seek(uint32_t offset) {
+    readBuffer += offset;
+  }
+
   virtual void closeStream() {
 
   }
@@ -111,7 +117,7 @@ class DataStream {
    * Retrieve size of data stream
    * @return size of data stream
    **/
-  const uint32_t getSize() const {
+  virtual const uint32_t getSize() const {
     return buffer.size();
   }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/io/FileStream.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/FileStream.h b/libminifi/include/io/FileStream.h
new file mode 100644
index 0000000..23a1f0b
--- /dev/null
+++ b/libminifi/include/io/FileStream.h
@@ -0,0 +1,136 @@
+/**
+ *
+ * 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_TLS_FILESTREAM_H_
+#define LIBMINIFI_INCLUDE_IO_TLS_FILESTREAM_H_
+
+#include <iostream>
+#include <cstdint>
+#include <string>
+#include "EndianCheck.h"
+#include "BaseStream.h"
+#include "Serializable.h"
+#include "core/logging/LoggerConfiguration.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace io {
+
+/**
+ * Purpose: File Stream Base stream extension. This is intended to be a thread safe access to
+ * read/write to the local file system.
+ *
+ * Design: Simply extends BaseStream and overrides readData/writeData to allow a sink to the
+ * fstream object.
+ */
+class FileStream : public io::BaseStream {
+ public:
+  /**
+   * File Stream constructor that accepts an fstream shared pointer.
+   * It must already be initialized for read and write.
+   */
+  explicit FileStream(const std::string &path, uint32_t offset,  bool write_enable = false);
+
+  /**
+   * File Stream constructor that accepts an fstream shared pointer.
+   * It must already be initialized for read and write.
+   */
+  explicit FileStream(const std::string &path);
+
+  virtual ~FileStream() {
+    closeStream();
+  }
+
+  virtual void closeStream();
+  /**
+   * Skip to the specified offset.
+   * @param offset offset to which we will skip
+   */
+  void seek(uint64_t offset);
+
+  const uint32_t getSize() const {
+    return length_;
+  }
+
+  // data stream extensions
+  /**
+   * Reads data and places it into buf
+   * @param buf buffer in which we extract data
+   * @param buflen
+   */
+  virtual int readData(std::vector<uint8_t> &buf, int buflen);
+  /**
+   * 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
+   *
+   */
+  virtual int writeData(std::vector<uint8_t> &buf, int buflen);
+
+  /**
+   * writes value to stream
+   * @param value value to write
+   * @param size size of value
+   */
+  virtual int writeData(uint8_t *value, int size);
+
+  /**
+   * Returns the underlying buffer
+   * @return vector's array
+   **/
+  const uint8_t *getBuffer() const {
+    throw std::runtime_error("Stream does not support this operation");
+  }
+
+ protected:
+
+  /**
+   * Creates a vector and returns the vector using the provided
+   * type name.
+   * @param t incoming object
+   * @returns vector.
+   */
+  template<typename T>
+  std::vector<uint8_t> readBuffer(const T&);
+  std::recursive_mutex file_lock_;
+  std::unique_ptr<std::fstream> file_stream_;
+  size_t offset_;
+  std::string path_;
+  size_t length_;
+
+ private:
+
+  std::shared_ptr<logging::Logger> logger_;
+
+};
+
+} /* namespace io */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_IO_TLS_FILESTREAM_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/processors/ExecuteProcess.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/ExecuteProcess.h b/libminifi/include/processors/ExecuteProcess.h
index 28dcf76..8cc7a25 100644
--- a/libminifi/include/processors/ExecuteProcess.h
+++ b/libminifi/include/processors/ExecuteProcess.h
@@ -31,6 +31,7 @@
 #include <iostream>
 #include <sys/types.h>
 #include <signal.h>
+#include "io/BaseStream.h"
 #include "FlowFileRecord.h"
 #include "core/Processor.h"
 #include "core/ProcessSession.h"
@@ -85,9 +86,12 @@ class ExecuteProcess : public core::Processor {
     }
     char *_data;
     uint64_t _dataSize;
-    void process(std::ofstream *stream) {
+    //void process(std::ofstream *stream) {
+    int64_t process(std::shared_ptr<io::BaseStream> stream) {
+      int64_t ret = 0;
       if (_data && _dataSize > 0)
-        stream->write(_data, _dataSize);
+        ret = stream->write(reinterpret_cast<uint8_t*>(_data), _dataSize);
+      return ret;
     }
   };
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/processors/GenerateFlowFile.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/GenerateFlowFile.h b/libminifi/include/processors/GenerateFlowFile.h
index abb5740..7551e88 100644
--- a/libminifi/include/processors/GenerateFlowFile.h
+++ b/libminifi/include/processors/GenerateFlowFile.h
@@ -68,9 +68,11 @@ class GenerateFlowFile : public core::Processor {
     }
     char *_data;
     uint64_t _dataSize;
-    void process(std::ofstream *stream) {
+    int64_t process(std::shared_ptr<io::BaseStream> stream) {
+      int64_t ret = 0;
       if (_data && _dataSize > 0)
-        stream->write(_data, _dataSize);
+        ret = stream->write(reinterpret_cast<uint8_t*>(_data), _dataSize);
+      return ret;
     }
   };
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/processors/InvokeHTTP.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/InvokeHTTP.h b/libminifi/include/processors/InvokeHTTP.h
index d55a5be..03a1611 100644
--- a/libminifi/include/processors/InvokeHTTP.h
+++ b/libminifi/include/processors/InvokeHTTP.h
@@ -104,6 +104,12 @@ class InvokeHTTP : public core::Processor {
   void onTrigger(core::ProcessContext *context, core::ProcessSession *session);
   void initialize();
   void onSchedule(core::ProcessContext *context, core::ProcessSessionFactory *sessionFactory);
+  /**
+   * Provides a reference to the URL.
+   */
+  const std::string &getUrl() {
+    return url_;
+  }
 
  protected:
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/processors/ListenHTTP.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/ListenHTTP.h b/libminifi/include/processors/ListenHTTP.h
index c9e42bc..1b58dcd 100644
--- a/libminifi/include/processors/ListenHTTP.h
+++ b/libminifi/include/processors/ListenHTTP.h
@@ -92,7 +92,7 @@ class ListenHTTP : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(struct mg_connection *conn, const struct mg_request_info *reqInfo);
-    void process(std::ofstream *stream);
+    int64_t process(std::shared_ptr<io::BaseStream> stream);
 
    private:
     // Logger

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/processors/ListenSyslog.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/ListenSyslog.h b/libminifi/include/processors/ListenSyslog.h
index ed54b44..25acac9 100644
--- a/libminifi/include/processors/ListenSyslog.h
+++ b/libminifi/include/processors/ListenSyslog.h
@@ -114,14 +114,16 @@ class ListenSyslog : public core::Processor {
   class WriteCallback : public OutputStreamCallback {
    public:
     WriteCallback(char *data, uint64_t size)
-        : _data(data),
+        : _data(reinterpret_cast<uint8_t*>(data)),
           _dataSize(size) {
     }
-    char *_data;
+    uint8_t *_data;
     uint64_t _dataSize;
-    void process(std::ofstream *stream) {
+    int64_t process(std::shared_ptr<io::BaseStream> stream) {
+      int64_t ret = 0;
       if (_data && _dataSize > 0)
-        stream->write(_data, _dataSize);
+        ret = stream->write(_data, _dataSize);
+      return ret;
     }
   };
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/processors/LogAttribute.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/LogAttribute.h b/libminifi/include/processors/LogAttribute.h
index 88230f7..b9e333f 100644
--- a/libminifi/include/processors/LogAttribute.h
+++ b/libminifi/include/processors/LogAttribute.h
@@ -87,25 +87,27 @@ class LogAttribute : public core::Processor {
   // Nest Callback Class for read stream
   class ReadCallback : public InputStreamCallback {
    public:
-    ReadCallback(uint64_t size) {
-      _bufferSize = size;
-      _buffer = new char[_bufferSize];
+    ReadCallback(uint64_t size)
+        : read_size_(0) {
+      buffer_size_ = size;
+      buffer_ = new uint8_t[buffer_size_];
     }
     ~ReadCallback() {
-      if (_buffer)
-        delete[] _buffer;
+      if (buffer_)
+        delete[] buffer_;
     }
-    void process(std::ifstream *stream) {
-
-      stream->read(_buffer, _bufferSize);
+    int64_t process(std::shared_ptr<io::BaseStream> stream) {
+      int64_t ret = 0;
+      ret = stream->read(buffer_, buffer_size_);
       if (!stream)
-        _readSize = stream->gcount();
+        read_size_ = stream->getSize();
       else
-        _readSize = _bufferSize;
+        read_size_ = buffer_size_;
+      return ret;
     }
-    char *_buffer;
-    uint64_t _bufferSize;
-    uint64_t _readSize;
+    uint8_t *buffer_;
+    uint64_t buffer_size_;
+    uint64_t read_size_;
   };
 
  public:

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/processors/PutFile.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/PutFile.h b/libminifi/include/processors/PutFile.h
index f67e512..c7f2823 100644
--- a/libminifi/include/processors/PutFile.h
+++ b/libminifi/include/processors/PutFile.h
@@ -80,7 +80,8 @@ class PutFile : public core::Processor {
    public:
     ReadCallback(const std::string &tmpFile, const std::string &destFile);
     ~ReadCallback();
-    virtual void process(std::ifstream *stream);bool commit();
+    virtual int64_t process(std::shared_ptr<io::BaseStream> stream);
+    bool commit();
 
    private:
     std::shared_ptr<logging::Logger> logger_;

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/properties/Configure.h
----------------------------------------------------------------------
diff --git a/libminifi/include/properties/Configure.h b/libminifi/include/properties/Configure.h
index 13da55a..341b89c 100644
--- a/libminifi/include/properties/Configure.h
+++ b/libminifi/include/properties/Configure.h
@@ -40,6 +40,7 @@ class Configure : public Properties {
   static const char *nifi_server_name;
   static const char *nifi_configuration_class_name;
   static const char *nifi_flow_repository_class_name;
+  static const char *nifi_content_repository_class_name;
   static const char *nifi_volatile_repository_options;
   static const char *nifi_provenance_repository_class_name;
   static const char *nifi_server_port;

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/provenance/Provenance.h
----------------------------------------------------------------------
diff --git a/libminifi/include/provenance/Provenance.h b/libminifi/include/provenance/Provenance.h
index 1479514..b9415dc 100644
--- a/libminifi/include/provenance/Provenance.h
+++ b/libminifi/include/provenance/Provenance.h
@@ -29,7 +29,8 @@
 #include <string>
 #include <thread>
 #include <vector>
-
+#include "core/Core.h"
+#include "core/SerializableComponent.h"
 #include "core/Repository.h"
 #include "core/Property.h"
 #include "properties/Configure.h"
@@ -50,7 +51,7 @@ namespace provenance {
 #define PROVENANCE_EVENT_RECORD_SEG_SIZE 2048
 
 // Provenance Event Record
-class ProvenanceEventRecord : protected org::apache::nifi::minifi::io::Serializable {
+class ProvenanceEventRecord : public core::SerializableComponent {
  public:
   enum ProvenanceEventType {
 
@@ -163,7 +164,8 @@ class ProvenanceEventRecord : protected org::apache::nifi::minifi::io::Serializa
    */
   ProvenanceEventRecord(ProvenanceEventType event, std::string componentId, std::string componentType);
 
-  ProvenanceEventRecord() {
+  ProvenanceEventRecord()
+      : core::SerializableComponent(core::getClassName<ProvenanceEventRecord>()) {
     _eventTime = getTimeMillis();
   }
 
@@ -172,7 +174,11 @@ class ProvenanceEventRecord : protected org::apache::nifi::minifi::io::Serializa
   }
   // Get the Event ID
   std::string getEventId() {
-    return _eventIdStr;
+    return uuidStr_;
+  }
+
+  void setEventId(const std::string &id) {
+    setUUIDStr(id);
   }
   // Get Attributes
   std::map<std::string, std::string> getAttributes() {
@@ -220,7 +226,7 @@ class ProvenanceEventRecord : protected org::apache::nifi::minifi::io::Serializa
   }
   // Get FlowFileUuid
   std::string getFlowFileUuid() {
-    return uuid_;
+    return flow_uuid_;
   }
   // Get content full path
   std::string getContentFullPath() {
@@ -333,7 +339,7 @@ class ProvenanceEventRecord : protected org::apache::nifi::minifi::io::Serializa
     _entryDate = flow->getEntryDate();
     _lineageStartDate = flow->getlineageStartDate();
     _lineageIdentifiers = flow->getlineageIdentifiers();
-    uuid_ = flow->getUUIDStr();
+    flow_uuid_ = flow->getUUIDStr();
     _attributes = flow->getAttributes();
     _size = flow->getSize();
     _offset = flow->getOffset();
@@ -344,15 +350,43 @@ class ProvenanceEventRecord : protected org::apache::nifi::minifi::io::Serializa
     }
   }
   // Serialize and Persistent to the repository
-  bool Serialize(const std::shared_ptr<core::Repository> &repo);
+  bool Serialize(const std::shared_ptr<core::SerializableComponent> &repo);
   // DeSerialize
-  bool DeSerialize(const uint8_t *buffer, const int bufferSize);
+  bool DeSerialize(const uint8_t *buffer, const size_t bufferSize);
   // DeSerialize
   bool DeSerialize(org::apache::nifi::minifi::io::DataStream &stream) {
     return DeSerialize(stream.getBuffer(), stream.getSize());
   }
   // DeSerialize
-  bool DeSerialize(const std::shared_ptr<core::Repository> &repo, std::string key);
+  bool DeSerialize(const std::shared_ptr<core::SerializableComponent> &repo);
+
+  uint64_t getEventTime(const uint8_t *buffer, const size_t bufferSize) {
+
+    int size = bufferSize > 72 ? 72 : bufferSize;
+    org::apache::nifi::minifi::io::DataStream outStream(buffer, size);
+
+    std::string uuid;
+    int ret = readUTF(uuid, &outStream);
+
+    if (ret <= 0) {
+      return 0;
+    }
+
+    uint32_t eventType;
+    ret = read(eventType, &outStream);
+    if (ret != 4) {
+      return 0;
+    }
+
+    uint64_t event_time;
+
+    ret = read(event_time, &outStream);
+    if (ret != 8) {
+      return 0;
+    }
+
+    return event_time;
+  }
 
  protected:
 
@@ -373,15 +407,13 @@ class ProvenanceEventRecord : protected org::apache::nifi::minifi::io::Serializa
   // Size in bytes of the data corresponding to this flow file
   uint64_t _size;
   // flow uuid
-  std::string uuid_;
+  std::string flow_uuid_;
   // Offset to the content
   uint64_t _offset;
   // Full path to the content
   std::string _contentFullPath;
   // Attributes key/values pairs for the flow record
   std::map<std::string, std::string> _attributes;
-  // provenance ID
-  uuid_t _eventId;
   // UUID string for all parents
   std::set<std::string> _lineageIdentifiers;
   // transitUri
@@ -396,8 +428,6 @@ class ProvenanceEventRecord : protected org::apache::nifi::minifi::io::Serializa
   std::string _details;
   // sourceQueueIdentifier
   std::string _sourceQueueIdentifier;
-  // event ID Str
-  std::string _eventIdStr;
   // relationship
   std::string _relationship;
   // alternateIdentifierUri;
@@ -437,6 +467,7 @@ class ProvenanceReporter {
   // Add event
   void add(ProvenanceEventRecord *event) {
     _events.insert(event);
+    logger_->log_debug("Prove reporter now %d", _events.size());
   }
   // Remove event
   void remove(ProvenanceEventRecord *event) {
@@ -496,10 +527,9 @@ class ProvenanceReporter {
 
  private:
 
+  std::shared_ptr<logging::Logger> logger_;
   // Incoming connection Iterator
   std::set<ProvenanceEventRecord *> _events;
-  // Logger
-  std::shared_ptr<logging::Logger> logger_;
   // provenance repository.
   std::shared_ptr<core::Repository> repo_;
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/provenance/ProvenanceRepository.h
----------------------------------------------------------------------
diff --git a/libminifi/include/provenance/ProvenanceRepository.h b/libminifi/include/provenance/ProvenanceRepository.h
index dd2c5ec..ea78a3c 100644
--- a/libminifi/include/provenance/ProvenanceRepository.h
+++ b/libminifi/include/provenance/ProvenanceRepository.h
@@ -42,12 +42,11 @@ class ProvenanceRepository : public core::Repository, public std::enable_shared_
   /*!
    * Create a new provenance repository
    */
-  ProvenanceRepository(const std::string repo_name = "", std::string directory = PROVENANCE_DIRECTORY, int64_t maxPartitionMillis =
-  MAX_PROVENANCE_ENTRY_LIFE_TIME,
-                       int64_t maxPartitionBytes = MAX_PROVENANCE_STORAGE_SIZE, uint64_t purgePeriod = PROVENANCE_PURGE_PERIOD)
+  ProvenanceRepository(const std::string repo_name = "", std::string directory = PROVENANCE_DIRECTORY, int64_t maxPartitionMillis = MAX_PROVENANCE_ENTRY_LIFE_TIME, int64_t maxPartitionBytes =
+  MAX_PROVENANCE_STORAGE_SIZE,
+                       uint64_t purgePeriod = PROVENANCE_PURGE_PERIOD)
       : Repository(repo_name.length() > 0 ? repo_name : core::getClassName<ProvenanceRepository>(), directory, maxPartitionMillis, maxPartitionBytes, purgePeriod),
         logger_(logging::LoggerFactory<ProvenanceRepository>::getLogger()) {
-
     db_ = NULL;
   }
 
@@ -62,9 +61,8 @@ class ProvenanceRepository : public core::Repository, public std::enable_shared_
       return;
     if (running_)
       return;
-    thread_ = std::thread(&ProvenanceRepository::run, shared_from_this());
-    thread_.detach();
     running_ = true;
+    thread_ = std::thread(&ProvenanceRepository::run, shared_from_this());
     logger_->log_info("%s Repository Monitor Thread Start", name_.c_str());
   }
 
@@ -98,7 +96,7 @@ class ProvenanceRepository : public core::Repository, public std::enable_shared_
     return true;
   }
   // Put
-  virtual bool Put(std::string key, uint8_t *buf, int bufLen) {
+  virtual bool Put(std::string key, const uint8_t *buf, size_t bufLen) {
 
     if (repo_full_)
       return false;
@@ -122,7 +120,7 @@ class ProvenanceRepository : public core::Repository, public std::enable_shared_
       return false;
   }
   // Get
-  virtual bool Get(std::string key, std::string &value) {
+  virtual bool Get(const std::string &key, std::string &value) {
     leveldb::Status status;
     status = db_->Get(leveldb::ReadOptions(), key, &value);
     if (status.ok())
@@ -130,17 +128,53 @@ class ProvenanceRepository : public core::Repository, public std::enable_shared_
     else
       return false;
   }
-  // Persistent event
-  void registerEvent(std::shared_ptr<ProvenanceEventRecord> &event) {
-    event->Serialize(std::static_pointer_cast<core::Repository>(shared_from_this()));
-  }
+
   // Remove event
   void removeEvent(ProvenanceEventRecord *event) {
     Delete(event->getEventId());
   }
+
+  virtual bool get(std::vector<std::shared_ptr<core::CoreComponent>> &store, size_t max_size) {
+    leveldb::Iterator* it = db_->NewIterator(leveldb::ReadOptions());
+    for (it->SeekToFirst(); it->Valid(); it->Next()) {
+      std::shared_ptr<ProvenanceEventRecord> eventRead = std::make_shared<ProvenanceEventRecord>();
+      std::string key = it->key().ToString();
+      if (store.size() >= max_size)
+        break;
+      if (eventRead->DeSerialize((uint8_t *) it->value().data(), (int) it->value().size())) {
+        store.push_back(std::dynamic_pointer_cast<core::CoreComponent>(eventRead));
+      }
+    }
+    delete it;
+    return true;
+  }
+
+  virtual bool DeSerialize(std::vector<std::shared_ptr<core::SerializableComponent>> &records, size_t &max_size, std::function<std::shared_ptr<core::SerializableComponent>()> lambda) {
+    leveldb::Iterator* it = db_->NewIterator(leveldb::ReadOptions());
+    size_t requested_batch = max_size;
+    max_size = 0;
+    for (it->SeekToFirst(); it->Valid(); it->Next()) {
+
+      if (max_size >= requested_batch)
+        break;
+      std::shared_ptr<core::SerializableComponent> eventRead = lambda();
+      std::string key = it->key().ToString();
+      if (eventRead->DeSerialize((uint8_t *) it->value().data(), (int) it->value().size())) {
+        max_size++;
+        records.push_back(eventRead);
+      }
+
+    }
+    delete it;
+
+    if (max_size > 0) {
+      return true;
+    } else {
+      return false;
+    }
+  }
   //! get record
   void getProvenanceRecord(std::vector<std::shared_ptr<ProvenanceEventRecord>> &records, int maxSize) {
-    std::lock_guard<std::mutex> lock(mutex_);
     leveldb::Iterator* it = db_->NewIterator(leveldb::ReadOptions());
     for (it->SeekToFirst(); it->Valid(); it->Next()) {
       std::shared_ptr<ProvenanceEventRecord> eventRead = std::make_shared<ProvenanceEventRecord>();
@@ -153,9 +187,29 @@ class ProvenanceRepository : public core::Repository, public std::enable_shared_
     }
     delete it;
   }
+
+  virtual bool DeSerialize(std::vector<std::shared_ptr<core::SerializableComponent>> &store, size_t &max_size) {
+    leveldb::Iterator* it = db_->NewIterator(leveldb::ReadOptions());
+    max_size = 0;
+    for (it->SeekToFirst(); it->Valid(); it->Next()) {
+      std::shared_ptr<ProvenanceEventRecord> eventRead = std::make_shared<ProvenanceEventRecord>();
+      std::string key = it->key().ToString();
+
+      if (store.at(max_size)->DeSerialize((uint8_t *) it->value().data(), (int) it->value().size())) {
+        max_size++;
+      }
+      if (store.size() >= max_size)
+        break;
+    }
+    delete it;
+    if (max_size > 0) {
+      return true;
+    } else {
+      return false;
+    }
+  }
   //! purge record
   void purgeProvenanceRecord(std::vector<std::shared_ptr<ProvenanceEventRecord>> &records) {
-    std::lock_guard<std::mutex> lock(mutex_);
     for (auto record : records) {
       Delete(record->getEventId());
     }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/utils/ByteInputCallBack.h
----------------------------------------------------------------------
diff --git a/libminifi/include/utils/ByteInputCallBack.h b/libminifi/include/utils/ByteInputCallBack.h
index a2b7838..86aae09 100644
--- a/libminifi/include/utils/ByteInputCallBack.h
+++ b/libminifi/include/utils/ByteInputCallBack.h
@@ -32,19 +32,27 @@ namespace utils {
  */
 class ByteInputCallBack : public InputStreamCallback {
  public:
-  ByteInputCallBack() {
+  ByteInputCallBack()
+      : ptr(nullptr) {
   }
 
   virtual ~ByteInputCallBack() {
 
   }
 
-  virtual void process(std::ifstream *stream) {
+  int64_t process(std::shared_ptr<io::BaseStream> stream) {
 
-    std::vector<char> nv = std::vector<char>(std::istreambuf_iterator<char>(*stream), std::istreambuf_iterator<char>());
-    vec = std::move(nv);
+    stream->seek(0);
 
-    ptr = &vec[0];
+    if (stream->getSize() > 0) {
+      vec.resize(stream->getSize());
+
+      stream->readData(vec, stream->getSize());
+    }
+
+    ptr = (char*) &vec[0];
+
+    return vec.size();
 
   }
 
@@ -58,7 +66,7 @@ class ByteInputCallBack : public InputStreamCallback {
 
  private:
   char *ptr;
-  std::vector<char> vec;
+  std::vector<uint8_t> vec;
 };
 
 } /* namespace utils */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/ConfigurationListener.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/ConfigurationListener.cpp b/libminifi/src/ConfigurationListener.cpp
index aaf50ce..858e455 100644
--- a/libminifi/src/ConfigurationListener.cpp
+++ b/libminifi/src/ConfigurationListener.cpp
@@ -35,14 +35,10 @@ void ConfigurationListener::start() {
   pull_interval_ = 60 * 1000;
   std::string value;
   // grab the value for configuration
-  if (configure_->get(Configure::nifi_configuration_listener_pull_interval,
-      value)) {
+  if (configure_->get(Configure::nifi_configuration_listener_pull_interval, value)) {
     core::TimeUnit unit;
-    if (core::Property::StringToTime(value, pull_interval_, unit)
-        && core::Property::ConvertTimeUnitToMS(pull_interval_, unit,
-            pull_interval_)) {
-      logger_->log_info("Configuration Listener pull interval: [%d] ms",
-           pull_interval_);
+    if (core::Property::StringToTime(value, pull_interval_, unit) && core::Property::ConvertTimeUnitToMS(pull_interval_, unit, pull_interval_)) {
+      logger_->log_info("Configuration Listener pull interval: [%d] ms", pull_interval_);
     }
   }
 
@@ -62,7 +58,7 @@ void ConfigurationListener::stop() {
 }
 
 void ConfigurationListener::run() {
-  std::unique_lock<std::mutex> lk(mutex_);
+  std::unique_lock < std::mutex > lk(mutex_);
   std::condition_variable cv;
   int64_t interval = 0;
   while (!cv.wait_for(lk, std::chrono::milliseconds(100), [this] {return (running_ == false);})) {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/Configure.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/Configure.cpp b/libminifi/src/Configure.cpp
index 8bbc5fc..acad1fd 100644
--- a/libminifi/src/Configure.cpp
+++ b/libminifi/src/Configure.cpp
@@ -31,7 +31,8 @@ const char *Configure::nifi_graceful_shutdown_seconds = "nifi.flowcontroller.gra
 const char *Configure::nifi_log_level = "nifi.log.level";
 const char *Configure::nifi_server_name = "nifi.server.name";
 const char *Configure::nifi_configuration_class_name = "nifi.flow.configuration.class.name";
-const char *Configure::nifi_flow_repository_class_name = "nifi.flow.repository.class.name";
+const char *Configure::nifi_flow_repository_class_name = "nifi.flowfile.repository.class.name";
+const char *Configure::nifi_content_repository_class_name = "nifi.content.repository.class.name";
 const char *Configure::nifi_volatile_repository_options = "nifi.volatile.repository.options.";
 const char *Configure::nifi_provenance_repository_class_name = "nifi.provenance.repository.class.name";
 const char *Configure::nifi_server_port = "nifi.server.port";
@@ -43,39 +44,22 @@ const char *Configure::nifi_flowfile_repository_max_storage_size = "nifi.flowfil
 const char *Configure::nifi_flowfile_repository_max_storage_time = "nifi.flowfile.repository.max.storage.time";
 const char *Configure::nifi_flowfile_repository_directory_default = "nifi.flowfile.repository.directory.default";
 const char *Configure::nifi_remote_input_secure = "nifi.remote.input.secure";
-const char *Configure::nifi_security_need_ClientAuth =
-    "nifi.security.need.ClientAuth";
-const char *Configure::nifi_security_client_certificate =
-    "nifi.security.client.certificate";
-const char *Configure::nifi_security_client_private_key =
-    "nifi.security.client.private.key";
-const char *Configure::nifi_security_client_pass_phrase =
-    "nifi.security.client.pass.phrase";
-const char *Configure::nifi_security_client_ca_certificate =
-    "nifi.security.client.ca.certificate";
-const char *Configure::nifi_configuration_listener_pull_interval =
-    "nifi.configuration.listener.pull.interval";
-const char *Configure::nifi_configuration_listener_http_url =
-    "nifi.configuration.listener.http.url";
-const char *Configure::nifi_configuration_listener_rest_url =
-    "nifi.configuration.listener.rest.url";
-const char *Configure::nifi_configuration_listener_type =
-    "nifi.configuration.listener.type";
-const char *Configure::nifi_https_need_ClientAuth =
-    "nifi.https.need.ClientAuth";
-const char *Configure::nifi_https_client_certificate =
-    "nifi.https.client.certificate";
-const char *Configure::nifi_https_client_private_key =
-    "nifi.https.client.private.key";
-const char *Configure::nifi_https_client_pass_phrase =
-    "nifi.https.client.pass.phrase";
-const char *Configure::nifi_https_client_ca_certificate =
-    "nifi.https.client.ca.certificate";
-const char *Configure::nifi_rest_api_user_name =
-    "nifi.rest.api.user.name";
-const char *Configure::nifi_rest_api_password =
-    "nifi.rest.api.password";
-
+const char *Configure::nifi_security_need_ClientAuth = "nifi.security.need.ClientAuth";
+const char *Configure::nifi_security_client_certificate = "nifi.security.client.certificate";
+const char *Configure::nifi_security_client_private_key = "nifi.security.client.private.key";
+const char *Configure::nifi_security_client_pass_phrase = "nifi.security.client.pass.phrase";
+const char *Configure::nifi_security_client_ca_certificate = "nifi.security.client.ca.certificate";
+const char *Configure::nifi_configuration_listener_pull_interval = "nifi.configuration.listener.pull.interval";
+const char *Configure::nifi_configuration_listener_http_url = "nifi.configuration.listener.http.url";
+const char *Configure::nifi_configuration_listener_rest_url = "nifi.configuration.listener.rest.url";
+const char *Configure::nifi_configuration_listener_type = "nifi.configuration.listener.type";
+const char *Configure::nifi_https_need_ClientAuth = "nifi.https.need.ClientAuth";
+const char *Configure::nifi_https_client_certificate = "nifi.https.client.certificate";
+const char *Configure::nifi_https_client_private_key = "nifi.https.client.private.key";
+const char *Configure::nifi_https_client_pass_phrase = "nifi.https.client.pass.phrase";
+const char *Configure::nifi_https_client_ca_certificate = "nifi.https.client.ca.certificate";
+const char *Configure::nifi_rest_api_user_name = "nifi.rest.api.user.name";
+const char *Configure::nifi_rest_api_password = "nifi.rest.api.password";
 
 } /* namespace minifi */
 } /* namespace nifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/Connection.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/Connection.cpp b/libminifi/src/Connection.cpp
index 0901a30..1d937b4 100644
--- a/libminifi/src/Connection.cpp
+++ b/libminifi/src/Connection.cpp
@@ -39,9 +39,11 @@ namespace apache {
 namespace nifi {
 namespace minifi {
 
-Connection::Connection(std::shared_ptr<core::Repository> flow_repository, std::string name, uuid_t uuid, uuid_t srcUUID, uuid_t destUUID)
+Connection::Connection(const std::shared_ptr<core::Repository> &flow_repository, const std::shared_ptr<core::ContentRepository> &content_repo, std::string name, uuid_t uuid, uuid_t srcUUID,
+                       uuid_t destUUID)
     : core::Connectable(name, uuid),
       flow_repository_(flow_repository),
+      content_repo_(content_repo),
       logger_(logging::LoggerFactory<Connection>::getLogger()) {
 
   if (srcUUID)
@@ -89,12 +91,12 @@ void Connection::put(std::shared_ptr<core::FlowFile> flow) {
 
     queued_data_size_ += flow->getSize();
 
-    logger_->log_debug("Enqueue flow file UUID %s to connection %s", flow->getUUIDStr().c_str(), name_.c_str());
+    logger_->log_debug("Enqueue flow file UUID %s to connection %s %d", flow->getUUIDStr(), name_, queue_.size());
   }
 
   if (!flow->isStored()) {
     // Save to the flowfile repo
-    FlowFileRecord event(flow_repository_, flow, this->uuidStr_);
+    FlowFileRecord event(flow_repository_, content_repo_, flow, this->uuidStr_);
     if (event.Serialize()) {
       flow->setStoredToRepository(true);
     }
@@ -102,6 +104,7 @@ void Connection::put(std::shared_ptr<core::FlowFile> flow) {
 
   // Notify receiving processor that work may be available
   if (dest_connectable_) {
+    logger_->log_debug("Notifying %s", dest_connectable_->getName());
     dest_connectable_->notifyWork();
   }
 }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/FlowControlProtocol.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/FlowControlProtocol.cpp b/libminifi/src/FlowControlProtocol.cpp
index dbe27e8..74a1573 100644
--- a/libminifi/src/FlowControlProtocol.cpp
+++ b/libminifi/src/FlowControlProtocol.cpp
@@ -63,15 +63,14 @@ int FlowControlProtocol::connectServer(const char *host, uint16_t port) {
       close(sock);
       return 0;
     }
-    if (setsockopt(sock, SOL_SOCKET, SO_REUSEADDR,
-            reinterpret_cast<char*>(&opt), sizeof(opt)) < 0) {
+    if (setsockopt(sock, SOL_SOCKET, SO_REUSEADDR, reinterpret_cast<char*>(&opt), sizeof(opt)) < 0) {
       logger_->log_error("setsockopt() SO_REUSEADDR failed");
       close(sock);
       return 0;
     }
   }
 
-  int sndsize = 256*1024;
+  int sndsize = 256 * 1024;
   if (setsockopt(sock, SOL_SOCKET, SO_SNDBUF, reinterpret_cast<char*>(&sndsize), sizeof(sndsize)) < 0) {
     logger_->log_error("setsockopt() SO_SNDBUF failed");
     close(sock);

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/FlowController.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/FlowController.cpp b/libminifi/src/FlowController.cpp
index 2c84811..6358ed0 100644
--- a/libminifi/src/FlowController.cpp
+++ b/libminifi/src/FlowController.cpp
@@ -39,8 +39,8 @@
 #include "utils/StringUtils.h"
 #include "core/Core.h"
 #include "core/controller/ControllerServiceProvider.h"
-#include "core/repository/FlowFileRepository.h"
 #include "core/logging/LoggerConfiguration.h"
+#include "core/repository/FlowFileRepository.h"
 
 namespace org {
 namespace apache {
@@ -52,7 +52,7 @@ std::shared_ptr<utils::IdGenerator> FlowController::id_generator_ = utils::IdGen
 #define DEFAULT_CONFIG_NAME "conf/flow.yml"
 
 FlowController::FlowController(std::shared_ptr<core::Repository> provenance_repo, std::shared_ptr<core::Repository> flow_file_repo, std::shared_ptr<Configure> configure,
-                               std::unique_ptr<core::FlowConfiguration> flow_configuration, const std::string name, bool headless_mode)
+                               std::unique_ptr<core::FlowConfiguration> flow_configuration, std::shared_ptr<core::ContentRepository> content_repo, const std::string name, bool headless_mode)
     : core::controller::ControllerServiceProvider(core::getClassName<FlowController>()),
       root_(nullptr),
       max_timer_driven_threads_(0),
@@ -68,6 +68,7 @@ FlowController::FlowController(std::shared_ptr<core::Repository> provenance_repo
       controller_service_provider_(nullptr),
       flow_configuration_(std::move(flow_configuration)),
       configuration_(configure),
+      content_repo_(content_repo),
       logger_(logging::LoggerFactory<FlowController>::getLogger()) {
   if (provenance_repo == nullptr)
     throw std::runtime_error("Provenance Repo should not be null");
@@ -159,8 +160,7 @@ bool FlowController::applyConfiguration(std::string &configurePayload) {
   std::unique_ptr<core::ProcessGroup> newRoot;
   try {
     newRoot = std::move(flow_configuration_->getRootFromPayload(configurePayload));
-  }
-  catch (const YAML::Exception& e) {
+  } catch (const YAML::Exception& e) {
     logger_->log_error("Invalid configuration payload");
     return false;
   }
@@ -168,10 +168,9 @@ bool FlowController::applyConfiguration(std::string &configurePayload) {
   if (newRoot == nullptr)
     return false;
 
-  logger_->log_info("Starting to reload Flow Controller with flow control name %s, version %d",
-      newRoot->getName().c_str(), newRoot->getVersion());
+  logger_->log_info("Starting to reload Flow Controller with flow control name %s, version %d", newRoot->getName().c_str(), newRoot->getVersion());
 
-  std::lock_guard<std::recursive_mutex> flow_lock(mutex_);
+  std::lock_guard < std::recursive_mutex > flow_lock(mutex_);
   stop(true);
   waitUnload(30000);
   this->root_ = std::move(newRoot);
@@ -181,7 +180,7 @@ bool FlowController::applyConfiguration(std::string &configurePayload) {
 }
 
 void FlowController::stop(bool force) {
-  std::lock_guard<std::recursive_mutex> flow_lock(mutex_);
+  std::lock_guard < std::recursive_mutex > flow_lock(mutex_);
   if (running_) {
     // immediately indicate that we are not running
     running_ = false;
@@ -222,7 +221,7 @@ void FlowController::waitUnload(const uint64_t timeToWaitMs) {
 }
 
 void FlowController::unload() {
-  std::lock_guard<std::recursive_mutex> flow_lock(mutex_);
+  std::lock_guard < std::recursive_mutex > flow_lock(mutex_);
   if (running_) {
     stop(true);
   }
@@ -237,7 +236,7 @@ void FlowController::unload() {
 }
 
 void FlowController::load() {
-  std::lock_guard<std::recursive_mutex> flow_lock(mutex_);
+  std::lock_guard < std::recursive_mutex > flow_lock(mutex_);
   if (running_) {
     stop(true);
   }
@@ -246,29 +245,30 @@ void FlowController::load() {
     // grab the value for configuration
     if (this->http_configuration_listener_ == nullptr && configuration_->get(Configure::nifi_configuration_listener_type, listenerType)) {
       if (listenerType == "http") {
-        this->http_configuration_listener_ =
-              std::unique_ptr<minifi::HttpConfigurationListener>(new minifi::HttpConfigurationListener(shared_from_this(), configuration_));
+        this->http_configuration_listener_ = std::unique_ptr < minifi::HttpConfigurationListener > (new minifi::HttpConfigurationListener(shared_from_this(), configuration_));
       }
     }
 
     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_);
+      timer_scheduler_ = std::make_shared < TimerDrivenSchedulingAgent
+          > (std::static_pointer_cast < core::controller::ControllerServiceProvider > (shared_from_this()), provenance_repo_, flow_file_repo_, content_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_);
+      event_scheduler_ = std::make_shared < EventDrivenSchedulingAgent
+          > (std::static_pointer_cast < core::controller::ControllerServiceProvider > (shared_from_this()), provenance_repo_, flow_file_repo_, content_repo_, configuration_);
     }
     logger_->log_info("Load Flow Controller from file %s", configuration_filename_.c_str());
 
-    this->root_ = std::shared_ptr<core::ProcessGroup>(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_));
+    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
@@ -279,7 +279,7 @@ void FlowController::load() {
 }
 
 void FlowController::reload(std::string yamlFile) {
-  std::lock_guard<std::recursive_mutex> flow_lock(mutex_);
+  std::lock_guard < std::recursive_mutex > flow_lock(mutex_);
   logger_->log_info("Starting to reload Flow Controller with yaml %s", yamlFile.c_str());
   stop(true);
   unload();
@@ -305,18 +305,18 @@ void FlowController::loadFlowRepo() {
       this->root_->getConnections(connectionMap);
     }
     logger_->log_debug("Number of connections from connectionMap %d", connectionMap.size());
-    auto rep = std::dynamic_pointer_cast<core::repository::FlowFileRepository>(flow_file_repo_);
+    auto rep = std::dynamic_pointer_cast < core::repository::FlowFileRepository > (flow_file_repo_);
     if (nullptr != rep) {
       rep->setConnectionMap(connectionMap);
     }
-    flow_file_repo_->loadComponent();
+    flow_file_repo_->loadComponent(content_repo_);
   } else {
     logger_->log_debug("Flow file repository is not set");
   }
 }
 
 bool FlowController::start() {
-  std::lock_guard<std::recursive_mutex> flow_lock(mutex_);
+  std::lock_guard < std::recursive_mutex > flow_lock(mutex_);
   if (!initialized_) {
     logger_->log_error("Can not start Flow Controller because it has not been initialized");
     return false;
@@ -349,8 +349,7 @@ bool FlowController::start() {
  * @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) {
+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);
 }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/FlowFileRecord.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/FlowFileRecord.cpp b/libminifi/src/FlowFileRecord.cpp
index 12711a9..efd6fa7 100644
--- a/libminifi/src/FlowFileRecord.cpp
+++ b/libminifi/src/FlowFileRecord.cpp
@@ -40,8 +40,10 @@ namespace minifi {
 std::shared_ptr<logging::Logger> FlowFileRecord::logger_ = logging::LoggerFactory<FlowFileRecord>::getLogger();
 std::atomic<uint64_t> FlowFileRecord::local_flow_seq_number_(0);
 
-FlowFileRecord::FlowFileRecord(std::shared_ptr<core::Repository> flow_repository, std::map<std::string, std::string> attributes, std::shared_ptr<ResourceClaim> claim)
+FlowFileRecord::FlowFileRecord(std::shared_ptr<core::Repository> flow_repository, const std::shared_ptr<core::ContentRepository> &content_repo, std::map<std::string, std::string> attributes,
+                               std::shared_ptr<ResourceClaim> claim)
     : FlowFile(),
+      content_repo_(content_repo),
       flow_repository_(flow_repository) {
   id_ = local_flow_seq_number_.load();
   claim_ = claim;
@@ -64,9 +66,11 @@ FlowFileRecord::FlowFileRecord(std::shared_ptr<core::Repository> flow_repository
     claim_->increaseFlowFileRecordOwnedCount();
 }
 
-FlowFileRecord::FlowFileRecord(std::shared_ptr<core::Repository> flow_repository, std::shared_ptr<core::FlowFile> &event, const std::string &uuidConnection)
+FlowFileRecord::FlowFileRecord(std::shared_ptr<core::Repository> flow_repository, const std::shared_ptr<core::ContentRepository> &content_repo, std::shared_ptr<core::FlowFile> &event,
+                               const std::string &uuidConnection)
     : FlowFile(),
       snapshot_(""),
+      content_repo_(content_repo),
       flow_repository_(flow_repository) {
   entry_date_ = event->getEntryDate();
   lineage_start_date_ = event->getlineageStartDate();
@@ -82,10 +86,11 @@ FlowFileRecord::FlowFileRecord(std::shared_ptr<core::Repository> flow_repository
   }
 }
 
-FlowFileRecord::FlowFileRecord(std::shared_ptr<core::Repository> flow_repository, std::shared_ptr<core::FlowFile> &event)
+FlowFileRecord::FlowFileRecord(std::shared_ptr<core::Repository> flow_repository, const std::shared_ptr<core::ContentRepository> &content_repo, std::shared_ptr<core::FlowFile> &event)
     : FlowFile(),
       uuid_connection_(""),
       snapshot_(""),
+      content_repo_(content_repo),
       flow_repository_(flow_repository) {
 }
 
@@ -101,7 +106,7 @@ FlowFileRecord::~FlowFileRecord() {
     if (claim_->getFlowFileRecordOwnedCount() <= 0) {
       logger_->log_debug("Delete Resource Claim %s", claim_->getContentFullPath().c_str());
       if (!this->stored || !flow_repository_->Get(uuid_str_, value)) {
-        std::remove(claim_->getContentFullPath().c_str());
+        content_repo_->remove(claim_);
       }
     }
   }
@@ -319,6 +324,9 @@ bool FlowFileRecord::DeSerialize(const uint8_t *buffer, const int bufferSize) {
     return false;
   }
 
+  if (nullptr == claim_) {
+    claim_ = std::make_shared<ResourceClaim>(content_full_fath_, content_repo_, true);
+  }
   return true;
 }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/HttpConfigurationListener.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/HttpConfigurationListener.cpp b/libminifi/src/HttpConfigurationListener.cpp
index 39da67b..c16ca75 100644
--- a/libminifi/src/HttpConfigurationListener.cpp
+++ b/libminifi/src/HttpConfigurationListener.cpp
@@ -63,17 +63,14 @@ bool HttpConfigurationListener::pullConfiguration(std::string &configuration) {
   }
 
   utils::HTTPRequestResponse content;
-  curl_easy_setopt(http_session, CURLOPT_WRITEFUNCTION,
-      &utils::HTTPRequestResponse::recieve_write);
+  curl_easy_setopt(http_session, CURLOPT_WRITEFUNCTION, &utils::HTTPRequestResponse::recieve_write);
 
-  curl_easy_setopt(http_session, CURLOPT_WRITEDATA,
-      static_cast<void*>(&content));
+  curl_easy_setopt(http_session, CURLOPT_WRITEDATA, static_cast<void*>(&content));
 
   CURLcode res = curl_easy_perform(http_session);
 
   if (res == CURLE_OK) {
-    logger_->log_debug("HttpConfigurationListener -- curl successful to %s",
-        fullUrl.c_str());
+    logger_->log_debug("HttpConfigurationListener -- curl successful to %s", fullUrl.c_str());
 
     std::string response_body(content.data.begin(), content.data.end());
     int64_t http_code = 0;
@@ -82,8 +79,7 @@ bool HttpConfigurationListener::pullConfiguration(std::string &configuration) {
     /* ask for the content-type */
     curl_easy_getinfo(http_session, CURLINFO_CONTENT_TYPE, &content_type);
 
-    bool isSuccess = ((int32_t) (http_code / 100)) == 2
-        && res != CURLE_ABORTED_BY_CALLBACK;
+    bool isSuccess = ((int32_t) (http_code / 100)) == 2 && res != CURLE_ABORTED_BY_CALLBACK;
     bool body_empty = IsNullOrEmpty(content.data);
 
     if (isSuccess && !body_empty) {
@@ -94,9 +90,7 @@ bool HttpConfigurationListener::pullConfiguration(std::string &configuration) {
       logger_->log_error("Cannot output body to content");
     }
   } else {
-    logger_->log_error(
-        "HttpConfigurationListener -- curl_easy_perform() failed %s\n",
-        curl_easy_strerror(res));
+    logger_->log_error("HttpConfigurationListener -- curl_easy_perform() failed %s\n", curl_easy_strerror(res));
   }
   curl_easy_cleanup(http_session);
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/Properties.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/Properties.cpp b/libminifi/src/Properties.cpp
index abebfbb..076cefc 100644
--- a/libminifi/src/Properties.cpp
+++ b/libminifi/src/Properties.cpp
@@ -34,7 +34,7 @@ Properties::Properties()
 
 // Get the config value
 bool Properties::get(std::string key, std::string &value) {
-  std::lock_guard<std::mutex> lock(mutex_);
+  std::lock_guard < std::mutex > lock(mutex_);
   auto it = properties_.find(key);
 
   if (it != properties_.end()) {
@@ -46,7 +46,7 @@ bool Properties::get(std::string key, std::string &value) {
 }
 
 int Properties::getInt(const std::string &key, int default_value) {
-  std::lock_guard<std::mutex> lock(mutex_);
+  std::lock_guard < std::mutex > lock(mutex_);
   auto it = properties_.find(key);
 
   if (it != properties_.end()) {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/RemoteProcessorGroupPort.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/RemoteProcessorGroupPort.cpp b/libminifi/src/RemoteProcessorGroupPort.cpp
index d1862cd..3c88e8f 100644
--- a/libminifi/src/RemoteProcessorGroupPort.cpp
+++ b/libminifi/src/RemoteProcessorGroupPort.cpp
@@ -18,7 +18,7 @@
  * limitations under the License.
  */
 
-#include "../include/RemoteProcessorGroupPort.h"
+#include "RemoteProcessorGroupPort.h"
 
 #include <curl/curl.h>
 #include <curl/curlbuild.h>
@@ -30,18 +30,20 @@
 #include <deque>
 #include <iostream>
 #include <set>
+
 #include <string>
 #include <type_traits>
 #include <utility>
 #include "json/json.h"
 #include "json/writer.h"
 
-#include "../include/core/logging/Logger.h"
-#include "../include/core/ProcessContext.h"
-#include "../include/core/ProcessorNode.h"
-#include "../include/core/Property.h"
-#include "../include/core/Relationship.h"
-#include "../include/Site2SitePeer.h"
+#include "Exception.h"
+#include "core/logging/Logger.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessorNode.h"
+#include "core/Property.h"
+#include "core/Relationship.h"
+#include "Site2SitePeer.h"
 
 namespace org {
 namespace apache {
@@ -54,8 +56,7 @@ core::Property RemoteProcessorGroupPort::port("Port", "Remote Port", "");
 core::Property RemoteProcessorGroupPort::portUUID("Port UUID", "Specifies remote NiFi Port UUID.", "");
 core::Relationship RemoteProcessorGroupPort::relation;
 
-std::unique_ptr<Site2SiteClientProtocol> RemoteProcessorGroupPort::getNextProtocol(
-bool create = true) {
+std::unique_ptr<Site2SiteClientProtocol> RemoteProcessorGroupPort::getNextProtocol(bool create = true) {
   std::unique_ptr<Site2SiteClientProtocol> nextProtocol = nullptr;
   if (!available_protocols_.try_dequeue(nextProtocol)) {
     if (create) {
@@ -170,31 +171,41 @@ void RemoteProcessorGroupPort::onTrigger(core::ProcessContext *context, core::Pr
     uuid_parse(value.c_str(), protocol_uuid_);
   }
 
-  std::unique_ptr<Site2SiteClientProtocol> protocol_ = getNextProtocol();
+  std::unique_ptr<Site2SiteClientProtocol> protocol_ = nullptr;
+  try {
+    protocol_ = getNextProtocol();
 
-  if (!protocol_) {
-    context->yield();
-    return;
-  }
+    if (!protocol_) {
+      context->yield();
+      return;
+    }
+    if (!protocol_->bootstrap()) {
+      // bootstrap the client protocol if needeed
+      context->yield();
+      std::shared_ptr<Processor> processor = std::static_pointer_cast < Processor > (context->getProcessorNode().getProcessor());
+      logger_->log_error("Site2Site bootstrap failed yield period %d peer ", processor->getYieldPeriodMsec());
+
+      return;
+    }
+
+    if (direction_ == RECEIVE) {
+      protocol_->receiveFlowFiles(context, session);
+    } else {
+      protocol_->transferFlowFiles(context, session);
+    }
 
-  if (!protocol_->bootstrap()) {
-    // bootstrap the client protocol if needeed
-    context->yield();
-    std::shared_ptr<Processor> processor = std::static_pointer_cast<Processor>(context->getProcessorNode().getProcessor());
-    logger_->log_error("Site2Site bootstrap failed yield period %d peer ", processor->getYieldPeriodMsec());
     returnProtocol(std::move(protocol_));
     return;
+  } catch (const minifi::Exception &ex2) {
+    context->yield();
+    session->rollback();
+  } catch (...) {
+    context->yield();
+    session->rollback();
   }
 
-  if (direction_ == RECEIVE) {
-    protocol_->receiveFlowFiles(context, session);
-  } else {
-    protocol_->transferFlowFiles(context, session);
-  }
-
-  returnProtocol(std::move(protocol_));
 
-  return;
+  throw std::exception();
 }
 
 void RemoteProcessorGroupPort::refreshRemoteSite2SiteInfo() {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/ResourceClaim.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/ResourceClaim.cpp b/libminifi/src/ResourceClaim.cpp
index 1a9f2fe..e7d4557 100644
--- a/libminifi/src/ResourceClaim.cpp
+++ b/libminifi/src/ResourceClaim.cpp
@@ -17,14 +17,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+#include "ResourceClaim.h"
 #include <uuid/uuid.h>
-
 #include <map>
 #include <queue>
 #include <string>
 #include <vector>
-
-#include "ResourceClaim.h"
+#include <memory>
+#include "core/StreamManager.h"
+#include "utils/Id.h"
 #include "core/logging/LoggerConfiguration.h"
 
 namespace org {
@@ -36,14 +37,22 @@ utils::NonRepeatingStringGenerator ResourceClaim::non_repeating_string_generator
 
 char *ResourceClaim::default_directory_path = const_cast<char*>(DEFAULT_CONTENT_DIRECTORY);
 
-ResourceClaim::ResourceClaim(const std::string contentDirectory)
+ResourceClaim::ResourceClaim(std::shared_ptr<core::StreamManager<ResourceClaim>> claim_manager, const std::string contentDirectory)
     : _flowFileRecordOwnedCount(0),
+      claim_manager_(claim_manager),
+      deleted_(false),
       logger_(logging::LoggerFactory<ResourceClaim>::getLogger()) {
   // Create the full content path for the content
   _contentFullPath = contentDirectory + "/" + non_repeating_string_generator_.generate();
   logger_->log_debug("Resource Claim created %s", _contentFullPath);
 }
 
+ResourceClaim::ResourceClaim(const std::string path, std::shared_ptr<core::StreamManager<ResourceClaim>> claim_manager, bool deleted)
+    : claim_manager_(claim_manager),
+      deleted_(deleted) {
+  _contentFullPath = path;
+}
+
 } /* namespace minifi */
 } /* namespace nifi */
 } /* namespace apache */


[6/6] nifi-minifi-cpp git commit: MINIFI-249: Update prov repo to better abstract deser.

Posted by al...@apache.org.
MINIFI-249: Update prov repo to better abstract deser.

- Deserialization and serialization are better abstracted
into SerializableComponent allowing us to use all repos with
the same [de]serialization interfaces.
- Update Test resources to use local http server when possible
- Allow for different volatile configurations
- Update removals and link FlowFileRepo to content repo

MINIFI-330: convert const char* to std::string

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

Branch: refs/heads/master
Commit: 7c16d1bbbc2bf7ab86c5eed26821aa26f761ac06
Parents: 20622f6
Author: Marc Parisi <ph...@apache.org>
Authored: Wed May 24 14:39:08 2017 -0400
Committer: Aldrin Piri <al...@apache.org>
Committed: Thu Jul 27 12:42:52 2017 -0400

----------------------------------------------------------------------
 cmake/BuildTests.cmake                          |   4 +-
 libminifi/include/Connection.h                  |   6 +-
 libminifi/include/EventDrivenSchedulingAgent.h  |   5 +-
 libminifi/include/FlowController.h              |  23 +-
 libminifi/include/FlowFileRecord.h              |  31 +-
 libminifi/include/ResourceClaim.h               |  35 +-
 libminifi/include/SchedulingAgent.h             |  12 +-
 libminifi/include/Site2SiteClientProtocol.h     |  35 +-
 libminifi/include/ThreadedSchedulingAgent.h     |   7 +-
 libminifi/include/TimerDrivenSchedulingAgent.h  |   7 +-
 libminifi/include/core/ConfigurationFactory.h   |  11 +-
 libminifi/include/core/ContentRepository.h      |  60 ++
 libminifi/include/core/Core.h                   |   3 +
 libminifi/include/core/FlowConfiguration.h      |  16 +-
 libminifi/include/core/ProcessContext.h         |  27 +-
 libminifi/include/core/ProcessGroup.h           |   3 +-
 libminifi/include/core/ProcessSession.h         |   2 +-
 libminifi/include/core/Repository.h             |  92 ++-
 libminifi/include/core/RepositoryFactory.h      |  16 +-
 libminifi/include/core/SerializableComponent.h  |  88 +++
 libminifi/include/core/StreamManager.h          |  81 +++
 .../include/core/logging/LoggerConfiguration.h  |   3 +-
 .../SiteToSiteProvenanceReportingTask.h         |   4 +-
 .../include/core/repository/AtomicRepoEntries.h | 501 ++++++++++++++++
 .../core/repository/FileSystemRepository.h      |  72 +++
 .../core/repository/FlowFileRepository.h        |  25 +-
 .../core/repository/VolatileContentRepository.h | 138 +++++
 .../repository/VolatileFlowFileRepository.h     |  82 +++
 .../repository/VolatileProvenanceRepository.h   |  60 ++
 .../core/repository/VolatileRepository.h        | 517 +++++++++--------
 libminifi/include/core/yaml/YamlConfiguration.h |  39 +-
 libminifi/include/io/AtomicEntryStream.h        | 205 +++++++
 libminifi/include/io/BaseStream.h               |  13 +
 libminifi/include/io/ClientSocket.h             |   6 +-
 libminifi/include/io/DataStream.h               |   8 +-
 libminifi/include/io/FileStream.h               | 136 +++++
 libminifi/include/processors/ExecuteProcess.h   |   8 +-
 libminifi/include/processors/GenerateFlowFile.h |   6 +-
 libminifi/include/processors/InvokeHTTP.h       |   6 +
 libminifi/include/processors/ListenHTTP.h       |   2 +-
 libminifi/include/processors/ListenSyslog.h     |  10 +-
 libminifi/include/processors/LogAttribute.h     |  28 +-
 libminifi/include/processors/PutFile.h          |   3 +-
 libminifi/include/properties/Configure.h        |   1 +
 libminifi/include/provenance/Provenance.h       |  62 +-
 .../include/provenance/ProvenanceRepository.h   |  82 ++-
 libminifi/include/utils/ByteInputCallBack.h     |  20 +-
 libminifi/src/ConfigurationListener.cpp         |  12 +-
 libminifi/src/Configure.cpp                     |  52 +-
 libminifi/src/Connection.cpp                    |   9 +-
 libminifi/src/FlowControlProtocol.cpp           |   5 +-
 libminifi/src/FlowController.cpp                |  47 +-
 libminifi/src/FlowFileRecord.cpp                |  16 +-
 libminifi/src/HttpConfigurationListener.cpp     |  16 +-
 libminifi/src/Properties.cpp                    |   4 +-
 libminifi/src/RemoteProcessorGroupPort.cpp      |  65 ++-
 libminifi/src/ResourceClaim.cpp                 |  17 +-
 libminifi/src/SchedulingAgent.cpp               |  12 +-
 libminifi/src/Site2SiteClientProtocol.cpp       |  19 +-
 libminifi/src/ThreadedSchedulingAgent.cpp       |   8 +-
 libminifi/src/controllers/SSLContextService.cpp |  14 +-
 libminifi/src/core/ClassLoader.cpp              |   6 +-
 libminifi/src/core/ConfigurableComponent.cpp    |  13 +-
 libminifi/src/core/ConfigurationFactory.cpp     |  12 +-
 libminifi/src/core/Connectable.cpp              |  12 +-
 libminifi/src/core/Core.cpp                     |   5 +
 libminifi/src/core/FlowConfiguration.cpp        |  12 +-
 libminifi/src/core/FlowFile.cpp                 |   2 +-
 libminifi/src/core/ProcessGroup.cpp             |  29 +-
 libminifi/src/core/ProcessSession.cpp           | 571 ++++++++++---------
 libminifi/src/core/ProcessSessionFactory.cpp    |   2 +-
 libminifi/src/core/Processor.cpp                |  20 +-
 libminifi/src/core/Repository.cpp               |   6 +-
 libminifi/src/core/RepositoryFactory.cpp        |  47 +-
 .../StandardControllerServiceNode.cpp           |   6 +-
 .../src/core/logging/LoggerConfiguration.cpp    |  16 +-
 .../SiteToSiteProvenanceReportingTask.cpp       |  31 +-
 .../core/repository/FileSystemRepository.cpp    |  54 ++
 .../src/core/repository/FlowFileRepository.cpp  |  54 +-
 .../repository/VolatileContentRepository.cpp    | 183 ++++++
 .../src/core/repository/VolatileRepository.cpp  |  29 +-
 libminifi/src/core/yaml/YamlConfiguration.cpp   |  66 ++-
 libminifi/src/io/AtomicEntryStream.cpp          |  34 ++
 libminifi/src/io/ClientSocket.cpp               |  14 +-
 libminifi/src/io/FileStream.cpp                 | 160 ++++++
 libminifi/src/io/StreamFactory.cpp              |   6 +-
 libminifi/src/processors/ExecuteProcess.cpp     |   6 +-
 libminifi/src/processors/GenerateFlowFile.cpp   |   4 +-
 libminifi/src/processors/GetFile.cpp            |   8 +-
 libminifi/src/processors/InvokeHTTP.cpp         |  33 +-
 libminifi/src/processors/ListenHTTP.cpp         |  10 +-
 libminifi/src/processors/ListenSyslog.cpp       |   2 +-
 libminifi/src/processors/LogAttribute.cpp       |   9 +-
 libminifi/src/processors/PutFile.cpp            |  45 +-
 libminifi/src/processors/TailFile.cpp           |  26 +-
 libminifi/src/provenance/Provenance.cpp         |  50 +-
 .../src/provenance/ProvenanceRepository.cpp     |  18 +-
 libminifi/test/CPPLINT.cfg                      |   1 +
 libminifi/test/TestBase.cpp                     | 211 +++++++
 libminifi/test/TestBase.h                       | 102 +++-
 libminifi/test/TestServer.h                     | 137 +++++
 .../ControllerServiceIntegrationTests.cpp       |  11 +-
 .../HttpConfigurationListenerTest.cpp           |  67 +--
 .../test/integration/HttpGetIntegrationTest.cpp |  67 ++-
 .../integration/HttpPostIntegrationTest.cpp     |  34 +-
 .../integration/ProvenanceReportingTest.cpp     |  11 +-
 .../test/integration/Site2SiteRestTest.cpp      |  61 +-
 .../test/integration/TestExecuteProcess.cpp     |  20 +-
 libminifi/test/resources/TestHTTPGet.yml        |   2 +-
 libminifi/test/resources/TestHTTPGetSecure.yml  |   2 +-
 libminifi/test/resources/TestHTTPPost.yml       |   2 +-
 libminifi/test/resources/cn.ckey.pem            |   1 -
 libminifi/test/resources/cn.crt.pem             |   1 -
 libminifi/test/resources/nifi-cert.pem          |  27 +
 libminifi/test/unit/FileStreamTests.cpp         | 210 +++++++
 libminifi/test/unit/InvokeHTTPTests.cpp         | 173 ++----
 libminifi/test/unit/ProcessorTests.cpp          | 289 ++++------
 libminifi/test/unit/ProvenanceTestHelper.h      |  70 ++-
 libminifi/test/unit/ProvenanceTests.cpp         |  33 +-
 libminifi/test/unit/RepoTests.cpp               |  14 +-
 libminifi/test/unit/TailFileTests.cpp           | 187 +++---
 libminifi/test/unit/YamlConfigurationTests.cpp  |   6 +-
 main/MiNiFiMain.cpp                             |  59 +-
 123 files changed, 4638 insertions(+), 1658 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/cmake/BuildTests.cmake
----------------------------------------------------------------------
diff --git a/cmake/BuildTests.cmake b/cmake/BuildTests.cmake
index 59f1d59..9500792 100644
--- a/cmake/BuildTests.cmake
+++ b/cmake/BuildTests.cmake
@@ -67,7 +67,7 @@ GETSOURCEFILES(INTEGRATION_TESTS "${TEST_DIR}/integration/")
 SET(UNIT_TEST_COUNT 0)
 FOREACH(testfile ${UNIT_TESTS})
 	get_filename_component(testfilename "${testfile}" NAME_WE)
-	add_executable("${testfilename}" "${TEST_DIR}/unit/${testfile}" ${SPD_SOURCES})
+	add_executable("${testfilename}" "${TEST_DIR}/unit/${testfile}" ${SPD_SOURCES} "${TEST_DIR}/TestBase.cpp")
 	createTests("${testfilename}")
  	MATH(EXPR UNIT_TEST_COUNT "${UNIT_TEST_COUNT}+1")
 	add_test(NAME "${testfilename}" COMMAND "${testfilename}" WORKING_DIRECTORY ${TEST_DIR})
@@ -77,7 +77,7 @@ message("-- Finished building ${UNIT_TEST_COUNT} unit test file(s)...")
 SET(INT_TEST_COUNT 0)
 FOREACH(testfile ${INTEGRATION_TESTS})
 	get_filename_component(testfilename "${testfile}" NAME_WE)
-	add_executable("${testfilename}" "${TEST_DIR}/integration/${testfile}" ${SPD_SOURCES})
+	add_executable("${testfilename}" "${TEST_DIR}/integration/${testfile}" ${SPD_SOURCES} "${TEST_DIR}/TestBase.cpp")
 	createTests("${testfilename}")
 	#message("Adding ${testfilename} from ${testfile}")
 	MATH(EXPR INT_TEST_COUNT "${INT_TEST_COUNT}+1")

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/Connection.h
----------------------------------------------------------------------
diff --git a/libminifi/include/Connection.h b/libminifi/include/Connection.h
index be51fce..ff32baf 100644
--- a/libminifi/include/Connection.h
+++ b/libminifi/include/Connection.h
@@ -47,7 +47,9 @@ class Connection : public core::Connectable, public std::enable_shared_from_this
   /*
    * Create a new processor
    */
-  explicit Connection(std::shared_ptr<core::Repository> flow_repository, std::string name, uuid_t uuid = NULL, uuid_t srcUUID = NULL, uuid_t destUUID = NULL);
+  explicit Connection(const std::shared_ptr<core::Repository> &flow_repository, const std::shared_ptr<core::ContentRepository> &content_repo, std::string name, uuid_t uuid = NULL,
+                      uuid_t srcUUID = NULL,
+                      uuid_t destUUID = NULL);
   // Destructor
   virtual ~Connection() {
   }
@@ -168,6 +170,8 @@ class Connection : public core::Connectable, public std::enable_shared_from_this
   std::atomic<uint64_t> expired_duration_;
   // flow file repository
   std::shared_ptr<core::Repository> flow_repository_;
+  // content repository reference.
+  std::shared_ptr<core::ContentRepository> content_repo_;
 
  private:
   // Mutex for protection

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/EventDrivenSchedulingAgent.h
----------------------------------------------------------------------
diff --git a/libminifi/include/EventDrivenSchedulingAgent.h b/libminifi/include/EventDrivenSchedulingAgent.h
index 6a63dc5..c838b11 100644
--- a/libminifi/include/EventDrivenSchedulingAgent.h
+++ b/libminifi/include/EventDrivenSchedulingAgent.h
@@ -38,8 +38,9 @@ class EventDrivenSchedulingAgent : public ThreadedSchedulingAgent {
   /*!
    * Create a new event driven scheduling agent.
    */
-  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) {
+  EventDrivenSchedulingAgent(std::shared_ptr<core::controller::ControllerServiceProvider> controller_service_provider, std::shared_ptr<core::Repository> repo,
+                             std::shared_ptr<core::Repository> flow_repo, std::shared_ptr<core::ContentRepository> content_repo, std::shared_ptr<Configure> configuration)
+      : ThreadedSchedulingAgent(controller_service_provider, repo, flow_repo, content_repo, configuration) {
   }
   // Destructor
   virtual ~EventDrivenSchedulingAgent() {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/FlowController.h
----------------------------------------------------------------------
diff --git a/libminifi/include/FlowController.h b/libminifi/include/FlowController.h
index c2fef2a..d9a0452 100644
--- a/libminifi/include/FlowController.h
+++ b/libminifi/include/FlowController.h
@@ -70,10 +70,23 @@ class FlowController : public core::controller::ControllerServiceProvider, publi
   /**
    * Flow controller constructor
    */
-  FlowController(std::shared_ptr<core::Repository> provenance_repo, std::shared_ptr<core::Repository> flow_file_repo, std::shared_ptr<Configure> configure,
-                 std::unique_ptr<core::FlowConfiguration> flow_configuration,
-                 const std::string name = DEFAULT_ROOT_GROUP_NAME,
-                 bool headless_mode = false);
+  explicit FlowController(std::shared_ptr<core::Repository> provenance_repo, std::shared_ptr<core::Repository> flow_file_repo, std::shared_ptr<Configure> configure,
+                          std::unique_ptr<core::FlowConfiguration> flow_configuration,
+                          std::shared_ptr<core::ContentRepository> content_repo, const std::string name, bool headless_mode);
+
+  explicit FlowController(std::shared_ptr<core::Repository> provenance_repo, std::shared_ptr<core::Repository> flow_file_repo, std::shared_ptr<Configure> configure,
+                          std::unique_ptr<core::FlowConfiguration> flow_configuration,
+                          std::shared_ptr<core::ContentRepository> content_repo)
+      : FlowController(provenance_repo, flow_file_repo, configure, std::move(flow_configuration), content_repo, DEFAULT_ROOT_GROUP_NAME, false)
+  {
+  }
+
+  explicit FlowController(std::shared_ptr<core::Repository> provenance_repo, std::shared_ptr<core::Repository> flow_file_repo, std::shared_ptr<Configure> configure,
+                          std::unique_ptr<core::FlowConfiguration> flow_configuration)
+      : FlowController(provenance_repo, flow_file_repo, configure, std::move(flow_configuration), std::make_shared<core::repository::FileSystemRepository>(), DEFAULT_ROOT_GROUP_NAME, false)
+  {
+    content_repo_->initialize(configure);
+  }
 
   // Destructor
   virtual ~FlowController();
@@ -301,6 +314,8 @@ class FlowController : public core::controller::ControllerServiceProvider, publi
   // FlowFile Repo
   std::shared_ptr<core::Repository> flow_file_repo_;
 
+  std::shared_ptr<core::ContentRepository> content_repo_;
+
   // Flow Engines
   // Flow Timer Scheduler
   std::shared_ptr<TimerDrivenSchedulingAgent> timer_scheduler_;

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/FlowFileRecord.h
----------------------------------------------------------------------
diff --git a/libminifi/include/FlowFileRecord.h b/libminifi/include/FlowFileRecord.h
index 3d6057d..d6e5f2e 100644
--- a/libminifi/include/FlowFileRecord.h
+++ b/libminifi/include/FlowFileRecord.h
@@ -30,7 +30,8 @@
 #include <sstream>
 #include <fstream>
 #include <set>
-
+#include "core/ContentRepository.h"
+#include "io/BaseStream.h"
 #include "io/Serializable.h"
 #include "core/FlowFile.h"
 #include "utils/TimeUtil.h"
@@ -81,11 +82,21 @@ inline const char *FlowAttributeKey(FlowAttribute attribute) {
 // throw exception for error
 class InputStreamCallback {
  public:
-  virtual void process(std::ifstream *stream) = 0;
+  virtual ~InputStreamCallback() {
+
+  }
+  //virtual void process(std::ifstream *stream) = 0;
+
+  virtual int64_t process(std::shared_ptr<io::BaseStream> stream) = 0;
 };
 class OutputStreamCallback {
  public:
-  virtual void process(std::ofstream *stream) = 0;
+  virtual ~OutputStreamCallback() {
+
+  }
+  //virtual void process(std::ofstream *stream) = 0;
+  virtual int64_t process(std::shared_ptr<io::BaseStream> stream) = 0;
+
 };
 
 class FlowFileRecord : public core::FlowFile, public io::Serializable {
@@ -94,14 +105,17 @@ class FlowFileRecord : public core::FlowFile, public io::Serializable {
   /*
    * Create a new flow record
    */
-  explicit FlowFileRecord(std::shared_ptr<core::Repository> flow_repository, std::map<std::string, std::string> attributes, std::shared_ptr<ResourceClaim> claim = nullptr);
+  explicit FlowFileRecord(std::shared_ptr<core::Repository> flow_repository, const std::shared_ptr<core::ContentRepository> &content_repo, std::map<std::string, std::string> attributes,
+                          std::shared_ptr<ResourceClaim> claim = nullptr);
 
-  explicit FlowFileRecord(std::shared_ptr<core::Repository> flow_repository, std::shared_ptr<core::FlowFile> &event);
+  explicit FlowFileRecord(std::shared_ptr<core::Repository> flow_repository, const std::shared_ptr<core::ContentRepository> &content_repo, std::shared_ptr<core::FlowFile> &event);
 
-  explicit FlowFileRecord(std::shared_ptr<core::Repository> flow_repository, std::shared_ptr<core::FlowFile> &event, const std::string &uuidConnection);
+  explicit FlowFileRecord(std::shared_ptr<core::Repository> flow_repository, const std::shared_ptr<core::ContentRepository> &content_repo, std::shared_ptr<core::FlowFile> &event,
+                          const std::string &uuidConnection);
 
-  explicit FlowFileRecord(std::shared_ptr<core::Repository> flow_repository)
+  explicit FlowFileRecord(std::shared_ptr<core::Repository> flow_repository, const std::shared_ptr<core::ContentRepository> &content_repo)
       : FlowFile(),
+        content_repo_(content_repo),
         flow_repository_(flow_repository),
         snapshot_("") {
 
@@ -168,6 +182,9 @@ class FlowFileRecord : public core::FlowFile, public io::Serializable {
   // repository reference.
   std::shared_ptr<core::Repository> flow_repository_;
 
+  // content repo reference.
+  std::shared_ptr<core::ContentRepository> content_repo_;
+
   // Snapshot flow record for session rollback
   bool snapshot_;
   // Prevent default copy constructor and assignment operation

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/ResourceClaim.h
----------------------------------------------------------------------
diff --git a/libminifi/include/ResourceClaim.h b/libminifi/include/ResourceClaim.h
index 49faed1..19a67fa 100644
--- a/libminifi/include/ResourceClaim.h
+++ b/libminifi/include/ResourceClaim.h
@@ -25,9 +25,11 @@
 #include <vector>
 #include <queue>
 #include <map>
+#include <memory>
 #include <mutex>
 #include <atomic>
 #include "core/Core.h"
+#include "core/StreamManager.h"
 #include "properties/Configure.h"
 #include "utils/Id.h"
 
@@ -40,7 +42,7 @@ namespace minifi {
 #define DEFAULT_CONTENT_DIRECTORY "./content_repository"
 
 // ResourceClaim Class
-class ResourceClaim {
+class ResourceClaim : public std::enable_shared_from_this<ResourceClaim> {
 
  public:
 
@@ -49,7 +51,9 @@ class ResourceClaim {
   /*!
    * Create a new resource claim
    */
-  ResourceClaim(const std::string contentDirectory = default_directory_path);
+  ResourceClaim(std::shared_ptr<core::StreamManager<ResourceClaim>> claim_manager, const std::string contentDirectory = default_directory_path);
+
+  ResourceClaim(const std::string path, std::shared_ptr<core::StreamManager<ResourceClaim>> claim_manager, bool deleted = false);
   // Destructor
   virtual ~ResourceClaim() {
   }
@@ -59,7 +63,11 @@ class ResourceClaim {
   }
   // decreaseFlowFileRecordOwenedCount
   void decreaseFlowFileRecordOwnedCount() {
-    --_flowFileRecordOwnedCount;
+
+    if (_flowFileRecordOwnedCount > 0) {
+      _flowFileRecordOwnedCount--;
+    }
+
   }
   // getFlowFileRecordOwenedCount
   uint64_t getFlowFileRecordOwnedCount() {
@@ -74,14 +82,35 @@ class ResourceClaim {
     _contentFullPath = path;
   }
 
+  void deleteClaim() {
+    if (!deleted_)
+    {
+      deleted_ = true;
+    }
+
+  }
+
+  friend std::ostream& operator<<(std::ostream& stream, const ResourceClaim& claim) {
+    stream << claim._contentFullPath;
+    return stream;
+  }
+
+  friend std::ostream& operator<<(std::ostream& stream, const std::shared_ptr<ResourceClaim>& claim) {
+    stream << claim->_contentFullPath;
+    return stream;
+  }
  protected:
+  std::atomic<bool> deleted_;
   // Full path to the content
   std::string _contentFullPath;
 
   // How many FlowFileRecord Own this cliam
   std::atomic<uint64_t> _flowFileRecordOwnedCount;
 
+  std::shared_ptr<core::StreamManager<ResourceClaim>> claim_manager_;
+
  private:
+
   // Logger
   std::shared_ptr<logging::Logger> logger_;
   // Prevent default copy constructor and assignment operation

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/SchedulingAgent.h
----------------------------------------------------------------------
diff --git a/libminifi/include/SchedulingAgent.h b/libminifi/include/SchedulingAgent.h
index 22f79db..1ff3fac 100644
--- a/libminifi/include/SchedulingAgent.h
+++ b/libminifi/include/SchedulingAgent.h
@@ -52,14 +52,19 @@ class SchedulingAgent {
   /*!
    * Create a new scheduling agent.
    */
-  SchedulingAgent(std::shared_ptr<core::controller::ControllerServiceProvider> controller_service_provider, std::shared_ptr<core::Repository> repo, std::shared_ptr<Configure> configuration)
+  SchedulingAgent(std::shared_ptr<core::controller::ControllerServiceProvider> controller_service_provider, std::shared_ptr<core::Repository> repo,
+		  std::shared_ptr<core::Repository> flow_repo,
+                  std::shared_ptr<core::ContentRepository> content_repo,
+                  std::shared_ptr<Configure> configuration)
       : configure_(configuration),
         admin_yield_duration_(0),
         bored_yield_duration_(0),
+        content_repo_(content_repo),
         controller_service_provider_(controller_service_provider),
         logger_(logging::LoggerFactory<SchedulingAgent>::getLogger()) {
     running_ = false;
     repo_ = repo;
+    flow_repo_ = flow_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();
@@ -77,7 +82,6 @@ class SchedulingAgent {
   // start
   void start() {
     running_ = true;
-
   }
   // stop
   void stop() {
@@ -108,6 +112,10 @@ class SchedulingAgent {
   std::shared_ptr<Configure> configure_;
 
   std::shared_ptr<core::Repository> repo_;
+  
+  std::shared_ptr<core::Repository> flow_repo_;
+
+  std::shared_ptr<core::ContentRepository> content_repo_;
   // thread pool for components.
   utils::ThreadPool<bool> component_lifecycle_thread_pool_;
   // controller service provider reference

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/Site2SiteClientProtocol.h
----------------------------------------------------------------------
diff --git a/libminifi/include/Site2SiteClientProtocol.h b/libminifi/include/Site2SiteClientProtocol.h
index 8d89004..dcb551a 100644
--- a/libminifi/include/Site2SiteClientProtocol.h
+++ b/libminifi/include/Site2SiteClientProtocol.h
@@ -549,7 +549,8 @@ class Site2SiteClientProtocol {
         : _packet(packet) {
     }
     DataPacket *_packet;
-    void process(std::ofstream *stream) {
+    //void process(std::ofstream *stream) {
+    int64_t process(std::shared_ptr<io::BaseStream> stream) {
       uint8_t buffer[8192];
       int len = _packet->_size;
       while (len > 0) {
@@ -557,11 +558,12 @@ class Site2SiteClientProtocol {
         int ret = _packet->_transaction->getStream().readData(buffer, size);
         if (ret != size) {
           _packet->_protocol->logger_->log_error("Site2Site Receive Flow Size %d Failed %d", size, ret);
-          break;
+          return -1;
         }
-        stream->write((const char *) buffer, size);
+        stream->write(buffer, size);
         len -= size;
       }
+      return len;
     }
   };
   // Nest Callback Class for read stream
@@ -571,22 +573,29 @@ class Site2SiteClientProtocol {
         : _packet(packet) {
     }
     DataPacket *_packet;
-    void process(std::ifstream *stream) {
+    int64_t process(std::shared_ptr<io::BaseStream> stream) {
       _packet->_size = 0;
-      uint8_t buffer[8192];
+      uint8_t buffer[8192] = { 0 };
       int readSize;
-      while (stream->good()) {
-        if (!stream->read((char *) buffer, 8192))
-          readSize = stream->gcount();
-        else
-          readSize = 8192;
+      size_t size = 0;
+      do {
+        readSize = stream->read(buffer, 8192);
+
+        if (readSize == 0) {
+          break;
+        }
+        if (readSize < 0) {
+          return -1;
+        }
         int ret = _packet->_transaction->getStream().writeData(buffer, readSize);
         if (ret != readSize) {
           _packet->_protocol->logger_->log_error("Site2Site Send Flow Size %d Failed %d", readSize, ret);
-          break;
+          return -1;
         }
-        _packet->_size += readSize;
-      }
+        size += readSize;
+      } while (size < stream->getSize());
+      _packet->_size = size;
+      return size;
     }
   };
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/ThreadedSchedulingAgent.h
----------------------------------------------------------------------
diff --git a/libminifi/include/ThreadedSchedulingAgent.h b/libminifi/include/ThreadedSchedulingAgent.h
index 50ab6c9..b4db4bf 100644
--- a/libminifi/include/ThreadedSchedulingAgent.h
+++ b/libminifi/include/ThreadedSchedulingAgent.h
@@ -42,8 +42,11 @@ class ThreadedSchedulingAgent : public SchedulingAgent {
   /*!
    * Create a new threaded scheduling agent.
    */
-  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),
+  ThreadedSchedulingAgent(std::shared_ptr<core::controller::ControllerServiceProvider> controller_service_provider, std::shared_ptr<core::Repository> repo,
+                          std::shared_ptr<core::Repository> flow_repo,
+                          std::shared_ptr<core::ContentRepository> content_repo,
+                          std::shared_ptr<Configure> configuration)
+      : SchedulingAgent(controller_service_provider, repo, flow_repo, content_repo, configuration),
         logger_(logging::LoggerFactory<ThreadedSchedulingAgent>::getLogger()) {
   }
   // Destructor

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/TimerDrivenSchedulingAgent.h
----------------------------------------------------------------------
diff --git a/libminifi/include/TimerDrivenSchedulingAgent.h b/libminifi/include/TimerDrivenSchedulingAgent.h
index 597dc76..816bcec 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::controller::ControllerServiceProvider> controller_service_provider, std::shared_ptr<core::Repository> repo, std::shared_ptr<Configure> configure)
-      : ThreadedSchedulingAgent(controller_service_provider, repo, configure) {
+  TimerDrivenSchedulingAgent(std::shared_ptr<core::controller::ControllerServiceProvider> controller_service_provider, std::shared_ptr<core::Repository> repo,
+                             std::shared_ptr<core::Repository> flow_repo,
+                             std::shared_ptr<core::ContentRepository> content_repo,
+                             std::shared_ptr<Configure> configure)
+      : ThreadedSchedulingAgent(controller_service_provider, repo, flow_repo, content_repo, configure) {
   }
   //  Destructor
   virtual ~TimerDrivenSchedulingAgent() {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/core/ConfigurationFactory.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ConfigurationFactory.h b/libminifi/include/core/ConfigurationFactory.h
index b58c170..61af8cd 100644
--- a/libminifi/include/core/ConfigurationFactory.h
+++ b/libminifi/include/core/ConfigurationFactory.h
@@ -30,6 +30,7 @@ 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::shared_ptr<core::ContentRepository> &content_repo,
                                                                            std::shared_ptr<Configure> configuration,
                                                                            const std::string path) {
   throw std::runtime_error("Cannot instantiate class");
@@ -37,16 +38,20 @@ typename std::enable_if<!class_operations<T>::value, T*>::type instantiate(const
 
 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::shared_ptr<core::ContentRepository> &content_repo,
                                                                           const std::shared_ptr<io::StreamFactory> &stream_factory,
-                                                                          std::shared_ptr<Configure> configuration, const std::string path) {
-  return new T(repo, flow_file_repo, stream_factory, configuration, path);
+                                                                          std::shared_ptr<Configure> configuration,
+                                                                          const std::string path) {
+  return new T(repo, flow_file_repo, content_repo, stream_factory, configuration, path);
 }
 
 /**
  * Configuration factory is used to create a new FlowConfiguration
  * object.
  */
-std::unique_ptr<core::FlowConfiguration> createFlowConfiguration(std::shared_ptr<core::Repository> repo, std::shared_ptr<core::Repository> flow_file_repo, std::shared_ptr<Configure> configure,
+std::unique_ptr<core::FlowConfiguration> createFlowConfiguration(std::shared_ptr<core::Repository> repo, std::shared_ptr<core::Repository> flow_file_repo,
+                                                                 std::shared_ptr<core::ContentRepository> content_repo,
+                                                                 std::shared_ptr<Configure> configure,
                                                                  std::shared_ptr<io::StreamFactory> stream_factory,
                                                                  const std::string configuration_class_name, const std::string path = "",
                                                                  bool fail_safe = false);

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/core/ContentRepository.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ContentRepository.h b/libminifi/include/core/ContentRepository.h
new file mode 100644
index 0000000..b544ca0
--- /dev/null
+++ b/libminifi/include/core/ContentRepository.h
@@ -0,0 +1,60 @@
+/**
+ *
+ * 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_CONTENTREPOSITORY_H_
+#define LIBMINIFI_INCLUDE_CORE_CONTENTREPOSITORY_H_
+
+#include "properties/Configure.h"
+#include "ResourceClaim.h"
+#include "io/DataStream.h"
+#include "io/BaseStream.h"
+#include "StreamManager.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+/**
+ * Content repository definition that extends StreamManager.
+ */
+class ContentRepository : public StreamManager<minifi::ResourceClaim> {
+ public:
+  virtual ~ContentRepository() {
+
+  }
+
+  /**
+   * initialize this content repository using the provided configuration.
+   */
+  virtual bool initialize(const std::shared_ptr<Configure> &configure) = 0;
+
+  /**
+   * Stops this repository.
+   */
+  virtual void stop() = 0;
+
+};
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_CONTENTREPOSITORY_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/core/Core.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/Core.h b/libminifi/include/core/Core.h
index c32eb59..1dc79e7 100644
--- a/libminifi/include/core/Core.h
+++ b/libminifi/include/core/Core.h
@@ -19,6 +19,7 @@
 #define LIBMINIFI_INCLUDE_CORE_CORE_H_
 
 #include <cstdlib>
+#include <iostream>
 #include <memory>
 #include <string>
 #include <uuid/uuid.h>
@@ -132,6 +133,8 @@ class CoreComponent {
    */
   void setUUID(uuid_t uuid);
 
+  void setUUIDStr(const std::string uuidStr);
+
   /**
    * Returns the UUID through the provided object.
    * @param uuid uuid struct to which we will copy the memory

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/core/FlowConfiguration.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/FlowConfiguration.h b/libminifi/include/core/FlowConfiguration.h
index 3429166..43d2bc0 100644
--- a/libminifi/include/core/FlowConfiguration.h
+++ b/libminifi/include/core/FlowConfiguration.h
@@ -58,11 +58,12 @@ class FlowConfiguration : public CoreComponent {
    * Constructor that will be used for configuring
    * the flow controller.
    */
-  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)
+  explicit FlowConfiguration(std::shared_ptr<core::Repository> repo, std::shared_ptr<core::Repository> flow_file_repo, std::shared_ptr<core::ContentRepository> content_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),
+        content_repo_(content_repo),
         config_path_(path),
         stream_factory_(stream_factory),
         configuration_(configuration),
@@ -76,8 +77,9 @@ class FlowConfiguration : public CoreComponent {
   // Create Processor (Node/Input/Output Port) based on the name
   std::shared_ptr<core::Processor> createProcessor(std::string name, uuid_t uuid);
   // Create Root Processor Group
-  std::unique_ptr<core::ProcessGroup> createRootProcessGroup(std::string name,
-                                                             uuid_t uuid, int version);
+
+  std::unique_ptr<core::ProcessGroup> createRootProcessGroup(std::string name, uuid_t uuid, int version);
+
   std::shared_ptr<core::controller::ControllerServiceNode> createControllerService(const std::string &class_name, const std::string &name, uuid_t uuid);
 
   // Create Remote Processor Group
@@ -100,7 +102,7 @@ class FlowConfiguration : public CoreComponent {
   }
 
   virtual std::unique_ptr<core::ProcessGroup> getRootFromPayload(
-      std::string &yamlConfigPayload) {
+                                                                 std::string &yamlConfigPayload) {
     return nullptr;
   }
 
@@ -127,6 +129,8 @@ class FlowConfiguration : public CoreComponent {
   std::string config_path_;
   // flow file repo
   std::shared_ptr<core::Repository> flow_file_repo_;
+  // content repository.
+  std::shared_ptr<core::ContentRepository> content_repo_;
   // stream factory
   std::shared_ptr<io::StreamFactory> stream_factory_;
   std::shared_ptr<Configure> configuration_;

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/core/ProcessContext.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ProcessContext.h b/libminifi/include/core/ProcessContext.h
index 48e0108..f6aaf5e 100644
--- a/libminifi/include/core/ProcessContext.h
+++ b/libminifi/include/core/ProcessContext.h
@@ -25,8 +25,10 @@
 #include <mutex>
 #include <atomic>
 #include <algorithm>
-
+#include <memory>
 #include "Property.h"
+#include "core/ContentRepository.h"
+#include "core/repository/FileSystemRepository.h"
 #include "core/controller/ControllerServiceProvider.h"
 #include "core/controller/ControllerServiceLookup.h"
 #include "core/logging/LoggerConfiguration.h"
@@ -46,10 +48,13 @@ class ProcessContext : public controller::ControllerServiceLookup {
   /*!
    * Create a new process context associated with the processor/controller service/state manager
    */
-  ProcessContext(ProcessorNode &processor, std::shared_ptr<controller::ControllerServiceProvider> &controller_service_provider, std::shared_ptr<core::Repository> repo)
+  ProcessContext(ProcessorNode &processor, std::shared_ptr<controller::ControllerServiceProvider> &controller_service_provider, std::shared_ptr<core::Repository> repo,
+		 std::shared_ptr<core::Repository> flow_repo,
+                 std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::FileSystemRepository>())
       : processor_node_(processor),
         controller_service_provider_(controller_service_provider),
-        logger_(logging::LoggerFactory<ProcessContext>::getLogger()) {
+        logger_(logging::LoggerFactory<ProcessContext>::getLogger()),
+        content_repo_(content_repo), flow_repo_(flow_repo) {
     repo_ = repo;
   }
   // Destructor
@@ -92,6 +97,18 @@ class ProcessContext : public controller::ControllerServiceLookup {
     return repo_;
   }
 
+  /**
+   * Returns a reference to the content repository for the running instance.
+   * @return content repository shared pointer.
+   */
+  std::shared_ptr<core::ContentRepository> getContentRepository() {
+    return content_repo_;
+  }
+  
+  std::shared_ptr<core::Repository> getFlowFileRepository() {
+    return flow_repo_;
+  }
+
   // Prevent default copy constructor and assignment operation
   // Only support pass by reference or pointer
   ProcessContext(const ProcessContext &parent) = delete;
@@ -145,6 +162,10 @@ class ProcessContext : public controller::ControllerServiceLookup {
   std::shared_ptr<controller::ControllerServiceProvider> controller_service_provider_;
   // repository shared pointer.
   std::shared_ptr<core::Repository> repo_;
+  std::shared_ptr<core::Repository> flow_repo_;
+
+  // repository shared pointer.
+  std::shared_ptr<core::ContentRepository> content_repo_;
   // Processor
   ProcessorNode processor_node_;
   // Logger

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/core/ProcessGroup.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ProcessGroup.h b/libminifi/include/core/ProcessGroup.h
index 410480a..a0e51e3 100644
--- a/libminifi/include/core/ProcessGroup.h
+++ b/libminifi/include/core/ProcessGroup.h
@@ -56,8 +56,7 @@ class ProcessGroup {
   /*!
    * Create a new process group
    */
-  ProcessGroup(ProcessGroupType type, std::string name, uuid_t uuid = NULL, int version = 0,
-               ProcessGroup *parent = NULL);
+  ProcessGroup(ProcessGroupType type, std::string name, uuid_t uuid = NULL, int version = 0, ProcessGroup *parent = NULL);
   // Destructor
   virtual ~ProcessGroup();
   // Set Processor Name

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/core/ProcessSession.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ProcessSession.h b/libminifi/include/core/ProcessSession.h
index ad79d12..d853e9b 100644
--- a/libminifi/include/core/ProcessSession.h
+++ b/libminifi/include/core/ProcessSession.h
@@ -50,7 +50,7 @@ class ProcessSession {
   ProcessSession(ProcessContext *processContext = NULL)
       : process_context_(processContext),
         logger_(logging::LoggerFactory<ProcessSession>::getLogger()) {
-    logger_->log_trace("ProcessSession created for %s", process_context_->getProcessorNode().getName().c_str());
+    logger_->log_trace("ProcessSession created for %s", process_context_->getProcessorNode().getName());
     auto repo = processContext->getProvenanceRepository();
     provenance_report_ = new provenance::ProvenanceReporter(repo, process_context_->getProcessorNode().getUUIDStr(), process_context_->getProcessorNode().getName());
   }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/core/Repository.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/Repository.h b/libminifi/include/core/Repository.h
index 5f7e6c2..f1b47ae 100644
--- a/libminifi/include/core/Repository.h
+++ b/libminifi/include/core/Repository.h
@@ -31,7 +31,8 @@
 #include <string>
 #include <thread>
 #include <vector>
-
+#include "core/ContentRepository.h"
+#include "core/SerializableComponent.h"
 #include "properties/Configure.h"
 #include "core/logging/LoggerConfiguration.h"
 #include "core/Property.h"
@@ -52,15 +53,15 @@ namespace core {
 #define MAX_REPOSITORY_ENTRY_LIFE_TIME (600000) // 10 minute
 #define REPOSITORY_PURGE_PERIOD (2500) // 2500 msec
 
-class Repository : public CoreComponent {
+class Repository : public core::SerializableComponent {
  public:
   /*
    * Constructor for the repository
    */
   Repository(std::string repo_name = "Repository", std::string directory = REPOSITORY_DIRECTORY, int64_t maxPartitionMillis = MAX_REPOSITORY_ENTRY_LIFE_TIME, int64_t maxPartitionBytes =
-  MAX_REPOSITORY_STORAGE_SIZE,
+                 MAX_REPOSITORY_STORAGE_SIZE,
              uint64_t purgePeriod = REPOSITORY_PURGE_PERIOD)
-      : CoreComponent(repo_name),
+      : core::SerializableComponent(repo_name),
         thread_(),
         logger_(logging::LoggerFactory<Repository>::getLogger()) {
     directory_ = directory;
@@ -81,7 +82,7 @@ class Repository : public CoreComponent {
     return true;
   }
   // Put
-  virtual bool Put(std::string key, uint8_t *buf, int bufLen) {
+  virtual bool Put(std::string key, const uint8_t *buf, size_t bufLen) {
     return true;
   }
   // Delete
@@ -89,7 +90,14 @@ class Repository : public CoreComponent {
     return true;
   }
 
-  virtual bool Get(std::string key, std::string &value) {
+  virtual bool Delete(std::vector<std::shared_ptr<core::SerializableComponent>> &storedValues) {
+    bool found = true;
+    for (auto storedValue : storedValues) {
+      found &= Delete(storedValue->getName());
+    }
+    return found;
+  }
+  virtual bool Get(const std::string &key, std::string &value) {
     return false;
   }
 
@@ -109,10 +117,82 @@ class Repository : public CoreComponent {
   virtual bool isRunning() {
     return running_;
   }
+
+  /**
+   * Specialization that allows us to serialize max_size objects into store.
+   * the lambdaConstructor will create objects to put into store
+   * @param store vector in which we can store serialized object
+   * @param max_size reference that stores the max number of objects to retrieve and serialize.
+   * upon return max_size will represent the number of serialized objects.
+   * @return status of this operation
+   *
+   * Base implementation returns true;
+   */
+  virtual bool Serialize(std::vector<std::shared_ptr<core::SerializableComponent>> &store, size_t max_size) {
+    return true;
+  }
+
+  /**
+   * Specialization that allows us to deserialize max_size objects into store.
+   * @param store vector in which we can store deserialized object
+   * @param max_size reference that stores the max number of objects to retrieve and deserialize.
+   * upon return max_size will represent the number of deserialized objects.
+   * @return status of this operation
+   *
+   * Base implementation returns true;
+   */
+  virtual bool DeSerialize(std::vector<std::shared_ptr<core::SerializableComponent>> &store, size_t &max_size) {
+    return true;
+  }
+
+  /**
+   * Specialization that allows us to deserialize max_size objects into store.
+   * the lambdaConstructor will create objects to put into store
+   * @param store vector in which we can store deserialized object
+   * @param max_size reference that stores the max number of objects to retrieve and deserialize.
+   * upon return max_size will represent the number of deserialized objects.
+   * @param lambdaConstructor reference that will create the objects for store
+   * @return status of this operation
+   *
+   * Base implementation returns true;
+   */
+  virtual bool DeSerialize(std::vector<std::shared_ptr<core::SerializableComponent>> &store, size_t &max_size, std::function<std::shared_ptr<core::SerializableComponent>()> lambdaConstructor) {
+    return true;
+  }
+
+  /**
+   * Base implementation returns true;
+   */
+  virtual bool Serialize(const std::shared_ptr<core::SerializableComponent> &store) {
+    return true;
+  }
+
+  /**
+   * Base implementation returns true;
+   */
+  virtual bool DeSerialize(const std::shared_ptr<core::SerializableComponent> &store) {
+    return true;
+  }
+
+  /**
+   * Base implementation returns true;
+   */
+  virtual bool DeSerialize(const uint8_t *buffer, const size_t bufferSize) {
+    return true;
+  }
+
+  virtual bool Serialize(const std::string &key, const uint8_t *buffer, const size_t bufferSize) {
+    return Put(key, buffer, bufferSize);
+  }
+
   uint64_t incrementSize(const char *fpath, const struct stat *sb, int typeflag) {
     return (repo_size_ += sb->st_size);
   }
 
+  virtual void loadComponent(const std::shared_ptr<core::ContentRepository> &content_repo) {
+
+  }
+
   // Prevent default copy constructor and assignment operation
   // Only support pass by reference or pointer
   Repository(const Repository &parent) = delete;

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/core/RepositoryFactory.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/RepositoryFactory.h b/libminifi/include/core/RepositoryFactory.h
index 9fafb57..b123a6d 100644
--- a/libminifi/include/core/RepositoryFactory.h
+++ b/libminifi/include/core/RepositoryFactory.h
@@ -19,8 +19,8 @@
 #ifndef LIBMINIFI_INCLUDE_CORE_REPOSITORYFACTORY_H_
 #define LIBMINIFI_INCLUDE_CORE_REPOSITORYFACTORY_H_
 
+#include "core/ContentRepository.h"
 #include "core/Repository.h"
-#include "core/repository/VolatileRepository.h"
 #include "Core.h"
 
 namespace org {
@@ -30,8 +30,22 @@ namespace minifi {
 
 namespace core {
 
+/**
+ * Create a repository represented by the configuration class name
+ * @param configuration_class_name configuration class name
+ * @param fail_safe determines whether or not to make the default class if configuration_class_name is invalid
+ * @param repo_name name of the repository
+ */
 std::shared_ptr<core::Repository> createRepository(const std::string configuration_class_name, bool fail_safe = false, const std::string repo_name = "");
 
+/**
+ * Create a context repository
+ * @param configuration_class_name configuration class name
+ * @param fail_safe determines whether or not to make the default class if configuration_class_name is invalid
+ * @param repo_name name of the repository
+ */
+std::shared_ptr<core::ContentRepository> createContentRepository(const std::string configuration_class_name, bool fail_safe = false, const std::string repo_name = "");
+
 } /* namespace core */
 } /* namespace minifi */
 } /* namespace nifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/core/SerializableComponent.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/SerializableComponent.h b/libminifi/include/core/SerializableComponent.h
new file mode 100644
index 0000000..f7f9feb
--- /dev/null
+++ b/libminifi/include/core/SerializableComponent.h
@@ -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.
+ */
+#ifndef LIBMINIFI_INCLUDE_CORE_SERIALIZABLECOMPONENT_H_
+#define LIBMINIFI_INCLUDE_CORE_SERIALIZABLECOMPONENT_H_
+
+#include "io/Serializable.h"
+#include "core/Core.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+/**
+ * Represents a component that is serializable and an extension point of core Component
+ */
+class SerializableComponent : public core::CoreComponent, public minifi::io::Serializable {
+
+ public:
+
+  SerializableComponent(const std::string name, uuid_t uuid = nullptr)
+      : core::CoreComponent(name, uuid) {
+
+  }
+
+  virtual ~SerializableComponent() {
+
+  }
+
+  /**
+   * Serialize this object into the the store
+   * @param store object in which we are serializing data into
+   * @return status of this serialization.
+   */
+  virtual bool Serialize(const std::shared_ptr<core::SerializableComponent> &store) = 0;
+
+  /**
+   * Deserialization from the parameter store into the current object
+   * @param store from which we are deserializing the current object
+   * @return status of this deserialization.
+   */
+  virtual bool DeSerialize(const std::shared_ptr<core::SerializableComponent> &store) = 0;
+
+  /**
+   * Deserializes the current object using buffer
+   * @param buffer buffer from which we can deserialize the currenet object
+   * @param bufferSize length of buffer from which we can deserialize the current object.
+   * @return status of the deserialization.
+   */
+  virtual bool DeSerialize(const uint8_t *buffer, const size_t bufferSize) = 0;
+
+  /**
+   * Serialization of this object into buffer
+   * @param key string that represents this objects identifier
+   * @param buffer buffer that contains the serialized object
+   * @param bufferSize length of buffer
+   * @return status of serialization
+   */
+  virtual bool Serialize(const std::string &key, const uint8_t *buffer, const size_t bufferSize) {
+    return false;
+  }
+
+};
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_SERIALIZABLECOMPONENT_H_ */
+

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/core/StreamManager.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/StreamManager.h b/libminifi/include/core/StreamManager.h
new file mode 100644
index 0000000..468526d
--- /dev/null
+++ b/libminifi/include/core/StreamManager.h
@@ -0,0 +1,81 @@
+/**
+ *
+ * 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_STREAMMANAGER_H_
+#define LIBMINIFI_INCLUDE_CORE_STREAMMANAGER_H_
+
+#include "properties/Configure.h"
+#include "ResourceClaim.h"
+#include "io/DataStream.h"
+#include "io/BaseStream.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+/**
+ * Purpose: Provides a base for all stream based managers. The goal here is to provide
+ * a small set of interfaces that provide a small set of operations to provide state 
+ * management for streams.
+ */
+template<typename T>
+class StreamManager {
+ public:
+  virtual ~StreamManager() {
+
+  }
+
+  /**
+   * Create a write stream using the streamId as a reference.
+   * @param streamId stream identifier
+   * @return stream pointer.
+   */
+  virtual std::shared_ptr<io::BaseStream> write(const std::shared_ptr<T> &streamId) = 0;
+
+  /**
+   * Create a read stream using the streamId as a reference.
+   * @param streamId stream identifier
+   * @return stream pointer.
+   */
+  virtual std::shared_ptr<io::BaseStream> read(const std::shared_ptr<T> &streamId) = 0;
+
+  /**
+   * Closes the stream
+   * @param streamId stream identifier
+   * @return result of operation.
+   */
+  virtual bool close(const std::shared_ptr<T> &streamId) = 0;
+
+  /**
+   * Removes the stream from this stream manager. The end result
+   * is dependent on the stream manager implementation.
+   * @param streamId stream identifier
+   * @return result of operation.
+   */
+  virtual bool remove(const std::shared_ptr<T> &streamId) = 0;
+
+};
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_STREAMMANAGER_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/core/logging/LoggerConfiguration.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/logging/LoggerConfiguration.h b/libminifi/include/core/logging/LoggerConfiguration.h
index aa4a1d0..787fec5 100644
--- a/libminifi/include/core/logging/LoggerConfiguration.h
+++ b/libminifi/include/core/logging/LoggerConfiguration.h
@@ -105,8 +105,7 @@ class LoggerConfiguration {
    protected:
   static std::shared_ptr<internal::LoggerNamespace> initialize_namespaces(const std::shared_ptr<LoggerProperties> &logger_properties);
   static std::shared_ptr<spdlog::logger> get_logger(std::shared_ptr<Logger> logger, const std::shared_ptr<internal::LoggerNamespace> &root_namespace, const std::string &name,
-                                                    std::shared_ptr<spdlog::formatter> formatter,
-                                                    bool remove_if_present = false);
+                                                    std::shared_ptr<spdlog::formatter> formatter, bool remove_if_present = false);
    private:
   static std::shared_ptr<internal::LoggerNamespace> create_default_root();
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/core/reporting/SiteToSiteProvenanceReportingTask.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/reporting/SiteToSiteProvenanceReportingTask.h b/libminifi/include/core/reporting/SiteToSiteProvenanceReportingTask.h
index e1d80e8..2bd4099 100644
--- a/libminifi/include/core/reporting/SiteToSiteProvenanceReportingTask.h
+++ b/libminifi/include/core/reporting/SiteToSiteProvenanceReportingTask.h
@@ -54,7 +54,6 @@ class SiteToSiteProvenanceReportingTask : public minifi::RemoteProcessorGroupPor
   }
   //! Destructor
   virtual ~SiteToSiteProvenanceReportingTask() {
-
   }
   //! Report Task Name
   static constexpr char const* ReportTaskName = "SiteToSiteProvenanceReportingTask";
@@ -62,7 +61,8 @@ class SiteToSiteProvenanceReportingTask : public minifi::RemoteProcessorGroupPor
 
  public:
   //! Get provenance json report
-  void getJsonReport(core::ProcessContext *context, core::ProcessSession *session, std::vector<std::shared_ptr<provenance::ProvenanceEventRecord>> &records, std::string &report);
+  void getJsonReport(core::ProcessContext *context, core::ProcessSession *session, std::vector<std::shared_ptr<core::SerializableComponent>> &records, std::string &report);
+
   void onSchedule(core::ProcessContext *context, core::ProcessSessionFactory *sessionFactory);
   //! OnTrigger method, implemented by NiFi SiteToSiteProvenanceReportingTask
   virtual void onTrigger(core::ProcessContext *context, core::ProcessSession *session);

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/core/repository/AtomicRepoEntries.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/repository/AtomicRepoEntries.h b/libminifi/include/core/repository/AtomicRepoEntries.h
new file mode 100644
index 0000000..c681060
--- /dev/null
+++ b/libminifi/include/core/repository/AtomicRepoEntries.h
@@ -0,0 +1,501 @@
+/**
+ *
+ * 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 ref_count_hip.
+ * 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_REPOSITORY_ATOMICREPOENTRIES_H_
+#define LIBMINIFI_INCLUDE_CORE_REPOSITORY_ATOMICREPOENTRIES_H_
+
+#include  <cstddef>
+#include <cstring>
+#include <iostream>
+#include <chrono>
+#include <functional>
+#include <atomic>
+#include <vector>
+#include <map>
+#include <iterator>
+
+static uint16_t accounting_size = sizeof(std::vector<uint8_t>) + sizeof(std::string) + sizeof(size_t);
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+namespace repository {
+
+/**
+ * Purpose: Repo value represents an item that will support a move operation within an AtomicEntry
+ *
+ * Justification: Since AtomicEntry is a static entry that does not move or change, the underlying
+ * RepoValue can be changed to support atomic operations.
+ */
+template<typename T>
+class RepoValue {
+ public:
+
+  explicit RepoValue() {
+  }
+
+  /**
+   * Constructor that populates the item allowing for a custom key comparator.
+   * @param key key for this repo value.
+   * @param ptr buffer
+   * @param size size buffer
+   * @param comparator custom comparator.
+   */
+  explicit RepoValue(T key, const uint8_t *ptr, size_t size, std::function<bool(T, T)> comparator = nullptr)
+      : key_(key),
+        comparator_(comparator) {
+    if (nullptr == ptr) {
+      size = 0;
+    }
+    buffer_.resize(size);
+    if (size > 0) {
+      std::memcpy(buffer_.data(), ptr, size);
+    }
+  }
+
+  /**
+   * RepoValue that moves the other object into this.
+   */
+  explicit RepoValue(RepoValue<T> &&other)
+noexcept      : key_(std::move(other.key_)),
+      buffer_(std::move(other.buffer_)),
+      comparator_(std::move(other.comparator_)) {
+      }
+
+      ~RepoValue()
+      {
+      }
+
+      T &getKey() {
+        return key_;
+      }
+
+      /**
+       * Sets the key, relacing the custom comparator if needed.
+       */
+      void setKey(const T key, std::function<bool(T,T)> comparator = nullptr) {
+        key_ = key;
+        comparator_ = comparator;
+      }
+
+      /**
+       * Determines if the key is the same using the custom comparator
+       * @param other object to compare against
+       * @return result of the comparison
+       */
+      inline bool isEqual(RepoValue<T> *other)
+      {
+        return comparator_ == nullptr ? key_ == other->key_ : comparator_(key_,other->key_);
+      }
+
+      /**
+       * Determines if the key is the same using the custom comparator
+       * @param other object to compare against
+       * @return result of the comparison
+       */
+      inline bool isKey(T other)
+      {
+        return comparator_ == nullptr ? key_ == other : comparator_(key_,other);
+      }
+
+      /**
+       * Clears the buffer.
+       */
+      void clearBuffer() {
+        buffer_.resize(0);
+        buffer_.clear();
+      }
+
+      /**
+       * Return the size of the memory within the key
+       * buffer, the size of timestamp, and the general
+       * system word size
+       */
+      uint64_t size() {
+        return buffer_.size();
+      }
+
+      size_t getBufferSize() {
+        return buffer_.size();
+      }
+
+      const uint8_t *getBuffer()
+      {
+        return buffer_.data();
+      }
+
+      /**
+       * Places the contents of buffer into str
+       * @param strnig into which we are placing the memory contained in buffer.
+       */
+      void emplace(std::string &str) {
+        str.insert(0, reinterpret_cast<const char*>(buffer_.data()), buffer_.size());
+      }
+
+      /**
+       * Appends ptr to the end of buffer.
+       * @param ptr pointer containing data to add to buffer_
+       */
+      void append(uint8_t *ptr, size_t size)
+      {
+        buffer_.insert(buffer_.end(), ptr, ptr + size);
+      }
+
+      RepoValue<T> &operator=(RepoValue<T> &&other) noexcept {
+        key_ = std::move(other.key_);
+        buffer_ = std::move(other.buffer_);
+        return *this;
+      }
+
+    private:
+      T key_;
+      std::function<bool(T,T)> comparator_;
+      std::vector<uint8_t> buffer_;
+    };
+
+    /**
+     * Purpose: Atomic Entry allows us to create a statically
+     * sized ring buffer, with the ability to create
+     *
+     **/
+template<typename T>
+class AtomicEntry {
+
+ public:
+  /**
+   * Constructor that accepts a max size and an atomic counter for the total
+   * size allowd by this and other atomic entries.
+   */
+  explicit AtomicEntry(std::atomic<size_t> *total_size, size_t *max_size)
+      : write_pending_(false),
+        has_value_(false),
+        accumulated_repo_size_(total_size),
+        max_repo_size_(max_size),
+        ref_count_(0),
+        free_required(false) {
+
+  }
+
+  /**
+   * Sets the repo value, moving the old value into old_value.
+   * @param new_value new value to move into value_.
+   * @param old_value the previous value of value_ will be moved into old_value
+   * @param prev_size size reclaimed.
+   * @return result of this set. If true old_value will be populated.
+   */
+  bool setRepoValue(RepoValue<T> &new_value, RepoValue<T> &old_value, size_t &prev_size) {
+    // delete the underlying pointer
+    bool lock = false;
+    if (!write_pending_.compare_exchange_weak(lock, true))
+    {
+      return false;
+    }
+    if (has_value_) {
+      prev_size = value_.size();
+    }
+    old_value = std::move(value_);
+    value_ = std::move(new_value);
+    has_value_ = true;
+    try_unlock();
+    return true;
+  }
+  
+  
+  AtomicEntry<T> *takeOwnership()
+  {
+      bool lock = false;
+      if (!write_pending_.compare_exchange_weak(lock, true) )
+	return nullptr;
+      
+      ref_count_++;
+      
+      try_unlock();
+      
+      return this;
+  }
+  /**
+   * A test and set operation, which is used to allow a function to test
+   * if an item can be released and a function used for reclaiming memory associated
+   * with said object.
+   * A custom comparator can be provided to augment the key being added into value_
+   */
+  bool testAndSetKey(const T str, std::function<bool(T)> releaseTest = nullptr, std::function<void(T)> reclaimer = nullptr, std::function<bool(T, T)> comparator = nullptr) {
+    bool lock = false;
+
+    if (!write_pending_.compare_exchange_weak(lock, true) )
+      return false;
+
+    if (has_value_) {
+      // we either don't have a release test or we cannot release this
+      // entity
+      if (releaseTest != nullptr && reclaimer != nullptr && releaseTest(value_.getKey()))
+                                                                        {
+        reclaimer(value_.getKey());
+      }
+      else if (free_required && ref_count_ == 0)
+      {
+	size_t bufferSize = value_.getBufferSize();
+	value_.clearBuffer();
+	has_value_ = false;
+	if (accumulated_repo_size_ != nullptr) {
+	  *accumulated_repo_size_ -= bufferSize;
+	}
+	free_required = false;  
+      }
+      else {
+        try_unlock();
+        return false;
+      }
+
+    }
+    ref_count_=1;
+    value_.setKey(str, comparator);
+    has_value_ = true;
+    try_unlock();
+    return true;
+  }
+
+  /**
+   * Moved the value into the argument
+   * @param value the previous value will be moved into this parameter
+   * @return  success of get operation based on whether or not this atomic entry has a value.
+   */
+  bool getValue(RepoValue<T> &value) {
+    try_lock();
+    if (!has_value_) {
+      try_unlock();
+      return false;
+    }
+    value = std::move(value_);
+    has_value_ = false;
+    try_unlock();
+    return true;
+  }
+
+  /**
+   * Moved the value into the argument
+   * @param value the previous value will be moved into this parameter
+   * @return  success of get operation based on whether or not this atomic entry has a value.
+   */
+  bool getValue(const T &key, RepoValue<T> &value) {
+    try_lock();
+    if (!has_value_) {
+      try_unlock();
+      return false;
+    }
+    if (!value_.isKey(key)) {
+      try_unlock();
+      return false;
+    }
+    value = std::move(value_);
+    has_value_ = false;
+    try_unlock();
+    return true;
+  }
+  
+  void decrementOwnership(){
+    try_lock();
+    if (!has_value_) {
+      try_unlock();
+      return;
+    }
+    if (ref_count_ > 0){
+      ref_count_--;
+    }
+    if (ref_count_ == 0 && free_required)
+    {
+      size_t bufferSize = value_.getBufferSize();
+      value_.clearBuffer();
+      has_value_ = false;
+      if (accumulated_repo_size_ != nullptr) {
+	*accumulated_repo_size_ -= bufferSize;
+      }
+      free_required = false;
+    }
+    else{
+    }
+    try_unlock();
+  }
+
+  /**
+   * Moved the value into the argument
+   * @param value the previous value will be moved into this parameter
+   * @return  success of get operation based on whether or not this atomic entry has a value.
+   */
+  bool getValue(const T &key, RepoValue<T> **value) {
+    try_lock();
+    if (!has_value_) {
+      try_unlock();
+      return false;
+    }
+    if (!value_.isKey(key)) {
+      try_unlock();
+      return false;
+    }
+    ref_count_++;
+    *value = &value_;
+    try_unlock();
+    return true;
+  }
+
+  /**
+   * Operation that will be used to test and free if a release is required without
+   * setting a new object.
+   * @param releaseTest function that will be used to free the RepoValue key from
+   * this atomic entry.
+   * @param freedValue informs the caller if an item was freed.
+   */
+  T testAndFree(std::function<bool(T)> releaseTest, bool &freedValue) {
+    try_lock();
+    T ref;
+    if (!has_value_) {
+      try_unlock();
+      return ref;
+    }
+
+    if (releaseTest(value_.getKey())) {
+      size_t bufferSize = value_.getBufferSize();
+      value_.clearBuffer();
+      ref = value_.getKey();
+      has_value_ = false;
+      if (accumulated_repo_size_ != nullptr) {
+        *accumulated_repo_size_ -= bufferSize;
+      }
+
+    }
+    try_unlock();
+    return ref;
+  }
+  
+  size_t getLength()
+  {
+    size_t size = 0;
+     try_lock();
+     size = value_.getBufferSize();
+     try_unlock();
+     return size;
+     
+  }
+
+  /**
+   * sets has_value to false; however, does not call
+   * any external entity to further free RepoValue
+   */
+  bool freeValue(const T &key) {
+    try_lock();
+    if (!has_value_) {
+      try_unlock();
+      return false;
+    }
+    if (!value_.isKey(key)) {
+      try_unlock();
+      return false;
+    }
+    if (ref_count_ > 0)
+    {
+       free_required = true;
+       try_unlock();
+       return true;
+    }
+    size_t bufferSize = value_.getBufferSize();
+    value_.clearBuffer();
+    has_value_ = false;
+    if (accumulated_repo_size_ != nullptr) {
+      *accumulated_repo_size_ -= bufferSize;
+    }
+    free_required = false;
+    try_unlock();
+    return true;
+  }
+
+  /**
+   * Appends buffer onto this atomic entry if key matches
+   * the current RepoValue's key.
+   */
+  bool insert(const T key, uint8_t *buffer, size_t size) {
+    try_lock();
+
+    if (!has_value_) {
+      try_unlock();
+      return false;
+    }
+
+    if (!value_.isKey(key)) {
+      try_unlock();
+      return false;
+    }
+
+    if ((accumulated_repo_size_ != nullptr && max_repo_size_ != nullptr) && (*accumulated_repo_size_ + size > *max_repo_size_)) {
+      // can't support this write
+      try_unlock();
+      return false;
+    }
+
+    value_.append(buffer, size);
+    (*accumulated_repo_size_) += size;
+    try_unlock();
+    return true;
+  }
+
+ private:
+
+  /**
+   * Spin lock to unlock the current atomic entry.
+   */
+  inline void try_lock() {
+    bool lock = false;
+    while (!write_pending_.compare_exchange_weak(lock, true,std::memory_order_acquire)) {
+      lock = false;
+      // attempt again
+    }
+  }
+
+  /**
+   * Spin lock to unlock the current atomic entry.
+   */
+  inline void try_unlock() {
+    bool lock = true;
+    while (!write_pending_.compare_exchange_weak(lock, false,std::memory_order_acquire)) {
+      lock = true;
+      // attempt again
+    }
+  }
+
+  // atomic size pointer.
+  std::atomic<size_t> *accumulated_repo_size_;
+  // max size
+  size_t *max_repo_size_;
+  // determines if a write is pending.
+  std::atomic<bool> write_pending_;
+  // used to determine if a value is present in this atomic entry.
+  std::atomic<bool> has_value_;
+  std::atomic<uint16_t> ref_count_;
+  std::atomic<bool> free_required;
+  // repo value.
+  RepoValue<T> value_;
+};
+
+} /* namespace repository */
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_REPOSITORY_ATOMICREPOENTRIES_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/core/repository/FileSystemRepository.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/repository/FileSystemRepository.h b/libminifi/include/core/repository/FileSystemRepository.h
new file mode 100644
index 0000000..84bf01e
--- /dev/null
+++ b/libminifi/include/core/repository/FileSystemRepository.h
@@ -0,0 +1,72 @@
+/**
+ *
+ * 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_REPOSITORY_FileSystemRepository_H_
+#define LIBMINIFI_INCLUDE_CORE_REPOSITORY_FileSystemRepository_H_
+
+#include "core/Core.h"
+#include "../ContentRepository.h"
+#include "properties/Configure.h"
+#include "core/logging/LoggerConfiguration.h"
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+namespace repository {
+
+/**
+ * FileSystemRepository is a content repository that stores data onto the local file system.
+ */
+class FileSystemRepository : public core::ContentRepository, public core::CoreComponent {
+ public:
+  FileSystemRepository(std::string name = getClassName<FileSystemRepository>())
+      : core::CoreComponent(name),
+        logger_(logging::LoggerFactory<FileSystemRepository>::getLogger()) {
+
+  }
+  virtual ~FileSystemRepository() {
+
+  }
+
+  virtual bool initialize(const std::shared_ptr<minifi::Configure> &configuration);
+
+  virtual void stop();
+
+  virtual std::shared_ptr<io::BaseStream> write(const std::shared_ptr<minifi::ResourceClaim> &claim);
+
+  virtual std::shared_ptr<io::BaseStream> read(const std::shared_ptr<minifi::ResourceClaim> &claim);
+
+  virtual bool close(const std::shared_ptr<minifi::ResourceClaim> &claim) {
+    return remove(claim);
+  }
+
+  virtual bool remove(const std::shared_ptr<minifi::ResourceClaim> &claim);
+
+ private:
+
+  std::shared_ptr<logging::Logger> logger_;
+};
+
+} /* namespace repository */
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_REPOSITORY_FileSystemRepository_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/core/repository/FlowFileRepository.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/repository/FlowFileRepository.h b/libminifi/include/core/repository/FlowFileRepository.h
index 2e19286..28b9c05 100644
--- a/libminifi/include/core/repository/FlowFileRepository.h
+++ b/libminifi/include/core/repository/FlowFileRepository.h
@@ -48,12 +48,10 @@ class FlowFileRepository : public core::Repository, public std::enable_shared_fr
   // Constructor
 
   FlowFileRepository(const std::string repo_name = "", std::string directory = FLOWFILE_REPOSITORY_DIRECTORY, int64_t maxPartitionMillis = MAX_FLOWFILE_REPOSITORY_ENTRY_LIFE_TIME,
-                     int64_t maxPartitionBytes = MAX_FLOWFILE_REPOSITORY_STORAGE_SIZE,
-                     uint64_t purgePeriod = FLOWFILE_REPOSITORY_PURGE_PERIOD)
+                     int64_t maxPartitionBytes = MAX_FLOWFILE_REPOSITORY_STORAGE_SIZE, uint64_t purgePeriod = FLOWFILE_REPOSITORY_PURGE_PERIOD)
       : Repository(repo_name.length() > 0 ? repo_name : core::getClassName<FlowFileRepository>(), directory, maxPartitionMillis, maxPartitionBytes, purgePeriod),
-        logger_(logging::LoggerFactory<FlowFileRepository>::getLogger())
-
-  {
+        logger_(logging::LoggerFactory<FlowFileRepository>::getLogger()),
+        content_repo_(nullptr) {
     db_ = NULL;
   }
 
@@ -95,11 +93,12 @@ class FlowFileRepository : public core::Repository, public std::enable_shared_fr
 
   virtual void run();
 
-  virtual bool Put(std::string key, uint8_t *buf, int bufLen) {
+  virtual bool Put(std::string key, const uint8_t *buf, size_t bufLen) {
 
     // persistent to the DB
     leveldb::Slice value((const char *) buf, bufLen);
     leveldb::Status status;
+    repo_size_+=bufLen;
     status = db_->Put(leveldb::WriteOptions(), key, value);
     if (status.ok())
       return true;
@@ -115,7 +114,9 @@ class FlowFileRepository : public core::Repository, public std::enable_shared_fr
     leveldb::Status status;
     status = db_->Delete(leveldb::WriteOptions(), key);
     if (status.ok())
+    {
       return true;
+    }
     else
       return false;
   }
@@ -123,7 +124,7 @@ class FlowFileRepository : public core::Repository, public std::enable_shared_fr
    * Sets the value from the provided key
    * @return status of the get operation.
    */
-  virtual bool Get(std::string key, std::string &value) {
+  virtual bool Get(const std::string &key, std::string &value) {
     leveldb::Status status;
     status = db_->Get(leveldb::ReadOptions(), key, &value);
     if (status.ok())
@@ -135,21 +136,25 @@ class FlowFileRepository : public core::Repository, public std::enable_shared_fr
   void setConnectionMap(std::map<std::string, std::shared_ptr<minifi::Connection>> &connectionMap) {
     this->connectionMap = connectionMap;
   }
-  void loadComponent();
+  virtual void loadComponent(const std::shared_ptr<core::ContentRepository> &content_repo);
 
   void start() {
     if (this->purge_period_ <= 0)
+    {
       return;
+    }
     if (running_)
+    {
       return;
-    thread_ = std::thread(&FlowFileRepository::run, shared_from_this());
-    thread_.detach();
+    }
     running_ = true;
+    thread_ = std::thread(&FlowFileRepository::run, shared_from_this());
     logger_->log_info("%s Repository Monitor Thread Start", name_.c_str());
   }
 
  private:
   std::map<std::string, std::shared_ptr<minifi::Connection>> connectionMap;
+  std::shared_ptr<core::ContentRepository> content_repo_;
   leveldb::DB* db_;
   std::shared_ptr<logging::Logger> logger_;
 };

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/core/repository/VolatileContentRepository.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/repository/VolatileContentRepository.h b/libminifi/include/core/repository/VolatileContentRepository.h
new file mode 100644
index 0000000..306a812
--- /dev/null
+++ b/libminifi/include/core/repository/VolatileContentRepository.h
@@ -0,0 +1,138 @@
+/**
+ *
+ * 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_REPOSITORY_VolatileContentRepository_H_
+#define LIBMINIFI_INCLUDE_CORE_REPOSITORY_VolatileContentRepository_H_
+
+#include "core/Core.h"
+#include "AtomicRepoEntries.h"
+#include "io/AtomicEntryStream.h"
+#include "../ContentRepository.h"
+#include "core/repository/VolatileRepository.h"
+#include "properties/Configure.h"
+#include "core/logging/LoggerConfiguration.h"
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+namespace repository {
+
+/**
+ * Purpose: Stages content into a volatile area of memory. Note that   when the maximum number
+ * of entries is consumed we will rollback a session to wait for others to be freed.
+ */
+class VolatileContentRepository : public core::ContentRepository, public core::repository::VolatileRepository<std::shared_ptr<minifi::ResourceClaim>> {
+ public:
+
+  static const char *minimal_locking;
+
+  explicit VolatileContentRepository(std::string name = getClassName<VolatileContentRepository>())
+      : core::repository::VolatileRepository<std::shared_ptr<minifi::ResourceClaim>>(name),
+        logger_(logging::LoggerFactory<VolatileContentRepository>::getLogger()),
+        minimize_locking_(true) {
+    max_count_ = 15000;
+  }
+  virtual ~VolatileContentRepository() {
+    if (!minimize_locking_) {
+      std::lock_guard<std::mutex> lock(map_mutex_);
+      for (const auto &item : master_list_)
+      {
+        delete item.second;
+      }
+      master_list_.clear();
+    }
+
+  }
+
+  /**
+   * Initialize the volatile content repo
+   * @param configure configuration
+   */
+  virtual bool initialize(const std::shared_ptr<Configure> &configure);
+
+  /**
+   * Stop any thread associated with the volatile content repository.
+   */
+  virtual void stop();
+
+  /**
+   * Creates writable stream.
+   * @param claim resource claim
+   * @return BaseStream shared pointer that represents the stream the consumer will write to.
+   */
+  virtual std::shared_ptr<io::BaseStream> write(const std::shared_ptr<minifi::ResourceClaim> &claim);
+
+  /**
+   * Creates readable stream.
+   * @param claim resource claim
+   * @return BaseStream shared pointer that represents the stream from which the consumer will read..
+   */
+  virtual std::shared_ptr<io::BaseStream> read(const std::shared_ptr<minifi::ResourceClaim> &claim);
+
+  /**
+   * Closes the claim.
+   * @return whether or not the claim is associated with content stored in volatile memory.
+   */
+  virtual bool close(const std::shared_ptr<minifi::ResourceClaim> &claim) {
+    return remove(claim);
+  }
+
+  /**
+   * Closes the claim.
+   * @return whether or not the claim is associated with content stored in volatile memory.
+   */
+  virtual bool remove(const std::shared_ptr<minifi::ResourceClaim> &claim);
+
+ protected:
+  virtual void start();
+
+  virtual void run();
+
+  template<typename T2>
+  std::shared_ptr<T2> shared_from_parent()
+  {
+    return std::static_pointer_cast<T2>(shared_from_this());
+  }
+
+ private:
+
+  bool minimize_locking_;
+
+  // function pointers that are associated with the claims.
+  std::function<bool(std::shared_ptr<minifi::ResourceClaim>, std::shared_ptr<minifi::ResourceClaim>)> resource_claim_comparator_;
+  std::function<bool(std::shared_ptr<minifi::ResourceClaim>)> resource_claim_check_;
+  std::function<void(std::shared_ptr<minifi::ResourceClaim>)> claim_reclaimer_;
+
+  // logger
+  std::shared_ptr<logging::Logger> logger_;
+
+  // mutex and master list that represent a cache of Atomic entries. this exists so that we don't have to walk the atomic entry list.
+  // The idea is to reduce the computational complexity while keeping access as maximally lock free as we can.
+  std::mutex map_mutex_;
+
+  std::map<std::string, AtomicEntry<std::shared_ptr<minifi::ResourceClaim>>*> master_list_;
+};
+
+} /* namespace repository */
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_REPOSITORY_VolatileContentRepository_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/include/core/repository/VolatileFlowFileRepository.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/repository/VolatileFlowFileRepository.h b/libminifi/include/core/repository/VolatileFlowFileRepository.h
new file mode 100644
index 0000000..059c1de
--- /dev/null
+++ b/libminifi/include/core/repository/VolatileFlowFileRepository.h
@@ -0,0 +1,82 @@
+/**
+ *
+ * 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_REPOSITORY_VOLATILEFLOWFILEREPOSITORY_H_
+#define LIBMINIFI_INCLUDE_CORE_REPOSITORY_VOLATILEFLOWFILEREPOSITORY_H_
+
+#include "VolatileRepository.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+namespace repository {
+
+/**
+ * Volatile flow file repository. keeps a running counter of the current location, freeing
+ * those which we no longer hold.
+ */
+class VolatileFlowFileRepository : public VolatileRepository<std::string>
+{
+ public:
+  explicit VolatileFlowFileRepository(std::string repo_name = "", std::string dir = REPOSITORY_DIRECTORY, int64_t maxPartitionMillis = MAX_REPOSITORY_ENTRY_LIFE_TIME, int64_t maxPartitionBytes =
+  MAX_REPOSITORY_STORAGE_SIZE,
+                                      uint64_t purgePeriod = REPOSITORY_PURGE_PERIOD)
+      : VolatileRepository(repo_name.length() > 0 ? repo_name : core::getClassName<VolatileRepository>(), "", maxPartitionMillis, maxPartitionBytes, purgePeriod)
+
+  {
+    purge_required_ = true;
+    content_repo_ = nullptr;
+  }
+
+  virtual void run() {
+    repo_full_ = false;
+    while (running_) {
+      std::this_thread::sleep_for(std::chrono::milliseconds(purge_period_));
+      if (purge_required_ && nullptr != content_repo_)
+          {
+        std::lock_guard<std::mutex> lock(purge_mutex_);
+        for (auto purgeItem : purge_list_)
+        {
+          std::shared_ptr<minifi::ResourceClaim> newClaim = std::make_shared<minifi::ResourceClaim>(purgeItem, content_repo_, true);
+          content_repo_->remove(newClaim);
+        }
+        purge_list_.resize(0);
+        purge_list_.clear();
+      }
+    }
+  }
+
+  void loadComponent(const std::shared_ptr<core::ContentRepository> &content_repo) {
+    content_repo_ = content_repo;
+
+  }
+
+ protected:
+
+  std::shared_ptr<core::ContentRepository> content_repo_;
+
+};
+} /* namespace repository */
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_REPOSITORY_VOLATILEFLOWFILEREPOSITORY_H_ */


[4/6] nifi-minifi-cpp git commit: MINIFI-249: Update prov repo to better abstract deser.

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/SchedulingAgent.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/SchedulingAgent.cpp b/libminifi/src/SchedulingAgent.cpp
index 24ba146..1060830 100644
--- a/libminifi/src/SchedulingAgent.cpp
+++ b/libminifi/src/SchedulingAgent.cpp
@@ -42,7 +42,7 @@ bool SchedulingAgent::hasWorkToDo(std::shared_ptr<core::Processor> processor) {
 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] {
+  std::function < bool() > f_ex = [serviceNode] {
     return serviceNode->enable();
   };
   // create a functor that will be submitted to the thread pool.
@@ -55,7 +55,7 @@ void SchedulingAgent::enableControllerService(std::shared_ptr<core::controller::
 
 void SchedulingAgent::disableControllerService(std::shared_ptr<core::controller::ControllerServiceNode> &serviceNode) {
   // reference the disable function from serviceNode
-  std::function<bool()> f_ex = [serviceNode] {
+  std::function < bool() > f_ex = [serviceNode] {
     return serviceNode->disable();
   };
   // create a functor that will be submitted to the thread pool.
@@ -77,13 +77,15 @@ bool SchedulingAgent::onTrigger(std::shared_ptr<core::Processor> processor, core
   // No need to yield, reset yield expiration to 0
   processor->clearYield();
 
-  if (!hasWorkToDo(processor))
+  if (!hasWorkToDo(processor)) {
     // No work to do, yield
     return true;
-
-  if (hasTooMuchOutGoing(processor))
+  }
+  if (hasTooMuchOutGoing(processor)) {
+    logger_->log_debug("backpressure applied because too much outgoing");
     // need to apply backpressure
     return true;
+  }
 
   processor->incrementActiveTasks();
   try {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/Site2SiteClientProtocol.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/Site2SiteClientProtocol.cpp b/libminifi/src/Site2SiteClientProtocol.cpp
index 7d6e3f3..024bd35 100644
--- a/libminifi/src/Site2SiteClientProtocol.cpp
+++ b/libminifi/src/Site2SiteClientProtocol.cpp
@@ -726,13 +726,19 @@ bool Site2SiteClientProtocol::send(std::string transactionID, DataPacket *packet
     if (ret != 8) {
       return false;
     }
-    if (flowFile->getSize()) {
+    if (flowFile->getSize() > 0) {
       Site2SiteClientProtocol::ReadCallback callback(packet);
       session->read(flowFile, &callback);
       if (flowFile->getSize() != packet->_size) {
         return false;
       }
     }
+    if (packet->payload_.length() == 0 && len == 0) {
+      if (flowFile->getResourceClaim() == nullptr)
+        logger_->log_debug("no claim");
+      else
+        logger_->log_debug("Flowfile empty %s", flowFile->getResourceClaim()->getContentFullPath());
+    }
   } else if (packet->payload_.length() > 0) {
     len = packet->payload_.length();
 
@@ -1101,8 +1107,9 @@ void Site2SiteClientProtocol::transferFlowFiles(core::ProcessContext *context, c
 
   Transaction *transaction = NULL;
 
-  if (!flow)
+  if (!flow) {
     return;
+  }
 
   if (_peerState != READY) {
     bootstrap();
@@ -1158,11 +1165,15 @@ void Site2SiteClientProtocol::transferFlowFiles(core::ProcessContext *context, c
     }  // while true
 
     if (!confirm(transactionID)) {
-      throw Exception(SITE2SITE_EXCEPTION, "Confirm Failed");
+      std::stringstream ss;
+      ss << "Confirm Failed for " << transactionID;
+      throw Exception(SITE2SITE_EXCEPTION, ss.str().c_str());
       return;
     }
     if (!complete(transactionID)) {
-      throw Exception(SITE2SITE_EXCEPTION, "Complete Failed");
+      std::stringstream ss;
+      ss << "Complete Failed for " << transactionID;
+      throw Exception(SITE2SITE_EXCEPTION, ss.str().c_str());
       return;
     }
     logger_->log_info("Site2Site transaction %s successfully send flow record %d, content bytes %d", transactionID.c_str(), transaction->_transfers, transaction->_bytes);

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/ThreadedSchedulingAgent.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/ThreadedSchedulingAgent.cpp b/libminifi/src/ThreadedSchedulingAgent.cpp
index 46a4710..7b4ce85 100644
--- a/libminifi/src/ThreadedSchedulingAgent.cpp
+++ b/libminifi/src/ThreadedSchedulingAgent.cpp
@@ -36,7 +36,7 @@ namespace nifi {
 namespace minifi {
 
 void ThreadedSchedulingAgent::schedule(std::shared_ptr<core::Processor> processor) {
-  std::lock_guard<std::mutex> lock(mutex_);
+  std::lock_guard < std::mutex > lock(mutex_);
 
   admin_yield_duration_ = 0;
   std::string yieldValue;
@@ -68,8 +68,8 @@ void ThreadedSchedulingAgent::schedule(std::shared_ptr<core::Processor> processo
   }
 
   core::ProcessorNode processor_node(processor);
-  auto processContext = std::make_shared<core::ProcessContext>(processor_node, controller_service_provider_, repo_);
-  auto sessionFactory = std::make_shared<core::ProcessSessionFactory>(processContext.get());
+  auto processContext = std::make_shared < core::ProcessContext > (processor_node, controller_service_provider_, repo_, flow_repo_, content_repo_);
+  auto sessionFactory = std::make_shared < core::ProcessSessionFactory > (processContext.get());
 
   processor->onSchedule(processContext.get(), sessionFactory.get());
 
@@ -89,7 +89,7 @@ void ThreadedSchedulingAgent::schedule(std::shared_ptr<core::Processor> processo
 }
 
 void ThreadedSchedulingAgent::unschedule(std::shared_ptr<core::Processor> processor) {
-  std::lock_guard<std::mutex> lock(mutex_);
+  std::lock_guard < std::mutex > lock(mutex_);
   logger_->log_info("Shutting down threads for processor %s/%s", processor->getName().c_str(), processor->getUUIDStr().c_str());
 
   if (processor->getScheduledState() != core::RUNNING) {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/controllers/SSLContextService.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/controllers/SSLContextService.cpp b/libminifi/src/controllers/SSLContextService.cpp
index a9450f6..73c9e35 100644
--- a/libminifi/src/controllers/SSLContextService.cpp
+++ b/libminifi/src/controllers/SSLContextService.cpp
@@ -35,7 +35,7 @@ void SSLContextService::initialize() {
   if (initialized_)
     return;
 
-  std::lock_guard<std::mutex> lock(initialization_mutex_);
+  std::lock_guard < std::mutex > lock(initialization_mutex_);
 
   ControllerService::initialize();
 
@@ -75,31 +75,31 @@ std::unique_ptr<SSLContext> SSLContextService::createSSLContext() {
   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));
+  return std::unique_ptr < SSLContext > (new SSLContext(ctx));
 }
 
 const std::string &SSLContextService::getCertificateFile() {
-  std::lock_guard<std::mutex> lock(initialization_mutex_);
+  std::lock_guard < std::mutex > lock(initialization_mutex_);
   return certificate;
 }
 
 const std::string &SSLContextService::getPassphrase() {
-  std::lock_guard<std::mutex> lock(initialization_mutex_);
+  std::lock_guard < std::mutex > lock(initialization_mutex_);
   return passphrase_;
 }
 
 const std::string &SSLContextService::getPassphraseFile() {
-  std::lock_guard<std::mutex> lock(initialization_mutex_);
+  std::lock_guard < std::mutex > lock(initialization_mutex_);
   return passphrase_file_;
 }
 
 const std::string &SSLContextService::getPrivateKeyFile() {
-  std::lock_guard<std::mutex> lock(initialization_mutex_);
+  std::lock_guard < std::mutex > lock(initialization_mutex_);
   return private_key_;
 }
 
 const std::string &SSLContextService::getCACertificate() {
-  std::lock_guard<std::mutex> lock(initialization_mutex_);
+  std::lock_guard < std::mutex > lock(initialization_mutex_);
   return ca_certificate_;
 }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/core/ClassLoader.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/ClassLoader.cpp b/libminifi/src/core/ClassLoader.cpp
index 9bead0e..fbd46f6 100644
--- a/libminifi/src/core/ClassLoader.cpp
+++ b/libminifi/src/core/ClassLoader.cpp
@@ -43,7 +43,7 @@ uint16_t ClassLoader::registerResource(const std::string &resource) {
     logger_->log_error("Cannot load library: %s", dlerror());
     return RESOURCE_FAILURE;
   } else {
-    std::lock_guard<std::mutex> lock(internal_mutex_);
+    std::lock_guard < std::mutex > lock(internal_mutex_);
     dl_handles_.push_back(resource_ptr);
   }
 
@@ -60,9 +60,9 @@ uint16_t ClassLoader::registerResource(const std::string &resource) {
 
   ObjectFactory *factory = create_factory_func();
 
-  std::lock_guard<std::mutex> lock(internal_mutex_);
+  std::lock_guard < std::mutex > lock(internal_mutex_);
 
-  loaded_factories_[factory->getClassName()] = std::unique_ptr<ObjectFactory>(factory);
+  loaded_factories_[factory->getClassName()] = std::unique_ptr < ObjectFactory > (factory);
 
   return RESOURCE_SUCCESS;
 }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/core/ConfigurableComponent.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/ConfigurableComponent.cpp b/libminifi/src/core/ConfigurableComponent.cpp
index f5247ac..62a08db 100644
--- a/libminifi/src/core/ConfigurableComponent.cpp
+++ b/libminifi/src/core/ConfigurableComponent.cpp
@@ -29,6 +29,7 @@ namespace apache {
 namespace nifi {
 namespace minifi {
 namespace core {
+
 ConfigurableComponent::ConfigurableComponent()
     : logger_(logging::LoggerFactory<ConfigurableComponent>::getLogger()) {
 }
@@ -42,7 +43,7 @@ ConfigurableComponent::~ConfigurableComponent() {
 }
 
 bool ConfigurableComponent::getProperty(const std::string &name, Property &prop) {
-  std::lock_guard<std::mutex> lock(configuration_mutex_);
+  std::lock_guard < std::mutex > lock(configuration_mutex_);
 
   auto &&it = properties_.find(name);
 
@@ -61,7 +62,7 @@ bool ConfigurableComponent::getProperty(const std::string &name, Property &prop)
  * @return result of getting property.
  */
 bool ConfigurableComponent::getProperty(const std::string name, std::string &value) {
-  std::lock_guard<std::mutex> lock(configuration_mutex_);
+  std::lock_guard < std::mutex > lock(configuration_mutex_);
 
   auto &&it = properties_.find(name);
   if (it != properties_.end()) {
@@ -80,7 +81,7 @@ bool ConfigurableComponent::getProperty(const std::string name, std::string &val
  * @return result of setting property.
  */
 bool ConfigurableComponent::setProperty(const std::string name, std::string value) {
-  std::lock_guard<std::mutex> lock(configuration_mutex_);
+  std::lock_guard < std::mutex > lock(configuration_mutex_);
   auto &&it = properties_.find(name);
 
   if (it != properties_.end()) {
@@ -101,7 +102,7 @@ bool ConfigurableComponent::setProperty(const std::string name, std::string valu
  * @return result of setting property.
  */
 bool ConfigurableComponent::updateProperty(const std::string &name, const std::string &value) {
-  std::lock_guard<std::mutex> lock(configuration_mutex_);
+  std::lock_guard < std::mutex > lock(configuration_mutex_);
   auto &&it = properties_.find(name);
 
   if (it != properties_.end()) {
@@ -122,7 +123,7 @@ bool ConfigurableComponent::updateProperty(const std::string &name, const std::s
  * @return whether property was set or not
  */
 bool ConfigurableComponent::setProperty(Property &prop, std::string value) {
-  std::lock_guard<std::mutex> lock(configuration_mutex_);
+  std::lock_guard < std::mutex > lock(configuration_mutex_);
   auto it = properties_.find(prop.getName());
 
   if (it != properties_.end()) {
@@ -150,7 +151,7 @@ bool ConfigurableComponent::setSupportedProperties(std::set<Property> properties
     return false;
   }
 
-  std::lock_guard<std::mutex> lock(configuration_mutex_);
+  std::lock_guard < std::mutex > lock(configuration_mutex_);
 
   properties_.clear();
   for (auto item : properties) {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/core/ConfigurationFactory.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/ConfigurationFactory.cpp b/libminifi/src/core/ConfigurationFactory.cpp
index ea2ed5c..0a0e911 100644
--- a/libminifi/src/core/ConfigurationFactory.cpp
+++ b/libminifi/src/core/ConfigurationFactory.cpp
@@ -39,7 +39,8 @@ namespace core {
 class YamlConfiguration;
 #endif
 
-std::unique_ptr<core::FlowConfiguration> createFlowConfiguration(std::shared_ptr<core::Repository> repo, std::shared_ptr<core::Repository> flow_file_repo, std::shared_ptr<Configure> configure,
+std::unique_ptr<core::FlowConfiguration> createFlowConfiguration(std::shared_ptr<core::Repository> repo, std::shared_ptr<core::Repository> flow_file_repo,
+                                                                 std::shared_ptr<core::ContentRepository> content_repo, std::shared_ptr<Configure> configure,
                                                                  std::shared_ptr<io::StreamFactory> stream_factory, const std::string configuration_class_name, const std::string path,
                                                                  bool fail_safe) {
   std::string class_name_lc = configuration_class_name;
@@ -47,22 +48,23 @@ std::unique_ptr<core::FlowConfiguration> createFlowConfiguration(std::shared_ptr
   try {
     if (class_name_lc == "flowconfiguration") {
       // load the base configuration.
-      return std::unique_ptr<core::FlowConfiguration>(new core::FlowConfiguration(repo, flow_file_repo, stream_factory, configure, path));
+
+      return std::unique_ptr < core::FlowConfiguration > (new core::FlowConfiguration(repo, flow_file_repo, content_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, configure, path));
+      return std::unique_ptr < core::FlowConfiguration > (instantiate<core::YamlConfiguration>(repo, flow_file_repo, content_repo, stream_factory, configure, path));
 
     } else {
       if (fail_safe) {
-        return std::unique_ptr<core::FlowConfiguration>(new core::FlowConfiguration(repo, flow_file_repo, stream_factory, configure, path));
+        return std::unique_ptr < core::FlowConfiguration > (new core::FlowConfiguration(repo, flow_file_repo, content_repo, stream_factory, configure, path));
       } else {
         throw std::runtime_error("Support for the provided configuration class could not be found");
       }
     }
   } catch (const std::runtime_error &r) {
     if (fail_safe) {
-      return std::unique_ptr<core::FlowConfiguration>(new core::FlowConfiguration(repo, flow_file_repo, stream_factory, configure, path));
+      return std::unique_ptr < core::FlowConfiguration > (new core::FlowConfiguration(repo, flow_file_repo, content_repo, stream_factory, configure, path));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/core/Connectable.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/Connectable.cpp b/libminifi/src/core/Connectable.cpp
index cf01f0c..9c3b26a 100644
--- a/libminifi/src/core/Connectable.cpp
+++ b/libminifi/src/core/Connectable.cpp
@@ -53,7 +53,7 @@ bool Connectable::setSupportedRelationships(std::set<core::Relationship> relatio
     return false;
   }
 
-  std::lock_guard<std::mutex> lock(relationship_mutex_);
+  std::lock_guard < std::mutex > lock(relationship_mutex_);
 
   relationships_.clear();
   for (auto item : relationships) {
@@ -67,7 +67,7 @@ bool Connectable::setSupportedRelationships(std::set<core::Relationship> relatio
 bool Connectable::isSupportedRelationship(core::Relationship relationship) {
   const bool requiresLock = isRunning();
 
-  const auto conditionalLock = !requiresLock ? std::unique_lock<std::mutex>() : std::unique_lock<std::mutex>(relationship_mutex_);
+  const auto conditionalLock = !requiresLock ? std::unique_lock<std::mutex>() : std::unique_lock < std::mutex > (relationship_mutex_);
 
   const auto &it = relationships_.find(relationship.getName());
   if (it != relationships_.end()) {
@@ -83,7 +83,7 @@ bool Connectable::setAutoTerminatedRelationships(std::set<Relationship> relation
     return false;
   }
 
-  std::lock_guard<std::mutex> lock(relationship_mutex_);
+  std::lock_guard < std::mutex > lock(relationship_mutex_);
 
   auto_terminated_relationships_.clear();
   for (auto item : relationships) {
@@ -97,7 +97,7 @@ bool Connectable::setAutoTerminatedRelationships(std::set<Relationship> relation
 bool Connectable::isAutoTerminated(core::Relationship relationship) {
   const bool requiresLock = isRunning();
 
-  const auto conditionalLock = !requiresLock ? std::unique_lock<std::mutex>() : std::unique_lock<std::mutex>(relationship_mutex_);
+  const auto conditionalLock = !requiresLock ? std::unique_lock<std::mutex>() : std::unique_lock < std::mutex > (relationship_mutex_);
 
   const auto &it = auto_terminated_relationships_.find(relationship.getName());
   if (it != auto_terminated_relationships_.end()) {
@@ -111,7 +111,7 @@ void Connectable::waitForWork(uint64_t timeoutMs) {
   has_work_.store(isWorkAvailable());
 
   if (!has_work_.load()) {
-    std::unique_lock<std::mutex> lock(work_available_mutex_);
+    std::unique_lock < std::mutex > lock(work_available_mutex_);
     work_condition_.wait_for(lock, std::chrono::milliseconds(timeoutMs), [&] {return has_work_.load();});
   }
 }
@@ -143,7 +143,7 @@ std::set<std::shared_ptr<Connectable>> Connectable::getOutGoingConnections(std::
 }
 
 std::shared_ptr<Connectable> Connectable::getNextIncomingConnection() {
-  std::lock_guard<std::mutex> lock(relationship_mutex_);
+  std::lock_guard < std::mutex > lock(relationship_mutex_);
 
   if (_incomingConnections.size() == 0)
     return NULL;

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/core/Core.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/Core.cpp b/libminifi/src/core/Core.cpp
index 304d4ce..995c001 100644
--- a/libminifi/src/core/Core.cpp
+++ b/libminifi/src/core/Core.cpp
@@ -35,6 +35,11 @@ void CoreComponent::setUUID(uuid_t uuid) {
   uuid_unparse_lower(uuid_, uuidStr);
   uuidStr_ = uuidStr;
 }
+
+void CoreComponent::setUUIDStr(const std::string uuidStr) {
+  uuid_parse(uuidStr.c_str(), uuid_);
+  uuidStr_ = uuidStr;
+}
 // Get UUID
 bool CoreComponent::getUUID(uuid_t uuid) {
   if (uuid) {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/core/FlowConfiguration.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/FlowConfiguration.cpp b/libminifi/src/core/FlowConfiguration.cpp
index c32add6..e8e7462 100644
--- a/libminifi/src/core/FlowConfiguration.cpp
+++ b/libminifi/src/core/FlowConfiguration.cpp
@@ -35,7 +35,7 @@ std::shared_ptr<core::Processor> FlowConfiguration::createProcessor(std::string
   if (nullptr == ptr) {
     logger_->log_error("No Processor defined for %s", name.c_str());
   }
-  std::shared_ptr<core::Processor> processor = std::static_pointer_cast<core::Processor>(ptr);
+  std::shared_ptr<core::Processor> processor = std::static_pointer_cast < core::Processor > (ptr);
 
   // initialize the processor
   processor->initialize();
@@ -53,18 +53,16 @@ std::shared_ptr<core::Processor> FlowConfiguration::createProvenanceReportTask()
   return processor;
 }
 
-std::unique_ptr<core::ProcessGroup> FlowConfiguration::createRootProcessGroup(
-    std::string name, uuid_t uuid, int version) {
-  return std::unique_ptr<core::ProcessGroup>(
-      new core::ProcessGroup(core::ROOT_PROCESS_GROUP, name, uuid, version));
+std::unique_ptr<core::ProcessGroup> FlowConfiguration::createRootProcessGroup(std::string name, uuid_t uuid, int version) {
+  return std::unique_ptr < core::ProcessGroup > (new core::ProcessGroup(core::ROOT_PROCESS_GROUP, name, uuid, version));
 }
 
 std::unique_ptr<core::ProcessGroup> FlowConfiguration::createRemoteProcessGroup(std::string name, uuid_t uuid) {
-  return std::unique_ptr<core::ProcessGroup>(new core::ProcessGroup(core::REMOTE_PROCESS_GROUP, name, uuid));
+  return std::unique_ptr < core::ProcessGroup > (new core::ProcessGroup(core::REMOTE_PROCESS_GROUP, name, uuid));
 }
 
 std::shared_ptr<minifi::Connection> FlowConfiguration::createConnection(std::string name, uuid_t uuid) {
-  return std::make_shared<minifi::Connection>(flow_file_repo_, name, uuid);
+  return std::make_shared < minifi::Connection > (flow_file_repo_, content_repo_, name, uuid);
 }
 
 std::shared_ptr<core::controller::ControllerServiceNode> FlowConfiguration::createControllerService(const std::string &class_name, const std::string &name, uuid_t uuid) {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/core/FlowFile.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/FlowFile.cpp b/libminifi/src/core/FlowFile.cpp
index d9057c5..6afd0fe 100644
--- a/libminifi/src/core/FlowFile.cpp
+++ b/libminifi/src/core/FlowFile.cpp
@@ -47,7 +47,7 @@ FlowFile::FlowFile()
   entry_date_ = getTimeMillis();
   lineage_start_date_ = entry_date_;
 
-  char uuidStr[37];
+  char uuidStr[37] = { 0 };
 
   // Generate the global UUID for the flow record
   id_generator_->generate(uuid_);

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/core/ProcessGroup.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/ProcessGroup.cpp b/libminifi/src/core/ProcessGroup.cpp
index 2cf3db0..db0fe08 100644
--- a/libminifi/src/core/ProcessGroup.cpp
+++ b/libminifi/src/core/ProcessGroup.cpp
@@ -39,8 +39,7 @@ namespace core {
 
 std::shared_ptr<utils::IdGenerator> ProcessGroup::id_generator_ = utils::IdGenerator::getIdGenerator();
 
-ProcessGroup::ProcessGroup(ProcessGroupType type, std::string name, uuid_t uuid, int version,
-                           ProcessGroup *parent)
+ProcessGroup::ProcessGroup(ProcessGroupType type, std::string name, uuid_t uuid, int version, ProcessGroup *parent)
     : logger_(logging::LoggerFactory<ProcessGroup>::getLogger()),
       name_(name),
       type_(type),
@@ -55,7 +54,7 @@ ProcessGroup::ProcessGroup(ProcessGroupType type, std::string name, uuid_t uuid,
   yield_period_msec_ = 0;
   transmitting_ = false;
 
-  logger_->log_info("ProcessGroup %s created", name_.c_str());
+  logger_->log_info("ProcessGroup %s created", name_);
 }
 
 ProcessGroup::~ProcessGroup() {
@@ -70,12 +69,12 @@ ProcessGroup::~ProcessGroup() {
 }
 
 bool ProcessGroup::isRootProcessGroup() {
-  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  std::lock_guard < std::recursive_mutex > lock(mutex_);
   return (type_ == ROOT_PROCESS_GROUP);
 }
 
 void ProcessGroup::addProcessor(std::shared_ptr<Processor> processor) {
-  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  std::lock_guard < std::recursive_mutex > lock(mutex_);
 
   if (processors_.find(processor) == processors_.end()) {
     // We do not have the same processor in this process group yet
@@ -85,7 +84,7 @@ void ProcessGroup::addProcessor(std::shared_ptr<Processor> processor) {
 }
 
 void ProcessGroup::removeProcessor(std::shared_ptr<Processor> processor) {
-  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  std::lock_guard < std::recursive_mutex > lock(mutex_);
 
   if (processors_.find(processor) != processors_.end()) {
     // We do have the same processor in this process group yet
@@ -95,7 +94,7 @@ void ProcessGroup::removeProcessor(std::shared_ptr<Processor> processor) {
 }
 
 void ProcessGroup::addProcessGroup(ProcessGroup *child) {
-  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  std::lock_guard < std::recursive_mutex > lock(mutex_);
 
   if (child_process_groups_.find(child) == child_process_groups_.end()) {
     // We do not have the same child process group in this process group yet
@@ -105,7 +104,7 @@ void ProcessGroup::addProcessGroup(ProcessGroup *child) {
 }
 
 void ProcessGroup::removeProcessGroup(ProcessGroup *child) {
-  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  std::lock_guard < std::recursive_mutex > lock(mutex_);
 
   if (child_process_groups_.find(child) != child_process_groups_.end()) {
     // We do have the same child process group in this process group yet
@@ -115,7 +114,7 @@ void ProcessGroup::removeProcessGroup(ProcessGroup *child) {
 }
 
 void ProcessGroup::startProcessing(TimerDrivenSchedulingAgent *timeScheduler, EventDrivenSchedulingAgent *eventScheduler) {
-  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  std::lock_guard < std::recursive_mutex > lock(mutex_);
 
   try {
     // Start all the processor node, input and output ports
@@ -143,7 +142,7 @@ void ProcessGroup::startProcessing(TimerDrivenSchedulingAgent *timeScheduler, Ev
 }
 
 void ProcessGroup::stopProcessing(TimerDrivenSchedulingAgent *timeScheduler, EventDrivenSchedulingAgent *eventScheduler) {
-  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  std::lock_guard < std::recursive_mutex > lock(mutex_);
 
   try {
     // Stop all the processor node, input and output ports
@@ -169,7 +168,7 @@ void ProcessGroup::stopProcessing(TimerDrivenSchedulingAgent *timeScheduler, Eve
 }
 
 std::shared_ptr<Processor> ProcessGroup::findProcessor(uuid_t uuid) {
-  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  std::lock_guard < std::recursive_mutex > lock(mutex_);
   std::shared_ptr<Processor> ret = NULL;
   for (auto processor : processors_) {
     logger_->log_info("find processor %s", processor->getName().c_str());
@@ -209,7 +208,7 @@ std::shared_ptr<core::controller::ControllerServiceNode> ProcessGroup::findContr
 }
 
 std::shared_ptr<Processor> ProcessGroup::findProcessor(const std::string &processorName) {
-  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  std::lock_guard < std::recursive_mutex > lock(mutex_);
   std::shared_ptr<Processor> ret = NULL;
   for (auto processor : processors_) {
     logger_->log_debug("Current processor is %s", processor->getName().c_str());
@@ -225,7 +224,7 @@ std::shared_ptr<Processor> ProcessGroup::findProcessor(const std::string &proces
 }
 
 void ProcessGroup::updatePropertyValue(std::string processorName, std::string propertyName, std::string propertyValue) {
-  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  std::lock_guard < std::recursive_mutex > lock(mutex_);
   for (auto processor : processors_) {
     if (processor->getName() == processorName) {
       processor->setProperty(propertyName, propertyValue);
@@ -247,7 +246,7 @@ void ProcessGroup::getConnections(std::map<std::string, std::shared_ptr<Connecti
 }
 
 void ProcessGroup::addConnection(std::shared_ptr<Connection> connection) {
-  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  std::lock_guard < std::recursive_mutex > lock(mutex_);
 
   if (connections_.find(connection) == connections_.end()) {
     // We do not have the same connection in this process group yet
@@ -269,7 +268,7 @@ void ProcessGroup::addConnection(std::shared_ptr<Connection> connection) {
 }
 
 void ProcessGroup::removeConnection(std::shared_ptr<Connection> connection) {
-  std::lock_guard<std::recursive_mutex> lock(mutex_);
+  std::lock_guard < std::recursive_mutex > lock(mutex_);
 
   if (connections_.find(connection) != connections_.end()) {
     // We do not have the same connection in this process group yet

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/core/ProcessSession.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/ProcessSession.cpp b/libminifi/src/core/ProcessSession.cpp
index df21a34..c69b361 100644
--- a/libminifi/src/core/ProcessSession.cpp
+++ b/libminifi/src/core/ProcessSession.cpp
@@ -38,19 +38,21 @@ namespace core {
 
 std::shared_ptr<core::FlowFile> ProcessSession::create() {
   std::map<std::string, std::string> empty;
-  std::shared_ptr<core::FlowFile> record = std::make_shared<FlowFileRecord>(process_context_->getProvenanceRepository(), empty);
+
+  std::shared_ptr<core::FlowFile> record = std::make_shared<FlowFileRecord>(process_context_->getFlowFileRepository(), process_context_->getContentRepository(), empty);
 
   _addedFlowFiles[record->getUUIDStr()] = record;
   logger_->log_debug("Create FlowFile with UUID %s", record->getUUIDStr().c_str());
-  std::string details = process_context_->getProcessorNode().getName() + " creates flow record " + record->getUUIDStr();
-  provenance_report_->create(record, details);
+  std::stringstream details;
+  details << process_context_->getProcessorNode().getName() << " creates flow record " << record->getUUIDStr();
+  provenance_report_->create(record, details.str());
 
   return record;
 }
 
 std::shared_ptr<core::FlowFile> ProcessSession::create(std::shared_ptr<core::FlowFile> &&parent) {
   std::map<std::string, std::string> empty;
-  std::shared_ptr<core::FlowFile> record = std::make_shared<FlowFileRecord>(process_context_->getProvenanceRepository(), empty);
+  std::shared_ptr<core::FlowFile> record = std::make_shared<FlowFileRecord>(process_context_->getFlowFileRepository(), process_context_->getContentRepository(), empty);
 
   if (record) {
     _addedFlowFiles[record->getUUIDStr()] = record;
@@ -92,7 +94,7 @@ std::shared_ptr<core::FlowFile> ProcessSession::clone(std::shared_ptr<core::Flow
 
 std::shared_ptr<core::FlowFile> ProcessSession::cloneDuringTransfer(std::shared_ptr<core::FlowFile> &parent) {
   std::map<std::string, std::string> empty;
-  std::shared_ptr<core::FlowFile> record = std::make_shared<FlowFileRecord>(process_context_->getProvenanceRepository(), empty);
+  std::shared_ptr<core::FlowFile> record = std::make_shared<FlowFileRecord>(process_context_->getFlowFileRepository(), process_context_->getContentRepository(), empty);
 
   if (record) {
     this->_clonedFlowFiles[record->getUUIDStr()] = record;
@@ -168,26 +170,30 @@ void ProcessSession::remove(std::shared_ptr<core::FlowFile> &&flow) {
 
 void ProcessSession::putAttribute(std::shared_ptr<core::FlowFile> &flow, std::string key, std::string value) {
   flow->setAttribute(key, value);
-  std::string details = process_context_->getProcessorNode().getName() + " modify flow record " + flow->getUUIDStr() + " attribute " + key + ":" + value;
-  provenance_report_->modifyAttributes(flow, details);
+  std::stringstream details;
+  details << process_context_->getProcessorNode().getName() << " modify flow record " << flow->getUUIDStr() << " attribute " << key << ":" << value;
+  provenance_report_->modifyAttributes(flow, details.str());
 }
 
 void ProcessSession::removeAttribute(std::shared_ptr<core::FlowFile> &flow, std::string key) {
   flow->removeAttribute(key);
-  std::string details = process_context_->getProcessorNode().getName() + " remove flow record " + flow->getUUIDStr() + " attribute " + key;
-  provenance_report_->modifyAttributes(flow, details);
+  std::stringstream details;
+  details << process_context_->getProcessorNode().getName() << " remove flow record " << flow->getUUIDStr() << " attribute " + key;
+  provenance_report_->modifyAttributes(flow, details.str());
 }
 
 void ProcessSession::putAttribute(std::shared_ptr<core::FlowFile> &&flow, std::string key, std::string value) {
   flow->setAttribute(key, value);
-  std::string details = process_context_->getProcessorNode().getName() + " modify flow record " + flow->getUUIDStr() + " attribute " + key + ":" + value;
-  provenance_report_->modifyAttributes(flow, details);
+  std::stringstream details;
+  details << process_context_->getProcessorNode().getName() << " modify flow record " << flow->getUUIDStr() << " attribute " << key << ":" << value;
+  provenance_report_->modifyAttributes(flow, details.str());
 }
 
 void ProcessSession::removeAttribute(std::shared_ptr<core::FlowFile> &&flow, std::string key) {
   flow->removeAttribute(key);
-  std::string details = process_context_->getProcessorNode().getName() + " remove flow record " + flow->getUUIDStr() + " attribute " + key;
-  provenance_report_->modifyAttributes(flow, details);
+  std::stringstream details;
+  details << process_context_->getProcessorNode().getName() << " remove flow record " << flow->getUUIDStr() << " attribute " << key;
+  provenance_report_->modifyAttributes(flow, details.str());
 }
 
 void ProcessSession::penalize(std::shared_ptr<core::FlowFile> &flow) {
@@ -207,41 +213,41 @@ void ProcessSession::transfer(std::shared_ptr<core::FlowFile> &&flow, Relationsh
 }
 
 void ProcessSession::write(std::shared_ptr<core::FlowFile> &flow, OutputStreamCallback *callback) {
-  std::shared_ptr<ResourceClaim> claim = std::make_shared<ResourceClaim>(
-  DEFAULT_CONTENT_DIRECTORY);
+  std::shared_ptr<ResourceClaim> claim = std::make_shared<ResourceClaim>(process_context_->getContentRepository());
 
   try {
-    std::ofstream fs;
     uint64_t startTime = getTimeMillis();
-    fs.open(claim->getContentFullPath().c_str(), std::fstream::out | std::fstream::binary | std::fstream::trunc);
-    if (fs.is_open()) {
-      // Call the callback to write the content
-      callback->process(&fs);
-      if (fs.good() && fs.tellp() >= 0) {
-        flow->setSize(fs.tellp());
-        flow->setOffset(0);
-        std::shared_ptr<ResourceClaim> flow_claim = flow->getResourceClaim();
-        if (flow_claim != nullptr) {
-          // Remove the old claim
-          flow_claim->decreaseFlowFileRecordOwnedCount();
-          flow->clearResourceClaim();
-        }
-        flow->setResourceClaim(claim);
-        claim->increaseFlowFileRecordOwnedCount();
-        /*
-         logger_->log_debug("Write offset %d length %d into content %s for FlowFile UUID %s",
-         flow->_offset, flow->_size, flow->_claim->getContentFullPath().c_str(), flow->getUUIDStr().c_str()); */
-        fs.close();
-        std::string details = process_context_->getProcessorNode().getName() + " modify flow record content " + flow->getUUIDStr();
-        uint64_t endTime = getTimeMillis();
-        provenance_report_->modifyContent(flow, details, endTime - startTime);
-      } else {
-        fs.close();
-        throw Exception(FILE_OPERATION_EXCEPTION, "File Write Error");
-      }
-    } else {
-      throw Exception(FILE_OPERATION_EXCEPTION, "File Open Error");
+    claim->increaseFlowFileRecordOwnedCount();
+//    fs.open(claim->getContentFullPath().c_str(), std::fstream::out | std::fstream::binary | std::fstream::trunc);
+    std::shared_ptr<io::BaseStream> stream = process_context_->getContentRepository()->write(claim);
+    // Call the callback to write the content
+    if (nullptr == stream) {
+      rollback();
+      return;
+    }
+    if (callback->process(stream) < 0) {
+      rollback();
+      return;
+    }
+
+    flow->setSize(stream->getSize());
+    flow->setOffset(0);
+    std::shared_ptr<ResourceClaim> flow_claim = flow->getResourceClaim();
+    if (flow_claim != nullptr) {
+      // Remove the old claim
+      flow_claim->decreaseFlowFileRecordOwnedCount();
+      flow->clearResourceClaim();
     }
+    flow->setResourceClaim(claim);
+
+    /*
+     logger_->log_debug("Write offset %d length %d into content %s for FlowFile UUID %s",
+     flow->_offset, flow->_size, flow->_claim->getContentFullPath().c_str(), flow->getUUIDStr().c_str()); */
+    stream->closeStream();
+    std::stringstream details;
+    details << process_context_->getProcessorNode().getName() << " modify flow record content " << flow->getUUIDStr();
+    uint64_t endTime = getTimeMillis();
+    provenance_report_->modifyContent(flow, details.str(), endTime - startTime);
   } catch (std::exception &exception) {
     if (flow && flow->getResourceClaim() == claim) {
       flow->getResourceClaim()->decreaseFlowFileRecordOwnedCount();
@@ -260,39 +266,34 @@ void ProcessSession::write(std::shared_ptr<core::FlowFile> &flow, OutputStreamCa
 }
 
 void ProcessSession::write(std::shared_ptr<core::FlowFile> &&flow, OutputStreamCallback *callback) {
-  std::shared_ptr<ResourceClaim> claim = std::make_shared<ResourceClaim>();
+  std::shared_ptr<ResourceClaim> claim = std::make_shared<ResourceClaim>(process_context_->getContentRepository());
   try {
-    std::ofstream fs;
     uint64_t startTime = getTimeMillis();
-    fs.open(claim->getContentFullPath().c_str(), std::fstream::out | std::fstream::binary | std::fstream::trunc);
-    if (fs.is_open()) {
-      // Call the callback to write the content
-      callback->process(&fs);
-      if (fs.good() && fs.tellp() >= 0) {
-        flow->setSize(fs.tellp());
-        flow->setOffset(0);
-        std::shared_ptr<ResourceClaim> flow_claim = flow->getResourceClaim();
-        if (flow_claim != nullptr) {
-          // Remove the old claim
-          flow_claim->decreaseFlowFileRecordOwnedCount();
-          flow->clearResourceClaim();
-        }
-        flow->setResourceClaim(claim);
-        claim->increaseFlowFileRecordOwnedCount();
-        /*
-         logger_->log_debug("Write offset %d length %d into content %s for FlowFile UUID %s",
-         flow->_offset, flow->_size, flow->_claim->getContentFullPath().c_str(), flow->getUUIDStr().c_str()); */
-        fs.close();
-        std::string details = process_context_->getProcessorNode().getName() + " modify flow record content " + flow->getUUIDStr();
-        uint64_t endTime = getTimeMillis();
-        provenance_report_->modifyContent(flow, details, endTime - startTime);
-      } else {
-        fs.close();
-        throw Exception(FILE_OPERATION_EXCEPTION, "File Write Error");
-      }
-    } else {
-      throw Exception(FILE_OPERATION_EXCEPTION, "File Open Error");
+    claim->increaseFlowFileRecordOwnedCount();
+    std::shared_ptr<io::BaseStream> stream = process_context_->getContentRepository()->write(claim);
+    if (nullptr == stream) {
+      rollback();
+      return;
+    }
+    // Call the callback to write the content
+    if (callback->process(stream) < 0) {
+      rollback();
+      return;
+    }
+    flow->setSize(stream->getSize());
+    flow->setOffset(0);
+    std::shared_ptr<ResourceClaim> flow_claim = flow->getResourceClaim();
+    if (flow_claim != nullptr) {
+      // Remove the old claim
+      flow_claim->decreaseFlowFileRecordOwnedCount();
+      flow->clearResourceClaim();
     }
+    flow->setResourceClaim(claim);
+
+    std::stringstream details;
+    details << process_context_->getProcessorNode().getName() << " modify flow record content " << flow->getUUIDStr();
+    uint64_t endTime = getTimeMillis();
+    provenance_report_->modifyContent(flow, details.str(), endTime - startTime);
   } catch (std::exception &exception) {
     if (flow && flow->getResourceClaim() == claim) {
       flow->getResourceClaim()->decreaseFlowFileRecordOwnedCount();
@@ -321,30 +322,25 @@ void ProcessSession::append(std::shared_ptr<core::FlowFile> &&flow, OutputStream
   claim = flow->getResourceClaim();
 
   try {
-    std::ofstream fs;
     uint64_t startTime = getTimeMillis();
-    fs.open(claim->getContentFullPath().c_str(), std::fstream::out | std::fstream::binary | std::fstream::app);
-    if (fs.is_open()) {
-      // Call the callback to write the content
-      std::streampos oldPos = fs.tellp();
-      callback->process(&fs);
-      if (fs.good() && fs.tellp() >= 0) {
-        uint64_t appendSize = fs.tellp() - oldPos;
-        flow->setSize(flow->getSize() + appendSize);
-        /*
-         logger_->log_debug("Append offset %d extra length %d to new size %d into content %s for FlowFile UUID %s",
-         flow->_offset, appendSize, flow->_size, claim->getContentFullPath().c_str(), flow->getUUIDStr().c_str()); */
-        fs.close();
-        std::string details = process_context_->getProcessorNode().getName() + " modify flow record content " + flow->getUUIDStr();
-        uint64_t endTime = getTimeMillis();
-        provenance_report_->modifyContent(flow, details, endTime - startTime);
-      } else {
-        fs.close();
-        throw Exception(FILE_OPERATION_EXCEPTION, "File Write Error");
-      }
-    } else {
-      throw Exception(FILE_OPERATION_EXCEPTION, "File Open Error");
-    }
+    std::shared_ptr<io::BaseStream> stream = process_context_->getContentRepository()->write(claim);
+    if (nullptr == stream) {
+      rollback();
+      return;
+    }
+    // Call the callback to write the content
+    size_t oldPos = stream->getSize();
+    stream->seek(oldPos + 1);
+    if (callback->process(stream) < 0) {
+      rollback();
+      return;
+    }
+    uint64_t appendSize = stream->getSize() - oldPos;
+    flow->setSize(flow->getSize() + appendSize);
+    std::stringstream details;
+    details << process_context_->getProcessorNode().getName() << " modify flow record content " << flow->getUUIDStr();
+    uint64_t endTime = getTimeMillis();
+    provenance_report_->modifyContent(flow, details.str(), endTime - startTime);
   } catch (std::exception &exception) {
     logger_->log_debug("Caught Exception %s", exception.what());
     throw;
@@ -365,30 +361,26 @@ void ProcessSession::append(std::shared_ptr<core::FlowFile> &flow, OutputStreamC
   claim = flow->getResourceClaim();
 
   try {
-    std::ofstream fs;
     uint64_t startTime = getTimeMillis();
-    fs.open(claim->getContentFullPath().c_str(), std::fstream::out | std::fstream::binary | std::fstream::app);
-    if (fs.is_open()) {
-      // Call the callback to write the content
-      std::streampos oldPos = fs.tellp();
-      callback->process(&fs);
-      if (fs.good() && fs.tellp() >= 0) {
-        uint64_t appendSize = fs.tellp() - oldPos;
-        flow->setSize(flow->getSize() + appendSize);
-        /*
-         logger_->log_debug("Append offset %d extra length %d to new size %d into content %s for FlowFile UUID %s",
-         flow->_offset, appendSize, flow->_size, claim->getContentFullPath().c_str(), flow->getUUIDStr().c_str()); */
-        fs.close();
-        std::string details = process_context_->getProcessorNode().getName() + " modify flow record content " + flow->getUUIDStr();
-        uint64_t endTime = getTimeMillis();
-        provenance_report_->modifyContent(flow, details, endTime - startTime);
-      } else {
-        fs.close();
-        throw Exception(FILE_OPERATION_EXCEPTION, "File Write Error");
-      }
-    } else {
-      throw Exception(FILE_OPERATION_EXCEPTION, "File Open Error");
-    }
+    std::shared_ptr<io::BaseStream> stream = process_context_->getContentRepository()->write(claim);
+    if (nullptr == stream) {
+      rollback();
+      return;
+    }
+    // Call the callback to write the content
+    size_t oldPos = stream->getSize();
+    stream->seek(oldPos + 1);
+    if (callback->process(stream) < 0) {
+      rollback();
+      return;
+    }
+    uint64_t appendSize = stream->getSize() - oldPos;
+    flow->setSize(flow->getSize() + appendSize);
+
+    std::stringstream details;
+    details << process_context_->getProcessorNode().getName() << " modify flow record content " << flow->getUUIDStr();
+    uint64_t endTime = getTimeMillis();
+    provenance_report_->modifyContent(flow, details.str(), endTime - startTime);
   } catch (std::exception &exception) {
     logger_->log_debug("Caught Exception %s", exception.what());
     throw;
@@ -408,23 +400,19 @@ void ProcessSession::read(std::shared_ptr<core::FlowFile> &flow, InputStreamCall
     }
 
     claim = flow->getResourceClaim();
-    std::ifstream fs;
-    fs.open(claim->getContentFullPath().c_str(), std::fstream::in | std::fstream::binary);
-    if (fs.is_open()) {
-      fs.seekg(flow->getOffset(), fs.beg);
-
-      if (fs.good()) {
-        callback->process(&fs);
-        /*
-         logger_->log_debug("Read offset %d size %d content %s for FlowFile UUID %s",
-         flow->_offset, flow->_size, claim->getContentFullPath().c_str(), flow->getUUIDStr().c_str()); */
-        fs.close();
-      } else {
-        fs.close();
-        throw Exception(FILE_OPERATION_EXCEPTION, "File Read Error");
-      }
-    } else {
-      throw Exception(FILE_OPERATION_EXCEPTION, "File Open Error");
+
+    std::shared_ptr<io::BaseStream> stream = process_context_->getContentRepository()->read(claim);
+
+    if (nullptr == stream) {
+      rollback();
+      return;
+    }
+
+    stream->seek(flow->getOffset());
+
+    if (callback->process(stream) < 0) {
+      rollback();
+      return;
     }
   } catch (std::exception &exception) {
     logger_->log_debug("Caught Exception %s", exception.what());
@@ -445,23 +433,17 @@ void ProcessSession::read(std::shared_ptr<core::FlowFile> &&flow, InputStreamCal
     }
 
     claim = flow->getResourceClaim();
-    std::ifstream fs;
-    fs.open(claim->getContentFullPath().c_str(), std::fstream::in | std::fstream::binary);
-    if (fs.is_open()) {
-      fs.seekg(flow->getOffset(), fs.beg);
-
-      if (fs.good()) {
-        callback->process(&fs);
-        /*
-         logger_->log_debug("Read offset %d size %d content %s for FlowFile UUID %s",
-         flow->_offset, flow->_size, claim->getContentFullPath().c_str(), flow->getUUIDStr().c_str()); */
-        fs.close();
-      } else {
-        fs.close();
-        throw Exception(FILE_OPERATION_EXCEPTION, "File Read Error");
-      }
-    } else {
-      throw Exception(FILE_OPERATION_EXCEPTION, "File Open Error");
+    std::shared_ptr<io::BaseStream> stream = process_context_->getContentRepository()->read(claim);
+
+    if (nullptr == stream) {
+      rollback();
+      return;
+    }
+    stream->seek(flow->getOffset());
+
+    if (callback->process(stream) < 0) {
+      rollback();
+      return;
     }
   } catch (std::exception &exception) {
     logger_->log_debug("Caught Exception %s", exception.what());
@@ -479,60 +461,55 @@ void ProcessSession::read(std::shared_ptr<core::FlowFile> &&flow, InputStreamCal
  *
  */
 void ProcessSession::importFrom(io::DataStream &stream, std::shared_ptr<core::FlowFile> &&flow) {
-  std::shared_ptr<ResourceClaim> claim = std::make_shared<ResourceClaim>();
-
+  std::shared_ptr<ResourceClaim> claim = std::make_shared<ResourceClaim>(process_context_->getContentRepository());
   int max_read = getpagesize();
   std::vector<uint8_t> charBuffer;
   charBuffer.resize(max_read);
 
   try {
-    std::ofstream fs;
     uint64_t startTime = getTimeMillis();
-    fs.open(claim->getContentFullPath().c_str(), std::fstream::out | std::fstream::binary | std::fstream::trunc);
-
-    if (fs.is_open()) {
-      size_t position = 0;
-      const size_t max_size = stream.getSize();
-      size_t read_size = max_read;
-      while (position < max_size) {
-        if ((max_size - position) > max_read) {
-          read_size = max_read;
-        } else {
-          read_size = max_size - position;
-        }
-        charBuffer.clear();
-        stream.readData(charBuffer, read_size);
-
-        fs.write((const char*) charBuffer.data(), read_size);
-        position += read_size;
+    claim->increaseFlowFileRecordOwnedCount();
+    std::shared_ptr<io::BaseStream> content_stream = process_context_->getContentRepository()->write(claim);
+
+    if (nullptr == content_stream) {
+      logger_->log_debug("Could not obtain claim for %s", claim->getContentFullPath());
+      rollback();
+      return;
+    }
+    size_t position = 0;
+    const size_t max_size = stream.getSize();
+    size_t read_size = max_read;
+    while (position < max_size) {
+      if ((max_size - position) > max_read) {
+        read_size = max_read;
+      } else {
+        read_size = max_size - position;
       }
-      // Open the source file and stream to the flow file
+      charBuffer.clear();
+      stream.readData(charBuffer, read_size);
 
-      if (fs.good() && fs.tellp() >= 0) {
-        flow->setSize(fs.tellp());
-        flow->setOffset(0);
-        if (flow->getResourceClaim() != nullptr) {
-          // Remove the old claim
-          flow->getResourceClaim()->decreaseFlowFileRecordOwnedCount();
-          flow->clearResourceClaim();
-        }
-        flow->setResourceClaim(claim);
-        claim->increaseFlowFileRecordOwnedCount();
-
-        logger_->log_debug("Import offset %d length %d into content %s for FlowFile UUID %s", flow->getOffset(), flow->getSize(), flow->getResourceClaim()->getContentFullPath().c_str(),
-                           flow->getUUIDStr().c_str());
+      content_stream->write(charBuffer.data(), read_size);
+      position += read_size;
+    }
+    // Open the source file and stream to the flow file
 
-        fs.close();
-        std::string details = process_context_->getProcessorNode().getName() + " modify flow record content " + flow->getUUIDStr();
-        uint64_t endTime = getTimeMillis();
-        provenance_report_->modifyContent(flow, details, endTime - startTime);
-      } else {
-        fs.close();
-        throw Exception(FILE_OPERATION_EXCEPTION, "File Import Error");
-      }
-    } else {
-      throw Exception(FILE_OPERATION_EXCEPTION, "File Import Error");
+    flow->setSize(content_stream->getSize());
+    flow->setOffset(0);
+    if (flow->getResourceClaim() != nullptr) {
+      // Remove the old claim
+      flow->getResourceClaim()->decreaseFlowFileRecordOwnedCount();
+      flow->clearResourceClaim();
     }
+    flow->setResourceClaim(claim);
+
+    logger_->log_debug("Import offset %d length %d into content %s for FlowFile UUID %s", flow->getOffset(), flow->getSize(), flow->getResourceClaim()->getContentFullPath().c_str(),
+                       flow->getUUIDStr().c_str());
+
+    content_stream->closeStream();
+    std::stringstream details;
+    details << process_context_->getProcessorNode().getName() << " modify flow record content " << flow->getUUIDStr();
+    uint64_t endTime = getTimeMillis();
+    provenance_report_->modifyContent(flow, details.str(), endTime - startTime);
   } catch (std::exception &exception) {
     if (flow && flow->getResourceClaim() == claim) {
       flow->getResourceClaim()->decreaseFlowFileRecordOwnedCount();
@@ -550,34 +527,44 @@ void ProcessSession::importFrom(io::DataStream &stream, std::shared_ptr<core::Fl
   }
 }
 
-void ProcessSession::import(std::string source, std::shared_ptr<core::FlowFile> &flow,
-bool keepSource,
-                            uint64_t offset) {
-  std::shared_ptr<ResourceClaim> claim = std::make_shared<ResourceClaim>();
+void ProcessSession::import(std::string source, std::shared_ptr<core::FlowFile> &flow, bool keepSource, uint64_t offset) {
+  std::shared_ptr<ResourceClaim> claim = std::make_shared<ResourceClaim>(process_context_->getContentRepository());
   char *buf = NULL;
   int size = 4096;
   buf = new char[size];
 
   try {
-    std::ofstream fs;
+    //  std::ofstream fs;
     uint64_t startTime = getTimeMillis();
-    fs.open(claim->getContentFullPath().c_str(), std::fstream::out | std::fstream::binary | std::fstream::trunc);
     std::ifstream input;
     input.open(source.c_str(), std::fstream::in | std::fstream::binary);
-
-    if (fs.is_open() && input.is_open()) {
+    claim->increaseFlowFileRecordOwnedCount();
+    std::shared_ptr<io::BaseStream> stream = process_context_->getContentRepository()->write(claim);
+    if (nullptr == stream) {
+      rollback();
+      return;
+    }
+    if (input.is_open()) {
       // Open the source file and stream to the flow file
-      input.seekg(offset, fs.beg);
+      input.seekg(offset);
+      bool invalidWrite = false;
       while (input.good()) {
         input.read(buf, size);
-        if (input)
-          fs.write(buf, size);
-        else
-          fs.write(buf, input.gcount());
+        if (input) {
+          if (stream->write(reinterpret_cast<uint8_t*>(buf), size) < 0) {
+            invalidWrite = true;
+            break;
+          }
+        } else {
+          if (stream->write(reinterpret_cast<uint8_t*>(buf), input.gcount()) < 0) {
+            invalidWrite = true;
+            break;
+          }
+        }
       }
 
-      if (fs.good() && fs.tellp() >= 0) {
-        flow->setSize(fs.tellp());
+      if (!invalidWrite) {
+        flow->setSize(stream->getSize());
         flow->setOffset(0);
         if (flow->getResourceClaim() != nullptr) {
           // Remove the old claim
@@ -585,20 +572,20 @@ bool keepSource,
           flow->clearResourceClaim();
         }
         flow->setResourceClaim(claim);
-        claim->increaseFlowFileRecordOwnedCount();
 
         logger_->log_debug("Import offset %d length %d into content %s for FlowFile UUID %s", flow->getOffset(), flow->getSize(), flow->getResourceClaim()->getContentFullPath().c_str(),
                            flow->getUUIDStr().c_str());
 
-        fs.close();
+        stream->closeStream();
         input.close();
         if (!keepSource)
           std::remove(source.c_str());
-        std::string details = process_context_->getProcessorNode().getName() + " modify flow record content " + flow->getUUIDStr();
+        std::stringstream details;
+        details << process_context_->getProcessorNode().getName() << " modify flow record content " << flow->getUUIDStr();
         uint64_t endTime = getTimeMillis();
-        provenance_report_->modifyContent(flow, details, endTime - startTime);
+        provenance_report_->modifyContent(flow, details.str(), endTime - startTime);
       } else {
-        fs.close();
+        stream->closeStream();
         input.close();
         throw Exception(FILE_OPERATION_EXCEPTION, "File Import Error");
       }
@@ -626,8 +613,7 @@ bool keepSource,
   }
 }
 
-void ProcessSession::import(std::string source, std::vector<std::shared_ptr<FlowFileRecord>> flows,
-  bool keepSource, uint64_t offset, char inputDelimiter) {
+void ProcessSession::import(std::string source, std::vector<std::shared_ptr<FlowFileRecord>> flows, bool keepSource, uint64_t offset, char inputDelimiter) {
   std::shared_ptr<ResourceClaim> claim;
 
   std::shared_ptr<FlowFileRecord> flowFile;
@@ -639,48 +625,61 @@ void ProcessSession::import(std::string source, std::vector<std::shared_ptr<Flow
   try {
     // Open the input file and seek to the appropriate location.
     std::ifstream input;
+    logger_->log_debug("Opening %s", source);
     input.open(source.c_str(), std::fstream::in | std::fstream::binary);
     if (input.is_open()) {
       input.seekg(offset, input.beg);
       while (input.good()) {
+        bool invalidWrite = false;
         flowFile = std::static_pointer_cast<FlowFileRecord>(create());
-        claim = std::make_shared<ResourceClaim>();
+        claim = std::make_shared<ResourceClaim>(process_context_->getContentRepository());
         uint64_t startTime = getTimeMillis();
         input.getline(buf, size, inputDelimiter);
-        std::ofstream fs;
-        fs.open(claim->getContentFullPath().c_str(), std::fstream::out | std::fstream::binary | std::fstream::trunc);
-
-        if (fs.is_open()) {
-          if (input)
-            fs.write(buf, strlen(buf));
-          else
-            fs.write(buf, input.gcount());
-
-          if (fs.good() && fs.tellp() >= 0) {
-            flowFile->setSize(fs.tellp());
-            flowFile->setOffset(0);
-            if (flowFile->getResourceClaim() != nullptr) {
-              // Remove the old claim
-              flowFile->getResourceClaim()->decreaseFlowFileRecordOwnedCount();
-              flowFile->clearResourceClaim();
-            }
-            flowFile->setResourceClaim(claim);
-            claim->increaseFlowFileRecordOwnedCount();
 
-            logger_->log_debug("Import offset %d length %d into content %s for FlowFile UUID %s", flowFile->getOffset(),
-                               flowFile->getSize(), flowFile->getResourceClaim()->getContentFullPath().c_str(),
-                               flowFile->getUUIDStr().c_str());
+        std::shared_ptr<io::BaseStream> stream = process_context_->getContentRepository()->write(claim);
+        if (nullptr == stream) {
+          logger_->log_debug("Stream is null");
+          rollback();
+          return;
+        }
 
-            fs.close();
-            std::string details = process_context_->getProcessorNode().getName() + " modify flow record content " + flowFile->getUUIDStr();
-            uint64_t endTime = getTimeMillis();
-            provenance_report_->modifyContent(flowFile, details, endTime - startTime);
-            flows.push_back(flowFile);
+        if (input) {
+          if (stream->write(reinterpret_cast<uint8_t*>(buf), size) < 0) {
+            invalidWrite = true;
+            break;
+          }
+        } else {
+          if (stream->write(reinterpret_cast<uint8_t*>(buf), input.gcount()) < 0) {
+            invalidWrite = true;
+            break;
+          }
+        }
 
-          } else {
-            fs.close();
-            throw Exception(FILE_OPERATION_EXCEPTION, "File Export Error creating Flowfile");
+        if (!invalidWrite) {
+          flowFile->setSize(stream->getSize());
+          flowFile->setOffset(0);
+          if (flowFile->getResourceClaim() != nullptr) {
+            // Remove the old claim
+            flowFile->getResourceClaim()->decreaseFlowFileRecordOwnedCount();
+            flowFile->clearResourceClaim();
           }
+          flowFile->setResourceClaim(claim);
+          claim->increaseFlowFileRecordOwnedCount();
+
+          logger_->log_debug("Import offset %d length %d into content %s for FlowFile UUID %s", flowFile->getOffset(),
+                             flowFile->getSize(),
+                             flowFile->getResourceClaim()->getContentFullPath().c_str(),
+                             flowFile->getUUIDStr().c_str());
+
+          stream->closeStream();
+          std::string details = process_context_->getProcessorNode().getName() + " modify flow record content " + flowFile->getUUIDStr();
+          uint64_t endTime = getTimeMillis();
+          provenance_report_->modifyContent(flowFile, details, endTime - startTime);
+          flows.push_back(flowFile);
+        } else {
+          logger_->log_debug("Error while writing");
+          stream->closeStream();
+          throw Exception(FILE_OPERATION_EXCEPTION, "File Export Error creating Flowfile");
         }
       }
       input.close();
@@ -711,35 +710,44 @@ void ProcessSession::import(std::string source, std::vector<std::shared_ptr<Flow
   }
 }
 
-void ProcessSession::import(std::string source, std::shared_ptr<core::FlowFile> &&flow,
-bool keepSource,
-                            uint64_t offset) {
-  std::shared_ptr<ResourceClaim> claim = std::make_shared<ResourceClaim>();
-
+void ProcessSession::import(std::string source, std::shared_ptr<core::FlowFile> &&flow, bool keepSource, uint64_t offset) {
+  std::shared_ptr<ResourceClaim> claim = std::make_shared<ResourceClaim>(process_context_->getContentRepository());
   char *buf = NULL;
   int size = 4096;
   buf = new char[size];
 
   try {
-    std::ofstream fs;
+    //  std::ofstream fs;
     uint64_t startTime = getTimeMillis();
-    fs.open(claim->getContentFullPath().c_str(), std::fstream::out | std::fstream::binary | std::fstream::trunc);
     std::ifstream input;
     input.open(source.c_str(), std::fstream::in | std::fstream::binary);
-
-    if (fs.is_open() && input.is_open()) {
+    claim->increaseFlowFileRecordOwnedCount();
+    std::shared_ptr<io::BaseStream> stream = process_context_->getContentRepository()->write(claim);
+    if (nullptr == stream) {
+      rollback();
+      return;
+    }
+    if (input.is_open()) {
       // Open the source file and stream to the flow file
-      input.seekg(offset, fs.beg);
+      input.seekg(offset);
+      int sizeWritten = 0;
+      bool invalidWrite = false;
       while (input.good()) {
         input.read(buf, size);
-        if (input)
-          fs.write(buf, size);
-        else
-          fs.write(buf, input.gcount());
+        if (input) {
+          if (stream->write(reinterpret_cast<uint8_t*>(buf), size) < 0) {
+            invalidWrite = true;
+            break;
+          }
+        } else {
+          if (stream->write(reinterpret_cast<uint8_t*>(buf), input.gcount()) < 0) {
+            invalidWrite = true;
+            break;
+          }
+        }
       }
-
-      if (fs.good() && fs.tellp() >= 0) {
-        flow->setSize(fs.tellp());
+      if (!invalidWrite) {
+        flow->setSize(stream->getSize());
         flow->setOffset(0);
         if (flow->getResourceClaim() != nullptr) {
           // Remove the old claim
@@ -747,20 +755,20 @@ bool keepSource,
           flow->clearResourceClaim();
         }
         flow->setResourceClaim(claim);
-        claim->increaseFlowFileRecordOwnedCount();
 
         logger_->log_debug("Import offset %d length %d into content %s for FlowFile UUID %s", flow->getOffset(), flow->getSize(), flow->getResourceClaim()->getContentFullPath().c_str(),
                            flow->getUUIDStr().c_str());
 
-        fs.close();
+        stream->closeStream();
         input.close();
         if (!keepSource)
           std::remove(source.c_str());
-        std::string details = process_context_->getProcessorNode().getName() + " modify flow record content " + flow->getUUIDStr();
+        std::stringstream details;
+        details << process_context_->getProcessorNode().getName() << " modify flow record content " << flow->getUUIDStr();
         uint64_t endTime = getTimeMillis();
-        provenance_report_->modifyContent(flow, details, endTime - startTime);
+        provenance_report_->modifyContent(flow, details.str(), endTime - startTime);
       } else {
-        fs.close();
+        stream->closeStream();
         input.close();
         throw Exception(FILE_OPERATION_EXCEPTION, "File Import Error");
       }
@@ -834,7 +842,7 @@ void ProcessSession::commit() {
       }
     }
 
-    // Do the samething for added flow file
+    // Do the same thing for added flow file
     for (const auto it : _addedFlowFiles) {
       std::shared_ptr<core::FlowFile> record = it.second;
       if (record->isDeleted())
@@ -851,6 +859,7 @@ void ProcessSession::commit() {
             std::string message = "Connect empty for non auto terminated relationship " + relationship.getName();
             throw Exception(PROCESS_SESSION_EXCEPTION, message.c_str());
           } else {
+            logger_->log_debug("added flow file is auto terminated");
             // Autoterminated
             remove(record);
           }
@@ -947,7 +956,7 @@ void ProcessSession::rollback() {
     _addedFlowFiles.clear();
     _updatedFlowFiles.clear();
     _deletedFlowFiles.clear();
-    logger_->log_trace("ProcessSession rollback for %s", process_context_->getProcessorNode().getName().c_str());
+    logger_->log_debug("ProcessSession rollback for %s", process_context_->getProcessorNode().getName().c_str());
   } catch (std::exception &exception) {
     logger_->log_debug("Caught Exception %s", exception.what());
     throw;
@@ -960,8 +969,10 @@ void ProcessSession::rollback() {
 std::shared_ptr<core::FlowFile> ProcessSession::get() {
   std::shared_ptr<Connectable> first = process_context_->getProcessorNode().getNextIncomingConnection();
 
-  if (first == NULL)
+  if (first == NULL) {
+    logger_->log_debug("Get is null for %s", process_context_->getProcessorNode().getName());
     return NULL;
+  }
 
   std::shared_ptr<Connection> current = std::static_pointer_cast<Connection>(first);
 
@@ -972,8 +983,9 @@ std::shared_ptr<core::FlowFile> ProcessSession::get() {
       // Remove expired flow record
       for (std::set<std::shared_ptr<core::FlowFile> >::iterator it = expired.begin(); it != expired.end(); ++it) {
         std::shared_ptr<core::FlowFile> record = *it;
-        std::string details = process_context_->getProcessorNode().getName() + " expire flow record " + record->getUUIDStr();
-        provenance_report_->expire(record, details);
+        std::stringstream details;
+        details << process_context_->getProcessorNode().getName() << " expire flow record " << record->getUUIDStr();
+        provenance_report_->expire(record, details.str());
       }
     }
     if (ret) {
@@ -981,10 +993,9 @@ std::shared_ptr<core::FlowFile> ProcessSession::get() {
       ret->setDeleted(false);
       _updatedFlowFiles[ret->getUUIDStr()] = ret;
       std::map<std::string, std::string> empty;
-      std::shared_ptr<core::FlowFile> snapshot = std::make_shared<FlowFileRecord>(process_context_->getProvenanceRepository(), empty);
+      std::shared_ptr<core::FlowFile> snapshot = std::make_shared<FlowFileRecord>(process_context_->getFlowFileRepository(), process_context_->getContentRepository(), empty);
       logger_->log_debug("Create Snapshot FlowFile with UUID %s", snapshot->getUUIDStr().c_str());
       snapshot = ret;
-//      snapshot->duplicate(ret);
       // save a snapshot
       _originalFlowFiles[snapshot->getUUIDStr()] = snapshot;
       return ret;

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/core/ProcessSessionFactory.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/ProcessSessionFactory.cpp b/libminifi/src/core/ProcessSessionFactory.cpp
index 31b7481..570d895 100644
--- a/libminifi/src/core/ProcessSessionFactory.cpp
+++ b/libminifi/src/core/ProcessSessionFactory.cpp
@@ -28,7 +28,7 @@ namespace minifi {
 namespace core {
 
 std::unique_ptr<ProcessSession> ProcessSessionFactory::createSession() {
-  return std::unique_ptr<ProcessSession>(new ProcessSession(process_context_));
+  return std::unique_ptr < ProcessSession > (new ProcessSession(process_context_));
 }
 
 } /* namespace core */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/core/Processor.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/Processor.cpp b/libminifi/src/core/Processor.cpp
index 7b07638..0c2e7cf 100644
--- a/libminifi/src/core/Processor.cpp
+++ b/libminifi/src/core/Processor.cpp
@@ -62,7 +62,7 @@ Processor::Processor(std::string name, uuid_t uuid)
   active_tasks_ = 0;
   yield_expiration_ = 0;
   incoming_connections_Iter = this->_incomingConnections.begin();
-  logger_->log_info("Processor %s created UUID %s", name_.c_str(), uuidStr_.c_str());
+  logger_->log_info("Processor %s created UUID %s", name_, uuidStr_);
 }
 
 bool Processor::isRunning() {
@@ -80,8 +80,8 @@ bool Processor::addConnection(std::shared_ptr<Connectable> conn) {
     logger_->log_info("Can not add connection while the process %s is running", name_.c_str());
     return false;
   }
-  std::shared_ptr<Connection> connection = std::static_pointer_cast<Connection>(conn);
-  std::lock_guard<std::mutex> lock(mutex_);
+  std::shared_ptr<Connection> connection = std::static_pointer_cast < Connection > (conn);
+  std::lock_guard < std::mutex > lock(mutex_);
 
   uuid_t srcUUID;
   uuid_t destUUID;
@@ -141,12 +141,12 @@ void Processor::removeConnection(std::shared_ptr<Connectable> conn) {
     return;
   }
 
-  std::lock_guard<std::mutex> lock(mutex_);
+  std::lock_guard < std::mutex > lock(mutex_);
 
   uuid_t srcUUID;
   uuid_t destUUID;
 
-  std::shared_ptr<Connection> connection = std::static_pointer_cast<Connection>(conn);
+  std::shared_ptr<Connection> connection = std::static_pointer_cast < Connection > (conn);
 
   connection->getSourceUUID(srcUUID);
   connection->getDestinationUUID(destUUID);
@@ -178,13 +178,13 @@ void Processor::removeConnection(std::shared_ptr<Connectable> conn) {
 }
 
 bool Processor::flowFilesQueued() {
-  std::lock_guard<std::mutex> lock(mutex_);
+  std::lock_guard < std::mutex > lock(mutex_);
 
   if (_incomingConnections.size() == 0)
     return false;
 
   for (auto &&conn : _incomingConnections) {
-    std::shared_ptr<Connection> connection = std::static_pointer_cast<Connection>(conn);
+    std::shared_ptr<Connection> connection = std::static_pointer_cast < Connection > (conn);
     if (connection->getQueueSize() > 0)
       return true;
   }
@@ -193,13 +193,13 @@ bool Processor::flowFilesQueued() {
 }
 
 bool Processor::flowFilesOutGoingFull() {
-  std::lock_guard<std::mutex> lock(mutex_);
+  std::lock_guard < std::mutex > lock(mutex_);
 
   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) {
-      std::shared_ptr<Connection> connection = std::static_pointer_cast<Connection>(conn);
+      std::shared_ptr < Connection > connection = std::static_pointer_cast < Connection > (conn);
       if (connection->isFull())
         return true;
     }
@@ -232,7 +232,7 @@ bool Processor::isWorkAvailable() {
 
   try {
     for (const auto &conn : _incomingConnections) {
-      std::shared_ptr<Connection> connection = std::static_pointer_cast<Connection>(conn);
+      std::shared_ptr<Connection> connection = std::static_pointer_cast < Connection > (conn);
       if (connection->getQueueSize() > 0) {
         hasWork = true;
         break;

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/core/Repository.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/Repository.cpp b/libminifi/src/core/Repository.cpp
index 50e8cd2..cf26a0d 100644
--- a/libminifi/src/core/Repository.cpp
+++ b/libminifi/src/core/Repository.cpp
@@ -19,13 +19,14 @@
 #include <arpa/inet.h>
 #include <cstdint>
 #include <vector>
+
+#include "../../include/core/repository/FlowFileRepository.h"
 #include "io/DataStream.h"
 #include "io/Serializable.h"
 #include "core/Relationship.h"
 #include "core/logging/Logger.h"
 #include "FlowController.h"
 #include "provenance/Provenance.h"
-#include "core/repository/FlowFileRepository.h"
 
 namespace org {
 namespace apache {
@@ -38,9 +39,8 @@ void Repository::start() {
     return;
   if (running_)
     return;
-  thread_ = std::thread(&Repository::threadExecutor, this);
-  thread_.detach();
   running_ = true;
+  thread_ = std::thread(&Repository::threadExecutor, this);
   logger_->log_info("%s Repository Monitor Thread Start", name_.c_str());
 }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/core/RepositoryFactory.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/RepositoryFactory.cpp b/libminifi/src/core/RepositoryFactory.cpp
index cf18601..9e99718 100644
--- a/libminifi/src/core/RepositoryFactory.cpp
+++ b/libminifi/src/core/RepositoryFactory.cpp
@@ -18,13 +18,17 @@
 #include <memory>
 #include <string>
 #include <algorithm>
+#include "core/ContentRepository.h"
+#include "core/repository/FileSystemRepository.h"
+#include "core/repository/VolatileContentRepository.h"
 #include "core/Repository.h"
 #ifdef LEVELDB_SUPPORT
 #include "core/repository/FlowFileRepository.h"
 #include "provenance/ProvenanceRepository.h"
 #endif
 
-#include "core/repository/VolatileRepository.h"
+#include "core/repository/VolatileProvenanceRepository.h"
+#include "core/repository/VolatileFlowFileRepository.h"
 
 namespace org {
 namespace apache {
@@ -48,14 +52,14 @@ std::shared_ptr<core::Repository> createRepository(const std::string configurati
   try {
     std::shared_ptr<core::Repository> return_obj = nullptr;
     if (class_name_lc == "flowfilerepository") {
-      std::cout << "creating flow" << std::endl;
       return_obj = instantiate<core::repository::FlowFileRepository>(repo_name);
     } else if (class_name_lc == "provenancerepository") {
       return_obj = instantiate<provenance::ProvenanceRepository>(repo_name);
-    } else if (class_name_lc == "volatilerepository") {
-      return_obj = instantiate<repository::VolatileRepository>(repo_name);
+    } else if (class_name_lc == "volatileflowfilerepository") {
+      return_obj = instantiate<repository::VolatileFlowFileRepository>(repo_name);
+    } else if (class_name_lc == "volatileprovenancefilerepository") {
+      return_obj = instantiate<repository::VolatileProvenanceRepository>(repo_name);
     } else if (class_name_lc == "nooprepository") {
-      std::cout << "creating noop" << std::endl;
       return_obj = instantiate<core::Repository>(repo_name);
     }
 
@@ -63,13 +67,42 @@ std::shared_ptr<core::Repository> createRepository(const std::string configurati
       return return_obj;
     }
     if (fail_safe) {
-      return std::make_shared<core::Repository>("fail_safe", "fail_safe", 1, 1, 1);
+      return std::make_shared < core::Repository > ("fail_safe", "fail_safe", 1, 1, 1);
     } else {
       throw std::runtime_error("Support for the provided configuration class could not be found");
     }
   } catch (const std::runtime_error &r) {
     if (fail_safe) {
-      return std::make_shared<core::Repository>("fail_safe", "fail_safe", 1, 1, 1);
+      return std::make_shared < core::Repository > ("fail_safe", "fail_safe", 1, 1, 1);
+    }
+  }
+
+  throw std::runtime_error("Support for the provided configuration class could not be found");
+}
+
+std::shared_ptr<core::ContentRepository> createContentRepository(const std::string configuration_class_name, bool fail_safe, const std::string repo_name) {
+  std::shared_ptr<core::ContentRepository> return_obj = nullptr;
+  std::string class_name_lc = configuration_class_name;
+  std::transform(class_name_lc.begin(), class_name_lc.end(), class_name_lc.begin(), ::tolower);
+  try {
+    std::shared_ptr<core::ContentRepository> return_obj = nullptr;
+    if (class_name_lc == "volatilecontentrepository") {
+      return_obj = instantiate<core::repository::VolatileContentRepository>(repo_name);
+    } else {
+      return_obj = instantiate<core::repository::FileSystemRepository>(repo_name);
+    }
+
+    if (return_obj) {
+      return return_obj;
+    }
+    if (fail_safe) {
+      return std::make_shared < core::repository::FileSystemRepository > ("fail_safe");
+    } else {
+      throw std::runtime_error("Support for the provided configuration class could not be found");
+    }
+  } catch (const std::runtime_error &r) {
+    if (fail_safe) {
+      return std::make_shared < core::repository::FileSystemRepository > ("fail_safe");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/core/controller/StandardControllerServiceNode.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/controller/StandardControllerServiceNode.cpp b/libminifi/src/core/controller/StandardControllerServiceNode.cpp
index 5c4aa70..69004c1 100644
--- a/libminifi/src/core/controller/StandardControllerServiceNode.cpp
+++ b/libminifi/src/core/controller/StandardControllerServiceNode.cpp
@@ -27,12 +27,12 @@ namespace minifi {
 namespace core {
 namespace controller {
 std::shared_ptr<core::ProcessGroup> &StandardControllerServiceNode::getProcessGroup() {
-  std::lock_guard<std::mutex> lock(mutex_);
+  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_);
+  std::lock_guard < std::mutex > lock(mutex_);
   process_group_ = processGroup;
 }
 
@@ -45,7 +45,7 @@ bool StandardControllerServiceNode::enable() {
     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_);
+        std::lock_guard < std::mutex > lock(mutex_);
         linked_controller_services_.push_back(csNode);
       }
     }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/core/logging/LoggerConfiguration.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/logging/LoggerConfiguration.cpp b/libminifi/src/core/logging/LoggerConfiguration.cpp
index c06239b..4b97055 100644
--- a/libminifi/src/core/logging/LoggerConfiguration.cpp
+++ b/libminifi/src/core/logging/LoggerConfiguration.cpp
@@ -56,19 +56,19 @@ std::vector<std::string> LoggerProperties::get_keys_of_type(const std::string &t
 LoggerConfiguration::LoggerConfiguration()
     : root_namespace_(create_default_root()),
       loggers(std::vector<std::shared_ptr<LoggerImpl>>()),
-      formatter_(std::make_shared<spdlog::pattern_formatter>(spdlog_default_pattern)) {
-  logger_ = std::shared_ptr<LoggerImpl>(new LoggerImpl(core::getClassName<LoggerConfiguration>(), get_logger(nullptr, root_namespace_, core::getClassName<LoggerConfiguration>(), formatter_)));
+      formatter_(std::make_shared < spdlog::pattern_formatter > (spdlog_default_pattern)) {
+  logger_ = std::shared_ptr < LoggerImpl > (new LoggerImpl(core::getClassName<LoggerConfiguration>(), get_logger(nullptr, root_namespace_, core::getClassName<LoggerConfiguration>(), formatter_)));
   loggers.push_back(logger_);
 }
 
 void LoggerConfiguration::initialize(const std::shared_ptr<LoggerProperties> &logger_properties) {
-  std::lock_guard<std::mutex> lock(mutex);
+  std::lock_guard < std::mutex > lock(mutex);
   root_namespace_ = initialize_namespaces(logger_properties);
   std::string spdlog_pattern;
   if (!logger_properties->get("spdlog.pattern", spdlog_pattern)) {
     spdlog_pattern = spdlog_default_pattern;
   }
-  formatter_ = std::make_shared<spdlog::pattern_formatter>(spdlog_pattern);
+  formatter_ = std::make_shared < spdlog::pattern_formatter > (spdlog_pattern);
   std::map<std::string, std::shared_ptr<spdlog::logger>> spdloggers;
   for (auto const & logger_impl : loggers) {
     std::shared_ptr<spdlog::logger> spdlogger;
@@ -85,8 +85,8 @@ void LoggerConfiguration::initialize(const std::shared_ptr<LoggerProperties> &lo
 }
 
 std::shared_ptr<Logger> LoggerConfiguration::getLogger(const std::string &name) {
-  std::lock_guard<std::mutex> lock(mutex);
-  std::shared_ptr<LoggerImpl> result = std::make_shared<LoggerImpl>(name, get_logger(logger_, root_namespace_, name, formatter_));
+  std::lock_guard < std::mutex > lock(mutex);
+  std::shared_ptr<LoggerImpl> result = std::make_shared < LoggerImpl > (name, get_logger(logger_, root_namespace_, name, formatter_));
   loggers.push_back(result);
   return result;
 }
@@ -130,7 +130,7 @@ std::shared_ptr<internal::LoggerNamespace> LoggerConfiguration::initialize_names
         } catch (const std::out_of_range &oor) {
         }
       }
-      sink_map[appender_name] = std::make_shared<spdlog::sinks::rotating_file_sink_mt>(file_name, max_file_size, max_files);
+      sink_map[appender_name] = std::make_shared < spdlog::sinks::rotating_file_sink_mt > (file_name, max_file_size, max_files);
     } else if ("stdout" == appender_type) {
       sink_map[appender_name] = spdlog::sinks::stdout_sink_mt::instance();
     } else {
@@ -227,7 +227,7 @@ std::shared_ptr<spdlog::logger> LoggerConfiguration::get_logger(std::shared_ptr<
   if (logger != nullptr) {
     logger->log_debug("%s logger got sinks from namespace %s and level %s from namespace %s", name, sink_namespace_str, spdlog::level::level_names[level], level_namespace_str);
   }
-  spdlogger = std::make_shared<spdlog::logger>(name, begin(sinks), end(sinks));
+  spdlogger = std::make_shared < spdlog::logger > (name, begin(sinks), end(sinks));
   spdlogger->set_level(level);
   spdlogger->set_formatter(formatter);
   spdlogger->flush_on(std::max(spdlog::level::info, current_namespace->level));

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/core/reporting/SiteToSiteProvenanceReportingTask.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/reporting/SiteToSiteProvenanceReportingTask.cpp b/libminifi/src/core/reporting/SiteToSiteProvenanceReportingTask.cpp
index 02ddb52..d4059d6 100644
--- a/libminifi/src/core/reporting/SiteToSiteProvenanceReportingTask.cpp
+++ b/libminifi/src/core/reporting/SiteToSiteProvenanceReportingTask.cpp
@@ -24,8 +24,10 @@
 #include <string>
 #include <memory>
 #include <sstream>
+#include <functional>
 #include <iostream>
 #include <utility>
+#include "core/Repository.h"
 #include "core/reporting/SiteToSiteProvenanceReportingTask.h"
 #include "../include/io/StreamFactory.h"
 #include "io/ClientSocket.h"
@@ -51,10 +53,14 @@ void SiteToSiteProvenanceReportingTask::initialize() {
   RemoteProcessorGroupPort::initialize();
 }
 
-void SiteToSiteProvenanceReportingTask::getJsonReport(core::ProcessContext *context, core::ProcessSession *session, std::vector<std::shared_ptr<provenance::ProvenanceEventRecord>> &records,
+void SiteToSiteProvenanceReportingTask::getJsonReport(core::ProcessContext *context, core::ProcessSession *session, std::vector<std::shared_ptr<core::SerializableComponent>> &records,
                                                       std::string &report) {
   Json::Value array;
-  for (auto record : records) {
+  for (auto sercomp : records) {
+    std::shared_ptr<provenance::ProvenanceEventRecord> record = std::dynamic_pointer_cast < provenance::ProvenanceEventRecord > (sercomp);
+    if (nullptr == record) {
+      break;
+    }
     Json::Value recordJson;
     Json::Value updatedAttributesJson;
     Json::Value parentUuidJson;
@@ -108,23 +114,32 @@ void SiteToSiteProvenanceReportingTask::onTrigger(core::ProcessContext *context,
     return;
   }
 
+  logger_->log_debug("SiteToSiteProvenanceReportingTask -- onTrigger");
+
   if (!protocol_->bootstrap()) {
     // bootstrap the client protocol if needeed
     context->yield();
-    std::shared_ptr<Processor> processor = std::static_pointer_cast<Processor>(context->getProcessorNode().getProcessor());
+    std::shared_ptr<Processor> processor = std::static_pointer_cast < Processor > (context->getProcessorNode().getProcessor());
     logger_->log_error("Site2Site bootstrap failed yield period %d peer ", processor->getYieldPeriodMsec());
     returnProtocol(std::move(protocol_));
     return;
   }
 
-  std::vector<std::shared_ptr<provenance::ProvenanceEventRecord>> records;
-  std::shared_ptr<provenance::ProvenanceRepository> repo = std::static_pointer_cast<provenance::ProvenanceRepository>(context->getProvenanceRepository());
-  repo->getProvenanceRecord(records, batch_size_);
-  if (records.size() <= 0) {
+  std::vector<std::shared_ptr<core::SerializableComponent>> records;
+
+  logger_->log_debug("batch size %d records", batch_size_);
+  size_t deserialized = batch_size_;
+  std::shared_ptr<core::Repository> repo = context->getProvenanceRepository();
+  std::function < std::shared_ptr<core::SerializableComponent>() > constructor = []() {return std::make_shared<provenance::ProvenanceEventRecord>();};
+  if (!repo->DeSerialize(records, deserialized, constructor) && deserialized == 0) {
+    logger_->log_debug("Not sending because deserialized is %d", deserialized);
     returnProtocol(std::move(protocol_));
     return;
   }
 
+  logger_->log_debug("batch size %d records", batch_size_, deserialized);
+
+  logger_->log_debug("Captured %d records", deserialized);
   std::string jsonStr;
   this->getJsonReport(context, session, records, jsonStr);
   if (jsonStr.length() <= 0) {
@@ -141,7 +156,7 @@ void SiteToSiteProvenanceReportingTask::onTrigger(core::ProcessContext *context,
   }
 
   // we transfer the record, purge the record from DB
-  repo->purgeProvenanceRecord(records);
+  repo->Delete(records);
   returnProtocol(std::move(protocol_));
 }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/core/repository/FileSystemRepository.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/repository/FileSystemRepository.cpp b/libminifi/src/core/repository/FileSystemRepository.cpp
new file mode 100644
index 0000000..fba1fe3
--- /dev/null
+++ b/libminifi/src/core/repository/FileSystemRepository.cpp
@@ -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.
+ */
+
+#include "core/repository/FileSystemRepository.h"
+#include <memory>
+#include "io/FileStream.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+namespace repository {
+
+bool FileSystemRepository::initialize(const std::shared_ptr<minifi::Configure> &configuration) {
+  return true;
+}
+void FileSystemRepository::stop() {
+}
+
+std::shared_ptr<io::BaseStream> FileSystemRepository::write(const std::shared_ptr<minifi::ResourceClaim> &claim) {
+  return std::make_shared<io::FileStream>(claim->getContentFullPath());
+}
+
+std::shared_ptr<io::BaseStream> FileSystemRepository::read(const std::shared_ptr<minifi::ResourceClaim> &claim) {
+  return std::make_shared<io::FileStream>(claim->getContentFullPath(), 0, false);
+}
+
+bool FileSystemRepository::remove(const std::shared_ptr<minifi::ResourceClaim> &claim) {
+  std::remove(claim->getContentFullPath().c_str());
+  return true;
+}
+
+} /* namespace repository */
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */


[3/6] nifi-minifi-cpp git commit: MINIFI-249: Update prov repo to better abstract deser.

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/core/repository/FlowFileRepository.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/repository/FlowFileRepository.cpp b/libminifi/src/core/repository/FlowFileRepository.cpp
index e6d561a..ac092ea 100644
--- a/libminifi/src/core/repository/FlowFileRepository.cpp
+++ b/libminifi/src/core/repository/FlowFileRepository.cpp
@@ -18,6 +18,7 @@
 #include "core/repository/FlowFileRepository.h"
 #include <memory>
 #include <string>
+#include <utility>
 #include <vector>
 #include "FlowFileRecord.h"
 
@@ -36,24 +37,36 @@ void FlowFileRepository::run() {
     uint64_t curTime = getTimeMillis();
     uint64_t size = repoSize();
     if (size >= purgeThreshold) {
-      std::vector<std::string> purgeList;
+      std::vector<std::shared_ptr<FlowFileRecord>> purgeList;
+      std::vector<std::pair<std::string, uint64_t>> keyRemovalList;
       leveldb::Iterator* it = db_->NewIterator(leveldb::ReadOptions());
 
       for (it->SeekToFirst(); it->Valid(); it->Next()) {
-        std::shared_ptr<FlowFileRecord> eventRead = std::make_shared<FlowFileRecord>(shared_from_this());
+        std::shared_ptr<FlowFileRecord> eventRead = std::make_shared<FlowFileRecord>(shared_from_this(), content_repo_);
         std::string key = it->key().ToString();
         if (eventRead->DeSerialize(reinterpret_cast<const uint8_t *>(it->value().data()), it->value().size())) {
-          if ((curTime - eventRead->getEventTime()) > max_partition_millis_)
-            purgeList.push_back(key);
+          if ((curTime - eventRead->getEventTime()) > max_partition_millis_) {
+            purgeList.push_back(eventRead);
+            keyRemovalList.push_back(std::make_pair(key, it->value().size()));
+          }
         } else {
           logger_->log_debug("NiFi %s retrieve event %s fail", name_.c_str(), key.c_str());
-          purgeList.push_back(key);
+          keyRemovalList.push_back(std::make_pair(key, it->value().size()));
         }
       }
       delete it;
-      for (auto eventId : purgeList) {
-        logger_->log_info("Repository Repo %s Purge %s", name_.c_str(), eventId.c_str());
-        Delete(eventId);
+      for (auto eventId : keyRemovalList) {
+        logger_->log_info("Repository Repo %s Purge %s", name_.c_str(), eventId.first.c_str());
+        if (Delete(eventId.first)) {
+          repo_size_ -= eventId.second;
+        }
+      }
+
+      for (const auto &ffr : purgeList) {
+        auto claim = ffr->getResourceClaim();
+        if (claim != nullptr) {
+          content_repo_->remove(claim);
+        }
       }
     }
     if (size > max_partition_bytes_)
@@ -61,22 +74,23 @@ void FlowFileRepository::run() {
     else
       repo_full_ = false;
   }
-  return;
 }
 
-void FlowFileRepository::loadComponent() {
-  std::vector<std::string> purgeList;
+void FlowFileRepository::loadComponent(const std::shared_ptr<core::ContentRepository> &content_repo) {
+  content_repo_ = content_repo;
+  std::vector<std::pair<std::string, uint64_t>> purgeList;
   leveldb::Iterator* it = db_->NewIterator(leveldb::ReadOptions());
 
   for (it->SeekToFirst(); it->Valid(); it->Next()) {
-    std::shared_ptr<FlowFileRecord> eventRead = std::make_shared<FlowFileRecord>(shared_from_this());
+    std::shared_ptr<FlowFileRecord> eventRead = std::make_shared<FlowFileRecord>(shared_from_this(), content_repo_);
     std::string key = it->key().ToString();
+    repo_size_ += it->value().size();
     if (eventRead->DeSerialize(reinterpret_cast<const uint8_t *>(it->value().data()), it->value().size())) {
       auto search = connectionMap.find(eventRead->getConnectionUuid());
       if (search != connectionMap.end()) {
         // we find the connection for the persistent flowfile, create the flowfile and enqueue that
         std::shared_ptr<core::FlowFile> flow_file_ref = std::static_pointer_cast<core::FlowFile>(eventRead);
-        std::shared_ptr<FlowFileRecord> record = std::make_shared<FlowFileRecord>(shared_from_this(), flow_file_ref);
+        std::shared_ptr<FlowFileRecord> record = std::make_shared<FlowFileRecord>(shared_from_this(), content_repo_);
         // set store to repo to true so that we do need to persistent again in enqueue
         record->setStoredToRepository(true);
         search->second->put(record);
@@ -84,19 +98,19 @@ void FlowFileRepository::loadComponent() {
         if (eventRead->getContentFullPath().length() > 0) {
           std::remove(eventRead->getContentFullPath().c_str());
         }
-        purgeList.push_back(key);
+        purgeList.push_back(std::make_pair(key, it->value().size()));
       }
     } else {
-      purgeList.push_back(key);
+      purgeList.push_back(std::make_pair(key, it->value().size()));
     }
   }
 
   delete it;
-  std::vector<std::string>::iterator itPurge;
-  for (itPurge = purgeList.begin(); itPurge != purgeList.end(); itPurge++) {
-    std::string eventId = *itPurge;
-    logger_->log_info("Repository Repo %s Purge %s", name_.c_str(), eventId.c_str());
-    Delete(eventId);
+  for (auto eventId : purgeList) {
+    logger_->log_info("Repository Repo %s Purge %s", name_.c_str(), eventId.first.c_str());
+    if (Delete(eventId.first)) {
+      repo_size_ -= eventId.second;
+    }
   }
 
   return;

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/core/repository/VolatileContentRepository.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/repository/VolatileContentRepository.cpp b/libminifi/src/core/repository/VolatileContentRepository.cpp
new file mode 100644
index 0000000..ac575c5
--- /dev/null
+++ b/libminifi/src/core/repository/VolatileContentRepository.cpp
@@ -0,0 +1,183 @@
+/**
+ *
+ * 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/repository/VolatileContentRepository.h"
+#include <cstdio>
+#include <string>
+#include <memory>
+#include <thread>
+#include "utils/StringUtils.h"
+#include "io/FileStream.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+namespace repository {
+
+const char *VolatileContentRepository::minimal_locking = "minimal.locking";
+
+bool VolatileContentRepository::initialize(const std::shared_ptr<Configure> &configure) {
+  VolatileRepository::initialize(configure);
+  resource_claim_comparator_ = [](std::shared_ptr<minifi::ResourceClaim> lhsPtr, std::shared_ptr<minifi::ResourceClaim> rhsPtr) {
+    if (lhsPtr == nullptr || rhsPtr == nullptr) {
+      return false;
+    }
+    return lhsPtr->getContentFullPath() == rhsPtr->getContentFullPath();};
+  resource_claim_check_ = [](std::shared_ptr<minifi::ResourceClaim> claim) {
+    return claim->getFlowFileRecordOwnedCount() <= 0;};
+  claim_reclaimer_ = [&](std::shared_ptr<minifi::ResourceClaim> claim) {if (claim->getFlowFileRecordOwnedCount() <= 0) {
+      remove(claim);
+    }
+  };
+
+  if (configure != nullptr) {
+    bool minimize_locking = false;
+    std::string value;
+    std::stringstream strstream;
+    strstream << Configure::nifi_volatile_repository_options << getName() << "." << minimal_locking;
+    if (configure->get(strstream.str(), value)) {
+      utils::StringUtils::StringToBool(value, minimize_locking);
+      minimize_locking_ = minimize_locking;
+    }
+  }
+  if (!minimize_locking_) {
+    for (auto ent : value_vector_) {
+      delete ent;
+    }
+    value_vector_.clear();
+  }
+  start();
+
+  return true;
+}
+
+void VolatileContentRepository::stop() {
+  running_ = false;
+}
+
+void VolatileContentRepository::run() {
+}
+
+void VolatileContentRepository::start() {
+  if (this->purge_period_ <= 0)
+    return;
+  if (running_)
+    return;
+  thread_ = std::thread(&VolatileContentRepository::run, shared_from_parent<VolatileContentRepository>());
+  thread_.detach();
+  running_ = true;
+  logger_->log_info("%s Repository Monitor Thread Start", name_);
+}
+
+std::shared_ptr<io::BaseStream> VolatileContentRepository::write(const std::shared_ptr<minifi::ResourceClaim> &claim) {
+  logger_->log_debug("enter write");
+  {
+    std::lock_guard<std::mutex> lock(map_mutex_);
+    auto claim_check = master_list_.find(claim->getContentFullPath());
+    if (claim_check != master_list_.end()) {
+      logger_->log_debug("Creating copy of atomic entry");
+      auto ent = claim_check->second->takeOwnership();
+      if (ent == nullptr) {
+        logger_->log_debug("write returns nullptr");
+        return nullptr;
+      }
+      return std::make_shared<io::AtomicEntryStream<std::shared_ptr<minifi::ResourceClaim>>>(claim, ent);
+    }
+  }
+
+  int size = 0;
+  if (__builtin_expect(minimize_locking_ == true, 1)) {
+    logger_->log_debug("Minimize locking");
+    for (auto ent : value_vector_) {
+      if (ent->testAndSetKey(claim, nullptr, nullptr, resource_claim_comparator_)) {
+        std::lock_guard<std::mutex> lock(map_mutex_);
+        master_list_[claim->getContentFullPath()] = ent;
+        return std::make_shared<io::AtomicEntryStream<std::shared_ptr<minifi::ResourceClaim>>>(claim, ent);
+      }
+      size++;
+    }
+  } else {
+    std::lock_guard < std::mutex > lock(map_mutex_);
+    auto claim_check = master_list_.find(claim->getContentFullPath());
+    if (claim_check != master_list_.end()) {
+      logger_->log_debug("Creating copy of atomic entry");
+      return std::make_shared < io::AtomicEntryStream<std::shared_ptr<minifi::ResourceClaim>>>(claim, claim_check->second);
+    } else {
+      logger_->log_debug("Creating new atomic entry");
+      AtomicEntry<std::shared_ptr<minifi::ResourceClaim>> *ent = new AtomicEntry<std::shared_ptr<minifi::ResourceClaim>>(&current_size_, &max_size_);
+      if (ent->testAndSetKey(claim, nullptr, nullptr, resource_claim_comparator_)) {
+        master_list_[claim->getContentFullPath()] = ent;
+        return std::make_shared< io::AtomicEntryStream<std::shared_ptr<minifi::ResourceClaim>>>(claim, ent);
+      }
+    }
+  }
+  logger_->log_debug("write returns nullptr %d", size);
+  return nullptr;
+}
+
+std::shared_ptr<io::BaseStream> VolatileContentRepository::read(const std::shared_ptr<minifi::ResourceClaim> &claim) {
+  logger_->log_debug("enter read");
+  int size = 0;
+  {
+    std::lock_guard<std::mutex> lock(map_mutex_);
+    auto claim_check = master_list_.find(claim->getContentFullPath());
+    if (claim_check != master_list_.end()) {
+      auto ent = claim_check->second->takeOwnership();
+      if (ent == nullptr) {
+        return nullptr;
+      }
+      return std::make_shared<io::AtomicEntryStream<std::shared_ptr<minifi::ResourceClaim>>>(claim, ent);
+    }
+  }
+  logger_->log_debug("enter read for %s after %d", claim->getContentFullPath(), size);
+  return nullptr;
+}
+
+bool VolatileContentRepository::remove(const std::shared_ptr<minifi::ResourceClaim> &claim) {
+  logger_->log_debug("enter remove for %s, reducing %d", claim->getContentFullPath(), current_size_.load());
+  if (__builtin_expect(minimize_locking_ == true, 1)) {
+    std::lock_guard<std::mutex> lock(map_mutex_);
+    auto ent = master_list_.find(claim->getContentFullPath());
+    if (ent != master_list_.end()) {
+      // if we cannot remove the entry we will let the owner's destructor
+      // decrement the reference count and free it
+      if (ent->second->freeValue(claim)) {
+        logger_->log_debug("removed %s", claim->getContentFullPath());
+        return true;
+      }
+      master_list_.erase(claim->getContentFullPath());
+    }
+  } else {
+    std::lock_guard<std::mutex> lock(map_mutex_);
+    delete master_list_[claim->getContentFullPath()];
+    master_list_.erase(claim->getContentFullPath());
+    return true;
+  }
+
+  logger_->log_debug("could not remove %s", claim->getContentFullPath());
+  return false;
+}
+
+} /* namespace repository */
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/core/repository/VolatileRepository.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/repository/VolatileRepository.cpp b/libminifi/src/core/repository/VolatileRepository.cpp
index a7e3a51..6e3e1c6 100644
--- a/libminifi/src/core/repository/VolatileRepository.cpp
+++ b/libminifi/src/core/repository/VolatileRepository.cpp
@@ -16,7 +16,9 @@
  * limitations under the License.
  */
 #include "core/repository/VolatileRepository.h"
+#include <map>
 #include <memory>
+#include <limits>
 #include <string>
 #include <vector>
 #include "FlowFileRecord.h"
@@ -28,33 +30,6 @@ namespace minifi {
 namespace core {
 namespace repository {
 
-const char *VolatileRepository::volatile_repo_max_count = "max.count";
-
-void VolatileRepository::run() {
-  repo_full_ = false;
-}
-
-/**
- * Purge
- */
-void VolatileRepository::purge() {
-  while (current_size_ > max_size_) {
-    for (auto ent : value_vector_) {
-      // let the destructor do the cleanup
-      RepoValue value;
-      if (ent->getValue(value)) {
-        current_size_ -= value.size();
-        logger_->log_info("VolatileRepository -- purge %s %d %d %d", value.getKey(), current_size_.load(), max_size_, current_index_.load());
-      }
-      if (current_size_ < max_size_)
-        break;
-    }
-  }
-}
-
-void VolatileRepository::loadComponent() {
-}
-
 } /* namespace repository */
 } /* namespace core */
 } /* namespace minifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/core/yaml/YamlConfiguration.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/yaml/YamlConfiguration.cpp b/libminifi/src/core/yaml/YamlConfiguration.cpp
index 4ce944e..b5d9a8f 100644
--- a/libminifi/src/core/yaml/YamlConfiguration.cpp
+++ b/libminifi/src/core/yaml/YamlConfiguration.cpp
@@ -35,7 +35,8 @@ core::ProcessGroup *YamlConfiguration::parseRootProcessGroupYaml(YAML::Node root
   uuid_t uuid;
   int64_t version = 0;
 
-  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);
@@ -47,10 +48,8 @@ core::ProcessGroup *YamlConfiguration::parseRootProcessGroupYaml(YAML::Node root
     }
   }
 
-  logger_->log_debug(
-      "parseRootProcessGroup: id => [%s], name => [%s]", id, flowName);
-  std::unique_ptr<core::ProcessGroup> group =
-      FlowConfiguration::createRootProcessGroup(flowName, uuid, version);
+  logger_->log_debug("parseRootProcessGroup: id => [%s], name => [%s]", id, flowName);
+  std::unique_ptr<core::ProcessGroup> group = FlowConfiguration::createRootProcessGroup(flowName, uuid, version);
 
   this->name_ = flowName;
 
@@ -77,7 +76,8 @@ void YamlConfiguration::parseProcessorNodeYaml(YAML::Node processorsNode, core::
         core::ProcessorConfig procCfg;
         YAML::Node procNode = iter->as<YAML::Node>();
 
-        checkRequiredField(&procNode, "name", CONFIG_YAML_PROCESSORS_KEY);
+        checkRequiredField(&procNode, "name",
+        CONFIG_YAML_PROCESSORS_KEY);
         procCfg.name = procNode["name"].as<std::string>();
         procCfg.id = getOrGenerateId(&procNode);
         uuid_parse(procCfg.id.c_str(), uuid);
@@ -101,11 +101,13 @@ void YamlConfiguration::parseProcessorNodeYaml(YAML::Node processorsNode, core::
         }
         processor->setName(procCfg.name);
 
-        checkRequiredField(&procNode, "scheduling strategy", CONFIG_YAML_PROCESSORS_KEY);
+        checkRequiredField(&procNode, "scheduling strategy",
+        CONFIG_YAML_PROCESSORS_KEY);
         procCfg.schedulingStrategy = procNode["scheduling strategy"].as<std::string>();
         logger_->log_debug("parseProcessorNode: scheduling strategy => [%s]", procCfg.schedulingStrategy);
 
-        checkRequiredField(&procNode, "scheduling period", CONFIG_YAML_PROCESSORS_KEY);
+        checkRequiredField(&procNode, "scheduling period",
+        CONFIG_YAML_PROCESSORS_KEY);
         procCfg.schedulingPeriod = procNode["scheduling period"].as<std::string>();
         logger_->log_debug("parseProcessorNode: scheduling period => [%s]", procCfg.schedulingPeriod);
 
@@ -224,13 +226,15 @@ void YamlConfiguration::parseRemoteProcessGroupYaml(YAML::Node *rpgNode, core::P
       for (YAML::const_iterator iter = rpgNode->begin(); iter != rpgNode->end(); ++iter) {
         YAML::Node currRpgNode = iter->as<YAML::Node>();
 
-        checkRequiredField(&currRpgNode, "name", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
+        checkRequiredField(&currRpgNode, "name",
+        CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
         auto name = currRpgNode["name"].as<std::string>();
         id = getOrGenerateId(&currRpgNode);
 
         logger_->log_debug("parseRemoteProcessGroupYaml: name => [%s], id => [%s]", name, id);
 
-        checkRequiredField(&currRpgNode, "url", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
+        checkRequiredField(&currRpgNode, "url",
+        CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
         std::string url = currRpgNode["url"].as<std::string>();
         logger_->log_debug("parseRemoteProcessGroupYaml: url => [%s]", url);
 
@@ -266,7 +270,8 @@ void YamlConfiguration::parseRemoteProcessGroupYaml(YAML::Node *rpgNode, core::P
         group->setTransmitting(true);
         group->setURL(url);
 
-        checkRequiredField(&currRpgNode, "Input Ports", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
+        checkRequiredField(&currRpgNode, "Input Ports",
+        CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
         YAML::Node inputPorts = currRpgNode["Input Ports"].as<YAML::Node>();
         if (inputPorts && inputPorts.IsSequence()) {
           for (YAML::const_iterator portIter = inputPorts.begin(); portIter != inputPorts.end(); ++portIter) {
@@ -312,9 +317,11 @@ void YamlConfiguration::parseProvenanceReportingYaml(YAML::Node *reportNode, cor
 
   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>();
 
   core::TimeUnit unit;
@@ -423,7 +430,8 @@ void YamlConfiguration::parseConnectionYaml(YAML::Node *connectionsNode, core::P
 
         // Configure basic connection
         uuid_t uuid;
-        checkRequiredField(&connectionNode, "name", CONFIG_YAML_CONNECTIONS_KEY);
+        checkRequiredField(&connectionNode, "name",
+        CONFIG_YAML_CONNECTIONS_KEY);
         std::string name = connectionNode["name"].as<std::string>();
         std::string id = getOrGenerateId(&connectionNode);
         uuid_parse(id.c_str(), uuid);
@@ -431,7 +439,8 @@ void YamlConfiguration::parseConnectionYaml(YAML::Node *connectionsNode, core::P
         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);
+        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);
@@ -441,6 +450,24 @@ void YamlConfiguration::parseConnectionYaml(YAML::Node *connectionsNode, core::P
 
         uuid_t srcUUID;
 
+        if (connectionNode["max work queue size"]) {
+          auto max_work_queue_str = connectionNode["max work queue size"].as<std::string>();
+          int64_t max_work_queue_size = 0;
+          if (core::Property::StringToInt(max_work_queue_str, max_work_queue_size)) {
+            connection->setMaxQueueSize(max_work_queue_size);
+          }
+          logger_->log_debug("Setting %d as the max queue size for %s", max_work_queue_size, name);
+        }
+
+        if (connectionNode["max work queue data size"]) {
+          auto max_work_queue_str = connectionNode["max work queue data size"].as<std::string>();
+          int64_t max_work_queue_data_size = 0;
+          if (core::Property::StringToInt(max_work_queue_str, max_work_queue_data_size)) {
+            connection->setMaxQueueDataSize(max_work_queue_data_size);
+          }
+          logger_->log_debug("Setting %d as the max queue data size for %s", max_work_queue_data_size, name);
+        }
+
         if (connectionNode["source id"]) {
           std::string connectionSrcProcId = connectionNode["source id"].as<std::string>();
           uuid_parse(connectionSrcProcId.c_str(), srcUUID);
@@ -449,7 +476,8 @@ void YamlConfiguration::parseConnectionYaml(YAML::Node *connectionsNode, core::P
                              name, connectionSrcProcId);
         } else {
           // if we don't have a source id, try to resolve using source name. config schema v2 will make this unnecessary
-          checkRequiredField(&connectionNode, "source name", CONFIG_YAML_CONNECTIONS_KEY);
+          checkRequiredField(&connectionNode, "source name",
+          CONFIG_YAML_CONNECTIONS_KEY);
           std::string connectionSrcProcName = connectionNode["source name"].as<std::string>();
           uuid_t tmpUUID;
           if (!uuid_parse(connectionSrcProcName.c_str(), tmpUUID) && NULL != parent->findProcessor(tmpUUID)) {
@@ -486,7 +514,8 @@ void YamlConfiguration::parseConnectionYaml(YAML::Node *connectionsNode, core::P
         } else {
           // we use the same logic as above for resolving the source processor
           // for looking up the destination processor in absence of a processor id
-          checkRequiredField(&connectionNode, "destination name", CONFIG_YAML_CONNECTIONS_KEY);
+          checkRequiredField(&connectionNode, "destination name",
+          CONFIG_YAML_CONNECTIONS_KEY);
           std::string connectionDestProcName = connectionNode["destination name"].as<std::string>();
           uuid_t tmpUUID;
           if (!uuid_parse(connectionDestProcName.c_str(), tmpUUID) &&
@@ -534,7 +563,8 @@ void YamlConfiguration::parsePortYaml(YAML::Node *portNode, core::ProcessGroup *
   YAML::Node inputPortsObj = portNode->as<YAML::Node>();
 
   // Check for required fields
-  checkRequiredField(&inputPortsObj, "name", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
+  checkRequiredField(&inputPortsObj, "name",
+  CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
   auto nameStr = inputPortsObj["name"].as<std::string>();
   checkRequiredField(&inputPortsObj, "id",
   CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY,

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/io/AtomicEntryStream.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/io/AtomicEntryStream.cpp b/libminifi/src/io/AtomicEntryStream.cpp
new file mode 100644
index 0000000..aac9723
--- /dev/null
+++ b/libminifi/src/io/AtomicEntryStream.cpp
@@ -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.
+ */
+#include "io/AtomicEntryStream.h"
+#include <vector>
+#include <mutex>
+#include <string>
+#include "io/Serializable.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace io {
+
+} /* namespace io */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/io/ClientSocket.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/io/ClientSocket.cpp b/libminifi/src/io/ClientSocket.cpp
index 57d6f03..5f7f5f4 100644
--- a/libminifi/src/io/ClientSocket.cpp
+++ b/libminifi/src/io/ClientSocket.cpp
@@ -170,6 +170,9 @@ int16_t Socket::initialize() {
   int hh_errno;
   gethostbyname_r(host, &he, buf, sizeof(buf), &h, &hh_errno);
 #endif
+  if (h == nullptr) {
+    return -1;
+  }
   memcpy(reinterpret_cast<char*>(&addr), h->h_addr_list[0], h->h_length);
 
   auto p = addr_info_;
@@ -197,7 +200,7 @@ int16_t Socket::select_descriptor(const uint16_t msec) {
   tv.tv_sec = msec / 1000;
   tv.tv_usec = (msec % 1000) * 1000;
 
-  std::lock_guard<std::recursive_mutex> guard(selection_mutex_);
+  std::lock_guard < std::recursive_mutex > guard(selection_mutex_);
 
   if (msec > 0)
     retval = select(socket_max_ + 1, &read_fds_, NULL, NULL, &tv);
@@ -241,14 +244,12 @@ int16_t Socket::setSocketOptions(const int sock) {
   bool nagle_off = true;
 #ifndef __MACH__
   if (nagle_off) {
-    if (setsockopt(sock, SOL_TCP, TCP_NODELAY, static_cast<void*>(&opt), sizeof(opt))
-        < 0) {
+    if (setsockopt(sock, SOL_TCP, TCP_NODELAY, static_cast<void*>(&opt), sizeof(opt)) < 0) {
       logger_->log_error("setsockopt() TCP_NODELAY failed");
       close(sock);
       return -1;
     }
-    if (setsockopt(sock, SOL_SOCKET, SO_REUSEADDR, reinterpret_cast<char*>(&opt),
-            sizeof(opt)) < 0) {
+    if (setsockopt(sock, SOL_SOCKET, SO_REUSEADDR, reinterpret_cast<char*>(&opt), sizeof(opt)) < 0) {
       logger_->log_error("setsockopt() SO_REUSEADDR failed");
       close(sock);
       return -1;
@@ -256,8 +257,7 @@ int16_t Socket::setSocketOptions(const int sock) {
   }
 
   int sndsize = 256 * 1024;
-  if (setsockopt(sock, SOL_SOCKET, SO_SNDBUF, reinterpret_cast<char *>( &sndsize),
-          sizeof(sndsize)) < 0) {
+  if (setsockopt(sock, SOL_SOCKET, SO_SNDBUF, reinterpret_cast<char *>(&sndsize), sizeof(sndsize)) < 0) {
     logger_->log_error("setsockopt() SO_SNDBUF failed");
     close(sock);
     return -1;

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/io/FileStream.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/io/FileStream.cpp b/libminifi/src/io/FileStream.cpp
new file mode 100644
index 0000000..3b2bfe1
--- /dev/null
+++ b/libminifi/src/io/FileStream.cpp
@@ -0,0 +1,160 @@
+/**
+ *
+ * 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 "io/FileStream.h"
+#include <fstream>
+#include <vector>
+#include <memory>
+#include <string>
+#include "io/validation.h"
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace io {
+
+FileStream::FileStream(const std::string &path)
+    : logger_(logging::LoggerFactory<FileStream>::getLogger()),
+      path_(path),
+      offset_(0) {
+  file_stream_ = std::unique_ptr<std::fstream>(new std::fstream());
+  file_stream_->open(path.c_str(), std::fstream::out | std::fstream::binary);
+  file_stream_->seekg(0, file_stream_->end);
+  file_stream_->seekp(0, file_stream_->end);
+  int len = file_stream_->tellg();
+  if (len > 0) {
+    length_ = len;
+  } else {
+    length_ = 0;
+  }
+  seek(offset_);
+}
+
+FileStream::FileStream(const std::string &path, uint32_t offset, bool write_enable)
+    : logger_(logging::LoggerFactory<FileStream>::getLogger()),
+      path_(path) {
+  file_stream_ = std::unique_ptr<std::fstream>(new std::fstream());
+  if (write_enable) {
+    file_stream_->open(path.c_str(), std::fstream::in | std::fstream::out | std::fstream::binary);
+  } else {
+    file_stream_->open(path.c_str(), std::fstream::in | std::fstream::binary);
+  }
+  file_stream_->seekg(0, file_stream_->end);
+  file_stream_->seekp(0, file_stream_->end);
+  int len = file_stream_->tellg();
+  if (len > 0) {
+    length_ = len;
+  } else {
+    length_ = 0;
+  }
+  seek(offset);
+}
+
+void FileStream::closeStream() {
+  std::lock_guard<std::recursive_mutex> lock(file_lock_);
+  if (file_stream_ != nullptr) {
+    file_stream_->close();
+    file_stream_ = nullptr;
+  }
+}
+
+void FileStream::seek(uint64_t offset) {
+  std::lock_guard<std::recursive_mutex> lock(file_lock_);
+  offset_ = offset;
+  file_stream_->clear();
+  file_stream_->seekg(offset_);
+  file_stream_->seekp(offset_);
+}
+
+int FileStream::writeData(std::vector<uint8_t> &buf, int buflen) {
+  if (buf.capacity() < buflen) {
+    return -1;
+  }
+  return writeData(reinterpret_cast<uint8_t *>(&buf[0]), buflen);
+}
+
+// data stream overrides
+
+int FileStream::writeData(uint8_t *value, int size) {
+  if (!IsNullOrEmpty(value)) {
+    std::lock_guard<std::recursive_mutex> lock(file_lock_);
+    if (file_stream_->write(reinterpret_cast<const char*>(value), size)) {
+      offset_ += size;
+      if (offset_ > length_) {
+        length_ = offset_;
+      }
+      file_stream_->seekg(offset_);
+      file_stream_->flush();
+      return size;
+    } else {
+      return -1;
+    }
+  } else {
+    return -1;
+  }
+}
+
+template<typename T>
+inline std::vector<uint8_t> FileStream::readBuffer(const T& t) {
+  std::vector<uint8_t> buf;
+  buf.resize(sizeof t);
+  readData(reinterpret_cast<uint8_t *>(&buf[0]), sizeof(t));
+  return buf;
+}
+
+int FileStream::readData(std::vector<uint8_t> &buf, int buflen) {
+  if (buf.capacity() < buflen) {
+    buf.resize(buflen);
+  }
+  int ret = readData(reinterpret_cast<uint8_t*>(&buf[0]), buflen);
+
+  if (ret < buflen) {
+    buf.resize(ret);
+  }
+  return ret;
+}
+
+int FileStream::readData(uint8_t *buf, int buflen) {
+  if (!IsNullOrEmpty(buf)) {
+    std::lock_guard<std::recursive_mutex> lock(file_lock_);
+    file_stream_->read(reinterpret_cast<char*>(buf), buflen);
+    if ((file_stream_->rdstate() & (file_stream_->eofbit | file_stream_->failbit)) != 0) {
+      file_stream_->clear();
+      file_stream_->seekg(0, file_stream_->end);
+      file_stream_->seekp(0, file_stream_->end);
+      int len = file_stream_->tellg();
+      offset_ = len;
+      length_ = len;
+      return offset_;
+    } else {
+      offset_ += buflen;
+      file_stream_->seekp(offset_);
+      return buflen;
+    }
+
+  } else {
+    return -1;
+  }
+}
+
+} /* namespace io */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/io/StreamFactory.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/io/StreamFactory.cpp b/libminifi/src/io/StreamFactory.cpp
index 7990edd..288f4d1 100644
--- a/libminifi/src/io/StreamFactory.cpp
+++ b/libminifi/src/io/StreamFactory.cpp
@@ -47,11 +47,11 @@ class SocketCreator : public AbstractStreamFactory {
  public:
   template<typename Q = V>
   ContextTypeCheck<true, std::shared_ptr<Q>> create(const std::shared_ptr<Configure> &configure) {
-    return std::make_shared<V>(configure);
+    return std::make_shared < V > (configure);
   }
   template<typename Q = V>
   ContextTypeCheck<false, std::shared_ptr<Q>> create(const std::shared_ptr<Configure> &configure) {
-    return std::make_shared<SocketContext>(configure);
+    return std::make_shared < SocketContext > (configure);
   }
 
   SocketCreator<T, V>(std::shared_ptr<Configure> configure) {
@@ -69,7 +69,7 @@ class SocketCreator : public AbstractStreamFactory {
 
   std::unique_ptr<Socket> createSocket(const std::string &host, const uint16_t port) {
     T *socket = create(host, port);
-    return std::unique_ptr<Socket>(socket);
+    return std::unique_ptr < Socket > (socket);
   }
 
  private:

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/processors/ExecuteProcess.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/ExecuteProcess.cpp b/libminifi/src/processors/ExecuteProcess.cpp
index 323d69a..9c6f732 100644
--- a/libminifi/src/processors/ExecuteProcess.cpp
+++ b/libminifi/src/processors/ExecuteProcess.cpp
@@ -145,7 +145,7 @@ void ExecuteProcess::onTrigger(core::ProcessContext *context, core::ProcessSessi
               break;
             logger_->log_info("Execute Command Respond %d", numRead);
             ExecuteProcess::WriteCallback callback(buffer, numRead);
-            std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast<FlowFileRecord>(session->create());
+            std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast < FlowFileRecord > (session->create());
             if (!flowFile)
               continue;
             flowFile->addAttribute("command", _command.c_str());
@@ -167,7 +167,7 @@ void ExecuteProcess::onTrigger(core::ProcessContext *context, core::ProcessSessi
                 // child exits and close the pipe
                 ExecuteProcess::WriteCallback callback(buffer, totalRead);
                 if (!flowFile) {
-                  flowFile = std::static_pointer_cast<FlowFileRecord>(session->create());
+                  flowFile = std::static_pointer_cast < FlowFileRecord > (session->create());
                   if (!flowFile)
                     break;
                   flowFile->addAttribute("command", _command.c_str());
@@ -185,7 +185,7 @@ void ExecuteProcess::onTrigger(core::ProcessContext *context, core::ProcessSessi
                 logger_->log_info("Execute Command Max Respond %d", sizeof(buffer));
                 ExecuteProcess::WriteCallback callback(buffer, sizeof(buffer));
                 if (!flowFile) {
-                  flowFile = std::static_pointer_cast<FlowFileRecord>(session->create());
+                  flowFile = std::static_pointer_cast < FlowFileRecord > (session->create());
                   if (!flowFile)
                     continue;
                   flowFile->addAttribute("command", _command.c_str());

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/processors/GenerateFlowFile.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/GenerateFlowFile.cpp b/libminifi/src/processors/GenerateFlowFile.cpp
index 3741a8f..2fee3f2 100644
--- a/libminifi/src/processors/GenerateFlowFile.cpp
+++ b/libminifi/src/processors/GenerateFlowFile.cpp
@@ -91,7 +91,7 @@ void GenerateFlowFile::onTrigger(core::ProcessContext *context, core::ProcessSes
     }
     for (int i = 0; i < batchSize; i++) {
       // For each batch
-      std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast<FlowFileRecord>(session->create());
+      std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast < FlowFileRecord > (session->create());
       if (!flowFile)
         return;
       if (fileSize > 0)
@@ -114,7 +114,7 @@ void GenerateFlowFile::onTrigger(core::ProcessContext *context, core::ProcessSes
     GenerateFlowFile::WriteCallback callback(_data, _dataSize);
     for (int i = 0; i < batchSize; i++) {
       // For each batch
-      std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast<FlowFileRecord>(session->create());
+      std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast < FlowFileRecord > (session->create());
       if (!flowFile)
         return;
       if (fileSize > 0)

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/processors/GetFile.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/GetFile.cpp b/libminifi/src/processors/GetFile.cpp
index f1dbb21..723d461 100644
--- a/libminifi/src/processors/GetFile.cpp
+++ b/libminifi/src/processors/GetFile.cpp
@@ -150,7 +150,7 @@ void GetFile::onTrigger(core::ProcessContext *context, core::ProcessSession *ses
         std::string fileName = list.front();
         list.pop();
         logger_->log_info("GetFile process %s", fileName.c_str());
-        std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast<FlowFileRecord>(session->create());
+        std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast < FlowFileRecord > (session->create());
         if (flowFile == nullptr)
           return;
         std::size_t found = fileName.find_last_of("/\\");
@@ -172,19 +172,19 @@ void GetFile::onTrigger(core::ProcessContext *context, core::ProcessSession *ses
 }
 
 bool GetFile::isListingEmpty() {
-  std::lock_guard<std::mutex> lock(mutex_);
+  std::lock_guard < std::mutex > lock(mutex_);
 
   return _dirList.empty();
 }
 
 void GetFile::putListing(std::string fileName) {
-  std::lock_guard<std::mutex> lock(mutex_);
+  std::lock_guard < std::mutex > lock(mutex_);
 
   _dirList.push(fileName);
 }
 
 void GetFile::pollListing(std::queue<std::string> &list, const GetFileRequest &request) {
-  std::lock_guard<std::mutex> lock(mutex_);
+  std::lock_guard < std::mutex > lock(mutex_);
 
   while (!_dirList.empty() && (request.maxSize == 0 || list.size() < request.maxSize)) {
     std::string fileName = _dirList.front();

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/processors/InvokeHTTP.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/InvokeHTTP.cpp b/libminifi/src/processors/InvokeHTTP.cpp
index d5097bb..7dc75d2 100644
--- a/libminifi/src/processors/InvokeHTTP.cpp
+++ b/libminifi/src/processors/InvokeHTTP.cpp
@@ -42,6 +42,8 @@
 #include "io/StreamFactory.h"
 #include "ResourceClaim.h"
 #include "utils/StringUtils.h"
+#include "utils/ByteInputCallBack.h"
+#include "utils/HTTPUtils.h"
 
 namespace org {
 namespace apache {
@@ -113,7 +115,7 @@ void InvokeHTTP::set_request_method(CURL *curl, const std::string &method) {
   if (my_method == "POST") {
     curl_easy_setopt(curl, CURLOPT_POST, 1);
   } else if (my_method == "PUT") {
-    curl_easy_setopt(curl, CURLOPT_UPLOAD, 1);
+    curl_easy_setopt(curl, CURLOPT_PUT, 1);
   } else if (my_method == "GET") {
   } else {
     curl_easy_setopt(curl, CURLOPT_CUSTOMREQUEST, my_method.c_str());
@@ -210,7 +212,7 @@ void InvokeHTTP::onSchedule(core::ProcessContext *context, core::ProcessSessionF
   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);
+      ssl_context_service_ = std::static_pointer_cast < minifi::controllers::SSLContextService > (service);
     }
   }
 }
@@ -283,14 +285,14 @@ void InvokeHTTP::configure_secure_connection(CURL *http_session) {
 }
 
 void InvokeHTTP::onTrigger(core::ProcessContext *context, core::ProcessSession *session) {
-  std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast<FlowFileRecord>(session->get());
+  std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast < FlowFileRecord > (session->get());
 
   logger_->log_info("onTrigger InvokeHTTP with  %s", method_.c_str());
 
   if (flowFile == nullptr) {
     if (!emitFlowFile(method_)) {
       logger_->log_info("InvokeHTTP -- create flow file with  %s", method_.c_str());
-      flowFile = std::static_pointer_cast<FlowFileRecord>(session->create());
+      flowFile = std::static_pointer_cast < FlowFileRecord > (session->create());
     } else {
       logger_->log_info("exiting because method is %s", method_.c_str());
       return;
@@ -317,9 +319,9 @@ void InvokeHTTP::onTrigger(core::ProcessContext *context, core::ProcessSession *
   if (read_timeout_ > 0) {
     curl_easy_setopt(http_session, CURLOPT_TIMEOUT, read_timeout_);
   }
+
   utils::HTTPRequestResponse content;
-  curl_easy_setopt(http_session, CURLOPT_WRITEFUNCTION,
-                   &utils::HTTPRequestResponse::recieve_write);
+  curl_easy_setopt(http_session, CURLOPT_WRITEFUNCTION, &utils::HTTPRequestResponse::recieve_write);
 
   curl_easy_setopt(http_session, CURLOPT_WRITEDATA, static_cast<void*>(&content));
 
@@ -333,13 +335,10 @@ void InvokeHTTP::onTrigger(core::ProcessContext *context, core::ProcessSession *
       callbackObj->ptr = callback;
       callbackObj->pos = 0;
       logger_->log_info("InvokeHTTP -- Setting callback");
-      curl_easy_setopt(http_session, CURLOPT_UPLOAD, 1L);
-      curl_easy_setopt(http_session, CURLOPT_INFILESIZE_LARGE,
-                       (curl_off_t)callback->getBufferSize());
-      curl_easy_setopt(http_session, CURLOPT_READFUNCTION,
-                       &utils::HTTPRequestResponse::send_write);
-      curl_easy_setopt(http_session, CURLOPT_READDATA,
-                       static_cast<void*>(callbackObj));
+      curl_easy_setopt(http_session, CURLOPT_INFILESIZE_LARGE, (curl_off_t)callback->getBufferSize());
+      curl_easy_setopt(http_session, CURLOPT_READFUNCTION, &utils::HTTPRequestResponse::send_write);
+      curl_easy_setopt(http_session, CURLOPT_READDATA, static_cast<void*>(callbackObj));
+
     } else {
       logger_->log_error("InvokeHTTP -- no resource claim");
     }
@@ -377,14 +376,14 @@ void InvokeHTTP::onTrigger(core::ProcessContext *context, core::ProcessSession *
     bool output_body_to_content = isSuccess && !putToAttribute;
     bool body_empty = IsNullOrEmpty(content.data);
 
-    logger_->log_info("isSuccess: %d", isSuccess);
+    logger_->log_info("isSuccess: %d, response code %d ", isSuccess, http_code);
     std::shared_ptr<FlowFileRecord> response_flow = nullptr;
 
     if (output_body_to_content) {
       if (flowFile != nullptr) {
-        response_flow = std::static_pointer_cast<FlowFileRecord>(session->create(flowFile));
+        response_flow = std::static_pointer_cast < FlowFileRecord > (session->create(flowFile));
       } else {
-        response_flow = std::static_pointer_cast<FlowFileRecord>(session->create());
+        response_flow = std::static_pointer_cast < FlowFileRecord > (session->create());
       }
 
       std::string ct = content_type;
@@ -398,7 +397,7 @@ void InvokeHTTP::onTrigger(core::ProcessContext *context, core::ProcessSession *
       session->importFrom(stream, response_flow);
     } else {
       logger_->log_info("Cannot output body to content");
-      response_flow = std::static_pointer_cast<FlowFileRecord>(session->create());
+      response_flow = std::static_pointer_cast < FlowFileRecord > (session->create());
     }
     route(flowFile, response_flow, session, context, isSuccess, http_code);
   } else {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/processors/ListenHTTP.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/ListenHTTP.cpp b/libminifi/src/processors/ListenHTTP.cpp
index c26b41d..d410547 100644
--- a/libminifi/src/processors/ListenHTTP.cpp
+++ b/libminifi/src/processors/ListenHTTP.cpp
@@ -201,7 +201,7 @@ ListenHTTP::~ListenHTTP() {
 }
 
 void ListenHTTP::onTrigger(core::ProcessContext *context, core::ProcessSession *session) {
-  std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast<FlowFileRecord>(session->get());
+  std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast < FlowFileRecord > (session->get());
 
   // Do nothing if there are no incoming files
   if (!flowFile) {
@@ -243,7 +243,7 @@ bool ListenHTTP::Handler::handlePost(CivetServer *server, struct mg_connection *
 
   auto session = _processSessionFactory->createSession();
   ListenHTTP::WriteCallback callback(conn, req_info);
-  auto flowFile = std::static_pointer_cast<FlowFileRecord>(session->create());
+  auto flowFile = std::static_pointer_cast < FlowFileRecord > (session->create());
 
   if (!flowFile) {
     sendErrorResponse(conn);
@@ -295,11 +295,11 @@ ListenHTTP::WriteCallback::WriteCallback(struct mg_connection *conn, const struc
   _reqInfo = reqInfo;
 }
 
-void ListenHTTP::WriteCallback::process(std::ofstream *stream) {
+int64_t ListenHTTP::WriteCallback::process(std::shared_ptr<io::BaseStream> stream) {
   int64_t rlen;
   int64_t nlen = 0;
   int64_t tlen = _reqInfo->content_length;
-  char buf[16384];
+  uint8_t buf[16384];
 
   while (nlen < tlen) {
     rlen = tlen - nlen;
@@ -320,6 +320,8 @@ void ListenHTTP::WriteCallback::process(std::ofstream *stream) {
 
     nlen += rlen;
   }
+
+  return nlen;
 }
 
 } /* namespace processors */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/processors/ListenSyslog.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/ListenSyslog.cpp b/libminifi/src/processors/ListenSyslog.cpp
index 054d585..a5fdf28 100644
--- a/libminifi/src/processors/ListenSyslog.cpp
+++ b/libminifi/src/processors/ListenSyslog.cpp
@@ -279,7 +279,7 @@ void ListenSyslog::onTrigger(core::ProcessContext *context, core::ProcessSession
     SysLogEvent event = eventQueue.front();
     eventQueue.pop();
     if (firstEvent) {
-      flowFile = std::static_pointer_cast<FlowFileRecord>(session->create());
+      flowFile = std::static_pointer_cast < FlowFileRecord > (session->create());
       if (!flowFile)
         return;
       ListenSyslog::WriteCallback callback(event.payload, event.len);

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/processors/LogAttribute.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/LogAttribute.cpp b/libminifi/src/processors/LogAttribute.cpp
index e308901..ad8e664 100644
--- a/libminifi/src/processors/LogAttribute.cpp
+++ b/libminifi/src/processors/LogAttribute.cpp
@@ -71,8 +71,9 @@ void LogAttribute::onTrigger(core::ProcessContext *context, core::ProcessSession
 
   std::shared_ptr<core::FlowFile> flow = session->get();
 
-  if (!flow)
+  if (!flow) {
     return;
+  }
 
   std::string value;
   if (context->getProperty(LogLevel.getName(), value)) {
@@ -107,10 +108,10 @@ void LogAttribute::onTrigger(core::ProcessContext *context, core::ProcessSession
     message << "\n" << "Payload:" << "\n";
     ReadCallback callback(flow->getSize());
     session->read(flow, &callback);
-    for (unsigned int i = 0, j = 0; i < callback._readSize; i++) {
-      message << std::hex << callback._buffer[i];
+    for (unsigned int i = 0, j = 0; i < callback.read_size_; i++) {
+      message << std::hex << callback.buffer_[i];
       j++;
-      if (j == 16) {
+      if (j == 80) {
         message << '\n';
         j = 0;
       }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/processors/PutFile.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/PutFile.cpp b/libminifi/src/processors/PutFile.cpp
index d72c56a..b425ae9 100644
--- a/libminifi/src/processors/PutFile.cpp
+++ b/libminifi/src/processors/PutFile.cpp
@@ -17,20 +17,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-#include "processors/PutFile.h"
-#include <stdio.h>
+
+#include "../../include/processors/PutFile.h"
+
+#include <sys/stat.h>
+#include <unistd.h>
 #include <uuid/uuid.h>
-#include <sstream>
-#include <string>
+#include <cstdint>
+#include <cstdio>
 #include <iostream>
 #include <memory>
 #include <set>
-#include <fstream>
-#include "io/validation.h"
-#include "utils/StringUtils.h"
-#include "utils/TimeUtil.h"
-#include "core/ProcessContext.h"
-#include "core/ProcessSession.h"
+#include <string>
+
+#include "../../include/core/logging/Logger.h"
+#include "../../include/core/ProcessContext.h"
+#include "../../include/core/Property.h"
+#include "../../include/core/Relationship.h"
+#include "../../include/io/BaseStream.h"
+#include "../../include/io/DataStream.h"
+#include "../../include/io/validation.h"
 
 namespace org {
 namespace apache {
@@ -76,7 +82,7 @@ void PutFile::onTrigger(core::ProcessContext *context, core::ProcessSession *ses
     return;
   }
 
-  std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast<FlowFileRecord>(session->get());
+  std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast < FlowFileRecord > (session->get());
 
   // Do nothing if there are no incoming files
   if (!flowFile) {
@@ -142,10 +148,23 @@ PutFile::ReadCallback::ReadCallback(const std::string &tmpFile, const std::strin
 }
 
 // Copy the entire file contents to the temporary file
-void PutFile::ReadCallback::process(std::ifstream *stream) {
+int64_t PutFile::ReadCallback::process(std::shared_ptr<io::BaseStream> stream) {
   // Copy file contents into tmp file
   _writeSucceeded = false;
-  _tmpFileOs << stream->rdbuf();
+  size_t size = 0;
+  uint8_t buffer[1024];
+  do {
+    int read = stream->read(buffer, 1024);
+    if (read < 0) {
+      return -1;
+    }
+    if (read == 0) {
+      break;
+    }
+    _tmpFileOs.write(reinterpret_cast<char*>(buffer), read);
+    size += read;
+  } while (size < stream->getSize());
+  return size;
   _writeSucceeded = true;
 }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/processors/TailFile.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/TailFile.cpp b/libminifi/src/processors/TailFile.cpp
index 46ed1fb..f87f4ec 100644
--- a/libminifi/src/processors/TailFile.cpp
+++ b/libminifi/src/processors/TailFile.cpp
@@ -51,7 +51,8 @@ core::Property TailFile::StateFile("State File", "Specifies the file that should
                                    " what data has been ingested so that upon restart NiFi can resume from where it left off",
                                    "TailFileState");
 core::Property TailFile::Delimiter("Input Delimiter", "Specifies the character that should be used for delimiting the data being tailed"
-                                    "from the incoming file.", "");
+                                   "from the incoming file.",
+                                   "");
 core::Relationship TailFile::Success("success", "All files are routed to success");
 
 void TailFile::initialize() {
@@ -240,10 +241,9 @@ void TailFile::onTrigger(core::ProcessContext *context, core::ProcessSession *se
       context->yield();
       return;
     }
-
-      std::size_t found = _currentTailFileName.find_last_of(".");
-      std::string baseName = _currentTailFileName.substr(0, found);
-      std::string extension = _currentTailFileName.substr(found + 1);
+    std::size_t found = _currentTailFileName.find_last_of(".");
+    std::string baseName = _currentTailFileName.substr(0, found);
+    std::string extension = _currentTailFileName.substr(found + 1);
 
     if (!this->_delimiter.empty()) {
       char delim = this->_delimiter.c_str()[0];
@@ -254,20 +254,20 @@ void TailFile::onTrigger(core::ProcessContext *context, core::ProcessSession *se
       for (std::shared_ptr<FlowFileRecord> ffr : flowFiles) {
         logger_->log_info("TailFile %s for %d bytes", _currentTailFileName, ffr->getSize());
         std::string logName = baseName + "." + std::to_string(_currentTailFilePosition) + "-" + std::to_string(_currentTailFilePosition + ffr->getSize()) + "." + extension;
-          ffr->updateKeyedAttribute(PATH, fileLocation);
-          ffr->addKeyedAttribute(ABSOLUTE_PATH, fullPath);
+        ffr->updateKeyedAttribute(PATH, fileLocation);
+        ffr->addKeyedAttribute(ABSOLUTE_PATH, fullPath);
         ffr->updateKeyedAttribute(FILENAME, logName);
-          session->transfer(ffr, Success);
+        session->transfer(ffr, Success);
         this->_currentTailFilePosition += ffr->getSize() + 1;
         storeState();
       }
 
     } else {
-        std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast<FlowFileRecord>(session->create());
-        if (!flowFile)
-            return;
-        flowFile->updateKeyedAttribute(PATH, fileLocation);
-        flowFile->addKeyedAttribute(ABSOLUTE_PATH, fullPath);
+      std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast<FlowFileRecord>(session->create());
+      if (!flowFile)
+        return;
+      flowFile->updateKeyedAttribute(PATH, fileLocation);
+      flowFile->addKeyedAttribute(ABSOLUTE_PATH, fullPath);
       session->import(fullPath, flowFile, true, this->_currentTailFilePosition);
       session->transfer(flowFile, Success);
       logger_->log_info("TailFile %s for %d bytes", _currentTailFileName, flowFile->getSize());

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/provenance/Provenance.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/provenance/Provenance.cpp b/libminifi/src/provenance/Provenance.cpp
index 8686a58..3e42a5a 100644
--- a/libminifi/src/provenance/Provenance.cpp
+++ b/libminifi/src/provenance/Provenance.cpp
@@ -17,13 +17,12 @@
  */
 
 #include "provenance/Provenance.h"
-
 #include <arpa/inet.h>
 #include <cstdint>
 #include <memory>
 #include <string>
 #include <vector>
-
+#include "core/Repository.h"
 #include "io/DataStream.h"
 #include "io/Serializable.h"
 #include "core/logging/Logger.h"
@@ -42,30 +41,35 @@ std::shared_ptr<logging::Logger> ProvenanceEventRecord::logger_ = logging::Logge
 const char *ProvenanceEventRecord::ProvenanceEventTypeStr[REPLAY + 1] = { "CREATE", "RECEIVE", "FETCH", "SEND", "DOWNLOAD", "DROP", "EXPIRE", "FORK", "JOIN", "CLONE", "CONTENT_MODIFIED",
     "ATTRIBUTES_MODIFIED", "ROUTE", "ADDINFO", "REPLAY" };
 
-ProvenanceEventRecord::ProvenanceEventRecord(ProvenanceEventRecord::ProvenanceEventType event, std::string componentId, std::string componentType) {
+ProvenanceEventRecord::ProvenanceEventRecord(ProvenanceEventRecord::ProvenanceEventType event, std::string componentId, std::string componentType)
+    : core::SerializableComponent(core::getClassName<ProvenanceEventRecord>()),
+      _eventDuration(0),
+      _entryDate(0),
+      _lineageStartDate(0) {
   _eventType = event;
   _componentId = componentId;
   _componentType = componentType;
   _eventTime = getTimeMillis();
-  char eventIdStr[37];
-  // Generate the global UUID for th event
-  id_generator_->generate(_eventId);
-  uuid_unparse_lower(_eventId, eventIdStr);
-  _eventIdStr = eventIdStr;
 }
 
 // DeSerialize
-bool ProvenanceEventRecord::DeSerialize(const std::shared_ptr<core::Repository> &repo, std::string key) {
+bool ProvenanceEventRecord::DeSerialize(const std::shared_ptr<core::SerializableComponent> &store) {
   std::string value;
   bool ret;
 
-  ret = repo->Get(key, value);
+  const std::shared_ptr<core::Repository> repo = std::dynamic_pointer_cast<core::Repository>(store);
+
+  if (nullptr == repo || IsNullOrEmpty(uuidStr_)) {
+    logger_->log_error("Repo could not be assigned");
+    return false;
+  }
+  ret = repo->Get(uuidStr_, value);
 
   if (!ret) {
-    logger_->log_error("NiFi Provenance Store event %s can not found", key.c_str());
+    logger_->log_error("NiFi Provenance Store event %s can not be found", uuidStr_);
     return false;
   } else {
-    logger_->log_debug("NiFi Provenance Read event %s length %d", key.c_str(), value.length());
+    logger_->log_debug("NiFi Provenance Read event %s length %d", uuidStr_, value.length());
   }
 
   org::apache::nifi::minifi::io::DataStream stream((const uint8_t*) value.data(), value.length());
@@ -73,20 +77,20 @@ bool ProvenanceEventRecord::DeSerialize(const std::shared_ptr<core::Repository>
   ret = DeSerialize(stream);
 
   if (ret) {
-    logger_->log_debug("NiFi Provenance retrieve event %s size %d eventType %d success", _eventIdStr.c_str(), stream.getSize(), _eventType);
+    logger_->log_debug("NiFi Provenance retrieve event %s size %d eventType %d success", uuidStr_, stream.getSize(), _eventType);
   } else {
-    logger_->log_debug("NiFi Provenance retrieve event %s size %d eventType %d fail", _eventIdStr.c_str(), stream.getSize(), _eventType);
+    logger_->log_debug("NiFi Provenance retrieve event %s size %d eventType %d fail", uuidStr_, stream.getSize(), _eventType);
   }
 
   return ret;
 }
 
-bool ProvenanceEventRecord::Serialize(const std::shared_ptr<core::Repository> &repo) {
+bool ProvenanceEventRecord::Serialize(const std::shared_ptr<core::SerializableComponent> &repo) {
   org::apache::nifi::minifi::io::DataStream outStream;
 
   int ret;
 
-  ret = writeUTF(this->_eventIdStr, &outStream);
+  ret = writeUTF(this->uuidStr_, &outStream);
   if (ret <= 0) {
     return false;
   }
@@ -127,7 +131,7 @@ bool ProvenanceEventRecord::Serialize(const std::shared_ptr<core::Repository> &r
     return false;
   }
 
-  ret = writeUTF(this->uuid_, &outStream);
+  ret = writeUTF(this->flow_uuid_, &outStream);
   if (ret <= 0) {
     return false;
   }
@@ -215,20 +219,20 @@ bool ProvenanceEventRecord::Serialize(const std::shared_ptr<core::Repository> &r
     }
   }
   // Persistent to the DB
-  if (repo->Put(_eventIdStr, const_cast<uint8_t*>(outStream.getBuffer()), outStream.getSize())) {
-    logger_->log_debug("NiFi Provenance Store event %s size %d success", _eventIdStr.c_str(), outStream.getSize());
+  if (repo->Serialize(uuidStr_, const_cast<uint8_t*>(outStream.getBuffer()), outStream.getSize())) {
+    logger_->log_debug("NiFi Provenance Store event %s size %d success", uuidStr_, outStream.getSize());
   } else {
-    logger_->log_error("NiFi Provenance Store event %s size %d fail", _eventIdStr.c_str(), outStream.getSize());
+    logger_->log_error("NiFi Provenance Store event %s size %d fail", uuidStr_, outStream.getSize());
   }
   return true;
 }
 
-bool ProvenanceEventRecord::DeSerialize(const uint8_t *buffer, const int bufferSize) {
+bool ProvenanceEventRecord::DeSerialize(const uint8_t *buffer, const size_t bufferSize) {
   int ret;
 
   org::apache::nifi::minifi::io::DataStream outStream(buffer, bufferSize);
 
-  ret = readUTF(this->_eventIdStr, &outStream);
+  ret = readUTF(this->uuidStr_, &outStream);
 
   if (ret <= 0) {
     return false;
@@ -271,7 +275,7 @@ bool ProvenanceEventRecord::DeSerialize(const uint8_t *buffer, const int bufferS
     return false;
   }
 
-  ret = readUTF(this->uuid_, &outStream);
+  ret = readUTF(this->flow_uuid_, &outStream);
   if (ret <= 0) {
     return false;
   }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/src/provenance/ProvenanceRepository.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/provenance/ProvenanceRepository.cpp b/libminifi/src/provenance/ProvenanceRepository.cpp
index e4a8ffa..ce19fe4 100644
--- a/libminifi/src/provenance/ProvenanceRepository.cpp
+++ b/libminifi/src/provenance/ProvenanceRepository.cpp
@@ -31,31 +31,23 @@ void ProvenanceRepository::run() {
   uint64_t purgeThreshold = max_partition_bytes_ * 3 / 4;
   while (running_) {
     std::this_thread::sleep_for(std::chrono::milliseconds(purge_period_));
-    std::this_thread::sleep_for(std::chrono::milliseconds(purge_period_));
     uint64_t curTime = getTimeMillis();
     uint64_t size = repoSize();
     if (size >= purgeThreshold) {
-      std::vector<std::string> purgeList;
       leveldb::Iterator* it = db_->NewIterator(leveldb::ReadOptions());
       for (it->SeekToFirst(); it->Valid(); it->Next()) {
         ProvenanceEventRecord eventRead;
         std::string key = it->key().ToString();
-        if (eventRead.DeSerialize(reinterpret_cast<uint8_t*>(const_cast<char*>(it->value().data())), it->value().size())) {
-          if ((curTime - eventRead.getEventTime()) > max_partition_millis_)
-            purgeList.push_back(key);
+        uint64_t eventTime = eventRead.getEventTime(reinterpret_cast<uint8_t*>(const_cast<char*>(it->value().data())), it->value().size());
+        if (eventTime > 0) {
+          if ((curTime - eventTime) > max_partition_millis_)
+            Delete(key);
         } else {
           logger_->log_debug("NiFi Provenance retrieve event %s fail", key.c_str());
-          purgeList.push_back(key);
+          Delete(key);
         }
       }
       delete it;
-      std::vector<std::string>::iterator itPurge;
-
-      for (itPurge = purgeList.begin(); itPurge != purgeList.end(); itPurge++) {
-        std::string eventId = *itPurge;
-        logger_->log_info("ProvenanceRepository Repo Purge %s", eventId.c_str());
-        Delete(eventId);
-      }
     }
     if (size > max_partition_bytes_)
       repo_full_ = true;

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/test/CPPLINT.cfg
----------------------------------------------------------------------
diff --git a/libminifi/test/CPPLINT.cfg b/libminifi/test/CPPLINT.cfg
index beed48a..a1e22ad 100644
--- a/libminifi/test/CPPLINT.cfg
+++ b/libminifi/test/CPPLINT.cfg
@@ -1,3 +1,4 @@
 set noparent
 filter=-build/include_order,-build/include_alpha
 exclude_files=Server.cpp
+exclude_files=TestBase.cpp
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/test/TestBase.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/TestBase.cpp b/libminifi/test/TestBase.cpp
new file mode 100644
index 0000000..4c0814d
--- /dev/null
+++ b/libminifi/test/TestBase.cpp
@@ -0,0 +1,211 @@
+/**
+ *
+ * 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 "./TestBase.h"
+#include <memory>
+#include <vector>
+#include <set>
+#include <string>
+
+TestPlan::TestPlan(std::shared_ptr<core::ContentRepository> content_repo, std::shared_ptr<core::Repository> flow_repo, std::shared_ptr<core::Repository> prov_repo)
+    :
+      content_repo_(content_repo),
+      flow_repo_(flow_repo),
+      prov_repo_(prov_repo),
+      location(-1),
+      finalized(false),
+      current_flowfile_(nullptr) {
+}
+
+std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::shared_ptr<core::Processor> &processor, const std::string &name, core::Relationship relationship,
+bool linkToPrevious) {
+  if (finalized) {
+    return nullptr;
+  }
+  std::lock_guard<std::recursive_mutex> guard(mutex);
+
+  uuid_t uuid;
+  uuid_generate(uuid);
+
+  // initialize the processor
+  processor->initialize();
+
+  processor_mapping_[processor->getUUIDStr()] = processor;
+
+  if (!linkToPrevious) {
+    termination_ = relationship;
+  } else {
+    std::shared_ptr<core::Processor> last = processor_queue_.back();
+
+    if (last == nullptr) {
+      last = processor;
+      termination_ = relationship;
+    }
+
+    std::stringstream connection_name;
+    connection_name << last->getUUIDStr() << "-to-" << processor->getUUIDStr();
+    std::shared_ptr<minifi::Connection> connection = std::make_shared<minifi::Connection>(flow_repo_, content_repo_, connection_name.str());
+    connection->setRelationship(relationship);
+
+    // link the connections so that we can test results at the end for this
+    connection->setSource(last);
+    connection->setDestination(processor);
+
+    uuid_t uuid_copy, uuid_copy_next;
+    last->getUUID(uuid_copy);
+    connection->setSourceUUID(uuid_copy);
+    processor->getUUID(uuid_copy_next);
+    connection->setDestinationUUID(uuid_copy_next);
+    last->addConnection(connection);
+    if (last != processor) {
+      processor->addConnection(connection);
+    }
+    relationships_.push_back(connection);
+  }
+
+  std::shared_ptr<core::ProcessorNode> node = std::make_shared<core::ProcessorNode>(processor);
+
+  processor_nodes_.push_back(node);
+
+  std::shared_ptr<core::ProcessContext> context = std::make_shared<core::ProcessContext>(*(node.get()), controller_services_provider_, flow_repo_, prov_repo_, content_repo_);
+  processor_contexts_.push_back(context);
+
+  processor_queue_.push_back(processor);
+
+  return processor;
+}
+
+std::shared_ptr<core::Processor> TestPlan::addProcessor(const std::string &processor_name, const std::string &name, core::Relationship relationship,
+bool linkToPrevious) {
+  if (finalized) {
+    return nullptr;
+  }
+  std::lock_guard<std::recursive_mutex> guard(mutex);
+
+  uuid_t uuid;
+  uuid_generate(uuid);
+
+  auto ptr = core::ClassLoader::getDefaultClassLoader().instantiate(processor_name, uuid);
+  if (nullptr == ptr) {
+    throw std::exception();
+  }
+  std::shared_ptr<core::Processor> processor = std::static_pointer_cast<core::Processor>(ptr);
+
+  processor->setName(name);
+
+  return addProcessor(processor, name, relationship, linkToPrevious);
+}
+
+bool TestPlan::setProperty(const std::shared_ptr<core::Processor> proc, const std::string &prop, const std::string &value) {
+  std::lock_guard<std::recursive_mutex> guard(mutex);
+  int i = 0;
+  for (i = 0; i < processor_queue_.size(); i++) {
+    if (processor_queue_.at(i) == proc) {
+      break;
+    }
+  }
+  if (i >= processor_queue_.size() || i < 0 || i >= processor_contexts_.size()) {
+    return false;
+  }
+
+  return processor_contexts_.at(i)->setProperty(prop, value);
+}
+
+void TestPlan::reset() {
+  std::lock_guard<std::recursive_mutex> guard(mutex);
+  process_sessions_.clear();
+  factories_.clear();
+  location = -1;
+  for (auto proc : processor_queue_) {
+    while (proc->getActiveTasks() > 0) {
+      proc->decrementActiveTask();
+    }
+  }
+}
+
+bool TestPlan::runNextProcessor(std::function<void(core::ProcessContext*, core::ProcessSession*)> verify) {
+  if (!finalized) {
+    finalize();
+  }
+  std::lock_guard<std::recursive_mutex> guard(mutex);
+  location++;
+  std::shared_ptr<core::Processor> processor = processor_queue_.at(location);
+  std::shared_ptr<core::ProcessContext> context = processor_contexts_.at(location);
+  std::shared_ptr<core::ProcessSessionFactory> factory = std::make_shared<core::ProcessSessionFactory>(context.get());
+  factories_.push_back(factory);
+  if (std::find(configured_processors_.begin(), configured_processors_.end(), processor) == configured_processors_.end()) {
+    processor->onSchedule(context.get(), factory.get());
+    configured_processors_.push_back(processor);
+  }
+  std::shared_ptr<core::ProcessSession> current_session = std::make_shared<core::ProcessSession>(context.get());
+  process_sessions_.push_back(current_session);
+  processor->incrementActiveTasks();
+  processor->setScheduledState(core::ScheduledState::RUNNING);
+  if (verify != nullptr) {
+    verify(context.get(), current_session.get());
+  } else {
+    processor->onTrigger(context.get(), current_session.get());
+  }
+  current_session->commit();
+  current_flowfile_ = current_session->get();
+  return location + 1 < processor_queue_.size();
+}
+
+std::set<provenance::ProvenanceEventRecord*> TestPlan::getProvenanceRecords() {
+  return process_sessions_.at(location)->getProvenanceReporter()->getEvents();
+}
+
+std::shared_ptr<core::FlowFile> TestPlan::getCurrentFlowFile() {
+  return current_flowfile_;
+}
+
+std::shared_ptr<minifi::Connection> TestPlan::buildFinalConnection(std::shared_ptr<core::Processor> processor, bool setDest) {
+  std::stringstream connection_name;
+  std::shared_ptr<core::Processor> last = processor;
+  connection_name << last->getUUIDStr() << "-to-" << processor->getUUIDStr();
+  std::shared_ptr<minifi::Connection> connection = std::make_shared<minifi::Connection>(flow_repo_, content_repo_, connection_name.str());
+  connection->setRelationship(termination_);
+
+  // link the connections so that we can test results at the end for this
+  connection->setSource(last);
+  if (setDest)
+    connection->setDestination(processor);
+
+  uuid_t uuid_copy;
+  last->getUUID(uuid_copy);
+  connection->setSourceUUID(uuid_copy);
+  if (setDest)
+    connection->setDestinationUUID(uuid_copy);
+
+  processor->addConnection(connection);
+  return connection;
+}
+
+void TestPlan::finalize() {
+  std::lock_guard<std::recursive_mutex> guard(mutex);
+  if (relationships_.size() > 0) {
+    relationships_.push_back(buildFinalConnection(processor_queue_.back()));
+  } else {
+    for (auto processor : processor_queue_) {
+      relationships_.push_back(buildFinalConnection(processor, true));
+    }
+  }
+
+  finalized = true;
+}
+

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/test/TestBase.h
----------------------------------------------------------------------
diff --git a/libminifi/test/TestBase.h b/libminifi/test/TestBase.h
index 331df08..47db4c3 100644
--- a/libminifi/test/TestBase.h
+++ b/libminifi/test/TestBase.h
@@ -25,6 +25,8 @@
 #include "ResourceClaim.h"
 #include "catch.hpp"
 #include <vector>
+#include <set>
+#include <map>
 #include "core/logging/Logger.h"
 #include "core/Core.h"
 #include "properties/Configure.h"
@@ -33,6 +35,14 @@
 #include "utils/Id.h"
 #include "spdlog/sinks/ostream_sink.h"
 #include "spdlog/sinks/dist_sink.h"
+#include "unit/ProvenanceTestHelper.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"
 
 class LogTestController {
  public:
@@ -105,7 +115,7 @@ class LogTestController {
   std::ostringstream log_output;
 
   std::shared_ptr<logging::Logger> logger_;
- private:
+   private:
   class TestBootstrapLogger : public logging::Logger {
    public:
     TestBootstrapLogger(std::shared_ptr<spdlog::logger> logger)
@@ -138,6 +148,73 @@ class LogTestController {
   std::vector<std::string> modified_loggers;
 };
 
+class TestPlan {
+ public:
+
+  explicit TestPlan(std::shared_ptr<core::ContentRepository> content_repo, std::shared_ptr<core::Repository> flow_repo, std::shared_ptr<core::Repository> prov_repo);
+
+  std::shared_ptr<core::Processor> addProcessor(const std::shared_ptr<core::Processor> &processor, const std::string &name,
+                                                core::Relationship relationship = core::Relationship("success", "description"),
+                                                bool linkToPrevious = false);
+
+  std::shared_ptr<core::Processor> addProcessor(const std::string &processor_name, const std::string &name, core::Relationship relationship = core::Relationship("success", "description"),
+  bool linkToPrevious = false);
+
+  bool setProperty(const std::shared_ptr<core::Processor> proc, const std::string &prop, const std::string &value);
+
+  void reset();
+
+  bool runNextProcessor(std::function<void(core::ProcessContext*, core::ProcessSession*)> verify = nullptr);
+
+  std::set<provenance::ProvenanceEventRecord*> getProvenanceRecords();
+
+  std::shared_ptr<core::FlowFile> getCurrentFlowFile();
+
+  std::shared_ptr<core::Repository> getFlowRepo() {
+    return flow_repo_;
+  }
+
+  std::shared_ptr<core::Repository> getProvenanceRepo() {
+    return prov_repo_;
+  }
+
+  std::shared_ptr<core::ContentRepository> getContentRepo() {
+    return content_repo_;
+  }
+
+ protected:
+
+  void finalize();
+
+  std::shared_ptr<minifi::Connection> buildFinalConnection(std::shared_ptr<core::Processor> processor, bool setDest = false);
+
+  std::atomic<bool> finalized;
+
+  std::shared_ptr<core::ContentRepository> content_repo_;
+
+  std::shared_ptr<core::Repository> flow_repo_;
+  std::shared_ptr<core::Repository> prov_repo_;
+
+  std::shared_ptr<core::controller::ControllerServiceProvider> controller_services_provider_;
+
+  std::recursive_mutex mutex;
+
+  int location;
+
+  std::shared_ptr<core::ProcessSession> current_session_;
+  std::shared_ptr<core::FlowFile> current_flowfile_;
+
+  std::map<std::string, std::shared_ptr<core::Processor>> processor_mapping_;
+  std::vector<std::shared_ptr<core::Processor>> processor_queue_;
+  std::vector<std::shared_ptr<core::Processor>> configured_processors_;
+  std::vector<std::shared_ptr<core::ProcessorNode>> processor_nodes_;
+  std::vector<std::shared_ptr<core::ProcessContext>> processor_contexts_;
+  std::vector<std::shared_ptr<core::ProcessSession>> process_sessions_;
+  std::vector<std::shared_ptr<core::ProcessSessionFactory>> factories_;
+  std::vector<std::shared_ptr<minifi::Connection>> relationships_;
+  core::Relationship termination_;
+};
+
 class TestController {
  public:
 
@@ -148,6 +225,25 @@ class TestController {
     utils::IdGenerator::getIdGenerator()->initialize(std::make_shared<minifi::Properties>());
   }
 
+  std::shared_ptr<TestPlan> createPlan()
+  {
+    std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
+    std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
+
+    content_repo->initialize(configuration);
+
+    std::shared_ptr<core::Repository> repo = std::make_shared<TestRepository>();
+    return std::make_shared<TestPlan>(content_repo, repo, repo);
+  }
+
+  void runSession(std::shared_ptr<TestPlan> &plan, bool runToCompletion = true, std::function<void(core::ProcessContext*, core::ProcessSession*)> verify = nullptr) {
+
+    while (plan->runNextProcessor(verify) && runToCompletion)
+    {
+
+    }
+  }
+
   ~TestController() {
     for (auto dir : directories) {
       DIR *created_dir;
@@ -176,6 +272,10 @@ class TestController {
   }
 
  protected:
+
+  std::mutex test_mutex;
+  //std::map<std::string,>
+
   LogTestController &log;
   std::vector<char*> directories;
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/test/TestServer.h
----------------------------------------------------------------------
diff --git a/libminifi/test/TestServer.h b/libminifi/test/TestServer.h
new file mode 100644
index 0000000..263a6b3
--- /dev/null
+++ b/libminifi/test/TestServer.h
@@ -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.
+ */
+#ifndef LIBMINIFI_TEST_TESTSERVER_H_
+#define LIBMINIFI_TEST_TESTSERVER_H_
+#include <regex.h>
+#include <string>
+#include <iostream>
+#include "civetweb.h"
+
+/* Server context handle */
+static struct mg_context *ctx;
+static std::string resp_str;
+
+static int responder(struct mg_connection *conn, void *response) {
+  const char *msg = resp_str.c_str();
+
+
+  mg_printf(conn, "HTTP/1.1 200 OK\r\n"
+            "Content-Length: %lu\r\n"
+            "Content-Type: text/plain\r\n"
+            "Connection: close\r\n\r\n",
+            resp_str.size());
+
+  mg_write(conn, msg, resp_str.size());
+
+  return 200;
+}
+
+void init_webserver() {
+  mg_init_library(0);
+}
+
+void start_webserver(std::string &port, std::string &rooturi, const std::string &response, struct mg_callbacks *callbacks, std::string &cert) {
+
+  std::cout << "root uri is " << rooturi << ":" << port << "/" << std::endl;
+  resp_str = response;
+  const char *options[] = { "listening_ports", port.c_str(), "ssl_certificate", cert.c_str(), "ssl_protocol_version", "3", "ssl_cipher_list",
+      "ECDHE-RSA-AES256-GCM-SHA384:DES-CBC3-SHA:AES128-SHA:AES128-GCM-SHA256", 0 };
+
+  if (!mg_check_feature(2)) {
+    std::cerr << "Error: Embedded example built with SSL support, " << "but civetweb library build without" << std::endl;
+    exit(1);
+  }
+
+  ctx = mg_start(callbacks, 0, options);
+  if (ctx == nullptr) {
+    std::cerr << "Cannot start CivetWeb - mg_start failed." << std::endl;
+    exit(1);
+  }
+
+  mg_set_request_handler(ctx, rooturi.c_str(), responder, (void*) &resp_str);
+
+}
+
+void start_webserver(std::string &port, std::string &rooturi, const std::string &response) {
+
+  std::cout << "root uri is " << rooturi << ":" << port << "/" << std::endl;
+  resp_str = response;
+
+  const char *options[] = { "listening_ports", port.c_str(), 0 };
+  ctx = mg_start(nullptr, 0, options);
+
+  if (ctx == nullptr) {
+    std::cerr << "Cannot start CivetWeb - mg_start failed." << std::endl;
+    exit(1);
+  }
+
+  mg_set_request_handler(ctx, rooturi.c_str(), responder, (void*) &resp_str);
+
+}
+
+bool parse_http_components(const std::string &url, std::string &port, std::string &scheme, std::string &path) {
+  regex_t regex;
+
+  const char *regexstr = "^(http|https)://(localhost:)([0-9]+)?(/.*)$";
+
+  int ret = regcomp(&regex, regexstr, REG_EXTENDED);
+  if (ret) {
+    return false;
+  }
+
+  size_t potentialGroups = regex.re_nsub + 1;
+  regmatch_t groups[potentialGroups];
+  if (regexec(&regex, url.c_str(), potentialGroups, groups, 0) == 0) {
+    for (int i = 0; i < potentialGroups; i++) {
+      if (groups[i].rm_so == -1)
+        break;
+
+      std::string str(url.data() + groups[i].rm_so, groups[i].rm_eo - groups[i].rm_so);
+      switch (i) {
+        case 1:
+          scheme = str;
+          break;
+        case 3:
+          port = str;
+          break;
+        case 4:
+          path = str;
+          break;
+        default:
+          break;
+      }
+    }
+  }
+  if (path.empty() || scheme.empty() || port.empty())
+    return false;
+
+  regfree(&regex);
+
+  return true;
+
+}
+
+static void stop_webserver() {
+  /* Stop the server */
+  mg_stop(ctx);
+
+  /* Un-initialize the library */
+  mg_exit_library();
+}
+
+#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/test/integration/ControllerServiceIntegrationTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/integration/ControllerServiceIntegrationTests.cpp b/libminifi/test/integration/ControllerServiceIntegrationTests.cpp
index 3f27b66..15720eb 100644
--- a/libminifi/test/integration/ControllerServiceIntegrationTests.cpp
+++ b/libminifi/test/integration/ControllerServiceIntegrationTests.cpp
@@ -80,18 +80,21 @@ int main(int argc, char **argv) {
   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<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
+  content_repo->initialize(configuration);
+  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr<core::YamlConfiguration>(
+      new core::YamlConfiguration(test_repo, test_repo, content_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,
+                                                                                                content_repo,
+                                                                                                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);
+  core::YamlConfiguration yaml_config(test_repo, test_repo, content_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());

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/7c16d1bb/libminifi/test/integration/HttpConfigurationListenerTest.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/integration/HttpConfigurationListenerTest.cpp b/libminifi/test/integration/HttpConfigurationListenerTest.cpp
index a86b884..b559f41 100644
--- a/libminifi/test/integration/HttpConfigurationListenerTest.cpp
+++ b/libminifi/test/integration/HttpConfigurationListenerTest.cpp
@@ -46,7 +46,7 @@ void waitToVerifyProcessor() {
   std::this_thread::sleep_for(std::chrono::seconds(10));
 }
 
-class ConfigHandler: public CivetHandler {
+class ConfigHandler : public CivetHandler {
  public:
   bool handleGet(CivetServer *server, struct mg_connection *conn) {
     std::ifstream myfile(test_file_location_.c_str());
@@ -57,8 +57,8 @@ class ConfigHandler: public CivetHandler {
       std::string str = buffer.str();
       myfile.close();
       mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
-          "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
-          str.length());
+                "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
+                str.length());
       mg_printf(conn, "%s", str.c_str());
     } else {
       mg_printf(conn, "HTTP/1.1 500 Internal Server Error\r\n");
@@ -75,7 +75,7 @@ int main(int argc, char **argv) {
   LogTestController::getInstance().setInfo<minifi::HttpConfigurationListener>();
 
   const char *options[] = { "document_root", ".", "listening_ports", "9090", 0 };
-  std::vector < std::string > cpp_options;
+  std::vector<std::string> cpp_options;
   for (int i = 0; i < (sizeof(options) / sizeof(options[0]) - 1); i++) {
     cpp_options.push_back(options[i]);
   }
@@ -89,45 +89,32 @@ int main(int argc, char **argv) {
     h_ex.test_file_location_ = test_file_location = argv[1];
     key_dir = argv[2];
   }
-  std::shared_ptr<minifi::Configure> configuration = std::make_shared<
-      minifi::Configure>();
+  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
   configuration->set(minifi::Configure::nifi_default_directory, key_dir);
-  configuration->set(minifi::Configure::nifi_configuration_listener_type,
-      "http");
-  configuration->set(
-      minifi::Configure::nifi_configuration_listener_pull_interval, "1 sec");
-  configuration->set(minifi::Configure::nifi_configuration_listener_http_url,
-      "http://localhost:9090/config");
+  configuration->set(minifi::Configure::nifi_configuration_listener_type, "http");
+  configuration->set(minifi::Configure::nifi_configuration_listener_pull_interval, "1 sec");
+  configuration->set(minifi::Configure::nifi_configuration_listener_http_url, "http://localhost:9090/config");
   mkdir("content_repository", S_IRWXU | S_IRWXG | S_IROTH | S_IXOTH);
 
-  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());
+  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::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
+  std::unique_ptr<core::FlowConfiguration> yaml_ptr = std::unique_ptr < core::YamlConfiguration
+      > (new core::YamlConfiguration(test_repo, test_repo, content_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), content_repo, DEFAULT_ROOT_GROUP_NAME, true);
+
+  core::YamlConfiguration yaml_config(test_repo, test_repo, content_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();