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/03/28 17:19:06 UTC

[01/16] nifi-minifi-cpp git commit: MINIFI-217: Updates namespaces and removes use of raw pointers for user facing API.

Repository: nifi-minifi-cpp
Updated Branches:
  refs/heads/master bca0a0661 -> 44704b363


http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/test/unit/TimeUtilsTest.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/TimeUtilsTest.cpp b/libminifi/test/unit/TimeUtilsTest.cpp
new file mode 100644
index 0000000..9470338
--- /dev/null
+++ b/libminifi/test/unit/TimeUtilsTest.cpp
@@ -0,0 +1,21 @@
+/**
+ *
+ * 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.
+ */
+
+
+
+

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/main/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/main/CMakeLists.txt b/main/CMakeLists.txt
index 126411b..f7bd6e3 100644
--- a/main/CMakeLists.txt
+++ b/main/CMakeLists.txt
@@ -23,7 +23,7 @@ IF(POLICY CMP0048)
   CMAKE_POLICY(SET CMP0048 OLD)
 ENDIF(POLICY CMP0048)
 
-include_directories(../include ../libminifi/include ../thirdparty/yaml-cpp-yaml-cpp-0.5.3/include ../thirdparty/civetweb-1.9.1/include ../thirdparty/leveldb-1.18/include ../thirdparty/)
+include_directories(../include ../libminifi/include  ../libminifi/include/core/yaml  ../libminifi/include/core ../thirdparty/yaml-cpp-yaml-cpp-0.5.3/include ../thirdparty/civetweb-1.9.1/include ../thirdparty/leveldb-1.18/include ../thirdparty/)
 
 find_package(Boost REQUIRED)
 include_directories(${Boost_INCLUDE_DIRS})

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/main/MiNiFiMain.cpp
----------------------------------------------------------------------
diff --git a/main/MiNiFiMain.cpp b/main/MiNiFiMain.cpp
index 3e5202d..6bfd9c9 100644
--- a/main/MiNiFiMain.cpp
+++ b/main/MiNiFiMain.cpp
@@ -28,12 +28,16 @@
 #include <yaml-cpp/yaml.h>
 #include <iostream>
 
-#include "../libminifi/include/BaseLogger.h"
-#include "../libminifi/include/LogAppenders.h"
-#include "spdlog/spdlog.h"
+#include "core/core.h"
 
-#include "Logger.h"
-#include "Configure.h"
+#include "core/logging/BaseLogger.h"
+#include "core/logging/LogAppenders.h"
+#include "spdlog/spdlog.h"
+#include "core/FlowConfiguration.h"
+#include "core/ConfigurationFactory.h"
+#include "core/RepositoryFactory.h"
+#include "core/logging/Logger.h"
+#include "properties/Configure.h"
 #include "FlowController.h"
 
 //! Main thread sleep interval 1 second
@@ -56,7 +60,7 @@
 // Variables that allow us to avoid a timed wait.
 sem_t *running;
 //! Flow Controller
-static FlowController *controller = NULL;
+static std::unique_ptr<minifi::FlowController> controller = nullptr;
 
 /**
  * Removed the stop command from the signal handler so that we could trigger
@@ -69,104 +73,131 @@ static FlowController *controller = NULL;
  */
 void sigHandler(int signal) {
 
-	if (signal == SIGINT || signal == SIGTERM) {
-		// avoid stopping the controller here.
-		sem_post(running);
-	}
+  if (signal == SIGINT || signal == SIGTERM) {
+    // avoid stopping the controller here.
+    sem_post(running);
+  }
 }
 
 int main(int argc, char **argv) {
-	std::shared_ptr<Logger> logger = Logger::getLogger();
-
-	logger->setLogLevel(info);
-
-	uint16_t stop_wait_time = STOP_WAIT_TIME_MS;
-
-	std::string graceful_shutdown_seconds = "";
-
-	running = sem_open("MiNiFiMain", O_CREAT, 0644, 0);
-	if (running == SEM_FAILED || running == 0) {
-
-		logger->log_error("could not initialize semaphore");
-		perror("initialization failure");
-	}
-	// assumes POSIX compliant environment
-	std::string minifiHome;
-	if (const char* env_p = std::getenv(MINIFI_HOME_ENV_KEY)) {
-		minifiHome = env_p;
-	} else {
-		logger->log_info(
-				"MINIFI_HOME was not found, determining based on executable path.");
-		char *path = NULL;
-		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
-		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) {
-		logger->log_error("Can not install signal handler");
-		return -1;
-	}
-
-	Configure *configure = Configure::getConfigure();
-	configure->setHome(minifiHome);
-	configure->loadConfigureFile(DEFAULT_NIFI_PROPERTIES_FILE);
-
-	if (configure->get(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",
-					Configure::nifi_graceful_shutdown_seconds, e.what());
-		} catch (const std::invalid_argument &e) {
-			logger->log_error("%s contains an invalid argument set. %s",
-					Configure::nifi_graceful_shutdown_seconds, e.what());
-		}
-	} else {
-		logger->log_debug("%s not set, defaulting to %d",
-				Configure::nifi_graceful_shutdown_seconds, STOP_WAIT_TIME_MS);
-	}
-
-	// set the log configuration.
-	std::unique_ptr<BaseLogger> configured_logger = LogInstance::getConfiguredLogger(
-			configure);
-
-	logger->updateLogger(std::move(configured_logger));
-
-	controller = FlowControllerFactory::getFlowController();
-
-	// Load flow from specified configuration file
-	controller->load();
-	// Start Processing the flow
-
-	controller->start();
-	logger->log_info("MiNiFi started");
-
-	/**
-	 * Sem wait provides us the ability to have a controlled
-	 * yield without the need for a more complex construct and
-	 * a spin lock
-	 */
-	if (sem_wait(running) != -1)
-		perror("sem_wait");
-
-	sem_unlink("MiNiFiMain");
-
-	/**
-	 * Trigger unload -- wait stop_wait_time
-	 */
-	controller->waitUnload(stop_wait_time);
-
-	delete controller;
-
-	logger->log_info("MiNiFi exit");
-
-	return 0;
+  std::shared_ptr<logging::Logger> logger = logging::Logger::getLogger();
+
+  logger->setLogLevel(logging::info);
+
+  uint16_t stop_wait_time = STOP_WAIT_TIME_MS;
+
+  std::string graceful_shutdown_seconds = "";
+  std::string prov_repo_class = "provenancerepository";
+  std::string flow_repo_class = "flowfilerepository";
+  std::string nifi_configuration_class_name = "yamlconfiguration";
+
+  running = sem_open("MiNiFiMain", O_CREAT, 0644, 0);
+  if (running == SEM_FAILED || running == 0) {
+
+    logger->log_error("could not initialize semaphore");
+    perror("initialization failure");
+  }
+  // assumes POSIX compliant environment
+  std::string minifiHome;
+  if (const char* env_p = std::getenv(MINIFI_HOME_ENV_KEY)) {
+    minifiHome = env_p;
+  } else {
+    logger->log_info(
+        "MINIFI_HOME was not found, determining based on executable path.");
+    char *path = NULL;
+    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
+    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) {
+    logger->log_error("Can not install signal handler");
+    return -1;
+  }
+
+  minifi::Configure *configure = minifi::Configure::getConfigure();
+  configure->setHome(minifiHome);
+  configure->loadConfigureFile(DEFAULT_NIFI_PROPERTIES_FILE);
+
+  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());
+    } catch (const std::invalid_argument &e) {
+      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);
+  }
+
+  // set the log configuration.
+  std::unique_ptr<logging::BaseLogger> configured_logger =
+      logging::LogInstance::getConfiguredLogger(configure);
+
+  logger->updateLogger(std::move(configured_logger));
+
+  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);
+  prov_repo->initialize();
+
+  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);
+
+  flow_repo->initialize();
+
+  configure->get(minifi::Configure::nifi_configuration_class_name,
+                 nifi_configuration_class_name);
+
+  std::unique_ptr<core::FlowConfiguration> flow_configuration = std::move(
+      core::createFlowConfiguration(prov_repo, flow_repo,
+                                   nifi_configuration_class_name));
+
+  controller = std::unique_ptr<minifi::FlowController>(
+      new minifi::FlowController(prov_repo, flow_repo,
+                                 std::move(flow_configuration)));
+
+  // Load flow from specified configuration file
+  controller->load();
+  // Start Processing the flow
+
+  controller->start();
+  logger->log_info("MiNiFi started");
+
+  /**
+   * Sem wait provides us the ability to have a controlled
+   * yield without the need for a more complex construct and
+   * a spin lock
+   */
+  if (sem_wait(running) != -1)
+    perror("sem_wait");
+
+  sem_unlink("MiNiFiMain");
+
+  /**
+   * Trigger unload -- wait stop_wait_time
+   */
+  controller->waitUnload(stop_wait_time);
+
+  logger->log_info("MiNiFi exit");
+
+  return 0;
 }


[16/16] nifi-minifi-cpp git commit: MINIFI-217: Updates namespaces and removes use of raw pointers for user facing API.

Posted by al...@apache.org.
MINIFI-217: Updates namespaces and removes use of raw pointers for user facing API.

MINIFI-226: Separate YamlConfiguration from FlowController.

This closes #63.

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

Branch: refs/heads/master
Commit: 44704b363ccc422bb4fe2e1a6dcc184e9213bd68
Parents: bca0a06
Author: Marc Parisi <ph...@apache.org>
Authored: Mon Feb 27 14:02:58 2017 -0500
Committer: Aldrin Piri <al...@apache.org>
Committed: Tue Mar 28 13:05:11 2017 -0400

----------------------------------------------------------------------
 CMakeLists.txt                                  |    6 +
 libminifi/CMakeLists.txt                        |    4 +-
 libminifi/include/AppendHostInfo.h              |   67 -
 libminifi/include/BaseLogger.h                  |  207 --
 libminifi/include/Configure.h                   |  127 -
 libminifi/include/Connection.h                  |  329 ++-
 libminifi/include/EventDrivenSchedulingAgent.h  |   62 +-
 libminifi/include/Exception.h                   |   28 +-
 libminifi/include/ExecuteProcess.h              |  112 -
 libminifi/include/FlowControlProtocol.h         |  506 ++--
 libminifi/include/FlowController.h              |  452 ++--
 libminifi/include/FlowFileRecord.h              |  330 +--
 libminifi/include/FlowFileRepository.h          |  204 --
 libminifi/include/GenerateFlowFile.h            |   87 -
 libminifi/include/GetFile.h                     |  117 -
 libminifi/include/ListenHTTP.h                  |  116 -
 libminifi/include/ListenSyslog.h                |  209 --
 libminifi/include/LogAppenders.h                |  298 ---
 libminifi/include/LogAttribute.h                |  128 -
 libminifi/include/Logger.h                      |  200 --
 libminifi/include/ProcessContext.h              |  113 -
 libminifi/include/ProcessGroup.h                |  187 --
 libminifi/include/ProcessSession.h              |  125 -
 libminifi/include/ProcessSessionFactory.h       |   52 -
 libminifi/include/Processor.h                   |  365 ---
 libminifi/include/Property.h                    |  259 --
 libminifi/include/Provenance.h                  |  604 -----
 libminifi/include/PutFile.h                     |   88 -
 libminifi/include/RealTimeDataCollector.h       |  131 -
 libminifi/include/Relationship.h                |   87 -
 libminifi/include/RemoteProcessorGroupPort.h    |  134 +-
 libminifi/include/Repository.h                  |  318 ---
 libminifi/include/ResourceClaim.h               |   46 +-
 libminifi/include/SchedulingAgent.h             |  138 +-
 libminifi/include/Site2SiteClientProtocol.h     | 1003 ++++----
 libminifi/include/Site2SitePeer.h               |  455 ++--
 libminifi/include/TailFile.h                    |   93 -
 libminifi/include/ThreadedSchedulingAgent.h     |   77 +-
 libminifi/include/TimerDrivenSchedulingAgent.h  |   63 +-
 libminifi/include/core/ConfigurableComponent.h  |  104 +
 libminifi/include/core/ConfigurationFactory.h   |   65 +
 libminifi/include/core/Connectable.h            |  165 ++
 libminifi/include/core/FlowConfiguration.h      |  118 +
 libminifi/include/core/FlowFile.h               |  283 +++
 libminifi/include/core/ProcessContext.h         |  114 +
 libminifi/include/core/ProcessGroup.h           |  190 ++
 libminifi/include/core/ProcessSession.h         |  167 ++
 libminifi/include/core/ProcessSessionFactory.h  |   64 +
 libminifi/include/core/Processor.h              |  270 ++
 libminifi/include/core/ProcessorConfig.h        |   53 +
 libminifi/include/core/ProcessorNode.h          |  246 ++
 libminifi/include/core/Property.h               |  264 ++
 libminifi/include/core/Relationship.h           |   96 +
 libminifi/include/core/Repository.h             |  153 ++
 libminifi/include/core/RepositoryFactory.h      |   44 +
 libminifi/include/core/Scheduling.h             |   64 +
 libminifi/include/core/core.h                   |  177 ++
 libminifi/include/core/logging/BaseLogger.h     |  224 ++
 libminifi/include/core/logging/LogAppenders.h   |  301 +++
 libminifi/include/core/logging/Logger.h         |  214 ++
 .../core/repository/FlowFileRepository.h        |  169 ++
 libminifi/include/core/yaml/YamlConfiguration.h |   99 +
 libminifi/include/io/BaseStream.h               |  256 +-
 libminifi/include/io/CRCStream.h                |  415 ++-
 libminifi/include/io/ClientSocket.h             |  411 +--
 libminifi/include/io/DataStream.h               |  188 +-
 libminifi/include/io/EndianCheck.h              |   44 +-
 libminifi/include/io/Serializable.h             |  309 +--
 libminifi/include/io/SocketFactory.h            |   91 -
 libminifi/include/io/Sockets.h                  |   27 +
 libminifi/include/io/StreamFactory.h            |  138 +
 libminifi/include/io/TLSSocket.h                |  187 --
 libminifi/include/io/tls/TLSSocket.h            |  198 ++
 libminifi/include/io/validation.h               |   29 +-
 libminifi/include/processors/AppendHostInfo.h   |   80 +
 libminifi/include/processors/ExecuteProcess.h   |  125 +
 libminifi/include/processors/GenerateFlowFile.h |   98 +
 libminifi/include/processors/GetFile.h          |  129 +
 libminifi/include/processors/ListenHTTP.h       |  126 +
 libminifi/include/processors/ListenSyslog.h     |  216 ++
 libminifi/include/processors/LogAttribute.h     |  130 +
 libminifi/include/processors/PutFile.h          |  101 +
 .../include/processors/RealTimeDataCollector.h  |  145 ++
 libminifi/include/processors/TailFile.h         |  105 +
 libminifi/include/properties/Configure.h        |  131 +
 libminifi/include/provenance/Provenance.h       |  560 +++++
 .../include/provenance/ProvenanceRepository.h   |  166 ++
 libminifi/include/utils/FailurePolicy.h         |   13 +
 libminifi/include/utils/StringUtils.h           |   15 +-
 libminifi/src/AppendHostInfo.cpp                |   97 -
 libminifi/src/BaseLogger.cpp                    |  153 --
 libminifi/src/Configure.cpp                     |   40 +-
 libminifi/src/Connection.cpp                    |  302 +--
 libminifi/src/EventDrivenSchedulingAgent.cpp    |   55 +-
 libminifi/src/ExecuteProcess.cpp                |  251 --
 libminifi/src/FlowControlProtocol.cpp           |  884 ++++---
 libminifi/src/FlowController.cpp                |  954 ++-----
 libminifi/src/FlowFileRecord.cpp                |  530 ++--
 libminifi/src/FlowFileRepository.cpp            |  282 ---
 libminifi/src/GenerateFlowFile.cpp              |  135 -
 libminifi/src/GetFile.cpp                       |  329 ---
 libminifi/src/ListenHTTP.cpp                    |  395 ---
 libminifi/src/ListenSyslog.cpp                  |  343 ---
 libminifi/src/LogAppenders.cpp                  |   25 -
 libminifi/src/LogAttribute.cpp                  |  159 --
 libminifi/src/Logger.cpp                        |   28 -
 libminifi/src/ProcessGroup.cpp                  |  307 ---
 libminifi/src/ProcessSession.cpp                |  790 ------
 libminifi/src/ProcessSessionFactory.cpp         |   28 -
 libminifi/src/Processor.cpp                     |  526 ----
 libminifi/src/Provenance.cpp                    |  566 -----
 libminifi/src/PutFile.cpp                       |  203 --
 libminifi/src/RealTimeDataCollector.cpp         |  481 ----
 libminifi/src/RemoteProcessorGroupPort.cpp      |  191 +-
 libminifi/src/Repository.cpp                    |  140 --
 libminifi/src/ResourceClaim.cpp                 |   47 +-
 libminifi/src/SchedulingAgent.cpp               |  107 +-
 libminifi/src/Site2SiteClientProtocol.cpp       | 2377 +++++++++---------
 libminifi/src/Site2SitePeer.cpp                 |   12 +
 libminifi/src/TailFile.cpp                      |  269 --
 libminifi/src/ThreadedSchedulingAgent.cpp       |  216 +-
 libminifi/src/TimerDrivenSchedulingAgent.cpp    |   50 +-
 libminifi/src/core/ConfigurableComponent.cpp    |  143 ++
 libminifi/src/core/ConfigurationFactory.cpp     |   81 +
 libminifi/src/core/Connectable.cpp              |  174 ++
 libminifi/src/core/Core.cpp                     |   51 +
 libminifi/src/core/FlowConfiguration.cpp        |  110 +
 libminifi/src/core/ProcessGroup.cpp             |  312 +++
 libminifi/src/core/ProcessSession.cpp           |  941 +++++++
 libminifi/src/core/ProcessSessionFactory.cpp    |   42 +
 libminifi/src/core/Processor.cpp                |  272 ++
 libminifi/src/core/ProcessorNode.cpp            |   65 +
 libminifi/src/core/Property.cpp                 |   57 +
 libminifi/src/core/Record.cpp                   |  223 ++
 libminifi/src/core/Repository.cpp               |   65 +
 libminifi/src/core/RepositoryFactory.cpp        |   69 +
 libminifi/src/core/logging/BaseLogger.cpp       |  161 ++
 libminifi/src/core/logging/LogAppenders.cpp     |   39 +
 libminifi/src/core/logging/Logger.cpp           |   40 +
 .../src/core/repository/FlowFileRepository.cpp  |  109 +
 libminifi/src/core/yaml/YamlConfiguration.cpp   |  490 ++++
 libminifi/src/io/BaseStream.cpp                 |   39 +-
 libminifi/src/io/ClientSocket.cpp               |  641 +++--
 libminifi/src/io/DataStream.cpp                 |   12 +
 libminifi/src/io/EndianCheck.cpp                |   13 +-
 libminifi/src/io/Serializable.cpp               |   13 +-
 libminifi/src/io/SocketFactory.cpp              |   24 -
 libminifi/src/io/StreamFactory.cpp              |   37 +
 libminifi/src/io/TLSSocket.cpp                  |  237 --
 libminifi/src/io/tls/TLSSocket.cpp              |  249 ++
 libminifi/src/processors/AppendHostInfo.cpp     |  124 +
 libminifi/src/processors/ExecuteProcess.cpp     |  255 ++
 libminifi/src/processors/GenerateFlowFile.cpp   |  145 ++
 libminifi/src/processors/GetFile.cpp            |  340 +++
 libminifi/src/processors/ListenHTTP.cpp         |  380 +++
 libminifi/src/processors/ListenSyslog.cpp       |  331 +++
 libminifi/src/processors/LogAttribute.cpp       |  176 ++
 libminifi/src/processors/PutFile.cpp            |  213 ++
 .../src/processors/RealTimeDataCollector.cpp    |  480 ++++
 libminifi/src/processors/TailFile.cpp           |  271 ++
 libminifi/src/provenance/Provenance.cpp         |  578 +++++
 .../src/provenance/ProvenanceRepository.cpp     |   75 +
 libminifi/test/Server.cpp                       |   38 +-
 libminifi/test/TestBase.h                       |   81 +-
 libminifi/test/nodefs/NoLevelDB.cpp             |   34 +
 libminifi/test/nodefs/NoYamlConfiguration.cpp   |   38 +
 libminifi/test/unit/CRCTests.cpp                |   76 +-
 libminifi/test/unit/LoggerTests.cpp             |  212 +-
 libminifi/test/unit/ProcessorTests.cpp          |  308 ++-
 libminifi/test/unit/PropertyTests.cpp           |   31 +-
 libminifi/test/unit/ProvenanceTestHelper.h      |  273 +-
 libminifi/test/unit/ProvenanceTests.cpp         |  112 +-
 libminifi/test/unit/SerializationTests.cpp      |   10 +-
 libminifi/test/unit/SocketTests.cpp             |  169 +-
 libminifi/test/unit/TimeUtilsTest.cpp           |   21 +
 main/CMakeLists.txt                             |    2 +-
 main/MiNiFiMain.cpp                             |  237 +-
 177 files changed, 19820 insertions(+), 17175 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 418220d..5d7875e 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -128,6 +128,12 @@ enable_testing(test)
     target_include_directories(tests PRIVATE BEFORE ${LEVELDB_INCLUDE_DIRS})
     target_include_directories(tests PRIVATE BEFORE "include")
     target_include_directories(tests PRIVATE BEFORE "libminifi/include/")
+    target_include_directories(tests PRIVATE BEFORE "libminifi/include/core")
+    target_include_directories(tests PRIVATE BEFORE "libminifi/include/core/repository")
+    target_include_directories(tests PRIVATE BEFORE "libminifi/include/io")
+    target_include_directories(tests PRIVATE BEFORE "libminifi/include/utils")
+    target_include_directories(tests PRIVATE BEFORE "libminifi/include/processors")
+    target_include_directories(tests PRIVATE BEFORE "libminifi/include/provenance")
     target_link_libraries(tests ${CMAKE_THREAD_LIBS_INIT} ${UUID_LIBRARIES} ${LEVELDB_LIBRARIES} ${OPENSSL_LIBRARIES} minifi yaml-cpp c-library civetweb-cpp)
     add_test(NAME LibMinifiTests COMMAND tests)
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/libminifi/CMakeLists.txt b/libminifi/CMakeLists.txt
index 69975ad..4c71cc1 100644
--- a/libminifi/CMakeLists.txt
+++ b/libminifi/CMakeLists.txt
@@ -48,7 +48,7 @@ include(CheckCXXCompilerFlag)
 CHECK_CXX_COMPILER_FLAG("-std=c++11" COMPILER_SUPPORTS_CXX11)
 CHECK_CXX_COMPILER_FLAG("-std=c++0x" COMPILER_SUPPORTS_CXX0X)
 if(COMPILER_SUPPORTS_CXX11)
-    set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++11 -g")
+    set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++11")
 elseif(COMPILER_SUPPORTS_CXX0X)
     set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++0x")
 else()
@@ -60,7 +60,7 @@ include_directories(../thirdparty/yaml-cpp-yaml-cpp-0.5.3/include)
 include_directories(../thirdparty/civetweb-1.9.1/include)
 include_directories(include)
 
-file(GLOB SOURCES "src/*.cpp" "src/io/*.cpp" "src/utils/*.cpp")
+file(GLOB SOURCES  "src/core/logging/*.cpp" "src/io/*.cpp" "src/io/tls/*.cpp" "src/core/*.cpp"  "src/core/repository/*.cpp" "src/core/yaml/*.cpp" "src/provenance/*.cpp" "src/processors/*.cpp" "src/*.cpp")
 file(GLOB SPD_SOURCES "../include/spdlog/*")
 
 # Workaround the limitations of having a

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/AppendHostInfo.h
----------------------------------------------------------------------
diff --git a/libminifi/include/AppendHostInfo.h b/libminifi/include/AppendHostInfo.h
deleted file mode 100644
index 8d9dd8f..0000000
--- a/libminifi/include/AppendHostInfo.h
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- * @file AppendHostInfo.h
- * AppendHostInfo class declaration
- *
- * 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 __APPEND_HOSTINFO_H__
-#define __APPEND_HOSTINFO_H__
-
-#include "FlowFileRecord.h"
-#include "Processor.h"
-#include "ProcessSession.h"
-
-//! AppendHostInfo Class
-class AppendHostInfo : public Processor
-{
-public:
-	//! Constructor
-	/*!
-	 * Create a new processor
-	 */
-	AppendHostInfo(std::string name, uuid_t uuid = NULL)
-	: Processor(name, uuid)
-	{
-		logger_ = Logger::getLogger();
-	}
-	//! Destructor
-	virtual ~AppendHostInfo()
-	{
-	}
-	//! Processor Name
-	static const std::string ProcessorName;
-	//! Supported Properties
-	static Property InterfaceName;
-	static Property HostAttribute;
-	static Property IPAttribute;
-
-	//! Supported Relationships
-	static Relationship Success;
-
-public:
-	//! OnTrigger method, implemented by NiFi AppendHostInfo
-	virtual void onTrigger(ProcessContext *context, ProcessSession *session);
-	//! Initialize, over write by NiFi AppendHostInfo
-	virtual void initialize(void);
-
-protected:
-
-private:
-	//! Logger
-	std::shared_ptr<Logger> logger_;
-};
-
-#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/BaseLogger.h
----------------------------------------------------------------------
diff --git a/libminifi/include/BaseLogger.h b/libminifi/include/BaseLogger.h
deleted file mode 100644
index e9d2a02..0000000
--- a/libminifi/include/BaseLogger.h
+++ /dev/null
@@ -1,207 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#ifndef LIBMINIFI_INCLUDE_BASELOGGER_H_
-#define LIBMINIFI_INCLUDE_BASELOGGER_H_
-
-#include <string>
-#include <memory>
-#include "spdlog/spdlog.h"
-#include <iostream>
-//! 5M default log file size
-#define DEFAULT_LOG_FILE_SIZE (5*1024*1024)
-//! 3 log files rotation
-#define DEFAULT_LOG_FILE_NUMBER 3
-#define LOG_NAME "minifi log"
-#define LOG_FILE_NAME "minifi-app.log"
-
-/**
- * Log level enumeration.
- */
-typedef enum {
-	trace = 0, debug = 1, info = 2, warn = 3, err = 4, critical = 5, off = 6
-} LOG_LEVEL_E;
-
-#define LOG_BUFFER_SIZE 1024
-#define FILL_BUFFER  char buffer[LOG_BUFFER_SIZE]; \
-    va_list args; \
-    va_start(args, format); \
-    std::vsnprintf(buffer, LOG_BUFFER_SIZE,format, args); \
-    va_end(args);
-
-/**
- * Base class that represents a logger configuration.
- */
-class BaseLogger {
-
-public:
-	static const char *nifi_log_level;
-	static const char *nifi_log_appender;
-
-	/**
-	 * Base Constructor
-	 */
-	BaseLogger() {
-		setLogLevel("info");
-		logger_ = nullptr;
-		stderr_ = nullptr;
-	}
-
-	/**
-	 * Logger configuration constructorthat will set the base log level.
-	 * @param config incoming configuration.
-	 */
-	BaseLogger(std::string log_level, std::shared_ptr<spdlog::logger> logger) : logger_(logger) {
-		setLogLevel(log_level);
-
-	}
-
-	virtual ~BaseLogger() {
-
-	}
-
-	/**
-	 * Move constructor that will atomically swap configuration
-	 * shared pointers.
-	 */
-	BaseLogger(const BaseLogger &&other) :
-			configured_level_(other.configured_level_.load()) {
-		// must atomically exchange the pointers
-		logger_ = std::move(other.logger_);
-		set_error_logger(other.stderr_);
-
-	}
-
-	/**
-	 * Returns the log level for this instance.
-	 */
-	virtual LOG_LEVEL_E getLogLevel() const {
-		return configured_level_;
-	}
-
-	/**
-	 * @brief Log error message
-	 * @param format format string ('man printf' for syntax)
-	 * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
-	 */
-	virtual void log_error(const char * const format, ...);
-	/**
-	 * @brief Log warn message
-	 * @param format format string ('man printf' for syntax)
-	 * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
-	 */
-	virtual void log_warn(const char * const format, ...);
-	/**
-	 * @brief Log info message
-	 * @param format format string ('man printf' for syntax)
-	 * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
-	 */
-	virtual void log_info(const char * const format, ...);
-	/**
-	 * @brief Log debug message
-	 * @param format format string ('man printf' for syntax)
-	 * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
-	 */
-	virtual void log_debug(const char * const format, ...);
-	/**
-	 * @brief Log trace message
-	 * @param format format string ('man printf' for syntax)
-	 * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
-	 */
-	virtual void log_trace(const char * const format, ...);
-
-	/**
-	 * @brief Log error message
-	 * @param format format string ('man printf' for syntax)
-	 * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
-	 */
-	virtual void log_str(LOG_LEVEL_E level,const std::string &buffer);
-
-	/**
-	 * Sets the log level for this instance based on the string
-	 * @param level desired log leve.
-	 * @param defaultLevel default level if we cannot match level.
-	 */
-	virtual void setLogLevel(const std::string &level,
-			LOG_LEVEL_E defaultLevel = info);
-
-	/**
-	 * Sets the log level atomic and sets it
-	 * within logger if it can
-	 * @param level desired log level.
-	 */
-	virtual void setLogLevel(LOG_LEVEL_E level) {
-		configured_level_ = level;
-		setLogLevel();
-	}
-
-	bool shouldLog(LOG_LEVEL_E level)
-	{
-		return level >= configured_level_.load(std::memory_order_relaxed);
-	}
-
-	/**
-	 * Move operator overload
-	 */
-	BaseLogger &operator=(const BaseLogger &&other) {
-		configured_level_ = (other.configured_level_.load());
-		// must atomically exchange the pointers
-		logger_ = std::move(other.logger_);
-		set_error_logger(other.stderr_);
-		return *this;
-	}
-
-protected:
-
-
-
-	/**
-	 * Logger configuration constructorthat will set the base log level.
-	 * @param config incoming configuration.
-	 */
-	BaseLogger(std::string log_level) : logger_(nullptr) {
-		setLogLevel(log_level);
-	}
-
-
-	void setLogger(std::shared_ptr<spdlog::logger> logger) {
-		logger_ = logger;
-	}
-
-	/**
-	 * Since a thread may be using stderr and it can be null,
-	 * we must atomically exchange the shared pointers.
-	 * @param other other shared pointer. can be null ptr
-	 */
-	void set_error_logger(std::shared_ptr<spdlog::logger> other);
-
-	/**
-	 * Sets the log level on the spdlogger if it is not null.
-	 */
-	void setLogLevel() {
-		if (logger_ != nullptr)
-		logger_->set_level(
-				(spdlog::level::level_enum) configured_level_.load());
-
-	}
-
-	std::atomic<LOG_LEVEL_E> configured_level_;
-	std::shared_ptr<spdlog::logger> logger_;
-	std::shared_ptr<spdlog::logger> stderr_;
-};
-
-#endif /* LIBMINIFI_INCLUDE_BASELOGGER_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/Configure.h
----------------------------------------------------------------------
diff --git a/libminifi/include/Configure.h b/libminifi/include/Configure.h
deleted file mode 100644
index 6f0d198..0000000
--- a/libminifi/include/Configure.h
+++ /dev/null
@@ -1,127 +0,0 @@
-/**
- * @file Configure.h
- * Configure class declaration
- *
- * 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 __CONFIGURE_H__
-#define __CONFIGURE_H__
-
-#include <stdio.h>
-#include <string>
-#include <map>
-#include <stdlib.h>
-#include <errno.h>
-#include <iostream>
-#include <fstream>
-#include "Logger.h"
-
-class Configure {
-public:
-	//! Get the singleton logger instance
-	static Configure * getConfigure()
-	{
-		if (!configure_)
-		{
-			configure_ = new Configure();
-		}
-		return configure_;
-	}
-	//! nifi.flow.configuration.file
-	static const char *nifi_flow_configuration_file;
-	static const char *nifi_administrative_yield_duration;
-	static const char *nifi_bored_yield_duration;
-	static const char *nifi_graceful_shutdown_seconds;
-	static const char *nifi_log_level;
-	static const char *nifi_server_name;
-	static const char *nifi_server_port;
-	static const char *nifi_server_report_interval;
-	static const char *nifi_provenance_repository_max_storage_time;
-	static const char *nifi_provenance_repository_max_storage_size;
-	static const char *nifi_provenance_repository_directory_default;
-	static const char *nifi_provenance_repository_enable;
-	static const char *nifi_flowfile_repository_max_storage_time;
-	static const char *nifi_flowfile_repository_max_storage_size;
-	static const char *nifi_flowfile_repository_directory_default;
-	static const char *nifi_flowfile_repository_enable;
-	static const char *nifi_remote_input_secure;
-	static const char *nifi_security_need_ClientAuth;
-	static const char *nifi_security_client_certificate;
-	static const char *nifi_security_client_private_key;
-	static const char *nifi_security_client_pass_phrase;
-	static const char *nifi_security_client_ca_certificate;
-
-	//! Clear the load config
-	void clear()
-	{
-		std::lock_guard<std::mutex> lock(_mtx);
-		_properties.clear();
-	}
-	//! Set the config value
-	void set(std::string key, std::string value)
-	{
-		std::lock_guard<std::mutex> lock(_mtx);
-		_properties[key] = value;
-	}
-	//! Check whether the config value existed
-	bool has(std::string key)
-	{
-		std::lock_guard<std::mutex> lock(_mtx);
-		return (_properties.find(key) != _properties.end());
-	}
-	//! Get the config value
-	bool get(std::string key, std::string &value);
-	//! Parse one line in configure file like key=value
-	void parseConfigureFileLine(char *buf);
-	//! Load Configure File
-	void loadConfigureFile(const char *fileName);
-    //! Set the determined MINIFI_HOME
-    void setHome(std::string minifiHome)
-    {
-        _minifiHome = minifiHome;
-    }
-
-    //! Get the determined MINIFI_HOME
-    std::string getHome()
-    {
-        return _minifiHome;
-    }
-    //! Parse Command Line
-    void parseCommandLine(int argc, char **argv);
-
-private:
-	//! Mutex for protection
-	std::mutex _mtx;
-	//! Logger
-	std::shared_ptr<Logger> logger_;
-	//! Home location for this executable
-	std::string _minifiHome;
-
-	Configure()
-	{
-		logger_ = Logger::getLogger();
-	}
-	virtual ~Configure()
-	{
-
-	}
-	static Configure *configure_;
-
-protected:
-	std::map<std::string,std::string> _properties;
-};
-
-#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/Connection.h
----------------------------------------------------------------------
diff --git a/libminifi/include/Connection.h b/libminifi/include/Connection.h
index 5af0d2f..1c7b9a4 100644
--- a/libminifi/include/Connection.h
+++ b/libminifi/include/Connection.h
@@ -27,180 +27,171 @@
 #include <mutex>
 #include <atomic>
 #include <algorithm>
+#include "core/core.h"
+#include "core/Connectable.h"
+#include "core/logging/Logger.h"
+#include "core/Relationship.h"
+#include "core/Connectable.h"
+#include "core/FlowFile.h"
+#include "core/Repository.h"
 
-#include "FlowFileRecord.h"
-#include "Logger.h"
-#include "Relationship.h"
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+// Connection Class
 
-//! Forwarder declaration
-class Processor;
+class Connection : public core::Connectable,
+    public std::enable_shared_from_this<Connection> {
+ public:
+  // Constructor
+  /*
+   * 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);
+  // Destructor
+  virtual ~Connection() {
+  }
 
-//! Connection Class
-class Connection
-{
-public:
-	//! Constructor
-	/*!
-	 * Create a new processor
-	 */
-	explicit Connection(std::string name, uuid_t uuid = NULL, uuid_t srcUUID = NULL, uuid_t destUUID = NULL);
-	//! Destructor
-	virtual ~Connection() {}
-	//! Set Connection Name
-	void setName(std::string name) {
-		_name = name;
-	}
-	//! Get Process Name
-	std::string getName(void) {
-		return (_name);
-	}
-	//! Set UUID
-	void setUUID(uuid_t uuid) {
-		uuid_copy(_uuid, uuid);
-	}
-	//! Set Source Processor UUID
-	void setSourceProcessorUUID(uuid_t uuid) {
-		uuid_copy(_srcUUID, uuid);
-	}
-	//! Set Destination Processor UUID
-	void setDestinationProcessorUUID(uuid_t uuid) {
-		uuid_copy(_destUUID, uuid);
-	}
-	//! Get Source Processor UUID
-	void getSourceProcessorUUID(uuid_t uuid) {
-		uuid_copy(uuid, _srcUUID);
-	}
-	//! Get Destination Processor UUID
-	void getDestinationProcessorUUID(uuid_t uuid) {
-		uuid_copy(uuid, _destUUID);
-	}
-	//! Get UUID
-	bool getUUID(uuid_t uuid) {
-		if (uuid)
-		{
-			uuid_copy(uuid, _uuid);
-			return true;
-		}
-		else
-			return false;
-	}
-	//! Get UUID Str
-	std::string getUUIDStr() {
-		return _uuidStr;
-	}
-	//! Set Connection Source Processor
-	void setSourceProcessor(Processor *source) {
-		_srcProcessor = source;
-	}
-	// ! Get Connection Source Processor
-	Processor *getSourceProcessor() {
-		return _srcProcessor;
-	}
-	//! Set Connection Destination Processor
-	void setDestinationProcessor(Processor *dest) {
-		_destProcessor = dest;
-	}
-	// ! Get Connection Destination Processor
-	Processor *getDestinationProcessor() {
-		return _destProcessor;
-	}
-	//! Set Connection relationship
-	void setRelationship(Relationship relationship) {
-		_relationship = relationship;
-	}
-	// ! Get Connection relationship
-	Relationship getRelationship() {
-		return _relationship;
-	}
-	//! Set Max Queue Size
-	void setMaxQueueSize(uint64_t size)
-	{
-		_maxQueueSize = size;
-	}
-	//! Get Max Queue Size
-	uint64_t getMaxQueueSize()
-	{
-		return _maxQueueSize;
-	}
-	//! Set Max Queue Data Size
-	void setMaxQueueDataSize(uint64_t size)
-	{
-		_maxQueueDataSize = size;
-	}
-	//! Get Max Queue Data Size
-	uint64_t getMaxQueueDataSize()
-	{
-		return _maxQueueDataSize;
-	}
-	//! Set Flow expiration duration in millisecond
-	void setFlowExpirationDuration(uint64_t duration)
-	{
-		_expiredDuration = duration;
-	}
-	//! Get Flow expiration duration in millisecond
-	uint64_t getFlowExpirationDuration()
-	{
-		return _expiredDuration;
-	}
-	//! Check whether the queue is empty
-	bool isEmpty();
-	//! Check whether the queue is full to apply back pressure
-	bool isFull();
-	//! Get queue size
-	uint64_t getQueueSize() {
-		std::lock_guard<std::mutex> lock(_mtx);
-		return _queue.size();
-	}
-	//! Get queue data size
-	uint64_t getQueueDataSize()
-	{
-		return _maxQueueDataSize;
-	}
-	//! Put the flow file into queue
-	void put(FlowFileRecord *flow);
-	//! Poll the flow file from queue, the expired flow file record also being returned
-	FlowFileRecord *poll(std::set<FlowFileRecord *> &expiredFlowRecords);
-	//! Drain the flow records
-	void drain();
+  // Set Source Processor UUID
+  void setSourceUUID(uuid_t uuid) {
+    uuid_copy(src_uuid_, uuid);
+  }
+  // Set Destination Processor UUID
+  void setDestinationUUID(uuid_t uuid) {
+    uuid_copy(dest_uuid_, uuid);
+  }
+  // Get Source Processor UUID
+  void getSourceUUID(uuid_t uuid) {
+    uuid_copy(uuid, src_uuid_);
+  }
+  // Get Destination Processor UUID
+  void getDestinationUUID(uuid_t uuid) {
+    uuid_copy(uuid, dest_uuid_);
+  }
 
-protected:
-	//! A global unique identifier
-	uuid_t _uuid;
-	//! Source Processor UUID
-	uuid_t _srcUUID;
-	//! Destination Processor UUID
-	uuid_t _destUUID;
-	//! Connection Name
-	std::string _name;
-	//! Relationship for this connection
-	Relationship _relationship;
-	//! Source Processor (ProcessNode/Port)
-	Processor *_srcProcessor;
-	//! Destination Processor (ProcessNode/Port)
-	Processor *_destProcessor;
-	//! Max queue size to apply back pressure
-	std::atomic<uint64_t> _maxQueueSize;
-	//! Max queue data size to apply back pressure
-	std::atomic<uint64_t> _maxQueueDataSize;
-	//! Flow File Expiration Duration in= MilliSeconds
-	std::atomic<uint64_t> _expiredDuration;
-	//! UUID string
-	std::string _uuidStr;
+  // Set Connection Source Processor
+  void setSource(std::shared_ptr<core::Connectable> source) {
+    source_connectable_ = source;
+  }
+  // ! Get Connection Source Processor
+  std::shared_ptr<core::Connectable> getSource() {
+    return source_connectable_;
+  }
+  // Set Connection Destination Processor
+  void setDestination(std::shared_ptr<core::Connectable> dest) {
+    dest_connectable_ = dest;
+  }
+  // ! Get Connection Destination Processor
+  std::shared_ptr<core::Connectable> getDestination() {
+    return dest_connectable_;
+  }
+  // Set Connection relationship
+  void setRelationship(core::Relationship relationship) {
+    relationship_ = relationship;
+  }
+  // ! Get Connection relationship
+  core::Relationship getRelationship() {
+    return relationship_;
+  }
+  // Set Max Queue Size
+  void setMaxQueueSize(uint64_t size) {
+    max_queue_size_ = size;
+  }
+  // Get Max Queue Size
+  uint64_t getMaxQueueSize() {
+    return max_queue_size_;
+  }
+  // Set Max Queue Data Size
+  void setMaxQueueDataSize(uint64_t size) {
+    max_data_queue_size_ = size;
+  }
+  // Get Max Queue Data Size
+  uint64_t getMaxQueueDataSize() {
+    return max_data_queue_size_;
+  }
+  // Set Flow expiration duration in millisecond
+  void setFlowExpirationDuration(uint64_t duration) {
+    expired_duration_ = duration;
+  }
+  // Get Flow expiration duration in millisecond
+  uint64_t getFlowExpirationDuration() {
+    return expired_duration_;
+  }
+  // Check whether the queue is empty
+  bool isEmpty();
+  // Check whether the queue is full to apply back pressure
+  bool isFull();
+  // Get queue size
+  uint64_t getQueueSize() {
+    std::lock_guard<std::mutex> lock(mutex_);
+    return queue_.size();
+  }
+  // Get queue data size
+  uint64_t getQueueDataSize() {
+    return max_data_queue_size_;
+  }
+  // Put the flow file into queue
+  void put(std::shared_ptr<core::FlowFile> flow);
+  // Poll the flow file from queue, the expired flow file record also being returned
+  std::shared_ptr<core::FlowFile> poll(
+      std::set<std::shared_ptr<core::FlowFile>> &expiredFlowRecords);
+  // Drain the flow records
+  void drain();
 
-private:
-	//! Mutex for protection
-	std::mutex _mtx;
-	//! Queued data size
-	std::atomic<uint64_t> _queuedDataSize;
-	//! Queue for the Flow File
-	std::queue<FlowFileRecord *> _queue;
-	//! Logger
-	std::shared_ptr<Logger> logger_;
-	// Prevent default copy constructor and assignment operation
-	// Only support pass by reference or pointer
-	Connection(const Connection &parent);
-	Connection &operator=(const Connection &parent);
+  void yield() {
 
-};
+  }
+
+  bool isWorkAvailable() {
+    return !isEmpty();
+  }
+
+  bool isRunning() {
+    return true;
+  }
 
+ protected:
+  // Source Processor UUID
+  uuid_t src_uuid_;
+  // Destination Processor UUID
+  uuid_t dest_uuid_;
+  // Relationship for this connection
+  core::Relationship relationship_;
+  // Source Processor (ProcessNode/Port)
+  std::shared_ptr<core::Connectable> source_connectable_;
+  // Destination Processor (ProcessNode/Port)
+  std::shared_ptr<core::Connectable> dest_connectable_;
+  // Max queue size to apply back pressure
+  std::atomic<uint64_t> max_queue_size_;
+  // Max queue data size to apply back pressure
+  std::atomic<uint64_t> max_data_queue_size_;
+  // Flow File Expiration Duration in= MilliSeconds
+  std::atomic<uint64_t> expired_duration_;
+  // flow file repository
+  std::shared_ptr<core::Repository> flow_repository_;
+
+ private:
+  // Mutex for protection
+  std::mutex mutex_;
+  // Queued data size
+  std::atomic<uint64_t> queued_data_size_;
+  // Queue for the Flow File
+  std::queue<std::shared_ptr<core::FlowFile>> queue_;
+  // flow repository
+  // Logger
+  std::shared_ptr<logging::Logger> logger_;
+  // Prevent default copy constructor and assignment operation
+  // Only support pass by reference or pointer
+  Connection(const Connection &parent);
+  Connection &operator=(const Connection &parent);
+
+};
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
 #endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/EventDrivenSchedulingAgent.h
----------------------------------------------------------------------
diff --git a/libminifi/include/EventDrivenSchedulingAgent.h b/libminifi/include/EventDrivenSchedulingAgent.h
index b32f84f..9d53c5c 100644
--- a/libminifi/include/EventDrivenSchedulingAgent.h
+++ b/libminifi/include/EventDrivenSchedulingAgent.h
@@ -20,36 +20,46 @@
 #ifndef __EVENT_DRIVEN_SCHEDULING_AGENT_H__
 #define __EVENT_DRIVEN_SCHEDULING_AGENT_H__
 
-#include "Logger.h"
-#include "Processor.h"
-#include "ProcessContext.h"
+#include "core/logging/Logger.h"
+#include "core/Processor.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSessionFactory.h"
 #include "ThreadedSchedulingAgent.h"
 
-//! EventDrivenSchedulingAgent Class
-class EventDrivenSchedulingAgent : public ThreadedSchedulingAgent
-{
-public:
-	//! Constructor
-	/*!
-	 * Create a new processor
-	 */
-	EventDrivenSchedulingAgent()
-	: ThreadedSchedulingAgent()
-	{
-	}
-	//! Destructor
-	virtual ~EventDrivenSchedulingAgent()
-	{
-	}
-	//! Run function for the thread
-	void run(Processor *processor, ProcessContext *processContext, ProcessSessionFactory *sessionFactory);
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
 
-private:
-	// Prevent default copy constructor and assignment operation
-	// Only support pass by reference or pointer
-	EventDrivenSchedulingAgent(const EventDrivenSchedulingAgent &parent);
-	EventDrivenSchedulingAgent &operator=(const EventDrivenSchedulingAgent &parent);
+// EventDrivenSchedulingAgent Class
+class EventDrivenSchedulingAgent : public ThreadedSchedulingAgent {
+ public:
+  // Constructor
+  /*!
+   * Create a new processor
+   */
+  EventDrivenSchedulingAgent(std::shared_ptr<core::Repository> repo)
+      : ThreadedSchedulingAgent(repo) {
+  }
+  // Destructor
+  virtual ~EventDrivenSchedulingAgent() {
+  }
+  // Run function for the thread
+  void run(std::shared_ptr<core::Processor> processor,
+           core::ProcessContext *processContext,
+           core::ProcessSessionFactory *sessionFactory);
+
+ private:
+  // Prevent default copy constructor and assignment operation
+  // Only support pass by reference or pointer
+  EventDrivenSchedulingAgent(const EventDrivenSchedulingAgent &parent);
+  EventDrivenSchedulingAgent &operator=(
+      const EventDrivenSchedulingAgent &parent);
 
 };
 
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
 #endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/Exception.h
----------------------------------------------------------------------
diff --git a/libminifi/include/Exception.h b/libminifi/include/Exception.h
index 1e02fa5..a0c70e6 100644
--- a/libminifi/include/Exception.h
+++ b/libminifi/include/Exception.h
@@ -26,7 +26,13 @@
 #include <errno.h>
 #include <string.h>
 
-//! ExceptionType 
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+// ExceptionType 
 enum ExceptionType 
 {
 	FILE_OPERATION_EXCEPTION = 0,
@@ -39,7 +45,7 @@ enum ExceptionType
 	MAX_EXCEPTION
 };
 
-//! Exception String 
+// Exception String 
 static const char *ExceptionStr[MAX_EXCEPTION] =
 {
 		"File Operation",
@@ -51,7 +57,7 @@ static const char *ExceptionStr[MAX_EXCEPTION] =
 		"General Operation"
 };
 
-//! Exception Type to String 
+// Exception Type to String 
 inline const char *ExceptionTypeToString(ExceptionType type)
 {
 	if (type < MAX_EXCEPTION)
@@ -60,17 +66,17 @@ inline const char *ExceptionTypeToString(ExceptionType type)
 		return NULL;
 }
 
-//! Exception Class
+// Exception Class
 class Exception : public std::exception
 {
 public:
-	//! Constructor
+	// Constructor
 	/*!
 	 * Create a new flow record
 	 */
 	Exception(ExceptionType type, const char *errorMsg) : _type(type), _errorMsg(errorMsg) {
 	}
-	//! Destructor
+	// Destructor
 	virtual ~Exception() throw () {}
 	virtual const char * what() const throw () {
 
@@ -82,13 +88,17 @@ public:
 
 
 private:
-	//! Exception type
+	// Exception type
 	ExceptionType _type;
-	//! Exception detailed information
+	// Exception detailed information
 	std::string _errorMsg;
-	//! Hold the what result
+	// Hold the what result
 	mutable std::string _whatStr;
 
 };
 
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
 #endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/ExecuteProcess.h
----------------------------------------------------------------------
diff --git a/libminifi/include/ExecuteProcess.h b/libminifi/include/ExecuteProcess.h
deleted file mode 100644
index 3ddd815..0000000
--- a/libminifi/include/ExecuteProcess.h
+++ /dev/null
@@ -1,112 +0,0 @@
-/**
- * @file ExecuteProcess.h
- * ExecuteProcess class declaration
- *
- * 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 __EXECUTE_PROCESS_H__
-#define __EXECUTE_PROCESS_H__
-
-#include <stdio.h>
-#include <unistd.h>
-#include <string>
-#include <errno.h>
-#include <chrono>
-#include <thread>
-#include <unistd.h>
-#include <sys/wait.h>
-#include <iostream>
-#include <sys/types.h>
-#include <signal.h>
-#include "FlowFileRecord.h"
-#include "Processor.h"
-#include "ProcessSession.h"
-
-//! ExecuteProcess Class
-class ExecuteProcess : public Processor
-{
-public:
-	//! Constructor
-	/*!
-	 * Create a new processor
-	 */
-	ExecuteProcess(std::string name, uuid_t uuid = NULL)
-	: Processor(name, uuid)
-	{
-		logger_ = Logger::getLogger();
-		_redirectErrorStream = false;
-		_batchDuration = 0;
-		_workingDir = ".";
-		_processRunning = false;
-		_pid = 0;
-	}
-	//! Destructor
-	virtual ~ExecuteProcess()
-	{
-		if (_processRunning && _pid > 0)
-			kill(_pid, SIGTERM);
-	}
-	//! Processor Name
-	static const std::string ProcessorName;
-	//! Supported Properties
-	static Property Command;
-	static Property CommandArguments;
-	static Property WorkingDir;
-	static Property BatchDuration;
-	static Property RedirectErrorStream;
-	//! Supported Relationships
-	static Relationship Success;
-
-	//! Nest Callback Class for write stream
-	class WriteCallback : public OutputStreamCallback
-	{
-		public:
-		WriteCallback(char *data, uint64_t size)
-		: _data(data), _dataSize(size) {}
-		char *_data;
-		uint64_t _dataSize;
-		void process(std::ofstream *stream) {
-			if (_data && _dataSize > 0)
-				stream->write(_data, _dataSize);
-		}
-	};
-
-public:
-	//! OnTrigger method, implemented by NiFi ExecuteProcess
-	virtual void onTrigger(ProcessContext *context, ProcessSession *session);
-	//! Initialize, over write by NiFi ExecuteProcess
-	virtual void initialize(void);
-
-protected:
-
-private:
-	//! Logger
-	std::shared_ptr<Logger> logger_;
-	//! Property
-	std::string _command;
-	std::string _commandArgument;
-	std::string _workingDir;
-	int64_t _batchDuration;
-	bool _redirectErrorStream;
-	//! Full command
-	std::string _fullCommand;
-	//! whether the process is running
-	bool _processRunning;
-	int _pipefd[2];
-	pid_t _pid;
-};
-
-#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/FlowControlProtocol.h
----------------------------------------------------------------------
diff --git a/libminifi/include/FlowControlProtocol.h b/libminifi/include/FlowControlProtocol.h
index ffc454a..c37c8f8 100644
--- a/libminifi/include/FlowControlProtocol.h
+++ b/libminifi/include/FlowControlProtocol.h
@@ -32,310 +32,288 @@
 #include <errno.h>
 #include <chrono>
 #include <thread>
-#include "Configure.h"
-#include "Logger.h"
-#include "Property.h"
 
-//! Forwarder declaration
+#include "core/Property.h"
+#include "properties/Configure.h"
+#include "core/logging/Logger.h"
+
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+// Forwarder declaration
 class FlowController;
 
 #define DEFAULT_NIFI_SERVER_PORT 9000
 #define DEFAULT_REPORT_INTERVAL 1000 // 1 sec
 #define MAX_READ_TIMEOUT 30000 // 30 seconds
 
-//! FlowControl Protocol Msg Type
+// FlowControl Protocol Msg Type
 typedef enum {
-	REGISTER_REQ, // Device Register Request from device to server which contain device serial number, current running flow YAML version
-	REGISTER_RESP, // Device Register Respond from server to device, may contain new flow.YAML from server ask device to apply and also device report interval
-	REPORT_REQ, // Period Device Report from device to server which contain device serial number, current running flow YAML name/version and other period report info
-	REPORT_RESP, // Report Respond from server to device, may ask device to update flow YAML or processor property
-	MAX_FLOW_CONTROL_MSG_TYPE
+  REGISTER_REQ,  // Device Register Request from device to server which contain device serial number, current running flow YAML version
+  REGISTER_RESP,  // Device Register Respond from server to device, may contain new flow.YAML from server ask device to apply and also device report interval
+  REPORT_REQ,  // Period Device Report from device to server which contain device serial number, current running flow YAML name/version and other period report info
+  REPORT_RESP,  // Report Respond from server to device, may ask device to update flow YAML or processor property
+  MAX_FLOW_CONTROL_MSG_TYPE
 } FlowControlMsgType;
 
-//! FlowControl Protocol Msg Type String
-static const char *FlowControlMsgTypeStr[MAX_FLOW_CONTROL_MSG_TYPE] =
-{
-		"REGISTER_REQ",
-		"REGISTER_RESP",
-		"REPORT_REQ",
-		"REPORT_RESP"
-};
+// FlowControl Protocol Msg Type String
+static const char *FlowControlMsgTypeStr[MAX_FLOW_CONTROL_MSG_TYPE] = {
+    "REGISTER_REQ", "REGISTER_RESP", "REPORT_REQ", "REPORT_RESP" };
 
-//! Flow Control Msg Type to String
-inline const char *FlowControlMsgTypeToStr(FlowControlMsgType type)
-{
-	if (type < MAX_FLOW_CONTROL_MSG_TYPE)
-		return FlowControlMsgTypeStr[type];
-	else
-		return NULL;
+// Flow Control Msg Type to String
+inline const char *FlowControlMsgTypeToStr(FlowControlMsgType type) {
+  if (type < MAX_FLOW_CONTROL_MSG_TYPE)
+    return FlowControlMsgTypeStr[type];
+  else
+    return NULL;
 }
 
-//! FlowControll Protocol Msg ID (Some Messages are fix length, Some are variable length (TLV)
+// FlowControll Protocol Msg ID (Some Messages are fix length, Some are variable length (TLV)
 typedef enum {
-	//Fix length 8 bytes: client to server in register request, required field
-	FLOW_SERIAL_NUMBER,
-	// Flow YAML name TLV: client to server in register request and report request, required field
-	FLOW_YML_NAME,
-	// Flow YAML content, TLV: server to client in register respond, option field in case server want to ask client to load YAML from server
-	FLOW_YML_CONTENT,
-	// Fix length, 4 bytes Report interval in msec: server to client in register respond, option field
-	REPORT_INTERVAL,
-	// Processor Name TLV:  server to client in report respond, option field in case server want to ask client to update processor property
-	PROCESSOR_NAME,
-	// Processor Property Name TLV: server to client in report respond, option field in case server want to ask client to update processor property
-	PROPERTY_NAME,
-	// Processor Property Value TLV: server to client in report respond, option field in case server want to ask client to update processor property
-	PROPERTY_VALUE,
-	// Report Blob TLV: client to server in report request, option field in case client want to pickyback the report blob in report request to server
-	REPORT_BLOB,
-	MAX_FLOW_MSG_ID
+  //Fix length 8 bytes: client to server in register request, required field
+  FLOW_SERIAL_NUMBER,
+  // Flow YAML name TLV: client to server in register request and report request, required field
+  FLOW_YML_NAME,
+  // Flow YAML content, TLV: server to client in register respond, option field in case server want to ask client to load YAML from server
+  FLOW_YML_CONTENT,
+  // Fix length, 4 bytes Report interval in msec: server to client in register respond, option field
+  REPORT_INTERVAL,
+  // Processor Name TLV:  server to client in report respond, option field in case server want to ask client to update processor property
+  PROCESSOR_NAME,
+  // Processor Property Name TLV: server to client in report respond, option field in case server want to ask client to update processor property
+  PROPERTY_NAME,
+  // Processor Property Value TLV: server to client in report respond, option field in case server want to ask client to update processor property
+  PROPERTY_VALUE,
+  // Report Blob TLV: client to server in report request, option field in case client want to pickyback the report blob in report request to server
+  REPORT_BLOB,
+  MAX_FLOW_MSG_ID
 } FlowControlMsgID;
 
-//! FlowControl Protocol Msg ID String
-static const char *FlowControlMsgIDStr[MAX_FLOW_MSG_ID] =
-{
-		"FLOW_SERIAL_NUMBER",
-		"FLOW_YAML_NAME",
-		"FLOW_YAML_CONTENT",
-		"REPORT_INTERVAL",
-		"PROCESSOR_NAME"
-		"PROPERTY_NAME",
-		"PROPERTY_VALUE",
-		"REPORT_BLOB"
-};
+// FlowControl Protocol Msg ID String
+static const char *FlowControlMsgIDStr[MAX_FLOW_MSG_ID] = {
+    "FLOW_SERIAL_NUMBER", "FLOW_YAML_NAME", "FLOW_YAML_CONTENT",
+    "REPORT_INTERVAL", "PROCESSOR_NAME"
+        "PROPERTY_NAME", "PROPERTY_VALUE", "REPORT_BLOB" };
 
 #define TYPE_HDR_LEN 4 // Fix Hdr Type
 #define TLV_HDR_LEN 8 // Type 4 bytes and Len 4 bytes
 
-//! FlowControl Protocol Msg Len
-inline int FlowControlMsgIDEncodingLen(FlowControlMsgID id, int payLoadLen)
-{
-	if (id == FLOW_SERIAL_NUMBER)
-		return (TYPE_HDR_LEN + 8);
-	else if (id == REPORT_INTERVAL)
-		return (TYPE_HDR_LEN + 4);
-	else if (id < MAX_FLOW_MSG_ID)
-		return (TLV_HDR_LEN + payLoadLen);
-	else
-		return -1;
+// FlowControl Protocol Msg Len
+inline int FlowControlMsgIDEncodingLen(FlowControlMsgID id, int payLoadLen) {
+  if (id == FLOW_SERIAL_NUMBER)
+    return (TYPE_HDR_LEN + 8);
+  else if (id == REPORT_INTERVAL)
+    return (TYPE_HDR_LEN + 4);
+  else if (id < MAX_FLOW_MSG_ID)
+    return (TLV_HDR_LEN + payLoadLen);
+  else
+    return -1;
 }
 
-//! Flow Control Msg Id to String
-inline const char *FlowControlMsgIdToStr(FlowControlMsgID id)
-{
-	if (id < MAX_FLOW_MSG_ID)
-		return FlowControlMsgIDStr[id];
-	else
-		return NULL;
+// Flow Control Msg Id to String
+inline const char *FlowControlMsgIdToStr(FlowControlMsgID id) {
+  if (id < MAX_FLOW_MSG_ID)
+    return FlowControlMsgIDStr[id];
+  else
+    return NULL;
 }
 
-//! Flow Control Respond status code
+// Flow Control Respond status code
 typedef enum {
-	RESP_SUCCESS,
-	RESP_TRIGGER_REGISTER, // Server respond to client report to re trigger register
-	RESP_START_FLOW_CONTROLLER, // Server respond to client to start flow controller
-	RESP_STOP_FLOW_CONTROLLER, // Server respond to client to stop flow controller
-	RESP_FAILURE,
-	MAX_RESP_CODE
+  RESP_SUCCESS,
+  RESP_TRIGGER_REGISTER,  // Server respond to client report to re trigger register
+  RESP_START_FLOW_CONTROLLER,  // Server respond to client to start flow controller
+  RESP_STOP_FLOW_CONTROLLER,  // Server respond to client to stop flow controller
+  RESP_FAILURE,
+  MAX_RESP_CODE
 } FlowControlRespCode;
 
-//! FlowControl Resp Code str
-static const char *FlowControlRespCodeStr[MAX_RESP_CODE] =
-{
-		"RESP_SUCCESS",
-		"RESP_TRIGGER_REGISTER",
-		"RESP_START_FLOW_CONTROLLER",
-		"RESP_STOP_FLOW_CONTROLLER",
-		"RESP_FAILURE"
-};
+// FlowControl Resp Code str
+static const char *FlowControlRespCodeStr[MAX_RESP_CODE] = { "RESP_SUCCESS",
+    "RESP_TRIGGER_REGISTER", "RESP_START_FLOW_CONTROLLER",
+    "RESP_STOP_FLOW_CONTROLLER", "RESP_FAILURE" };
 
-//! Flow Control Resp Code to String
-inline const char *FlowControlRespCodeToStr(FlowControlRespCode code)
-{
-	if (code < MAX_RESP_CODE)
-		return FlowControlRespCodeStr[code];
-	else
-		return NULL;
+// Flow Control Resp Code to String
+inline const char *FlowControlRespCodeToStr(FlowControlRespCode code) {
+  if (code < MAX_RESP_CODE)
+    return FlowControlRespCodeStr[code];
+  else
+    return NULL;
 }
 
-//! Common FlowControlProtocol Header
+// Common FlowControlProtocol Header
 typedef struct {
-	uint32_t msgType; //! Msg Type
-	uint32_t seqNumber; //! Seq Number to match Req with Resp
-	uint32_t status; //! Resp Code, see FlowControlRespCode
-	uint32_t payloadLen; //! Msg Payload length
+  uint32_t msgType;  // Msg Type
+  uint32_t seqNumber;  // Seq Number to match Req with Resp
+  uint32_t status;  // Resp Code, see FlowControlRespCode
+  uint32_t payloadLen;  // Msg Payload length
 } FlowControlProtocolHeader;
 
-//! FlowControlProtocol Class
-class FlowControlProtocol
-{
-public:
-	//! Constructor
-	/*!
-	 * Create a new control protocol
-	 */
-	FlowControlProtocol(FlowController *controller) {
-		_controller = controller;
-		logger_ = Logger::getLogger();
-		configure_ = Configure::getConfigure();
-		_socket = 0;
-		_serverName = "localhost";
-		_serverPort = DEFAULT_NIFI_SERVER_PORT;
-		_registered = false;
-		_seqNumber = 0;
-		_reportBlob = NULL;
-		_reportBlobLen = 0;
-		_reportInterval = DEFAULT_REPORT_INTERVAL;
-		_running = false;
+// FlowControlProtocol Class
+class FlowControlProtocol {
+ public:
+  // Constructor
+  /*!
+   * Create a new control protocol
+   */
+  FlowControlProtocol(FlowController *controller) {
+    _controller = controller;
+    logger_ = logging::Logger::getLogger();
+    configure_ = Configure::getConfigure();
+    _socket = 0;
+    _serverName = "localhost";
+    _serverPort = DEFAULT_NIFI_SERVER_PORT;
+    _registered = false;
+    _seqNumber = 0;
+    _reportBlob = NULL;
+    _reportBlobLen = 0;
+    _reportInterval = DEFAULT_REPORT_INTERVAL;
+    running_ = false;
 
-		std::string value;
+    std::string value;
 
-		if (configure_->get(Configure::nifi_server_name, value))
-		{
-			_serverName = value;
-			logger_->log_info("NiFi Server Name %s", _serverName.c_str());
-		}
-		if (configure_->get(Configure::nifi_server_port, value) && Property::StringToInt(value, _serverPort))
-		{
-			logger_->log_info("NiFi Server Port: [%d]", _serverPort);
-		}
-		if (configure_->get(Configure::nifi_server_report_interval, value))
-		{
-			TimeUnit unit;
-			if (Property::StringToTime(value, _reportInterval, unit) &&
-						Property::ConvertTimeUnitToMS(_reportInterval, unit, _reportInterval))
-			{
-				logger_->log_info("NiFi server report interval: [%d] ms", _reportInterval);
-			}
-		}
-		else
-			_reportInterval = 0;
-	}
-	//! Destructor
-	virtual ~FlowControlProtocol()
-	{
-		stop();
-		if (_socket)
-			close(_socket);
-		if (_reportBlob)
-			delete [] _reportBlob;
-		if (this->_thread)
-			delete this->_thread;
-	}
+    if (configure_->get(Configure::nifi_server_name, value)) {
+      _serverName = value;
+      logger_->log_info("NiFi Server Name %s", _serverName.c_str());
+    }
+    if (configure_->get(Configure::nifi_server_port, value)
+        && core::Property::StringToInt(
+            value, _serverPort)) {
+      logger_->log_info("NiFi Server Port: [%d]", _serverPort);
+    }
+    if (configure_->get(Configure::nifi_server_report_interval, value)) {
+      core::TimeUnit unit;
+      if (core::Property::StringToTime(
+          value, _reportInterval, unit)
+          && core::Property::ConvertTimeUnitToMS(
+              _reportInterval, unit, _reportInterval)) {
+        logger_->log_info("NiFi server report interval: [%d] ms",
+                          _reportInterval);
+      }
+    } else
+      _reportInterval = 0;
+  }
+  // Destructor
+  virtual ~FlowControlProtocol() {
+    stop();
+    if (_socket)
+      close(_socket);
+    if (_reportBlob)
+      delete[] _reportBlob;
+    if (this->_thread)
+      delete this->_thread;
+  }
 
-public:
+ public:
 
-	//! SendRegisterRequest and Process Register Respond, return 0 for success
-	int sendRegisterReq();
-	//! SendReportReq and Process Report Respond, return 0 for success
-	int sendReportReq();
-	//! Start the flow control protocol
-	void start();
-	//! Stop the flow control protocol
-	void stop();
-	//! Set Report BLOB for periodically report
-	void setReportBlob(char *blob, int len)
-	{
-		std::lock_guard<std::mutex> lock(_mtx);
-		if (_reportBlob && _reportBlobLen >= len)
-		{
-			memcpy(_reportBlob, blob, len);
-			_reportBlobLen = len;
-		}
-		else
-		{
-			if (_reportBlob)
-				delete[] _reportBlob;
-			_reportBlob = new char[len];
-			_reportBlobLen = len;
-		}
-	}
-	//! Run function for the thread
-	static void run(FlowControlProtocol *protocol);
-	//! set 8 bytes SerialNumber
-	void setSerialNumber(uint8_t *number)
-	{
-		memcpy(_serialNumber, number, 8);
-	}
+  // SendRegisterRequest and Process Register Respond, return 0 for success
+  int sendRegisterReq();
+  // SendReportReq and Process Report Respond, return 0 for success
+  int sendReportReq();
+  // Start the flow control protocol
+  void start();
+  // Stop the flow control protocol
+  void stop();
+  // Set Report BLOB for periodically report
+  void setReportBlob(char *blob, int len) {
+    std::lock_guard<std::mutex> lock(mutex_);
+    if (_reportBlob && _reportBlobLen >= len) {
+      memcpy(_reportBlob, blob, len);
+      _reportBlobLen = len;
+    } else {
+      if (_reportBlob)
+        delete[] _reportBlob;
+      _reportBlob = new char[len];
+      _reportBlobLen = len;
+    }
+  }
+  // Run function for the thread
+  static void run(FlowControlProtocol *protocol);
+  // set 8 bytes SerialNumber
+  void setSerialNumber(uint8_t *number) {
+    memcpy(_serialNumber, number, 8);
+  }
 
-protected:
+ protected:
 
-private:
-	//! Connect to the socket, return sock descriptor if success, 0 for failure
-	int connectServer(const char *host, uint16_t port);
-	//! Send Data via the socket, return -1 for failure
-	int sendData(uint8_t *buf, int buflen);
-	//! Read length into buf, return -1 for failure and 0 for EOF
-	int readData(uint8_t *buf, int buflen);
-	//! Select on the socket
-	int selectClient(int msec);
-	//! Read the header
-	int readHdr(FlowControlProtocolHeader *hdr);
-	//! encode uint32_t
-	uint8_t *encode(uint8_t *buf, uint32_t value)
-	{
-		*buf++ = (value & 0xFF000000) >> 24;
-		*buf++ = (value & 0x00FF0000) >> 16;
-		*buf++ = (value & 0x0000FF00) >> 8;
-		*buf++ = (value & 0x000000FF);
-		return buf;
-	}
-	//! encode uint32_t
-	uint8_t *decode(uint8_t *buf, uint32_t &value)
-	{
-		value = ((buf[0]<<24)|(buf[1]<<16)|(buf[2]<<8)|(buf[3]));
-		return (buf + 4);
-	}
-	//! encode byte array
-	uint8_t *encode(uint8_t *buf, uint8_t *bufArray, int size)
-	{
-		memcpy(buf, bufArray, size);
-		buf += size;
-		return buf;
-	}
-	//! encode std::string
-	uint8_t *encode(uint8_t *buf, std::string value)
-	{
-		// add the \0 for size
-		buf = encode(buf, value.size()+1);
-		buf = encode(buf, (uint8_t *) value.c_str(), value.size()+1);
-		return buf;
-	}
-	//! Mutex for protection
-	std::mutex _mtx;
-	//! Logger
-	std::shared_ptr<Logger> logger_;
-	//! Configure
-	Configure *configure_ = NULL;
-	//! NiFi server Name
-	std::string _serverName;
-	//! NiFi server port
-	int64_t _serverPort;
-	//! Serial Number
-	uint8_t _serialNumber[8];
-	//! socket to server
-	int _socket;
-	//! report interal in msec
-	int64_t _reportInterval;
-	//! whether it was registered to the NiFi server
-	bool _registered;
-	//! seq number
-	uint32_t _seqNumber;
-	//! FlowController
-	FlowController *_controller = NULL;
-	//! report Blob
-	char *_reportBlob;
-	//! report Blob len;
-	int _reportBlobLen;
-	//! thread
-	std::thread *_thread = NULL;
-	//! whether it is running
-	bool _running;
-	// Prevent default copy constructor and assignment operation
-	// Only support pass by reference or pointer
-	FlowControlProtocol(const FlowControlProtocol &parent);
-	FlowControlProtocol &operator=(const FlowControlProtocol &parent);
+ private:
+  // Connect to the socket, return sock descriptor if success, 0 for failure
+  int connectServer(const char *host, uint16_t port);
+  // Send Data via the socket, return -1 for failure
+  int sendData(uint8_t *buf, int buflen);
+  // Read length into buf, return -1 for failure and 0 for EOF
+  int readData(uint8_t *buf, int buflen);
+  // Select on the socket
+  int selectClient(int msec);
+  // Read the header
+  int readHdr(FlowControlProtocolHeader *hdr);
+  // encode uint32_t
+  uint8_t *encode(uint8_t *buf, uint32_t value) {
+    *buf++ = (value & 0xFF000000) >> 24;
+    *buf++ = (value & 0x00FF0000) >> 16;
+    *buf++ = (value & 0x0000FF00) >> 8;
+    *buf++ = (value & 0x000000FF);
+    return buf;
+  }
+  // encode uint32_t
+  uint8_t *decode(uint8_t *buf, uint32_t &value) {
+    value = ((buf[0] << 24) | (buf[1] << 16) | (buf[2] << 8) | (buf[3]));
+    return (buf + 4);
+  }
+  // encode byte array
+  uint8_t *encode(uint8_t *buf, uint8_t *bufArray, int size) {
+    memcpy(buf, bufArray, size);
+    buf += size;
+    return buf;
+  }
+  // encode std::string
+  uint8_t *encode(uint8_t *buf, std::string value) {
+    // add the \0 for size
+    buf = encode(buf, value.size() + 1);
+    buf = encode(buf, (uint8_t *) value.c_str(), value.size() + 1);
+    return buf;
+  }
+  // Mutex for protection
+  std::mutex mutex_;
+  // Logger
+  std::shared_ptr<logging::Logger> logger_;
+  // Configure
+  Configure *configure_ = NULL;
+  // NiFi server Name
+  std::string _serverName;
+  // NiFi server port
+  int64_t _serverPort;
+  // Serial Number
+  uint8_t _serialNumber[8];
+  // socket to server
+  int _socket;
+  // report interal in msec
+  int64_t _reportInterval;
+  // whether it was registered to the NiFi server
+  bool _registered;
+  // seq number
+  uint32_t _seqNumber;
+  // FlowController
+  FlowController *_controller = NULL;
+  // report Blob
+  char *_reportBlob;
+  // report Blob len;
+  int _reportBlobLen;
+  // thread
+  std::thread *_thread = NULL;
+  // whether it is running
+  bool running_;
+  // Prevent default copy constructor and assignment operation
+  // Only support pass by reference or pointer
+  FlowControlProtocol(const FlowControlProtocol &parent);
+  FlowControlProtocol &operator=(const FlowControlProtocol &parent);
 
 };
 
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
 #endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/FlowController.h
----------------------------------------------------------------------
diff --git a/libminifi/include/FlowController.h b/libminifi/include/FlowController.h
index 9db13ff..0475623 100644
--- a/libminifi/include/FlowController.h
+++ b/libminifi/include/FlowController.h
@@ -28,324 +28,168 @@
 #include <atomic>
 #include <algorithm>
 #include <set>
-#ifdef YAML_SUPPORT
-#include "yaml-cpp/yaml.h"
-#endif
-#include "Configure.h"
-#include "Property.h"
-#include "Relationship.h"
+#include "properties/Configure.h"
+#include "core/Relationship.h"
 #include "FlowFileRecord.h"
 #include "Connection.h"
-#include "Processor.h"
-#include "ProcessContext.h"
-#include "ProcessSession.h"
-#include "ProcessGroup.h"
-#include "GenerateFlowFile.h"
-#include "LogAttribute.h"
-#include "RealTimeDataCollector.h"
+#include "core/Processor.h"
+#include "core/logging/Logger.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/ProcessGroup.h"
+#include "core/FlowConfiguration.h"
 #include "TimerDrivenSchedulingAgent.h"
 #include "EventDrivenSchedulingAgent.h"
 #include "FlowControlProtocol.h"
-#include "RemoteProcessorGroupPort.h"
-#include "Provenance.h"
-#include "FlowFileRepository.h"
-#include "GetFile.h"
-#include "PutFile.h"
-#include "TailFile.h"
-#include "ListenSyslog.h"
-#include "ListenHTTP.h"
-#include "ExecuteProcess.h"
-#include "AppendHostInfo.h"
-// OpenSSL related
-#ifdef OPENSSL_SUPPORT
-#include <openssl/ssl.h>
-#include <openssl/err.h>
-#endif
 
-//! Default NiFi Root Group Name
-#define DEFAULT_ROOT_GROUP_NAME ""
-#define DEFAULT_FLOW_YAML_FILE_NAME "conf/flow.yml"
-#define CONFIG_YAML_PROCESSORS_KEY "Processors"
+#include "core/Property.h"
 
-struct ProcessorConfig {
-	std::string id;
-	std::string name;
-	std::string javaClass;
-	std::string maxConcurrentTasks;
-	std::string schedulingStrategy;
-	std::string schedulingPeriod;
-	std::string penalizationPeriod;
-	std::string yieldPeriod;
-	std::string runDurationNanos;
-	std::vector<std::string> autoTerminatedRelationships;
-	std::vector<Property> properties;
-};
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+// Default NiFi Root Group Name
+#define DEFAULT_ROOT_GROUP_NAME ""
 
 /**
  * Flow Controller class. Generally used by FlowController factory
  * as a singleton.
  */
-class FlowController {
-public:
-  	static const int DEFAULT_MAX_TIMER_DRIVEN_THREAD = 10;
-	static const int DEFAULT_MAX_EVENT_DRIVEN_THREAD = 5;
-
-
-	//! Destructor
-	virtual ~FlowController(){
-	}
-	//! Set FlowController Name
-	virtual void setName(std::string name) {
-		_name = name;
-	}
-	//! Get Flow Controller Name
-	virtual std::string getName(void) {
-		return (_name);
-	}
-	//! Set UUID
-	virtual void setUUID(uuid_t uuid) {
-		uuid_copy(_uuid, uuid);
-	}
-	//! Get UUID
-	virtual bool getUUID(uuid_t uuid) {
-		if (uuid) {
-			uuid_copy(uuid, _uuid);
-			return true;
-		} else
-			return false;
-	}
-	//! Set MAX TimerDrivenThreads
-	virtual void setMaxTimerDrivenThreads(int number) {
-		_maxTimerDrivenThreads = number;
-	}
-	//! Get MAX TimerDrivenThreads
-	virtual int getMaxTimerDrivenThreads() {
-		return _maxTimerDrivenThreads;
-	}
-	//! Set MAX EventDrivenThreads
-	virtual void setMaxEventDrivenThreads(int number) {
-		_maxEventDrivenThreads = number;
-	}
-	//! Get MAX EventDrivenThreads
-	virtual int getMaxEventDrivenThreads() {
-		return _maxEventDrivenThreads;
-	}
-	//! Get the provenance repository
-	virtual ProvenanceRepository *getProvenanceRepository() {
-		return this->_provenanceRepo;
-	}
-	//! Get the flowfile repository
-	virtual FlowFileRepository *getFlowFileRepository() {
-		return this->_flowfileRepo;
-	}
-	//! Load flow xml from disk, after that, create the root process group and its children, initialize the flows
-	virtual void load() = 0;
-
-	//! Whether the Flow Controller is start running
-	virtual bool isRunning() {
-		return _running.load();
-	}
-	//! Whether the Flow Controller has already been initialized (loaded flow XML)
-	virtual bool isInitialized() {
-		return _initialized.load();
-	}
-	//! Start to run the Flow Controller which internally start the root process group and all its children
-	virtual bool start() = 0;
-	//! Unload the current flow YAML, clean the root process group and all its children
-	virtual void stop(bool force) = 0;
-	//! Asynchronous function trigger unloading and wait for a period of time
-	virtual void waitUnload(const uint64_t timeToWaitMs) = 0;
-	//! Unload the current flow xml, clean the root process group and all its children
-	virtual void unload() = 0;
-	//! Load new xml
-	virtual void reload(std::string yamlFile) = 0;
-	//! update property value
-	void updatePropertyValue(std::string processorName,
-			std::string propertyName, std::string propertyValue) {
-		if (_root)
-			_root->updatePropertyValue(processorName, propertyName,
-					propertyValue);
-	}
-
-	//! Create Processor (Node/Input/Output Port) based on the name
-	virtual Processor *createProcessor(std::string name, uuid_t uuid) = 0;
-	//! Create Root Processor Group
-	virtual ProcessGroup *createRootProcessGroup(std::string name, uuid_t uuid) = 0;
-	//! Create Remote Processor Group
-	virtual ProcessGroup *createRemoteProcessGroup(std::string name,
-			uuid_t uuid) = 0;
-	//! Create Connection
-	virtual Connection *createConnection(std::string name, uuid_t uuid) = 0;
-	//! set 8 bytes SerialNumber
-	virtual void setSerialNumber(uint8_t *number) {
-		_protocol->setSerialNumber(number);
-	}
-
-protected:
-  
-	//! A global unique identifier
-	uuid_t _uuid;
-	//! FlowController Name
-	std::string _name;
-	//! Configuration File Name
-	std::string _configurationFileName;
-	//! NiFi property File Name
-	std::string _propertiesFileName;
-	//! Root Process Group
-	ProcessGroup *_root;
-	//! MAX Timer Driven Threads
-	int _maxTimerDrivenThreads;
-	//! MAX Event Driven Threads
-	int _maxEventDrivenThreads;
-	//! Config
-	//! FlowFile Repo
-	//! Whether it is running
-	std::atomic<bool> _running;
-	//! Whether it has already been initialized (load the flow XML already)
-	std::atomic<bool> _initialized;
-	//! Provenance Repo
-	ProvenanceRepository *_provenanceRepo;
-	//! FlowFile Repo
-	FlowFileRepository *_flowfileRepo;
-	//! Flow Engines
-	//! Flow Timer Scheduler
-	TimerDrivenSchedulingAgent _timerScheduler;
-	//! Flow Event Scheduler
-	EventDrivenSchedulingAgent _eventScheduler;
-	//! Controller Service
-	//! Config
-	//! Site to Site Server Listener
-	//! Heart Beat
-	//! FlowControl Protocol
-	FlowControlProtocol *_protocol;
-	
-
-	FlowController() :
-			_root(0), _maxTimerDrivenThreads(0), _maxEventDrivenThreads(0), _running(
-					false), _initialized(false), _provenanceRepo(0), _flowfileRepo(0), _protocol(
-					0), logger_(Logger::getLogger()){
-	}
-
-private:
-
-	//! Logger
-	std::shared_ptr<Logger> logger_;
+class FlowController : public core::CoreComponent {
+ public:
+  static const int DEFAULT_MAX_TIMER_DRIVEN_THREAD = 10;
+  static const int DEFAULT_MAX_EVENT_DRIVEN_THREAD = 5;
+
+  /**
+   * Flow controller constructor
+   */
+  FlowController(std::shared_ptr<core::Repository> provenance_repo,
+                 std::shared_ptr<core::Repository> flow_file_repo,
+                 std::unique_ptr<core::FlowConfiguration> flow_configuration,
+                 const std::string name = DEFAULT_ROOT_GROUP_NAME,bool headless_mode=false);
+
+  // Destructor
+  virtual ~FlowController();
+
+  // Set MAX TimerDrivenThreads
+  virtual void setMaxTimerDrivenThreads(int number) {
+    max_timer_driven_threads_ = number;
+  }
+  // Get MAX TimerDrivenThreads
+  virtual int getMaxTimerDrivenThreads() {
+    return max_timer_driven_threads_;
+  }
+  // Set MAX EventDrivenThreads
+  virtual void setMaxEventDrivenThreads(int number) {
+    max_event_driven_threads_ = number;
+  }
+  // Get MAX EventDrivenThreads
+  virtual int getMaxEventDrivenThreads() {
+    return max_event_driven_threads_;
+  }
+  // Get the provenance repository
+  virtual std::shared_ptr<core::Repository> getProvenanceRepository() {
+    return this->provenance_repo_;
+  }
+
+  // Get the flowfile repository
+  virtual std::shared_ptr<core::Repository> getFlowFileRepository() {
+    return this->flow_file_repo_;
+  }
+
+  // Load flow xml from disk, after that, create the root process group and its children, initialize the flows
+  virtual void load();
+
+  // Whether the Flow Controller is start running
+  virtual bool isRunning() {
+    return running_.load();
+  }
+  // Whether the Flow Controller has already been initialized (loaded flow XML)
+  virtual bool isInitialized() {
+    return initialized_.load();
+  }
+  // Start to run the Flow Controller which internally start the root process group and all its children
+  virtual bool start();
+  // Unload the current flow YAML, clean the root process group and all its children
+  virtual void stop(bool force);
+  // Asynchronous function trigger unloading and wait for a period of time
+  virtual void waitUnload(const uint64_t timeToWaitMs);
+  // Unload the current flow xml, clean the root process group and all its children
+  virtual void unload();
+  // Load new xml
+  virtual void reload(std::string yamlFile);
+  // update property value
+  void updatePropertyValue(std::string processorName, std::string propertyName,
+                           std::string propertyValue) {
+    if (root_  != nullptr)
+      root_->updatePropertyValue(processorName, propertyName, propertyValue);
+  }
+
+  // set 8 bytes SerialNumber
+  virtual void setSerialNumber(uint8_t *number) {
+    protocol_->setSerialNumber(number);
+  }
+
+ protected:
+
+  // function to load the flow file repo.
+  void loadFlowRepo();
+
+  /**
+   * Initializes flow controller paths.
+   */
+  virtual void initializePaths(const std::string &adjustedFilename);
+
+  // flow controller mutex
+  std::recursive_mutex mutex_;
+
+  // configuration object
+  Configure *configure_;
+
+  // Configuration File Name
+  std::string configuration_file_name_;
+  // NiFi property File Name
+  std::string properties_file_name_;
+  // Root Process Group
+  std::unique_ptr<core::ProcessGroup> root_;
+  // MAX Timer Driven Threads
+  int max_timer_driven_threads_;
+  // MAX Event Driven Threads
+  int max_event_driven_threads_;
+  // FlowFile Repo
+  // Whether it is running
+  std::atomic<bool> running_;
+  // conifiguration filename
+  std::string configuration_filename_;
+  // Whether it has already been initialized (load the flow XML already)
+  std::atomic<bool> initialized_;
+  // Provenance Repo
+  std::shared_ptr<core::Repository> provenance_repo_;
+
+  // FlowFile Repo
+  std::shared_ptr<core::Repository> flow_file_repo_;
+
+  // Flow Engines
+  // Flow Timer Scheduler
+  TimerDrivenSchedulingAgent _timerScheduler;
+  // Flow Event Scheduler
+  EventDrivenSchedulingAgent _eventScheduler;
+  // Controller Service
+  // Config
+  // Site to Site Server Listener
+  // Heart Beat
+  // FlowControl Protocol
+  FlowControlProtocol *protocol_;
+
+  // flow configuration object.
+  std::unique_ptr<core::FlowConfiguration> flow_configuration_;
 
 };
 
-/**
- * Flow Controller implementation that defines the typical flow.
- * of events.
- */
-class FlowControllerImpl: public FlowController {
-public:
-
-	//! Destructor
-	virtual ~FlowControllerImpl();
-
-	//! Life Cycle related function
-	//! Load flow xml from disk, after that, create the root process group and its children, initialize the flows
-	void load();
-	//! Start to run the Flow Controller which internally start the root process group and all its children
-	bool start();
-	//! Stop to run the Flow Controller which internally stop the root process group and all its children
-	void stop(bool force);
-	//! Asynchronous function trigger unloading and wait for a period of time
-	void waitUnload(const uint64_t timeToWaitMs);
-	//! Unload the current flow xml, clean the root process group and all its children
-	void unload();
-	//! Load new xml
-	void reload(std::string yamlFile);
-	//! Load Flow File from persistent Flow Repo
-	void loadFlowRepo();
-	//! update property value
-	void updatePropertyValue(std::string processorName,
-			std::string propertyName, std::string propertyValue) {
-		if (_root)
-			_root->updatePropertyValue(processorName, propertyName,
-					propertyValue);
-	}
-
-	//! Create Processor (Node/Input/Output Port) based on the name
-	Processor *createProcessor(std::string name, uuid_t uuid);
-	//! Create Root Processor Group
-	ProcessGroup *createRootProcessGroup(std::string name, uuid_t uuid);
-	//! Create Remote Processor Group
-	ProcessGroup *createRemoteProcessGroup(std::string name, uuid_t uuid);
-	//! Create Connection
-	Connection *createConnection(std::string name, uuid_t uuid);
-
-	//! Constructor
-	/*!
-	 * Create a new Flow Controller
-	 */
-	FlowControllerImpl(std::string name = DEFAULT_ROOT_GROUP_NAME);
-
-	
-	
-	
-	friend class FlowControlFactory;
-
-private:
-  
-	
-
-	//! Mutex for protection
-	std::mutex _mtx;
-	//! Logger
-	std::shared_ptr<Logger> logger_;
-	Configure *configure_;
-
-#ifdef YAML_SUPPORT
-	//! Process Processor Node YAML
-	void parseProcessorNodeYaml(YAML::Node processorNode, ProcessGroup *parent);
-	//! Process Port YAML
-	void parsePortYaml(YAML::Node *portNode, ProcessGroup *parent,
-			TransferDirection direction);
-	//! Process Root Processor Group YAML
-	void parseRootProcessGroupYaml(YAML::Node rootNode);
-	//! Process Property YAML
-	void parseProcessorPropertyYaml(YAML::Node *doc, YAML::Node *node,
-			Processor *processor);
-	//! Process connection YAML
-	void parseConnectionYaml(YAML::Node *node, ProcessGroup *parent);
-	//! Process Remote Process Group YAML
-	void parseRemoteProcessGroupYaml(YAML::Node *node, ProcessGroup *parent);
-	//! Parse Properties Node YAML for a processor
-	void parsePropertiesNodeYaml(YAML::Node *propertiesNode,
-			Processor *processor);
-#endif
-
-	// Prevent default copy constructor and assignment operation
-	// Only support pass by reference or pointer
-	FlowControllerImpl(const FlowController &parent);
-	FlowControllerImpl &operator=(const FlowController &parent);
-
-};
-
-/**
- * Flow Controller factory that creates flow controllers or gets the
- * assigned instance.
- */
-class FlowControllerFactory {
-public:
-	//! Get the singleton flow controller
-	static FlowController * getFlowController(FlowController *instance = 0) {
-		if (!_flowController) {
-			if (NULL == instance)
-				_flowController = createFlowController();
-			else
-				_flowController = instance;
-		}
-		return _flowController;
-	}
-
-	//! Get the singleton flow controller
-	static FlowController * createFlowController() {
-		 return dynamic_cast<FlowController*>(new FlowControllerImpl());
-	}
-private:
-	static FlowController *_flowController;
-};
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
 
 #endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/FlowFileRecord.h
----------------------------------------------------------------------
diff --git a/libminifi/include/FlowFileRecord.h b/libminifi/include/FlowFileRecord.h
index ded0623..ca0856c 100644
--- a/libminifi/include/FlowFileRecord.h
+++ b/libminifi/include/FlowFileRecord.h
@@ -31,215 +31,155 @@
 #include <fstream>
 #include <set>
 
+#include "io/Serializable.h"
+#include "core/FlowFile.h"
 #include "utils/TimeUtil.h"
-#include "Logger.h"
+#include "core/logging/Logger.h"
 #include "ResourceClaim.h"
+#include "Connection.h"
 
-class ProcessSession;
-class Connection;
-class FlowFileEventRecord;
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
 
 #define DEFAULT_FLOWFILE_PATH "."
 
-//! FlowFile Attribute
-enum FlowAttribute
-{
-	//! The flowfile's path indicates the relative directory to which a FlowFile belongs and does not contain the filename
-	PATH = 0,
-	//! The flowfile's absolute path indicates the absolute directory to which a FlowFile belongs and does not contain the filename
-	ABSOLUTE_PATH,
-	//! The filename of the FlowFile. The filename should not contain any directory structure.
-	FILENAME,
-	//! A unique UUID assigned to this FlowFile.
-	UUID,
-	//! A numeric value indicating the FlowFile priority
-	priority,
-	//! The MIME Type of this FlowFile
-	MIME_TYPE,
-	//! Specifies the reason that a FlowFile is being discarded
-	DISCARD_REASON,
-	//! Indicates an identifier other than the FlowFile's UUID that is known to refer to this FlowFile.
-	ALTERNATE_IDENTIFIER,
-	MAX_FLOW_ATTRIBUTES
+// FlowFile Attribute
+enum FlowAttribute {
+  // The flowfile's path indicates the relative directory to which a FlowFile belongs and does not contain the filename
+  PATH = 0,
+  // The flowfile's absolute path indicates the absolute directory to which a FlowFile belongs and does not contain the filename
+  ABSOLUTE_PATH,
+  // The filename of the FlowFile. The filename should not contain any directory structure.
+  FILENAME,
+  // A unique UUID assigned to this FlowFile.
+  UUID,
+  // A numeric value indicating the FlowFile priority
+  priority,
+  // The MIME Type of this FlowFile
+  MIME_TYPE,
+  // Specifies the reason that a FlowFile is being discarded
+  DISCARD_REASON,
+  // Indicates an identifier other than the FlowFile's UUID that is known to refer to this FlowFile.
+  ALTERNATE_IDENTIFIER,
+  MAX_FLOW_ATTRIBUTES
 };
 
-//! FlowFile Attribute Key
-static const char *FlowAttributeKeyArray[MAX_FLOW_ATTRIBUTES] =
-{
-		"path",
-		"absolute.path",
-		"filename",
-		"uuid",
-		"priority",
-		"mime.type",
-		"discard.reason",
-		"alternate.identifier"
-};
-
-//! FlowFile Attribute Enum to Key
-inline const char *FlowAttributeKey(FlowAttribute attribute)
-{
-	if (attribute < MAX_FLOW_ATTRIBUTES)
-		return FlowAttributeKeyArray[attribute];
-	else
-		return NULL;
+// FlowFile Attribute Key
+static const char *FlowAttributeKeyArray[MAX_FLOW_ATTRIBUTES] = { "path",
+    "absolute.path", "filename", "uuid", "priority", "mime.type",
+    "discard.reason", "alternate.identifier" };
+
+// FlowFile Attribute Enum to Key
+inline const char *FlowAttributeKey(FlowAttribute attribute) {
+  if (attribute < MAX_FLOW_ATTRIBUTES)
+    return FlowAttributeKeyArray[attribute];
+  else
+    return NULL;
 }
 
-//! FlowFile IO Callback functions for input and output
-//! throw exception for error
-class InputStreamCallback
-{
-public:
-	virtual void process(std::ifstream *stream) = 0;
+// FlowFile IO Callback functions for input and output
+// throw exception for error
+class InputStreamCallback {
+ public:
+  virtual void process(std::ifstream *stream) = 0;
 };
-class OutputStreamCallback
-{
-public:
-	virtual void process(std::ofstream *stream) = 0;
+class OutputStreamCallback {
+ public:
+  virtual void process(std::ofstream *stream) = 0;
 };
 
-
-//! FlowFile Record Class
-class FlowFileRecord
-{
-	friend class ProcessSession;
-public:
-	//! Constructor
-	/*!
-	 * Create a new flow record
-	 */
-	explicit FlowFileRecord(std::map<std::string, std::string> attributes, ResourceClaim *claim = NULL);
-	/*!
-	 * Create a new flow record from repo flow event
-	 */
-	explicit FlowFileRecord(FlowFileEventRecord *event);
-	//! Destructor
-	virtual ~FlowFileRecord();
-	//! addAttribute key is enum
-	bool addAttribute(FlowAttribute key, std::string value);
-	//! addAttribute key is string
-	bool addAttribute(std::string key, std::string value);
-	//! removeAttribute key is enum
-	bool removeAttribute(FlowAttribute key);
-	//! removeAttribute key is string
-	bool removeAttribute(std::string key);
-	//! updateAttribute key is enum
-	bool updateAttribute(FlowAttribute key, std::string value);
-	//! updateAttribute key is string
-	bool updateAttribute(std::string key, std::string value);
-	//! getAttribute key is enum
-	bool getAttribute(FlowAttribute key, std::string &value);
-	//! getAttribute key is string
-	bool getAttribute(std::string key, std::string &value);
-	//! setAttribute, if attribute already there, update it, else, add it
-	void setAttribute(std::string key, std::string value) {
-		_attributes[key] = value;
-	}
-	//! Get the UUID as string
-	std::string getUUIDStr() {
-		return _uuidStr;
-	}
-	//! Get Attributes
-	std::map<std::string, std::string> getAttributes() {
-		return _attributes;
-	}
-	//! Check whether it is still being penalized
-	bool isPenalized() {
-		return (_penaltyExpirationMs > 0 ? _penaltyExpirationMs > getTimeMillis() : false);
-	}
-	//! Get Size
-	uint64_t getSize() {
-		return _size;
-	}
-	// ! Get Offset
-	uint64_t getOffset() {
-		return _offset;
-	}
-	// ! Get Entry Date
-	uint64_t getEntryDate() {
-		return _entryDate;
-	}
-	// ! Get Lineage Start Date
-	uint64_t getlineageStartDate() {
-		return _lineageStartDate;
-	}
-	// ! Set Original connection
-	void setOriginalConnection (Connection *connection) {
-		_orginalConnection = connection;
-	}
-	//! Get Original connection
-	Connection * getOriginalConnection() {
-		return _orginalConnection;
-	}
-	//! Get Resource Claim
-	ResourceClaim *getResourceClaim() {
-		return _claim;
-	}
-	//! Get lineageIdentifiers
-	std::set<std::string> getlineageIdentifiers()
-	{
-		return _lineageIdentifiers;
-	}
-	//! Check whether it is stored to DB already
-	bool isStoredToRepository()
-	{
-		return _isStoredToRepo;
-	}
-	void setStoredToRepository(bool value)
-	{
-		_isStoredToRepo = value;
-	}
-
-protected:
-
-	//! Date at which the flow file entered the flow
-	uint64_t _entryDate;
-	//! Date at which the origin of this flow file entered the flow
-	uint64_t _lineageStartDate;
-	//! Date at which the flow file was queued
-	uint64_t _lastQueueDate;
-	//! Size in bytes of the data corresponding to this flow file
-	uint64_t _size;
-	//! A global unique identifier
-	uuid_t _uuid;
-	//! A local unique identifier
-	uint64_t _id;
-	//! Offset to the content
-	uint64_t _offset;
-	//! Penalty expiration
-	uint64_t _penaltyExpirationMs;
-	//! Attributes key/values pairs for the flow record
-	std::map<std::string, std::string> _attributes;
-	//! Pointer to the associated content resource claim
-	ResourceClaim *_claim;
-	//! UUID string
-	std::string _uuidStr;
-	//! UUID string for all parents
-	std::set<std::string> _lineageIdentifiers;
-	//! whether it is stored to DB
-	bool _isStoredToRepo;
-	//! duplicate the original flow file
-	void duplicate(FlowFileRecord *original);
-
-private:
-
-	//! Local flow sequence ID
-	static std::atomic<uint64_t> _localFlowSeqNumber;
-	//! Mark for deletion
-	bool _markedDelete;
-	//! Connection queue that this flow file will be transfer or current in
-	Connection *_connection;
-	//! Orginal connection queue that this flow file was dequeued from
-	Connection *_orginalConnection;
-	//! Logger
-	std::shared_ptr<Logger> logger_;
-	//! Snapshot flow record for session rollback
-	bool _snapshot;
-	// Prevent default copy constructor and assignment operation
-	// Only support pass by reference or pointer
-	FlowFileRecord(const FlowFileRecord &parent);
-	FlowFileRecord &operator=(const FlowFileRecord &parent);
+class FlowFileRecord : public core::FlowFile, public io::Serializable {
+ public:
+  // Constructor
+  /*
+   * 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,
+                          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)
+      : FlowFile(),
+        flow_repository_(flow_repository),
+        snapshot_("") {
+
+  }
+  // Destructor
+  virtual ~FlowFileRecord();
+  // addAttribute key is enum
+  bool addKeyedAttribute(FlowAttribute key, std::string value);
+  // removeAttribute key is enum
+  bool removeKeyedAttribute(FlowAttribute key);
+  // updateAttribute key is enum
+  bool updateKeyedAttribute(FlowAttribute key, std::string value);
+  // getAttribute key is enum
+  bool getKeyedAttribute(FlowAttribute key, std::string &value);
+
+  //! Serialize and Persistent to the repository
+  bool Serialize();
+  //! DeSerialize
+  bool DeSerialize(const uint8_t *buffer, const int bufferSize);
+  //! DeSerialize
+  bool DeSerialize(io::DataStream &stream) {
+    return DeSerialize(stream.getBuffer(), stream.getSize());
+  }
+  //! DeSerialize
+  bool DeSerialize(std::string key);
+
+  void setSnapShot(bool snapshot) {
+    snapshot_ = snapshot;
+  }
+
+  /**
+   * gets the UUID connection.
+   * @return uuidConnection
+   */
+  const std::string getConnectionUuid() {
+    return uuid_connection_;
+  }
+  
+  const std::string getContentFullPath()
+  {
+    return content_full_fath_;
+  }
+  
+
+  FlowFileRecord &operator=(const FlowFileRecord &);
+
+  FlowFileRecord(const FlowFileRecord &parent) = delete;
+
+ protected:
+
+  // connection uuid
+  std::string uuid_connection_;
+  // Full path to the content
+  std::string content_full_fath_;
+
+  // Local flow sequence ID
+  static std::atomic<uint64_t> local_flow_seq_number_;
+
+  // repository reference.
+  std::shared_ptr<core::Repository> flow_repository_;
+
+  // Snapshot flow record for session rollback
+  bool snapshot_;
+  // Prevent default copy constructor and assignment operation
+  // Only support pass by reference or pointer
 
 };
 
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
 #endif


[05/16] nifi-minifi-cpp git commit: MINIFI-217: Updates namespaces and removes use of raw pointers for user facing API.

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/core/ConfigurableComponent.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/ConfigurableComponent.cpp b/libminifi/src/core/ConfigurableComponent.cpp
new file mode 100644
index 0000000..e5703d1
--- /dev/null
+++ b/libminifi/src/core/ConfigurableComponent.cpp
@@ -0,0 +1,143 @@
+/**
+ *
+ * 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/ConfigurableComponent.h"
+
+#include "core/Property.h"
+#include "core/logging/Logger.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+ConfigurableComponent::ConfigurableComponent(std::shared_ptr<logging::Logger> logger)
+    : logger_(logger) {
+
+}
+
+ConfigurableComponent::ConfigurableComponent(
+    const ConfigurableComponent &&other)
+    : properties_(std::move(other.properties_)),
+      logger_(std::move(other.logger_)) {
+
+}
+ConfigurableComponent::~ConfigurableComponent() {
+
+}
+
+/**
+ * Get property using the provided name.
+ * @param name property name.
+ * @param value value passed in by reference
+ * @return result of getting property.
+ */
+bool ConfigurableComponent::getProperty(const std::string name,
+                                        std::string &value) {
+  std::lock_guard<std::mutex> lock(configuration_mutex_);
+
+  auto &&it = properties_.find(name);
+
+  if (it != properties_.end()) {
+    Property item = it->second;
+    value = item.getValue();
+    logger_->log_info("Processor %s property name %s value %s", name.c_str(),
+                      item.getName().c_str(), value.c_str());
+    return true;
+  } else {
+    return false;
+  }
+}
+/**
+ * Sets the property using the provided name
+ * @param property name
+ * @param value property value.
+ * @return result of setting property.
+ */
+bool ConfigurableComponent::setProperty(const std::string name,
+                                        std::string value) {
+  std::lock_guard<std::mutex> lock(configuration_mutex_);
+  auto &&it = properties_.find(name);
+
+  if (it != properties_.end()) {
+    Property item = it->second;
+    item.setValue(value);
+    properties_[item.getName()] = item;
+    logger_->log_info("Component %s property name %s value %s", name.c_str(),
+                      item.getName().c_str(), value.c_str());
+    return true;
+  } else {
+    return false;
+  }
+}
+
+/**
+ * Sets the property using the provided name
+ * @param property name
+ * @param value property value.
+ * @return whether property was set or not
+ */
+bool ConfigurableComponent::setProperty(Property &prop, std::string value) {
+  std::lock_guard<std::mutex> lock(configuration_mutex_);
+  auto it = properties_.find(prop.getName());
+
+  if (it != properties_.end()) {
+    Property item = it->second;
+    item.setValue(value);
+    properties_[item.getName()] = item;
+    logger_->log_info("property name %s value %s", prop.getName().c_str(),
+                      item.getName().c_str(), value.c_str());
+    return true;
+  } else {
+    Property newProp(prop);
+    newProp.setValue(value);
+    properties_.insert(
+        std::pair<std::string, Property>(prop.getName(), newProp));
+    return true;
+
+  }
+  return false;
+}
+
+/**
+ * Sets supported properties for the ConfigurableComponent
+ * @param supported properties
+ * @return result of set operation.
+ */
+bool ConfigurableComponent::setSupportedProperties(
+    std::set<Property> properties) {
+  if (!canEdit()) {
+    return false;
+  }
+
+  std::lock_guard<std::mutex> lock(configuration_mutex_);
+
+  properties_.clear();
+  for (auto item : properties) {
+    properties_[item.getName()] = item;
+  }
+
+  return true;
+}
+
+} /* namespace components */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/core/ConfigurationFactory.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/ConfigurationFactory.cpp b/libminifi/src/core/ConfigurationFactory.cpp
new file mode 100644
index 0000000..52bde69
--- /dev/null
+++ b/libminifi/src/core/ConfigurationFactory.cpp
@@ -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.
+ */
+
+#include "core/ConfigurationFactory.h"
+#include "core/FlowConfiguration.h"
+#include  <type_traits>
+#ifdef YAML_SUPPORT
+#include "core/yaml/YamlConfiguration.h"
+#endif
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+#ifndef YAML_SUPPORT
+  class YamlConfiguration;
+#endif
+
+  std::unique_ptr<core::FlowConfiguration> createFlowConfiguration(
+      std::shared_ptr<core::Repository> repo,
+      std::shared_ptr<core::Repository> flow_file_repo,
+      const std::string configuration_class_name, const std::string path,
+      bool fail_safe) {
+
+    std::string class_name_lc = configuration_class_name;
+    std::transform(class_name_lc.begin(), class_name_lc.end(),
+                   class_name_lc.begin(), ::tolower);
+    try {
+
+      if (class_name_lc == "flowconfiguration") {
+	// load the base configuration.
+        return std::unique_ptr<core::FlowConfiguration>(
+            new core::FlowConfiguration(repo, flow_file_repo, 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, path));
+            
+
+      } else {
+        if (fail_safe) {
+          return std::unique_ptr<core::FlowConfiguration>(
+              new core::FlowConfiguration(repo, flow_file_repo, 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, path));
+      }
+    }
+
+    throw std::runtime_error(
+        "Support for the provided configuration class could not be found");
+  }
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/core/Connectable.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/Connectable.cpp b/libminifi/src/core/Connectable.cpp
new file mode 100644
index 0000000..ac61568
--- /dev/null
+++ b/libminifi/src/core/Connectable.cpp
@@ -0,0 +1,174 @@
+/*
+ * Connectable.cpp
+ *
+ *  Created on: Feb 27, 2017
+ *      Author: mparisi
+ */
+
+#include "../../include/core/Connectable.h"
+
+#include <uuid/uuid.h>
+#include "core/logging/Logger.h"
+#include "core/Relationship.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+Connectable::Connectable(std::string name, uuid_t uuid)
+    : CoreComponent(name, uuid),
+      max_concurrent_tasks_(1) {
+
+}
+
+Connectable::Connectable(const Connectable &&other)
+    : CoreComponent(std::move(other)),
+      max_concurrent_tasks_(std::move(other.max_concurrent_tasks_)) {
+  has_work_ = other.has_work_.load();
+  strategy_ = other.strategy_.load();
+}
+
+Connectable::~Connectable() {
+
+}
+
+bool Connectable::setSupportedRelationships(
+    std::set<core::Relationship> relationships) {
+  if (isRunning()) {
+    logger_->log_info(
+        "Can not set processor supported relationship while the process %s is running",
+        name_.c_str());
+    return false;
+  }
+
+  std::lock_guard<std::mutex> lock(relationship_mutex_);
+
+  relationships_.clear();
+  for (auto item : relationships) {
+    relationships_[item.getName()] = item;
+    logger_->log_info("Processor %s supported relationship name %s",
+                      name_.c_str(), item.getName().c_str());
+  }
+
+  return true;
+}
+
+// Whether the relationship is supported
+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 &it = relationships_.find(relationship.getName());
+  if (it != relationships_.end()) {
+    return true;
+  } else {
+    return false;
+  }
+}
+
+bool Connectable::setAutoTerminatedRelationships(
+    std::set<Relationship> relationships) {
+  if (isRunning()) {
+    logger_->log_info(
+        "Can not set processor auto terminated relationship while the process %s is running",
+        name_.c_str());
+    return false;
+  }
+
+  std::lock_guard<std::mutex> lock(relationship_mutex_);
+
+  auto_terminated_relationships_.clear();
+  for (auto item : relationships) {
+    auto_terminated_relationships_[item.getName()] = item;
+    logger_->log_info("Processor %s auto terminated relationship name %s",
+                      name_.c_str(), item.getName().c_str());
+  }
+
+  return true;
+}
+
+// Check whether the relationship is auto terminated
+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 &it = auto_terminated_relationships_.find(relationship.getName());
+  if (it != auto_terminated_relationships_.end()) {
+    return true;
+  } else {
+    return false;
+  }
+}
+
+void Connectable::waitForWork(uint64_t timeoutMs) {
+  has_work_.store(isWorkAvailable());
+
+  if (!has_work_.load()) {
+    std::unique_lock<std::mutex> lock(work_available_mutex_);
+    work_condition_.wait_for(lock, std::chrono::milliseconds(timeoutMs),
+                             [&] {return has_work_.load();});
+  }
+
+}
+
+void Connectable::notifyWork() {
+  // Do nothing if we are not event-driven
+  if (strategy_ != EVENT_DRIVEN) {
+    return;
+  }
+
+  {
+    has_work_.store(isWorkAvailable());
+
+    if (has_work_.load()) {
+      work_condition_.notify_one();
+    }
+  }
+
+}
+
+std::set<std::shared_ptr<Connectable>> Connectable::getOutGoingConnections(
+    std::string relationship) {
+  std::set<std::shared_ptr<Connectable>> empty;
+
+  auto &&it = _outGoingConnections.find(relationship);
+  if (it != _outGoingConnections.end()) {
+    return _outGoingConnections[relationship];
+  } else {
+    return empty;
+  }
+}
+
+std::shared_ptr<Connectable> Connectable::getNextIncomingConnection() {
+  std::lock_guard<std::mutex> lock(relationship_mutex_);
+
+  if (_incomingConnections.size() == 0)
+    return NULL;
+
+  if (incoming_connections_Iter == _incomingConnections.end())
+    incoming_connections_Iter = _incomingConnections.begin();
+
+  std::shared_ptr<Connectable> ret = *incoming_connections_Iter;
+  incoming_connections_Iter++;
+
+  if (incoming_connections_Iter == _incomingConnections.end())
+    incoming_connections_Iter = _incomingConnections.begin();
+
+  return ret;
+}
+
+} /* namespace components */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/core/Core.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/Core.cpp b/libminifi/src/core/Core.cpp
new file mode 100644
index 0000000..39969f6
--- /dev/null
+++ b/libminifi/src/core/Core.cpp
@@ -0,0 +1,51 @@
+/*
+ * Core.cpp
+ *
+ *  Created on: Mar 10, 2017
+ *      Author: mparisi
+ */
+
+#include "core/core.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+// Set UUID
+void CoreComponent::setUUID(uuid_t uuid) {
+  uuid_copy(uuid_, uuid);
+  char uuidStr[37];
+  uuid_unparse_lower(uuid_, uuidStr);
+  uuidStr_ = uuidStr;
+}
+// Get UUID
+bool CoreComponent::getUUID(uuid_t uuid) {
+  if (uuid) {
+    uuid_copy(uuid, uuid_);
+    return true;
+  } else {
+    return false;
+  }
+}
+
+// Get UUID
+unsigned const char *CoreComponent::getUUID() {
+  return uuid_;
+}
+
+// Set Processor Name
+void CoreComponent::setName(const std::string name) {
+  name_ = name;
+
+}
+// Get Process Name
+std::string CoreComponent::getName() {
+  return name_;
+}
+}
+}
+}
+}
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/core/FlowConfiguration.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/FlowConfiguration.cpp b/libminifi/src/core/FlowConfiguration.cpp
new file mode 100644
index 0000000..c6472cc
--- /dev/null
+++ b/libminifi/src/core/FlowConfiguration.cpp
@@ -0,0 +1,110 @@
+/**
+ *
+ * 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/FlowConfiguration.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+FlowConfiguration::~FlowConfiguration() {
+
+}
+
+std::shared_ptr<core::Processor> FlowConfiguration::createProcessor(
+    std::string name, uuid_t uuid) {
+  std::shared_ptr<core::Processor> processor = nullptr;
+  if (name
+      == org::apache::nifi::minifi::processors::GenerateFlowFile::ProcessorName) {
+    processor = std::make_shared<
+        org::apache::nifi::minifi::processors::GenerateFlowFile>(name, uuid);
+  } else if (name
+      == org::apache::nifi::minifi::processors::LogAttribute::ProcessorName) {
+    processor = std::make_shared<
+        org::apache::nifi::minifi::processors::LogAttribute>(name, uuid);
+  } else if (name
+      == org::apache::nifi::minifi::processors::RealTimeDataCollector::ProcessorName) {
+    processor = std::make_shared<
+        org::apache::nifi::minifi::processors::RealTimeDataCollector>(name,
+                                                                      uuid);
+  } else if (name
+      == org::apache::nifi::minifi::processors::GetFile::ProcessorName) {
+    processor =
+        std::make_shared<org::apache::nifi::minifi::processors::GetFile>(name,
+                                                                         uuid);
+  } else if (name
+      == org::apache::nifi::minifi::processors::PutFile::ProcessorName) {
+    processor =
+        std::make_shared<org::apache::nifi::minifi::processors::PutFile>(name,
+                                                                         uuid);
+  } else if (name
+      == org::apache::nifi::minifi::processors::TailFile::ProcessorName) {
+    processor =
+        std::make_shared<org::apache::nifi::minifi::processors::TailFile>(name,
+                                                                          uuid);
+  } else if (name
+      == org::apache::nifi::minifi::processors::ListenSyslog::ProcessorName) {
+    processor = std::make_shared<
+        org::apache::nifi::minifi::processors::ListenSyslog>(name, uuid);
+  } else if (name
+      == org::apache::nifi::minifi::processors::ListenHTTP::ProcessorName) {
+    processor = std::make_shared<
+        org::apache::nifi::minifi::processors::ListenHTTP>(name, uuid);
+  } else if (name
+      == org::apache::nifi::minifi::processors::ExecuteProcess::ProcessorName) {
+    processor = std::make_shared<
+        org::apache::nifi::minifi::processors::ExecuteProcess>(name, uuid);
+  } else if (name
+      == org::apache::nifi::minifi::processors::AppendHostInfo::ProcessorName) {
+    processor = std::make_shared<
+        org::apache::nifi::minifi::processors::AppendHostInfo>(name, uuid);
+  } else {
+    logger_->log_error("No Processor defined for %s", name.c_str());
+    return nullptr;
+  }
+
+  // initialize the processor
+  processor->initialize();
+
+  return processor;
+}
+
+std::unique_ptr<core::ProcessGroup> FlowConfiguration::createRootProcessGroup(
+    std::string name, uuid_t uuid) {
+  return std::unique_ptr<core::ProcessGroup>(
+      new core::ProcessGroup(core::ROOT_PROCESS_GROUP, name, uuid));
+}
+
+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));
+}
+
+std::shared_ptr<minifi::Connection> FlowConfiguration::createConnection(
+    std::string name, uuid_t uuid) {
+  return std::make_shared<minifi::Connection>(flow_file_repo_, name, uuid);
+}
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/core/ProcessGroup.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/ProcessGroup.cpp b/libminifi/src/core/ProcessGroup.cpp
new file mode 100644
index 0000000..baa3ebd
--- /dev/null
+++ b/libminifi/src/core/ProcessGroup.cpp
@@ -0,0 +1,312 @@
+/**
+ * @file ProcessGroup.cpp
+ * ProcessGroup class implementation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <vector>
+#include <queue>
+#include <map>
+#include <set>
+#include <sys/time.h>
+#include <time.h>
+#include <chrono>
+#include <thread>
+
+#include "core/ProcessGroup.h"
+#include "core/Processor.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+ProcessGroup::ProcessGroup(ProcessGroupType type, std::string name, uuid_t uuid,
+                           ProcessGroup *parent)
+    : name_(name),
+      type_(type),
+      parent_process_group_(parent) {
+  if (!uuid)
+    // Generate the global UUID for the flow record
+    uuid_generate(uuid_);
+  else
+    uuid_copy(uuid_, uuid);
+
+  yield_period_msec_ = 0;
+  transmitting_ = false;
+
+  logger_ = logging::Logger::getLogger();
+  logger_->log_info("ProcessGroup %s created", name_.c_str());
+}
+
+ProcessGroup::~ProcessGroup() {
+  for (auto &&connection : connections_) {
+    connection->drain();
+  }
+
+  for (std::set<ProcessGroup *>::iterator it = child_process_groups_.begin();
+      it != child_process_groups_.end(); ++it) {
+    ProcessGroup *processGroup(*it);
+    delete processGroup;
+  }
+
+}
+
+bool ProcessGroup::isRootProcessGroup() {
+  std::lock_guard<std::mutex> lock(mutex_);
+  return (type_ == ROOT_PROCESS_GROUP);
+}
+
+void ProcessGroup::addProcessor(std::shared_ptr<Processor> processor) {
+  std::lock_guard<std::mutex> lock(mutex_);
+
+  if (processors_.find(processor) == processors_.end()) {
+    // We do not have the same processor in this process group yet
+    processors_.insert(processor);
+    logger_->log_info("Add processor %s into process group %s",
+                      processor->getName().c_str(), name_.c_str());
+  }
+}
+
+void ProcessGroup::removeProcessor(std::shared_ptr<Processor> processor) {
+  std::lock_guard<std::mutex> lock(mutex_);
+
+  if (processors_.find(processor) != processors_.end()) {
+    // We do have the same processor in this process group yet
+    processors_.erase(processor);
+    logger_->log_info("Remove processor %s from process group %s",
+                      processor->getName().c_str(), name_.c_str());
+  }
+}
+
+void ProcessGroup::addProcessGroup(ProcessGroup *child) {
+  std::lock_guard<std::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
+    child_process_groups_.insert(child);
+    logger_->log_info("Add child process group %s into process group %s",
+                      child->getName().c_str(), name_.c_str());
+  }
+}
+
+void ProcessGroup::removeProcessGroup(ProcessGroup *child) {
+  std::lock_guard<std::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
+    child_process_groups_.erase(child);
+    logger_->log_info("Remove child process group %s from process group %s",
+                      child->getName().c_str(), name_.c_str());
+  }
+}
+
+void ProcessGroup::startProcessing(TimerDrivenSchedulingAgent *timeScheduler,
+                                   EventDrivenSchedulingAgent *eventScheduler) {
+  std::lock_guard<std::mutex> lock(mutex_);
+
+  try {
+    // Start all the processor node, input and output ports
+    for (auto processor : processors_) {
+      logger_->log_debug("Starting %s", processor->getName().c_str());
+
+      if (!processor->isRunning()
+          && processor->getScheduledState() != DISABLED) {
+        if (processor->getSchedulingStrategy() == TIMER_DRIVEN)
+          timeScheduler->schedule(processor);
+        else if (processor->getSchedulingStrategy() == EVENT_DRIVEN)
+          eventScheduler->schedule(processor);
+      }
+    }
+    // Start processing the group
+    for (auto processGroup : child_process_groups_) {
+      processGroup->startProcessing(timeScheduler, eventScheduler);
+    }
+  } catch (std::exception &exception) {
+    logger_->log_debug("Caught Exception %s", exception.what());
+    throw;
+  } catch (...) {
+    logger_->log_debug(
+        "Caught Exception during process group start processing");
+    throw;
+  }
+}
+
+void ProcessGroup::stopProcessing(TimerDrivenSchedulingAgent *timeScheduler,
+                                  EventDrivenSchedulingAgent *eventScheduler) {
+  std::lock_guard<std::mutex> lock(mutex_);
+
+  try {
+    // Stop all the processor node, input and output ports
+    for (std::set<std::shared_ptr<Processor> >::iterator it =
+        processors_.begin(); it != processors_.end(); ++it) {
+      std::shared_ptr<Processor> processor(*it);
+      if (processor->getSchedulingStrategy() == TIMER_DRIVEN)
+        timeScheduler->unschedule(processor);
+      else if (processor->getSchedulingStrategy() == EVENT_DRIVEN)
+        eventScheduler->unschedule(processor);
+    }
+
+    for (std::set<ProcessGroup *>::iterator it = child_process_groups_.begin();
+        it != child_process_groups_.end(); ++it) {
+      ProcessGroup *processGroup(*it);
+      processGroup->stopProcessing(timeScheduler, eventScheduler);
+    }
+  } catch (std::exception &exception) {
+    logger_->log_debug("Caught Exception %s", exception.what());
+    throw;
+  } catch (...) {
+    logger_->log_debug("Caught Exception during process group stop processing");
+    throw;
+  }
+}
+
+std::shared_ptr<Processor> ProcessGroup::findProcessor(uuid_t uuid) {
+
+  std::shared_ptr<Processor> ret = NULL;
+  // std::lock_guard<std::mutex> lock(mutex_);
+
+  for (auto processor : processors_) {
+    logger_->log_info("find processor %s", processor->getName().c_str());
+    uuid_t processorUUID;
+
+    if (processor->getUUID(processorUUID)) {
+
+      char uuid_str[37];  // ex. "1b4e28ba-2fa1-11d2-883f-0016d3cca427" + "\0"
+      uuid_unparse_lower(processorUUID, uuid_str);
+      std::string processorUUIDstr = uuid_str;
+      uuid_unparse_lower(uuid, uuid_str);
+      std::string uuidStr = uuid_str;
+      if (processorUUIDstr == uuidStr) {
+        return processor;
+      }
+    }
+
+  }
+  for (auto processGroup : child_process_groups_) {
+
+    logger_->log_info("find processor child %s",
+                      processGroup->getName().c_str());
+    std::shared_ptr<Processor> processor = processGroup->findProcessor(uuid);
+    if (processor)
+      return processor;
+  }
+
+  return ret;
+}
+
+std::shared_ptr<Processor> ProcessGroup::findProcessor(
+    const std::string &processorName) {
+  std::shared_ptr<Processor> ret = NULL;
+
+  for (auto processor : processors_) {
+    logger_->log_debug("Current processor is %s", processor->getName().c_str());
+    if (processor->getName() == processorName)
+      return processor;
+  }
+
+  for (auto processGroup : child_process_groups_) {
+    std::shared_ptr<Processor> processor = processGroup->findProcessor(
+        processorName);
+    if (processor)
+      return processor;
+  }
+
+  return ret;
+}
+
+void ProcessGroup::updatePropertyValue(std::string processorName,
+                                       std::string propertyName,
+                                       std::string propertyValue) {
+  std::lock_guard<std::mutex> lock(mutex_);
+
+  for (auto processor : processors_) {
+    if (processor->getName() == processorName) {
+      processor->setProperty(propertyName, propertyValue);
+    }
+  }
+
+  for (auto processGroup : child_process_groups_) {
+    processGroup->updatePropertyValue(processorName, propertyName,
+                                      propertyValue);
+  }
+
+  return;
+}
+
+void ProcessGroup::getConnections(
+    std::map<std::string, std::shared_ptr<Connection>> &connectionMap) {
+  for (auto connection : connections_) {
+    connectionMap[connection->getUUIDStr()] = connection;
+  }
+
+  for (auto processGroup : child_process_groups_) {
+    processGroup->getConnections(connectionMap);
+  }
+}
+
+void ProcessGroup::addConnection(std::shared_ptr<Connection> connection) {
+  std::lock_guard<std::mutex> lock(mutex_);
+
+  if (connections_.find(connection) == connections_.end()) {
+    // We do not have the same connection in this process group yet
+    connections_.insert(connection);
+    logger_->log_info("Add connection %s into process group %s",
+                      connection->getName().c_str(), name_.c_str());
+    uuid_t sourceUUID;
+    std::shared_ptr<Processor> source = NULL;
+    connection->getSourceUUID(sourceUUID);
+    source = this->findProcessor(sourceUUID);
+    if (source)
+      source->addConnection(connection);
+    std::shared_ptr<Processor> destination = NULL;
+    uuid_t destinationUUID;
+    connection->getDestinationUUID(destinationUUID);
+    destination = this->findProcessor(destinationUUID);
+    if (destination && destination != source)
+      destination->addConnection(connection);
+  }
+}
+
+void ProcessGroup::removeConnection(std::shared_ptr<Connection> connection) {
+  std::lock_guard<std::mutex> lock(mutex_);
+
+  if (connections_.find(connection) != connections_.end()) {
+    // We do not have the same connection in this process group yet
+    connections_.erase(connection);
+    logger_->log_info("Remove connection %s into process group %s",
+                      connection->getName().c_str(), name_.c_str());
+    uuid_t sourceUUID;
+    std::shared_ptr<Processor> source = NULL;
+    connection->getSourceUUID(sourceUUID);
+    source = this->findProcessor(sourceUUID);
+    if (source)
+      source->removeConnection(connection);
+    std::shared_ptr<Processor> destination = NULL;
+    uuid_t destinationUUID;
+    connection->getDestinationUUID(destinationUUID);
+    destination = this->findProcessor(destinationUUID);
+    if (destination && destination != source)
+      destination->removeConnection(connection);
+  }
+}
+
+} /* namespace processor */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/core/ProcessSession.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/ProcessSession.cpp b/libminifi/src/core/ProcessSession.cpp
new file mode 100644
index 0000000..e6fa7c4
--- /dev/null
+++ b/libminifi/src/core/ProcessSession.cpp
@@ -0,0 +1,941 @@
+/**
+ * @file ProcessSession.cpp
+ * ProcessSession class implementation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <vector>
+#include <queue>
+#include <map>
+#include <set>
+#include <sys/time.h>
+#include <time.h>
+#include <chrono>
+#include <thread>
+#include <iostream>
+
+#include "core/ProcessSession.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+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);
+
+  _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);
+
+  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);
+
+  if (record) {
+    _addedFlowFiles[record->getUUIDStr()] = record;
+    logger_->log_debug("Create FlowFile with UUID %s",
+                       record->getUUIDStr().c_str());
+  }
+
+  if (record) {
+    // Copy attributes
+    std::map<std::string, std::string> parentAttributes =
+        parent->getAttributes();
+    std::map<std::string, std::string>::iterator it;
+    for (it = parentAttributes.begin(); it != parentAttributes.end(); it++) {
+      if (it->first == FlowAttributeKey(ALTERNATE_IDENTIFIER)
+          || it->first == FlowAttributeKey(DISCARD_REASON)
+          || it->first == FlowAttributeKey(UUID))
+        // Do not copy special attributes from parent
+        continue;
+      record->setAttribute(it->first, it->second);
+    }
+    record->setLineageStartDate(parent->getlineageStartDate());
+    record->setLineageIdentifiers(parent->getlineageIdentifiers());
+    parent->getlineageIdentifiers().insert(parent->getUUIDStr());
+
+  }
+  return record;
+}
+
+std::shared_ptr<core::FlowFile> ProcessSession::clone(
+    std::shared_ptr<core::FlowFile> &parent) {
+  std::shared_ptr<core::FlowFile> record = this->create(parent);
+  if (record) {
+    // Copy Resource Claim
+    std::shared_ptr<ResourceClaim> parent_claim = parent->getResourceClaim();
+    record->setResourceClaim(parent_claim);
+    if (parent_claim != nullptr) {
+      record->setOffset(parent->getOffset());
+      record->setSize(parent->getSize());
+      record->getResourceClaim()->increaseFlowFileRecordOwnedCount();
+      ;
+    }
+    provenance_report_->clone(parent, record);
+  }
+  return record;
+}
+
+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);
+
+  if (record) {
+    this->_clonedFlowFiles[record->getUUIDStr()] = record;
+    logger_->log_debug("Clone FlowFile with UUID %s during transfer",
+                       record->getUUIDStr().c_str());
+    // Copy attributes
+    std::map<std::string, std::string> parentAttributes =
+        parent->getAttributes();
+    std::map<std::string, std::string>::iterator it;
+    for (it = parentAttributes.begin(); it != parentAttributes.end(); it++) {
+      if (it->first == FlowAttributeKey(ALTERNATE_IDENTIFIER)
+          || it->first == FlowAttributeKey(DISCARD_REASON)
+          || it->first == FlowAttributeKey(UUID))
+        // Do not copy special attributes from parent
+        continue;
+      record->setAttribute(it->first, it->second);
+    }
+    record->setLineageStartDate(parent->getlineageStartDate());
+
+    record->setLineageIdentifiers(parent->getlineageIdentifiers());
+    record->getlineageIdentifiers().insert(parent->getUUIDStr());
+
+    // Copy Resource Claim
+    std::shared_ptr<ResourceClaim> parent_claim = parent->getResourceClaim();
+    record->setResourceClaim(parent_claim);
+    if (parent_claim != nullptr) {
+      record->setOffset(parent->getOffset());
+      record->setSize(parent->getSize());
+      record->getResourceClaim()->increaseFlowFileRecordOwnedCount();
+      ;
+    }
+    provenance_report_->clone(parent, record);
+  }
+
+  return record;
+}
+
+std::shared_ptr<core::FlowFile> ProcessSession::clone(
+    std::shared_ptr<core::FlowFile> &parent, long offset, long size) {
+  std::shared_ptr<core::FlowFile> record = this->create(parent);
+  if (record) {
+
+    if (parent->getResourceClaim()) {
+      if ((offset + size) > (long) parent->getSize()) {
+        // Set offset and size
+        logger_->log_error("clone offset %d and size %d exceed parent size %d",
+                           offset, size, parent->getSize());
+        // Remove the Add FlowFile for the session
+        std::map<std::string, std::shared_ptr<core::FlowFile> >::iterator it =
+            this->_addedFlowFiles.find(record->getUUIDStr());
+        if (it != this->_addedFlowFiles.end())
+          this->_addedFlowFiles.erase(record->getUUIDStr());
+        return nullptr;
+      }
+      record->setOffset(parent->getOffset() + parent->getOffset());
+      record->setSize(size);
+      // Copy Resource Claim
+      std::shared_ptr<ResourceClaim> parent_claim = parent->getResourceClaim();
+      record->setResourceClaim(parent_claim);
+      if (parent_claim != nullptr) {
+
+        record->getResourceClaim()->increaseFlowFileRecordOwnedCount();
+      }
+    }
+    provenance_report_->clone(parent, record);
+  }
+  return record;
+}
+
+void ProcessSession::remove(std::shared_ptr<core::FlowFile> &flow) {
+  flow->setDeleted(true);
+  _deletedFlowFiles[flow->getUUIDStr()] = flow;
+  std::string reason = process_context_->getProcessorNode().getName()
+      + " drop flow record " + flow->getUUIDStr();
+  provenance_report_->drop(flow, reason);
+}
+
+void ProcessSession::remove(std::shared_ptr<core::FlowFile> &&flow) {
+  flow->setDeleted(true);
+  _deletedFlowFiles[flow->getUUIDStr()] = flow;
+  std::string reason = process_context_->getProcessorNode().getName()
+      + " drop flow record " + flow->getUUIDStr();
+  provenance_report_->drop(flow, reason);
+}
+
+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);
+}
+
+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);
+}
+
+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);
+}
+
+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);
+}
+
+void ProcessSession::penalize(std::shared_ptr<core::FlowFile> &flow) {
+  flow->setPenaltyExpiration(
+      getTimeMillis()
+          + process_context_->getProcessorNode().getPenalizationPeriodMsec());
+}
+
+void ProcessSession::penalize(std::shared_ptr<core::FlowFile> &&flow) {
+  flow->setPenaltyExpiration(
+      getTimeMillis()
+          + process_context_->getProcessorNode().getPenalizationPeriodMsec());
+}
+
+void ProcessSession::transfer(std::shared_ptr<core::FlowFile> &flow,
+                              Relationship relationship) {
+  _transferRelationship[flow->getUUIDStr()] = relationship;
+}
+
+void ProcessSession::transfer(std::shared_ptr<core::FlowFile> &&flow,
+                              Relationship relationship) {
+  _transferRelationship[flow->getUUIDStr()] = relationship;
+}
+
+void ProcessSession::write(std::shared_ptr<core::FlowFile> &flow,
+                           OutputStreamCallback *callback) {
+  std::shared_ptr<ResourceClaim> claim = std::make_shared<ResourceClaim>(
+  DEFAULT_CONTENT_DIRECTORY);
+
+  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");
+    }
+  } catch (std::exception &exception) {
+    if (flow && flow->getResourceClaim() == claim) {
+      flow->getResourceClaim()->decreaseFlowFileRecordOwnedCount();
+      flow->clearResourceClaim();
+    }
+    logger_->log_debug("Caught Exception %s", exception.what());
+    throw;
+  } catch (...) {
+    if (flow && flow->getResourceClaim() == claim) {
+      flow->getResourceClaim()->decreaseFlowFileRecordOwnedCount();
+      flow->clearResourceClaim();
+    }
+    logger_->log_debug("Caught Exception during process session write");
+    throw;
+  }
+}
+
+void ProcessSession::write(std::shared_ptr<core::FlowFile> &&flow,
+                           OutputStreamCallback *callback) {
+  std::shared_ptr<ResourceClaim> claim = std::make_shared<ResourceClaim>();
+  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");
+    }
+  } catch (std::exception &exception) {
+    if (flow && flow->getResourceClaim() == claim) {
+      flow->getResourceClaim()->decreaseFlowFileRecordOwnedCount();
+      flow->clearResourceClaim();
+    }
+    logger_->log_debug("Caught Exception %s", exception.what());
+    throw;
+  } catch (...) {
+    if (flow && flow->getResourceClaim() == claim) {
+      flow->getResourceClaim()->decreaseFlowFileRecordOwnedCount();
+      flow->clearResourceClaim();
+    }
+    logger_->log_debug("Caught Exception during process session write");
+    throw;
+  }
+}
+
+void ProcessSession::append(std::shared_ptr<core::FlowFile> &&flow,
+                            OutputStreamCallback *callback) {
+  std::shared_ptr<ResourceClaim> claim = nullptr;
+
+  if (flow->getResourceClaim() == nullptr) {
+    // No existed claim for append, we need to create new claim
+    return write(flow, callback);
+  }
+
+  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");
+    }
+  } catch (std::exception &exception) {
+    logger_->log_debug("Caught Exception %s", exception.what());
+    throw;
+  } catch (...) {
+    logger_->log_debug("Caught Exception during process session append");
+    throw;
+  }
+}
+
+void ProcessSession::append(std::shared_ptr<core::FlowFile> &flow,
+                            OutputStreamCallback *callback) {
+  std::shared_ptr<ResourceClaim> claim = nullptr;
+
+  if (flow->getResourceClaim() == nullptr) {
+    // No existed claim for append, we need to create new claim
+    return write(flow, callback);
+  }
+
+  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");
+    }
+  } catch (std::exception &exception) {
+    logger_->log_debug("Caught Exception %s", exception.what());
+    throw;
+  } catch (...) {
+    logger_->log_debug("Caught Exception during process session append");
+    throw;
+  }
+}
+
+void ProcessSession::read(std::shared_ptr<core::FlowFile> &flow,
+                          InputStreamCallback *callback) {
+  try {
+    std::shared_ptr<ResourceClaim> claim = nullptr;
+
+    if (flow->getResourceClaim() == nullptr) {
+      // No existed claim for read, we throw exception
+      throw Exception(FILE_OPERATION_EXCEPTION,
+                      "No Content Claim existed for read");
+    }
+
+    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");
+    }
+  } catch (std::exception &exception) {
+    logger_->log_debug("Caught Exception %s", exception.what());
+    throw;
+  } catch (...) {
+    logger_->log_debug("Caught Exception during process session read");
+    throw;
+  }
+}
+
+void ProcessSession::read(std::shared_ptr<core::FlowFile> &&flow,
+                          InputStreamCallback *callback) {
+  try {
+    std::shared_ptr<ResourceClaim> claim = nullptr;
+
+    if (flow->getResourceClaim() == nullptr) {
+      // No existed claim for read, we throw exception
+      throw Exception(FILE_OPERATION_EXCEPTION,
+                      "No Content Claim existed for read");
+    }
+
+    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");
+    }
+  } catch (std::exception &exception) {
+    logger_->log_debug("Caught Exception %s", exception.what());
+    throw;
+  } catch (...) {
+    logger_->log_debug("Caught Exception during process session read");
+    throw;
+  }
+}
+
+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>();
+  char *buf = NULL;
+  int size = 4096;
+  buf = new char[size];
+
+  try {
+    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()) {
+      // Open the source file and stream to the flow file
+      input.seekg(offset, fs.beg);
+      while (input.good()) {
+        input.read(buf, size);
+        if (input)
+          fs.write(buf, size);
+        else
+          fs.write(buf, input.gcount());
+      }
+
+      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());
+
+        fs.close();
+        input.close();
+        if (!keepSource)
+          std::remove(source.c_str());
+        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();
+        input.close();
+        throw Exception(FILE_OPERATION_EXCEPTION, "File Import Error");
+      }
+    } else {
+      throw Exception(FILE_OPERATION_EXCEPTION, "File Import Error");
+    }
+
+    delete[] buf;
+  } catch (std::exception &exception) {
+    if (flow && flow->getResourceClaim() == claim) {
+      flow->getResourceClaim()->decreaseFlowFileRecordOwnedCount();
+      flow->clearResourceClaim();
+    }
+    logger_->log_debug("Caught Exception %s", exception.what());
+    delete[] buf;
+    throw;
+  } catch (...) {
+    if (flow && flow->getResourceClaim() == claim) {
+      flow->getResourceClaim()->decreaseFlowFileRecordOwnedCount();
+      flow->clearResourceClaim();
+    }
+    logger_->log_debug("Caught Exception during process session write");
+    delete[] buf;
+    throw;
+  }
+}
+
+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>();
+
+  char *buf = NULL;
+  int size = 4096;
+  buf = new char[size];
+
+  try {
+    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()) {
+      // Open the source file and stream to the flow file
+      input.seekg(offset, fs.beg);
+      while (input.good()) {
+        input.read(buf, size);
+        if (input)
+          fs.write(buf, size);
+        else
+          fs.write(buf, input.gcount());
+      }
+
+      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());
+
+        fs.close();
+        input.close();
+        if (!keepSource)
+          std::remove(source.c_str());
+        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();
+        input.close();
+        throw Exception(FILE_OPERATION_EXCEPTION, "File Import Error");
+      }
+    } else {
+      throw Exception(FILE_OPERATION_EXCEPTION, "File Import Error");
+    }
+
+    delete[] buf;
+  } catch (std::exception &exception) {
+    if (flow && flow->getResourceClaim() == claim) {
+      flow->getResourceClaim()->decreaseFlowFileRecordOwnedCount();
+      flow->clearResourceClaim();
+    }
+    logger_->log_debug("Caught Exception %s", exception.what());
+    delete[] buf;
+    throw;
+  } catch (...) {
+    if (flow && flow->getResourceClaim() == claim) {
+      flow->getResourceClaim()->decreaseFlowFileRecordOwnedCount();
+      flow->clearResourceClaim();
+    }
+    logger_->log_debug("Caught Exception during process session write");
+    delete[] buf;
+    throw;
+  }
+}
+
+void ProcessSession::commit() {
+
+  try {
+    // First we clone the flow record based on the transfered relationship for updated flow record
+    for (auto && it : _updatedFlowFiles) {
+      std::shared_ptr<core::FlowFile> record = it.second;
+      if (record->isDeleted())
+        continue;
+      std::map<std::string, Relationship>::iterator itRelationship = this
+          ->_transferRelationship.find(record->getUUIDStr());
+      if (itRelationship != _transferRelationship.end()) {
+        Relationship relationship = itRelationship->second;
+        // Find the relationship, we need to find the connections for that relationship
+        std::set<std::shared_ptr<Connectable>> connections = process_context_
+            ->getProcessorNode().getOutGoingConnections(relationship.getName());
+        if (connections.empty()) {
+          // No connection
+          if (!process_context_->getProcessorNode().isAutoTerminated(
+              relationship)) {
+            // Not autoterminate, we should have the connect
+            std::string message =
+                "Connect empty for non auto terminated relationship"
+                    + relationship.getName();
+            throw Exception(PROCESS_SESSION_EXCEPTION, message.c_str());
+          } else {
+            // Autoterminated
+            remove(record);
+          }
+        } else {
+          // We connections, clone the flow and assign the connection accordingly
+          for (std::set<std::shared_ptr<Connectable>>::iterator itConnection =
+              connections.begin(); itConnection != connections.end();
+              ++itConnection) {
+            std::shared_ptr<Connectable> connection = *itConnection;
+            if (itConnection == connections.begin()) {
+              // First connection which the flow need be routed to
+              record->setConnection(connection);
+            } else {
+              // Clone the flow file and route to the connection
+              std::shared_ptr<core::FlowFile> cloneRecord;
+              cloneRecord = this->cloneDuringTransfer(record);
+              if (cloneRecord)
+                cloneRecord->setConnection(connection);
+              else
+                throw Exception(PROCESS_SESSION_EXCEPTION,
+                                "Can not clone the flow for transfer");
+            }
+          }
+        }
+      } else {
+        // Can not find relationship for the flow
+        throw Exception(PROCESS_SESSION_EXCEPTION,
+                        "Can not find the transfer relationship for the flow");
+      }
+    }
+
+    // Do the samething for added flow file
+    for (const auto it : _addedFlowFiles) {
+      std::shared_ptr<core::FlowFile> record = it.second;
+      if (record->isDeleted())
+        continue;
+      std::map<std::string, Relationship>::iterator itRelationship = this
+          ->_transferRelationship.find(record->getUUIDStr());
+      if (itRelationship != _transferRelationship.end()) {
+        Relationship relationship = itRelationship->second;
+        // Find the relationship, we need to find the connections for that relationship
+        std::set<std::shared_ptr<Connectable>> connections = process_context_
+            ->getProcessorNode().getOutGoingConnections(relationship.getName());
+        if (connections.empty()) {
+          // No connection
+          if (!process_context_->getProcessorNode().isAutoTerminated(
+              relationship)) {
+            // Not autoterminate, we should have the connect
+            std::string message =
+                "Connect empty for non auto terminated relationship "
+                    + relationship.getName();
+            throw Exception(PROCESS_SESSION_EXCEPTION, message.c_str());
+          } else {
+            // Autoterminated
+            remove(record);
+          }
+        } else {
+          // We connections, clone the flow and assign the connection accordingly
+          for (std::set<std::shared_ptr<Connectable>>::iterator itConnection =
+              connections.begin(); itConnection != connections.end();
+              ++itConnection) {
+            std::shared_ptr<Connectable> connection(*itConnection);
+            if (itConnection == connections.begin()) {
+              // First connection which the flow need be routed to
+              record->setConnection(connection);
+            } else {
+              // Clone the flow file and route to the connection
+              std::shared_ptr<core::FlowFile> cloneRecord;
+              cloneRecord = this->cloneDuringTransfer(record);
+              if (cloneRecord)
+                cloneRecord->setConnection(connection);
+              else
+                throw Exception(PROCESS_SESSION_EXCEPTION,
+                                "Can not clone the flow for transfer");
+            }
+          }
+        }
+      } else {
+        // Can not find relationship for the flow
+        throw Exception(PROCESS_SESSION_EXCEPTION,
+                        "Can not find the transfer relationship for the flow");
+      }
+    }
+
+    std::shared_ptr<Connection> connection = nullptr;
+    // Complete process the added and update flow files for the session, send the flow file to its queue
+    for (const auto &it : _updatedFlowFiles) {
+      std::shared_ptr<core::FlowFile> record = it.second;
+      if (record->isDeleted()) {
+        continue;
+      }
+
+      connection = std::static_pointer_cast<Connection>(
+          record->getConnection());
+      if ((connection) != nullptr)
+        connection->put(record);
+    }
+    for (const auto &it : _addedFlowFiles) {
+      std::shared_ptr<core::FlowFile> record = it.second;
+      if (record->isDeleted()) {
+        continue;
+      }
+      connection = std::static_pointer_cast<Connection>(
+          record->getConnection());
+      if ((connection) != nullptr)
+        connection->put(record);
+    }
+    // Process the clone flow files
+    for (const auto &it : _clonedFlowFiles) {
+      std::shared_ptr<core::FlowFile> record = it.second;
+      if (record->isDeleted()) {
+        continue;
+      }
+      connection = std::static_pointer_cast<Connection>(
+          record->getConnection());
+      if ((connection) != nullptr)
+        connection->put(record);
+    }
+
+    // All done
+    _updatedFlowFiles.clear();
+    _addedFlowFiles.clear();
+    _clonedFlowFiles.clear();
+    _deletedFlowFiles.clear();
+    _originalFlowFiles.clear();
+    // persistent the provenance report
+    this->provenance_report_->commit();
+    logger_->log_trace("ProcessSession committed for %s",
+                       process_context_->getProcessorNode().getName().c_str());
+  } catch (std::exception &exception) {
+    logger_->log_debug("Caught Exception %s", exception.what());
+    throw;
+  } catch (...) {
+    logger_->log_debug("Caught Exception during process session commit");
+    throw;
+  }
+}
+
+void ProcessSession::rollback() {
+  try {
+    std::shared_ptr<Connection> connection = nullptr;
+    // Requeue the snapshot of the flowfile back
+    for (const auto &it : _originalFlowFiles) {
+      std::shared_ptr<core::FlowFile> record = it.second;
+      connection = std::static_pointer_cast<Connection>(
+          record->getOriginalConnection());
+      if ((connection) != nullptr) {
+        std::shared_ptr<FlowFileRecord> flowf = std::static_pointer_cast<
+            FlowFileRecord>(record);
+        flowf->setSnapShot(false);
+        connection->put(record);
+      }
+
+    }
+    _originalFlowFiles.clear();
+
+    _clonedFlowFiles.clear();
+    _addedFlowFiles.clear();
+    _updatedFlowFiles.clear();
+    _deletedFlowFiles.clear();
+    logger_->log_trace("ProcessSession rollback for %s",
+                       process_context_->getProcessorNode().getName().c_str());
+  } catch (std::exception &exception) {
+    logger_->log_debug("Caught Exception %s", exception.what());
+    throw;
+  } catch (...) {
+    logger_->log_debug("Caught Exception during process session roll back");
+    throw;
+  }
+}
+
+std::shared_ptr<core::FlowFile> ProcessSession::get() {
+  std::shared_ptr<Connectable> first = process_context_->getProcessorNode()
+      .getNextIncomingConnection();
+
+  if (first == NULL)
+    return NULL;
+
+  std::shared_ptr<Connection> current = std::static_pointer_cast<Connection>(
+      first);
+
+  do {
+    std::set<std::shared_ptr<core::FlowFile> > expired;
+    std::shared_ptr<core::FlowFile> ret = current->poll(expired);
+    if (expired.size() > 0) {
+      // 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);
+      }
+    }
+    if (ret) {
+      // add the flow record to the current process session update map
+      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);
+      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;
+    }
+    current = std::static_pointer_cast<Connection>(
+        process_context_->getProcessorNode().getNextIncomingConnection());
+  } while (current != NULL && current != first);
+
+  return NULL;
+}
+
+} /* namespace processor */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/core/ProcessSessionFactory.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/ProcessSessionFactory.cpp b/libminifi/src/core/ProcessSessionFactory.cpp
new file mode 100644
index 0000000..445ca58
--- /dev/null
+++ b/libminifi/src/core/ProcessSessionFactory.cpp
@@ -0,0 +1,42 @@
+/**
+ * @file ProcessSessionFactory.cpp
+ * ProcessSessionFactory class implementation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "core/ProcessSessionFactory.h"
+
+#include <memory>
+
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+std::unique_ptr<ProcessSession> ProcessSessionFactory::createSession()
+{
+	return std::unique_ptr<ProcessSession>(new ProcessSession(process_context_));
+}
+
+
+} /* namespace processor */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/core/Processor.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/Processor.cpp b/libminifi/src/core/Processor.cpp
new file mode 100644
index 0000000..ba52c28
--- /dev/null
+++ b/libminifi/src/core/Processor.cpp
@@ -0,0 +1,272 @@
+/**
+ * @file Processor.cpp
+ * Processor class implementation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <vector>
+#include <queue>
+#include <map>
+#include <set>
+#include <sys/time.h>
+#include <time.h>
+#include <chrono>
+#include <thread>
+#include <memory>
+#include <functional>
+
+#include "core/Processor.h"
+
+#include "Connection.h"
+#include "core/Connectable.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/ProcessSessionFactory.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+Processor::Processor(std::string name, uuid_t uuid)
+    : Connectable(name, uuid),
+      ConfigurableComponent(logging::Logger::getLogger()) {
+
+  has_work_.store(false);
+  // Setup the default values
+  state_ = DISABLED;
+  strategy_ = TIMER_DRIVEN;
+  loss_tolerant_ = false;
+  _triggerWhenEmpty = false;
+  scheduling_period_nano_ = MINIMUM_SCHEDULING_NANOS;
+  run_durantion_nano_ = 0;
+  yield_period_msec_ = DEFAULT_YIELD_PERIOD_SECONDS * 1000;
+  _penalizationPeriodMsec = DEFAULT_PENALIZATION_PERIOD_SECONDS * 1000;
+  max_concurrent_tasks_ = 1;
+  active_tasks_ = 0;
+  yield_expiration_ = 0;
+  incoming_connections_Iter = this->_incomingConnections.begin();
+  logger_ = logging::Logger::getLogger();
+  logger_->log_info("Processor %s created UUID %s", name_.c_str(),
+                    uuidStr_.c_str());
+}
+
+bool Processor::isRunning() {
+  return (state_ == RUNNING && active_tasks_ > 0);
+}
+
+void Processor::setScheduledState(ScheduledState state) {
+  state_ = state;
+}
+
+bool Processor::addConnection(std::shared_ptr<Connectable> conn) {
+  
+  bool ret = false;
+
+  if (isRunning()) {
+    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_);
+
+  uuid_t srcUUID;
+  uuid_t destUUID;
+
+  connection->getSourceUUID(srcUUID);
+  connection->getDestinationUUID(destUUID);
+  char uuid_str[37];
+
+  uuid_unparse_lower(uuid_, uuid_str);
+  std::string my_uuid = uuid_str;
+  uuid_unparse_lower(destUUID, uuid_str);
+  std::string destination_uuid = uuid_str;
+  if (my_uuid == destination_uuid) {
+    // Connection is destination to the current processor
+    if (_incomingConnections.find(connection) == _incomingConnections.end()) {
+      _incomingConnections.insert(connection);
+      connection->setDestination(shared_from_this());
+      logger_->log_info(
+          "Add connection %s into Processor %s incoming connection",
+          connection->getName().c_str(), name_.c_str());
+      incoming_connections_Iter = this->_incomingConnections.begin();
+      ret = true;
+    }
+  }
+  uuid_unparse_lower(srcUUID, uuid_str);
+  std::string source_uuid = uuid_str;
+  if (my_uuid == source_uuid) {
+    std::string relationship = connection->getRelationship().getName();
+    // Connection is source from the current processor
+    auto &&it = _outGoingConnections.find(relationship);
+    if (it != _outGoingConnections.end()) {
+      // We already has connection for this relationship
+      std::set<std::shared_ptr<Connectable>> existedConnection = it->second;
+      if (existedConnection.find(connection) == existedConnection.end()) {
+        // We do not have the same connection for this relationship yet
+        existedConnection.insert(connection);
+        connection->setSource(shared_from_this());
+        _outGoingConnections[relationship] = existedConnection;
+        logger_->log_info(
+            "Add connection %s into Processor %s outgoing connection for relationship %s",
+            connection->getName().c_str(), name_.c_str(), relationship.c_str());
+        ret = true;
+      }
+    } else {
+
+      // We do not have any outgoing connection for this relationship yet
+      std::set<std::shared_ptr<Connectable>> newConnection;
+      newConnection.insert(connection);
+      connection->setSource(shared_from_this());
+      _outGoingConnections[relationship] = newConnection;
+      logger_->log_info(
+          "Add connection %s into Processor %s outgoing connection for relationship %s",
+          connection->getName().c_str(), name_.c_str(), relationship.c_str());
+      ret = true;
+    }
+  }
+
+  return ret;
+}
+
+void Processor::removeConnection(std::shared_ptr<Connectable> conn) {
+  if (isRunning()) {
+    logger_->log_info(
+        "Can not remove connection while the process %s is running",
+        name_.c_str());
+    return;
+  }
+
+  std::lock_guard<std::mutex> lock(mutex_);
+
+  uuid_t srcUUID;
+  uuid_t destUUID;
+  
+  std::shared_ptr<Connection> connection = std::static_pointer_cast<Connection>(conn);
+
+  connection->getSourceUUID(srcUUID);
+  connection->getDestinationUUID(destUUID);
+
+  if (uuid_compare(uuid_, destUUID) == 0) {
+    // Connection is destination to the current processor
+    if (_incomingConnections.find(connection) != _incomingConnections.end()) {
+      _incomingConnections.erase(connection);
+      connection->setDestination(NULL);
+      logger_->log_info(
+          "Remove connection %s into Processor %s incoming connection",
+          connection->getName().c_str(), name_.c_str());
+      incoming_connections_Iter = this->_incomingConnections.begin();
+    }
+  }
+
+  if (uuid_compare(uuid_, srcUUID) == 0) {
+    std::string relationship = connection->getRelationship().getName();
+    // Connection is source from the current processor
+    auto &&it = _outGoingConnections.find(relationship);
+    if (it == _outGoingConnections.end()) {
+      return;
+    } else {
+      if (_outGoingConnections[relationship].find(connection)
+          != _outGoingConnections[relationship].end()) {
+        _outGoingConnections[relationship].erase(connection);
+        connection->setSource(NULL);
+        logger_->log_info(
+            "Remove connection %s into Processor %s outgoing connection for relationship %s",
+            connection->getName().c_str(), name_.c_str(), relationship.c_str());
+      }
+    }
+  }
+}
+
+
+
+bool Processor::flowFilesQueued() {
+  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);
+    if (connection->getQueueSize() > 0)
+      return true;
+  }
+
+  return false;
+}
+
+bool Processor::flowFilesOutGoingFull() {
+  std::lock_guard<std::mutex> lock(mutex_);
+
+  for (auto &&connection : _outGoingConnections) {
+    // 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);
+      if (connection->isFull())
+        return true;
+    }
+  }
+
+  return false;
+}
+
+void Processor::onTrigger(ProcessContext *context,
+                          ProcessSessionFactory *sessionFactory) {
+  auto session = sessionFactory->createSession();
+
+  try {
+    // Call the virtual trigger function
+    onTrigger(context, session.get());
+    session->commit();
+  } catch (std::exception &exception) {
+    logger_->log_debug("Caught Exception %s", exception.what());
+    session->rollback();
+    throw;
+  } catch (...) {
+    logger_->log_debug("Caught Exception Processor::onTrigger");
+    session->rollback();
+    throw;
+  }
+}
+
+bool Processor::isWorkAvailable() {
+  // We have work if any incoming connection has work
+  bool hasWork = false;
+
+  try {
+    for (const auto &conn : _incomingConnections) {
+      std::shared_ptr<Connection> connection = std::static_pointer_cast<Connection>(conn);
+      if (connection->getQueueSize() > 0) {
+        hasWork = true;
+        break;
+      }
+    }
+  } catch (...) {
+    logger_->log_error(
+        "Caught an exception while checking if work is available; unless it was positively determined that work is available, assuming NO work is available!");
+  }
+
+  return hasWork;
+}
+
+} /* namespace processor */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/core/ProcessorNode.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/ProcessorNode.cpp b/libminifi/src/core/ProcessorNode.cpp
new file mode 100644
index 0000000..44491d3
--- /dev/null
+++ b/libminifi/src/core/ProcessorNode.cpp
@@ -0,0 +1,65 @@
+/**
+ * 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/ProcessorNode.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+ProcessorNode::ProcessorNode(const std::shared_ptr<Connectable> processor)
+    : processor_(processor),
+      Connectable(processor->getName(),0),
+      ConfigurableComponent(logging::Logger::getLogger()) {
+	
+	uuid_t copy;
+	processor->getUUID(copy);
+	setUUID( copy );
+
+
+}
+
+ProcessorNode::ProcessorNode(const ProcessorNode &other)
+    : processor_(other.processor_),
+      Connectable(other.getName(), 0),
+      ConfigurableComponent(logging::Logger::getLogger()) {
+	
+	uuid_t copy;
+	processor_->getUUID(copy);
+	setUUID( copy );
+
+}
+
+ProcessorNode::~ProcessorNode() {
+
+}
+
+bool ProcessorNode::isWorkAvailable() {
+  return processor_->isWorkAvailable();
+}
+
+bool ProcessorNode::isRunning() {
+  return processor_->isRunning();
+}
+
+} /* namespace processor */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/core/Property.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/Property.cpp b/libminifi/src/core/Property.cpp
new file mode 100644
index 0000000..287b7ec
--- /dev/null
+++ b/libminifi/src/core/Property.cpp
@@ -0,0 +1,57 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     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/Property.h"
+
+namespace org {
+namespace apache {
+namespace nifi {namespace minifi {
+namespace core {
+
+// Get Name for the property
+std::string Property::getName() const {
+  return name_;
+}
+// Get Description for the property
+std::string Property::getDescription() {
+  return description_;
+}
+// Get value for the property
+std::string Property::getValue() const {
+  return value_;
+}
+// Set value for the property
+void Property::setValue(std::string value) {
+  value_ = value;
+}
+// Compare
+bool Property::operator <(const Property & right) const {
+  return name_ < right.name_;
+}
+
+const Property &Property::operator=(const Property &other) {
+  name_ = other.name_;
+  value_ = other.value_;
+  return *this;
+}
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/core/Record.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/Record.cpp b/libminifi/src/core/Record.cpp
new file mode 100644
index 0000000..dbf0102
--- /dev/null
+++ b/libminifi/src/core/Record.cpp
@@ -0,0 +1,223 @@
+/*
+ * Copyright 2017 <copyright holder> <email>
+ * 
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * 
+ *     http://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/FlowFile.h"
+#include "core/logging/Logger.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+FlowFile::FlowFile()
+    : size_(0),
+      id_(0),
+      stored(false),
+      offset_(0),
+      last_queue_date_(0),
+      penaltyExpiration_ms_(0),
+      claim_(nullptr),
+      marked_delete_(false),
+      connection_(nullptr),
+      original_connection_() {
+  entry_date_ = getTimeMillis();
+  lineage_start_date_ = entry_date_;
+
+  char uuidStr[37];
+
+  // Generate the global UUID for the flow record
+  uuid_generate(uuid_);
+
+  uuid_unparse_lower(uuid_, uuidStr);
+  uuid_str_ = uuidStr;
+  
+  logger_ = logging::Logger::getLogger();
+
+}
+
+FlowFile::~FlowFile() {
+
+}
+
+FlowFile& FlowFile::operator=(const FlowFile& other) {
+
+  uuid_copy(uuid_, other.uuid_);
+  stored = other.stored;
+  marked_delete_ = other.marked_delete_;
+  entry_date_ = other.entry_date_;
+  lineage_start_date_ = other.lineage_start_date_;
+  lineage_Identifiers_ = other.lineage_Identifiers_;
+  last_queue_date_ = other.last_queue_date_;
+  size_ = other.size_;
+  penaltyExpiration_ms_ = other.penaltyExpiration_ms_;
+  attributes_ = other.attributes_;
+  claim_ = other.claim_;
+  if (claim_ != nullptr)
+    this->claim_->increaseFlowFileRecordOwnedCount();
+  uuid_str_ = other.uuid_str_;
+  connection_ = other.connection_;
+  original_connection_ = other.original_connection_;
+
+  return *this;
+}
+
+/**
+ * Returns whether or not this flow file record
+ * is marked as deleted.
+ * @return marked deleted
+ */
+bool FlowFile::isDeleted() {
+  return marked_delete_;
+}
+
+/**
+ * Sets whether to mark this flow file record
+ * as deleted
+ * @param deleted deleted flag
+ */
+void FlowFile::setDeleted(const bool deleted) {
+  marked_delete_ = deleted;
+}
+
+std::shared_ptr<ResourceClaim> FlowFile::getResourceClaim() {
+  return claim_;
+}
+
+void FlowFile::clearResourceClaim() {
+  claim_ = nullptr;
+}
+void FlowFile::setResourceClaim(std::shared_ptr<ResourceClaim> &claim) {
+  claim_ = claim;
+}
+
+// ! Get Entry Date
+uint64_t FlowFile::getEntryDate() {
+  return entry_date_;
+}
+uint64_t FlowFile::getEventTime() {
+  return event_time_;
+}
+// ! Get Lineage Start Date
+uint64_t FlowFile::getlineageStartDate() {
+  return lineage_start_date_;
+}
+
+std::set<std::string> &FlowFile::getlineageIdentifiers() {
+  return lineage_Identifiers_;
+}
+
+bool FlowFile::getAttribute(std::string key, std::string &value) {
+  auto it = attributes_.find(key);
+  if (it != attributes_.end()) {
+    value = it->second;
+    return true;
+  } else {
+    return false;
+  }
+}
+
+// Get Size
+uint64_t FlowFile::getSize() {
+  return size_;
+}
+// ! Get Offset
+uint64_t FlowFile::getOffset() {
+  return offset_;
+}
+
+bool FlowFile::removeAttribute(const std::string key) {
+  auto it = attributes_.find(key);
+  if (it != attributes_.end()) {
+    attributes_.erase(key);
+    return true;
+  } else {
+    return false;
+  }
+}
+
+bool FlowFile::updateAttribute(const std::string key, const std::string value) {
+  auto it = attributes_.find(key);
+  if (it != attributes_.end()) {
+    attributes_[key] = value;
+    return true;
+  } else {
+    return false;
+  }
+}
+
+bool FlowFile::addAttribute(const std::string &key, const std::string &value) {
+  auto it = attributes_.find(key);
+  if (it != attributes_.end()) {
+    // attribute already there in the map
+    return false;
+  } else {
+    attributes_[key] = value;
+    return true;
+  }
+}
+
+void FlowFile::setLineageStartDate(const uint64_t date) {
+  lineage_start_date_ = date;
+}
+
+/**
+ * Sets the original connection with a shared pointer.
+ * @param connection shared connection.
+ */
+void FlowFile::setOriginalConnection(
+    std::shared_ptr<core::Connectable> &connection) {
+  original_connection_ = connection;
+}
+
+/**
+ * Sets the connection with a shared pointer.
+ * @param connection shared connection.
+ */
+void FlowFile::setConnection(std::shared_ptr<core::Connectable> &connection) {
+  connection_ = connection;
+}
+
+/**
+ * Sets the connection with a shared pointer.
+ * @param connection shared connection.
+ */
+void FlowFile::setConnection(std::shared_ptr<core::Connectable> &&connection) {
+  connection_ = connection;
+}
+
+/**
+ * Returns the connection referenced by this record.
+ * @return shared connection pointer.
+ */
+std::shared_ptr<core::Connectable> FlowFile::getConnection() {
+  return connection_;
+}
+
+/**
+ * Returns the original connection referenced by this record.
+ * @return shared original connection pointer.
+ */
+std::shared_ptr<core::Connectable> FlowFile::getOriginalConnection() {
+  return original_connection_;
+}
+
+}
+}
+}
+}
+}


[15/16] nifi-minifi-cpp git commit: MINIFI-217: Updates namespaces and removes use of raw pointers for user facing API.

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/FlowFileRepository.h
----------------------------------------------------------------------
diff --git a/libminifi/include/FlowFileRepository.h b/libminifi/include/FlowFileRepository.h
deleted file mode 100644
index 50d2c41..0000000
--- a/libminifi/include/FlowFileRepository.h
+++ /dev/null
@@ -1,204 +0,0 @@
-/**
- * @file FlowFileRepository 
- * Flow file repository class declaration
- *
- * 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 __FLOWFILE_REPOSITORY_H__
-#define __FLOWFILE_REPOSITORY_H__
-
-#include <ftw.h>
-#include <uuid/uuid.h>
-#include <atomic>
-#include <cstdint>
-#include <cstring>
-#include <iostream>
-#include <map>
-#include <set>
-#include <string>
-#include <thread>
-#include <vector>
-
-#include "Configure.h"
-#include "Connection.h"
-#include "FlowFileRecord.h"
-#include "Logger.h"
-#include "Property.h"
-#include "ResourceClaim.h"
-#include "io/Serializable.h"
-#include "utils/TimeUtil.h"
-#include "Repository.h"
-
-#define FLOWFILE_REPOSITORY_DIRECTORY "./flowfile_repository"
-#define MAX_FLOWFILE_REPOSITORY_STORAGE_SIZE (10*1024*1024) // 10M
-#define MAX_FLOWFILE_REPOSITORY_ENTRY_LIFE_TIME (600000) // 10 minute
-#define FLOWFILE_REPOSITORY_PURGE_PERIOD (2500) // 2500 msec
-
-//! FlowFile Repository
-class FlowFileRepository : public Repository
-{
-public:
-	//! Constructor
-	/*!
-	 * Create a new provenance repository
-	 */
-	FlowFileRepository()
-	 : Repository(Repository::FLOWFILE, FLOWFILE_REPOSITORY_DIRECTORY,
-			MAX_FLOWFILE_REPOSITORY_ENTRY_LIFE_TIME, MAX_FLOWFILE_REPOSITORY_STORAGE_SIZE, FLOWFILE_REPOSITORY_PURGE_PERIOD)
-	{
-	}
-	//! Destructor
-	virtual ~FlowFileRepository() {
-	}
-	//! Load Repo to Connections
-	void loadFlowFileToConnections(std::map<std::string, Connection *> *connectionMap);
-
-protected:
-
-private:
-
-	// Prevent default copy constructor and assignment operation
-	// Only support pass by reference or pointer
-	FlowFileRepository(const FlowFileRepository &parent);
-	FlowFileRepository &operator=(const FlowFileRepository &parent);
-};
-
-//! FlowFile Event Record
-class FlowFileEventRecord : protected Serializable
-{
-public:
-	//! Constructor
-	/*!
-	 * Create a new provenance event record
-	 */
-	FlowFileEventRecord()
-	: _entryDate(0), _lineageStartDate(0), _size(0), _offset(0)  
-	{
-		_eventTime = getTimeMillis();
-		logger_ = Logger::getLogger();
-	}
-
-	//! Destructor
-	virtual ~FlowFileEventRecord() {
-	}
-	//! Get Attributes
-	std::map<std::string, std::string> getAttributes() {
-		return _attributes;
-	}
-	//! Get Size
-	uint64_t getFileSize() {
-		return _size;
-	}
-	// ! Get Offset
-	uint64_t getFileOffset() {
-		return _offset;
-	}
-	// ! Get Entry Date
-	uint64_t getFlowFileEntryDate() {
-		return _entryDate;
-	}
-	// ! Get Lineage Start Date
-	uint64_t getlineageStartDate() {
-		return _lineageStartDate;
-	}
-	// ! Get Event Time
-	uint64_t getEventTime() {
-		return _eventTime;
-	}
-	//! Get FlowFileUuid
-	std::string getFlowFileUuid()
-	{
-		return _uuid;
-	}
-	//! Get ConnectionUuid
-	std::string getConnectionUuid()
-	{
-		return _uuidConnection;
-	}
-	//! Get content full path
-	std::string getContentFullPath()
-	{
-		return _contentFullPath;
-	}
-	//! Get LineageIdentifiers
-	std::set<std::string> getLineageIdentifiers()
-	{
-		return _lineageIdentifiers;
-	}
-	//! fromFlowFile
-	void fromFlowFile(FlowFileRecord *flow, std::string uuidConnection)
-	{
-		_entryDate = flow->getEntryDate();
-		_lineageStartDate = flow->getlineageStartDate();
-		_lineageIdentifiers = flow->getlineageIdentifiers();
-		_uuid = flow->getUUIDStr();
-		_attributes = flow->getAttributes();
-		_size = flow->getSize();
-		_offset = flow->getOffset();
-		_uuidConnection = uuidConnection;
-		if (flow->getResourceClaim())
-		{
-			_contentFullPath = flow->getResourceClaim()->getContentFullPath();
-		}
-	}
-	//! Serialize and Persistent to the repository
-	bool Serialize(FlowFileRepository *repo);
-	//! DeSerialize
-	bool DeSerialize(const uint8_t *buffer, const int bufferSize);
-	//! DeSerialize
-	bool DeSerialize(DataStream &stream)
-	{
-		return DeSerialize(stream.getBuffer(),stream.getSize());
-	}
-	//! DeSerialize
-	bool DeSerialize(FlowFileRepository *repo, std::string key);
-
-protected:
-
-	//! Date at which the event was created
-	uint64_t _eventTime;
-	//! Date at which the flow file entered the flow
-	uint64_t _entryDate;
-	//! Date at which the origin of this flow file entered the flow
-	uint64_t _lineageStartDate;
-	//! Size in bytes of the data corresponding to this flow file
-	uint64_t _size;
-	//! flow uuid
-	std::string _uuid;
-	//! connection uuid
-	std::string _uuidConnection;
-	//! 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;
-	//! UUID string for all parents
-	std::set<std::string> _lineageIdentifiers;
-
-private:
-
-	//! Logger
-	std::shared_ptr<Logger> logger_;
-	
-	// Prevent default copy constructor and assignment operation
-	// Only support pass by reference or pointer
-	FlowFileEventRecord(const FlowFileEventRecord &parent);
-	FlowFileEventRecord &operator=(const FlowFileEventRecord &parent);
-
-};
-
-#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/GenerateFlowFile.h
----------------------------------------------------------------------
diff --git a/libminifi/include/GenerateFlowFile.h b/libminifi/include/GenerateFlowFile.h
deleted file mode 100644
index 27aa43b..0000000
--- a/libminifi/include/GenerateFlowFile.h
+++ /dev/null
@@ -1,87 +0,0 @@
-/**
- * @file GenerateFlowFile.h
- * GenerateFlowFile class declaration
- *
- * 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 __GENERATE_FLOW_FILE_H__
-#define __GENERATE_FLOW_FILE_H__
-
-#include "FlowFileRecord.h"
-#include "Processor.h"
-#include "ProcessSession.h"
-
-//! GenerateFlowFile Class
-class GenerateFlowFile : public Processor
-{
-public:
-	//! Constructor
-	/*!
-	 * Create a new processor
-	 */
-	GenerateFlowFile(std::string name, uuid_t uuid = NULL)
-	: Processor(name, uuid)
-	{
-		_data = NULL;
-		_dataSize = 0;
-	}
-	//! Destructor
-	virtual ~GenerateFlowFile()
-	{
-		if (_data)
-			delete[] _data;
-	}
-	//! Processor Name
-	static const std::string ProcessorName;
-	//! Supported Properties
-	static Property FileSize;
-	static Property BatchSize;
-	static Property DataFormat;
-	static Property UniqueFlowFiles;
-	static const char *DATA_FORMAT_BINARY;
-	static const char *DATA_FORMAT_TEXT;
-	//! Supported Relationships
-	static Relationship Success;
-	//! Nest Callback Class for write stream
-	class WriteCallback : public OutputStreamCallback
-	{
-		public:
-		WriteCallback(char *data, uint64_t size)
-		: _data(data), _dataSize(size) {}
-		char *_data;
-		uint64_t _dataSize;
-		void process(std::ofstream *stream) {
-			if (_data && _dataSize > 0)
-				stream->write(_data, _dataSize);
-		}
-	};
-
-public:
-	//! OnTrigger method, implemented by NiFi GenerateFlowFile
-	virtual void onTrigger(ProcessContext *context, ProcessSession *session);
-	//! Initialize, over write by NiFi GenerateFlowFile
-	virtual void initialize(void);
-
-protected:
-
-private:
-	//! Generated data
-	char * _data;
-	//! Size of the generate data
-	uint64_t _dataSize;
-};
-
-#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/GetFile.h
----------------------------------------------------------------------
diff --git a/libminifi/include/GetFile.h b/libminifi/include/GetFile.h
deleted file mode 100644
index 8f8068c..0000000
--- a/libminifi/include/GetFile.h
+++ /dev/null
@@ -1,117 +0,0 @@
-/**
- * @file GetFile.h
- * GetFile class declaration
- *
- * 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 __GET_FILE_H__
-#define __GET_FILE_H__
-
-#include "FlowFileRecord.h"
-#include "Processor.h"
-#include "ProcessSession.h"
-
-//! GetFile Class
-class GetFile : public Processor
-{
-public:
-	//! Constructor
-	/*!
-	 * Create a new processor
-	 */
-	GetFile(std::string name, uuid_t uuid = NULL)
-	: Processor(name, uuid)
-	{
-		logger_ = Logger::getLogger();
-		_directory = ".";
-		_recursive = true;
-		_keepSourceFile = false;
-		_minAge = 0;
-		_maxAge = 0;
-		_minSize = 0;
-		_maxSize = 0;
-		_ignoreHiddenFile = true;
-		_pollInterval = 0;
-		_batchSize = 10;
-		_lastDirectoryListingTime = getTimeMillis();
-		_fileFilter = "[^\\.].*";
-	}
-	//! Destructor
-	virtual ~GetFile()
-	{
-	}
-	//! Processor Name
-	static const std::string ProcessorName;
-	//! Supported Properties
-	static Property Directory;
-	static Property Recurse;
-	static Property KeepSourceFile;
-	static Property MinAge;
-	static Property MaxAge;
-	static Property MinSize;
-	static Property MaxSize;
-	static Property IgnoreHiddenFile;
-	static Property PollInterval;
-	static Property BatchSize;
-	static Property FileFilter;
-	//! Supported Relationships
-	static Relationship Success;
-
-public:
-	//! OnTrigger method, implemented by NiFi GetFile
-	virtual void onTrigger(ProcessContext *context, ProcessSession *session);
-	//! Initialize, over write by NiFi GetFile
-	virtual void initialize(void);
-	//! perform directory listing
-	void performListing(std::string dir);
-
-protected:
-
-private:
-	//! Logger
-	std::shared_ptr<Logger> logger_;
-	//! Queue for store directory list
-	std::queue<std::string> _dirList;
-	//! Get Listing size
-	uint64_t getListingSize() {
-		std::lock_guard<std::mutex> lock(_mtx);
-		return _dirList.size();
-	}
-	//! Whether the directory listing is empty
-	bool isListingEmpty();
-	//! Put full path file name into directory listing
-	void putListing(std::string fileName);
-	//! Poll directory listing for files
-	void pollListing(std::queue<std::string> &list, int maxSize);
-	//! Check whether file can be added to the directory listing
-	bool acceptFile(std::string fullName, std::string name);
-	//! Mutex for protection of the directory listing
-	std::mutex _mtx;
-	std::string _directory;
-	bool _recursive;
-	bool _keepSourceFile;
-	int64_t _minAge;
-	int64_t _maxAge;
-	int64_t _minSize;
-	int64_t _maxSize;
-	bool _ignoreHiddenFile;
-	int64_t _pollInterval;
-	int64_t _batchSize;
-	uint64_t _lastDirectoryListingTime;
-	std::string _fileFilter;
-};
-
-#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/ListenHTTP.h
----------------------------------------------------------------------
diff --git a/libminifi/include/ListenHTTP.h b/libminifi/include/ListenHTTP.h
deleted file mode 100644
index 5a467f2..0000000
--- a/libminifi/include/ListenHTTP.h
+++ /dev/null
@@ -1,116 +0,0 @@
-/**
- * @file ListenHTTP.h
- * ListenHTTP class declaration
- *
- * 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 __LISTEN_HTTP_H__
-#define __LISTEN_HTTP_H__
-
-#include <memory>
-#include <regex>
-
-#include <CivetServer.h>
-
-#include "FlowFileRecord.h"
-#include "Processor.h"
-#include "ProcessSession.h"
-
-
-//! ListenHTTP Class
-class ListenHTTP : public Processor
-{
-public:
-
-	//! Constructor
-	/*!
-	 * Create a new processor
-	 */
-	ListenHTTP(std::string name, uuid_t uuid = NULL)
-	: Processor(name, uuid)
-	{
-		_logger = Logger::getLogger();
-	}
-	//! Destructor
-	~ListenHTTP()
-	{
-	}
-	//! Processor Name
-	static const std::string ProcessorName;
-	//! Supported Properties
-	static Property BasePath;
-	static Property Port;
-	static Property AuthorizedDNPattern;
-	static Property SSLCertificate;
-	static Property SSLCertificateAuthority;
-	static Property SSLVerifyPeer;
-	static Property SSLMinimumVersion;
-	static Property HeadersAsAttributesRegex;
-	//! Supported Relationships
-	static Relationship Success;
-
-	void onTrigger(ProcessContext *context, ProcessSession *session);
-	void initialize();
-	void onSchedule(ProcessContext *context, ProcessSessionFactory *sessionFactory);
-
-	//! HTTP request handler
-	class Handler : public CivetHandler
-	{
-	public:
-		Handler(ProcessContext *context,
-				ProcessSessionFactory *sessionFactory,
-				std::string &&authDNPattern,
-				std::string &&headersAsAttributesPattern);
-		bool handlePost(CivetServer *server, struct mg_connection *conn);
-
-	private:
-		//! Send HTTP 500 error response to client
-		void sendErrorResponse(struct mg_connection *conn);
-		//! Logger
-		std::shared_ptr<Logger> _logger;
-
-		std::regex _authDNRegex;
-		std::regex _headersAsAttributesRegex;
-		ProcessContext *_processContext;
-		ProcessSessionFactory *_processSessionFactory;
-	};
-
-	//! Write callback for transferring data from HTTP request to content repo
-	class WriteCallback : public OutputStreamCallback
-	{
-	public:
-		WriteCallback(struct mg_connection *conn, const struct mg_request_info *reqInfo);
-		void process(std::ofstream *stream);
-
-	private:
-		//! Logger
-		std::shared_ptr<Logger> _logger;
-
-		struct mg_connection *_conn;
-		const struct mg_request_info *_reqInfo;
-	};
-
-protected:
-
-private:
-	//! Logger
-	std::shared_ptr<Logger> _logger;
-
-	std::unique_ptr<CivetServer> _server;
-	std::unique_ptr<Handler> _handler;
-};
-
-#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/ListenSyslog.h
----------------------------------------------------------------------
diff --git a/libminifi/include/ListenSyslog.h b/libminifi/include/ListenSyslog.h
deleted file mode 100644
index 339dbc1..0000000
--- a/libminifi/include/ListenSyslog.h
+++ /dev/null
@@ -1,209 +0,0 @@
-/**
- * @file ListenSyslog.h
- * ListenSyslog class declaration
- *
- * 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 __LISTEN_SYSLOG_H__
-#define __LISTEN_SYSLOG_H__
-
-#include <stdio.h>
-#include <unistd.h>
-#include <sys/types.h>
-#include <sys/socket.h>
-#include <netinet/in.h>
-#include <arpa/inet.h>
-#include <errno.h>
-#include <sys/select.h>
-#include <sys/time.h>
-#include <sys/types.h>
-#include <chrono>
-#include <thread>
-#include "FlowFileRecord.h"
-#include "Processor.h"
-#include "ProcessSession.h"
-
-//! SyslogEvent
-typedef struct {
-	uint8_t *payload;
-	uint64_t len;
-} SysLogEvent;
-
-//! ListenSyslog Class
-class ListenSyslog : public Processor
-{
-public:
-	//! Constructor
-	/*!
-	 * Create a new processor
-	 */
-	ListenSyslog(std::string name, uuid_t uuid = NULL)
-	: Processor(name, uuid)
-	{
-		logger_ = Logger::getLogger();
-		_eventQueueByteSize = 0;
-		_serverSocket = 0;
-		_recvBufSize = 65507;
-		_maxSocketBufSize = 1024*1024;
-		_maxConnections = 2;
-		_maxBatchSize = 1;
-		_messageDelimiter = "\n";
-		_protocol = "UDP";
-		_port = 514;
-		_parseMessages = false;
-		_serverSocket = 0;
-		_maxFds = 0;
-		FD_ZERO(&_readfds);
-		_thread = NULL;
-		_resetServerSocket = false;
-		_serverTheadRunning = false;
-	}
-	//! Destructor
-	virtual ~ListenSyslog()
-	{
-		_serverTheadRunning = false;
-		if (this->_thread)
-			delete this->_thread;
-		// need to reset the socket
-		std::vector<int>::iterator it;
-		for (it = _clientSockets.begin(); it != _clientSockets.end(); ++it)
-		{
-			int clientSocket = *it;
-			close(clientSocket);
-		}
-		_clientSockets.clear();
-		if (_serverSocket > 0)
-		{
-			logger_->log_info("ListenSysLog Server socket %d close", _serverSocket);
-			close(_serverSocket);
-			_serverSocket = 0;
-		}
-	}
-	//! Processor Name
-	static const std::string ProcessorName;
-	//! Supported Properties
-	static Property RecvBufSize;
-	static Property MaxSocketBufSize;
-	static Property MaxConnections;
-	static Property MaxBatchSize;
-	static Property MessageDelimiter;
-	static Property ParseMessages;
-	static Property Protocol;
-	static Property Port;
-	//! Supported Relationships
-	static Relationship Success;
-	static Relationship Invalid;
-	//! Nest Callback Class for write stream
-	class WriteCallback : public OutputStreamCallback
-	{
-		public:
-		WriteCallback(char *data, uint64_t size)
-		: _data(data), _dataSize(size) {}
-		char *_data;
-		uint64_t _dataSize;
-		void process(std::ofstream *stream) {
-			if (_data && _dataSize > 0)
-				stream->write(_data, _dataSize);
-		}
-	};
-
-public:
-	//! OnTrigger method, implemented by NiFi ListenSyslog
-	virtual void onTrigger(ProcessContext *context, ProcessSession *session);
-	//! Initialize, over write by NiFi ListenSyslog
-	virtual void initialize(void);
-
-protected:
-
-private:
-	//! Logger
-	std::shared_ptr<Logger> logger_;
-	//! Run function for the thread
-	static void run(ListenSyslog *process);
-	//! Run Thread
-	void runThread();
-	//! Queue for store syslog event
-	std::queue<SysLogEvent> _eventQueue;
-	//! Size of Event queue in bytes
-	uint64_t _eventQueueByteSize;
-	//! Get event queue size
-	uint64_t getEventQueueSize() {
-		std::lock_guard<std::mutex> lock(_mtx);
-		return _eventQueue.size();
-	}
-	//! Get event queue byte size
-	uint64_t getEventQueueByteSize() {
-		std::lock_guard<std::mutex> lock(_mtx);
-		return _eventQueueByteSize;
-	}
-	//! Whether the event queue  is empty
-	bool isEventQueueEmpty()
-	{
-		std::lock_guard<std::mutex> lock(_mtx);
-		return _eventQueue.empty();
-	}
-	//! Put event into directory listing
-	void putEvent(uint8_t *payload, uint64_t len)
-	{
-		std::lock_guard<std::mutex> lock(_mtx);
-		SysLogEvent event;
-		event.payload = payload;
-		event.len = len;
-		_eventQueue.push(event);
-		_eventQueueByteSize += len;
-	}
-	//! Read \n terminated line from TCP socket
-	int readline( int fd, char *bufptr, size_t len );
-	//! start server socket and handling client socket
-	void startSocketThread();
-	//! Poll event
-	void pollEvent(std::queue<SysLogEvent> &list, int maxSize)
-	{
-		std::lock_guard<std::mutex> lock(_mtx);
-
-		while (!_eventQueue.empty() && (maxSize == 0 || list.size() < maxSize))
-		{
-			SysLogEvent event = _eventQueue.front();
-			_eventQueue.pop();
-			_eventQueueByteSize -= event.len;
-			list.push(event);
-		}
-		return;
-	}
-	//! Mutex for protection of the directory listing
-	std::mutex _mtx;
-	int64_t _recvBufSize;
-	int64_t _maxSocketBufSize;
-	int64_t _maxConnections;
-	int64_t _maxBatchSize;
-	std::string _messageDelimiter;
-	std::string _protocol;
-	int64_t _port;
-	bool _parseMessages;
-	int _serverSocket;
-	std::vector<int> _clientSockets;
-	int _maxFds;
-	fd_set _readfds;
-	//! thread
-	std::thread *_thread;
-	//! whether to reset the server socket
-	bool _resetServerSocket;
-	bool _serverTheadRunning;
-	//! buffer for read socket
-	uint8_t _buffer[2048];
-};
-
-#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/LogAppenders.h
----------------------------------------------------------------------
diff --git a/libminifi/include/LogAppenders.h b/libminifi/include/LogAppenders.h
deleted file mode 100644
index ef28bb8..0000000
--- a/libminifi/include/LogAppenders.h
+++ /dev/null
@@ -1,298 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#ifndef LIBMINIFI_INCLUDE_LOGAPPENDERS_H_
-#define LIBMINIFI_INCLUDE_LOGAPPENDERS_H_
-
-#include "BaseLogger.h"
-#include "spdlog/sinks/null_sink.h"
-#include "spdlog/sinks/ostream_sink.h"
-#include <cxxabi.h>
-#include "Configure.h"
-
-template<typename T>
-static std::string getUniqueName() {
-	std::string name = LOG_NAME;
-	name += " -- ";
-	name += abi::__cxa_demangle(typeid(T).name(), 0, 0, 0);
-	spdlog::drop(name);
-	return name;
-}
-
-/**
- * Null appender sets a null sink, thereby performing no logging.
- */
-class NullAppender: public BaseLogger {
-public:
-	/**
-	 * Base constructor that creates the null sink.
-	 */
-	explicit NullAppender() :
-			::BaseLogger("off") {
-		auto null_sink = std::make_shared<spdlog::sinks::null_sink_st>();
-		std::string unique_name = getUniqueName<NullAppender>();
-		logger_ = std::make_shared<spdlog::logger>(unique_name, null_sink);
-		configured_level_ = off;
-		setLogLevel();
-	}
-
-	/**
-	 * Move constructor for the null appender.
-	 */
-	explicit NullAppender(const NullAppender &&other) :
-			::BaseLogger(std::move(other)) {
-
-	}
-
-};
-
-/**
- * Basic output stream configuration that uses a supplied ostream
- *
- * Design : extends LoggerConfiguration using the logger and log level
- * encapsulated within the base configuration class.
- */
-class OutputStreamAppender: public BaseLogger {
-
-public:
-
-	static const char *nifi_log_output_stream_error_stderr;
-
-	/**
-	 * Output stream move constructor.
-	 */
-	explicit OutputStreamAppender(const OutputStreamAppender &&other) :
-			::BaseLogger(std::move(other)) {
-
-	}
-
-	/**
-	 * Base constructor. Creates a ostream sink.
-	 * @param stream incoming stream reference.
-	 * @param config configuration.
-	 */
-	explicit OutputStreamAppender(Configure *config) :
-			::BaseLogger("info") {
-		auto ostream_sink = std::make_shared<spdlog::sinks::ostream_sink_mt>(
-				std::cout);
-
-		std::string unique_name = getUniqueName<OutputStreamAppender>();
-		logger_ = std::make_shared<spdlog::logger>(unique_name, ostream_sink);
-
-		std::string use_std_err;
-
-		if (NULL != config
-				&& config->get(nifi_log_output_stream_error_stderr,
-						use_std_err)) {
-
-			std::transform(use_std_err.begin(), use_std_err.end(),
-					use_std_err.begin(), ::tolower);
-
-			if (use_std_err == "true") {
-				std::string err_unique_name =
-						getUniqueName<OutputStreamAppender>();
-				auto error_ostream_sink = std::make_shared<
-						spdlog::sinks::ostream_sink_mt>(std::cerr);
-				stderr_ = std::make_shared<spdlog::logger>(err_unique_name,
-						error_ostream_sink);
-			}
-		} else {
-			stderr_ = nullptr;
-		}
-
-		std::string log_level;
-		if (NULL != config
-				&& config->get(BaseLogger::nifi_log_level, log_level)) {
-			setLogLevel(log_level);
-		} else{
-			setLogLevel("info");
-		}
-
-	}
-
-	/**
-	 * Base constructor. Creates a ostream sink.
-	 * @param stream incoming stream reference.
-	 * @param config configuration.
-	 */
-	OutputStreamAppender(std::ostream &stream, Configure *config) :
-			::BaseLogger("info") {
-		auto ostream_sink = std::make_shared<spdlog::sinks::ostream_sink_mt>(
-				stream);
-		std::string unique_name = getUniqueName<OutputStreamAppender>();
-		logger_ = std::make_shared<spdlog::logger>(unique_name, ostream_sink);
-
-		stderr_ = nullptr;
-
-		std::string log_level;
-		if (NULL != config
-				&& config->get(BaseLogger::nifi_log_level, log_level)) {
-			setLogLevel(log_level);
-		} else {
-			setLogLevel("info");
-		}
-
-	}
-
-protected:
-
-};
-
-/**
- * Rolling configuration
- * Design : extends LoggerConfiguration using the logger and log level
- * encapsulated within the base configuration class.
- */
-class RollingAppender: public BaseLogger {
-public:
-	static const char *nifi_log_rolling_apender_file;
-	static const char *nifi_log_rolling_appender_max_files;
-	static const char *nifi_log_rolling_appender_max_file_size;
-
-	/**
-	 * RollingAppenderConfiguration move constructor.
-	 */
-	explicit RollingAppender(const RollingAppender&& other) :
-			::BaseLogger(std::move(other)), max_files_(
-					std::move(other.max_files_)), file_name_(
-					std::move(other.file_name_)), max_file_size_(
-					std::move(other.max_file_size_)) {
-	}
-	/**
-	 * Base Constructor.
-	 * @param config pointer to the configuration for this instance.
-	 */
-	explicit RollingAppender(Configure * config = 0) :
-			::BaseLogger("info") {
-		std::string file_name = "";
-		if (NULL != config
-				&& config->get(nifi_log_rolling_apender_file, file_name)) {
-			file_name_ = file_name;
-		} else{
-			file_name_ = LOG_FILE_NAME;
-		}
-
-		std::string max_files = "";
-		if (NULL != config
-				&& config->get(nifi_log_rolling_appender_max_files,
-						max_files)) {
-			try {
-				max_files_ = std::stoi(max_files);
-			} catch (const std::invalid_argument &ia) {
-				max_files_ = DEFAULT_LOG_FILE_NUMBER;
-			} catch (const std::out_of_range &oor) {
-				max_files_ = DEFAULT_LOG_FILE_NUMBER;
-			}
-		} else {
-			max_files_ = DEFAULT_LOG_FILE_NUMBER;
-		}
-
-		std::string max_file_size = "";
-		if (NULL != config
-				&& config->get(nifi_log_rolling_appender_max_file_size,
-						max_file_size)) {
-			try {
-				max_file_size_ = std::stoi(max_file_size);
-			} catch (const std::invalid_argument &ia) {
-				max_file_size_ = DEFAULT_LOG_FILE_SIZE;
-			} catch (const std::out_of_range &oor) {
-				max_file_size_ = DEFAULT_LOG_FILE_SIZE;
-			}
-		} else {
-			max_file_size_ = DEFAULT_LOG_FILE_SIZE;
-		}
-
-		std::string unique_name = getUniqueName<OutputStreamAppender>();
-		logger_ = spdlog::rotating_logger_mt(unique_name, file_name_,
-				max_file_size_, max_files_);
-
-		std::string log_level;
-		if (NULL != config
-				&& config->get(BaseLogger::nifi_log_level, log_level)) {
-			setLogLevel(log_level);
-		}
-	}
-
-	/**
-	 * To maintain current functionality we will flush on write.
-	 */
-	void log_str(LOG_LEVEL_E level, const std::string &buffer) {
-		::BaseLogger::log_str(level, buffer);
-		logger_->flush();
-	}
-
-protected:
-
-	/**
-	 * file name.
-	 */
-	std::string file_name_;
-	/**
-	 * maximum number of files to keep in the rotation.
-	 */
-	size_t max_files_;
-	/**
-	 * Maximum file size per rotated file.
-	 */
-	size_t max_file_size_;
-
-};
-
-class LogInstance {
-public:
-	/**
-	 * Returns a logger configuration based on
-	 * the configuration within this instance.
-	 * @param config configuration for this instance.
-	 */
-	static std::unique_ptr<BaseLogger> getConfiguredLogger(Configure *config) {
-		std::string appender = "";
-
-		if (config->get(BaseLogger::nifi_log_appender, appender)) {
-			std::transform(appender.begin(), appender.end(), appender.begin(),
-					::tolower);
-
-			if ("nullappender" == appender || "null appender" == appender
-					|| "null" == appender) {
-
-				return std::move(
-						std::unique_ptr<BaseLogger>(new NullAppender()));
-
-			} else if ("rollingappender" == appender
-					|| "rolling appender" == appender
-					|| "rolling" == appender) {
-
-				return std::move(
-						std::unique_ptr<BaseLogger>(
-								new RollingAppender(config)));
-
-			} else if ("outputstream" == appender
-					|| "outputstreamappender" == appender
-					|| "outputstream appender" == appender) {
-
-				return std::move(
-						std::unique_ptr<BaseLogger>(
-								new OutputStreamAppender(config)));
-
-			}
-		}
-		return nullptr;
-
-	}
-};
-
-#endif /* LIBMINIFI_INCLUDE_LOGAPPENDERS_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/LogAttribute.h
----------------------------------------------------------------------
diff --git a/libminifi/include/LogAttribute.h b/libminifi/include/LogAttribute.h
deleted file mode 100644
index 429a594..0000000
--- a/libminifi/include/LogAttribute.h
+++ /dev/null
@@ -1,128 +0,0 @@
-/**
- * @file LogAttribute.h
- * LogAttribute class declaration
- *
- * 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 __LOG_ATTRIBUTE_H__
-#define __LOG_ATTRIBUTE_H__
-
-#include "FlowFileRecord.h"
-#include "Processor.h"
-#include "ProcessSession.h"
-
-//! LogAttribute Class
-class LogAttribute : public Processor
-{
-public:
-	//! Constructor
-	/*!
-	 * Create a new processor
-	 */
-	LogAttribute(std::string name, uuid_t uuid = NULL)
-	: Processor(name, uuid)
-	{
-		logger_ = Logger::getLogger();
-	}
-	//! Destructor
-	virtual ~LogAttribute()
-	{
-	}
-	//! Processor Name
-	static const std::string ProcessorName;
-	//! Supported Properties
-	static Property LogLevel;
-	static Property AttributesToLog;
-	static Property AttributesToIgnore;
-	static Property LogPayload;
-	static Property LogPrefix;
-	//! Supported Relationships
-	static Relationship Success;
-	enum LogAttrLevel {
-        LogAttrLevelTrace, LogAttrLevelDebug, LogAttrLevelInfo, LogAttrLevelWarn, LogAttrLevelError
-    };
-	//! Convert log level from string to enum
-	bool logLevelStringToEnum(std::string logStr, LogAttrLevel &level)
-	{
-		if (logStr == "trace")
-		{
-			level = LogAttrLevelTrace;
-			return true;
-		}
-		else if (logStr == "debug")
-		{
-			level = LogAttrLevelDebug;
-			return true;
-		}
-		else if (logStr == "info")
-		{
-			level = LogAttrLevelInfo;
-			return true;
-		}
-		else if (logStr == "warn")
-		{
-			level = LogAttrLevelWarn;
-			return true;
-		}
-		else if (logStr == "error")
-		{
-			level = LogAttrLevelError;
-			return true;
-		}
-		else
-			return false;
-	}
-	//! Nest Callback Class for read stream
-	class ReadCallback : public InputStreamCallback
-	{
-		public:
-		ReadCallback(uint64_t size)
-		{
-			_bufferSize = size;
-			_buffer = new char[_bufferSize];
-		}
-		~ReadCallback()
-		{
-			if (_buffer)
-				delete[] _buffer;
-		}
-		void process(std::ifstream *stream) {
-
-			stream->read(_buffer, _bufferSize);
-			if (!stream)
-				_readSize = stream->gcount();
-			else
-				_readSize = _bufferSize;
-		}
-		char  *_buffer;
-		uint64_t _bufferSize;
-		uint64_t _readSize;
-	};
-
-public:
-	//! OnTrigger method, implemented by NiFi LogAttribute
-	virtual void onTrigger(ProcessContext *context, ProcessSession *session);
-	//! Initialize, over write by NiFi LogAttribute
-	virtual void initialize(void);
-
-protected:
-
-private:
-	//! Logger
-	std::shared_ptr<Logger> logger_;
-};
-
-#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/Logger.h
----------------------------------------------------------------------
diff --git a/libminifi/include/Logger.h b/libminifi/include/Logger.h
deleted file mode 100644
index 81d4446..0000000
--- a/libminifi/include/Logger.h
+++ /dev/null
@@ -1,200 +0,0 @@
-/**
- * @file Logger.h
- * Logger class declaration
- * This is a C++ wrapper for spdlog, a lightweight C++ logging library
- *
- * 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 __LOGGER_H__
-#define __LOGGER_H__
-
-#include <string>
-#include <atomic>
-#include <memory>
-#include <utility>
-#include <algorithm>
-#include <cstdio>
-#include <iostream>
-
-#include "BaseLogger.h"
-#include "spdlog/spdlog.h"
-
-/**
- * Logger class
- * Design: Extends BaseLogger, leaving this class to be the facade to the underlying
- * logging mechanism. Is a facade to BaseLogger's underlying log stream. This allows
- * the underlying implementation to be replaced real time.
- */
-class Logger: public BaseLogger {
-protected:
-	struct singleton;
-public:
-
-	/**
-	 * Returns a shared pointer to the logger instance.
-	 * Note that while there is no synchronization this is expected
-	 * to be called and initialized first
-	 * @returns shared pointer to the base logger.
-	 */
-	static std::shared_ptr<Logger> getLogger() {
-
-		if (singleton_logger_ == nullptr)
-			singleton_logger_ = std::make_shared<Logger>(singleton { 0 });
-		return singleton_logger_;
-	}
-
-	/**
-	 * Returns the log level for this instance.
-	 */
-	LOG_LEVEL_E getLogLevel() const {
-		return current_logger_.load()->getLogLevel();
-	}
-
-	/**
-	 * Sets the log level atomic and sets it
-	 * within logger if it can
-	 * @param level desired log level.
-	 */
-	void setLogLevel(LOG_LEVEL_E level) {
-		current_logger_.load()->setLogLevel(level);
-	}
-
-	/**
-	 * Sets the log level for this instance based on the string
-	 * @param level desired log leve.
-	 * @param defaultLevel default level if we cannot match level.
-	 */
-	void setLogLevel(const std::string &level,
-			LOG_LEVEL_E defaultLevel = info) {
-		current_logger_.load()->setLogLevel(level, info);
-	}
-
-	void updateLogger(std::unique_ptr<BaseLogger> logger) {
-
-		if (logger == nullptr  )
-			return;
-		current_logger_.store(logger.release());
-	}
-
-	/**
-	 * @brief Log error message
-	 * @param format format string ('man printf' for syntax)
-	 * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
-	 */
-	void log_error(const char * const format, ...) {
-		if (!current_logger_.load()->shouldLog(err))
-			return;
-		FILL_BUFFER
-		current_logger_.load()->log_str(err, buffer);
-	}
-	/**
-	 * @brief Log warn message
-	 * @param format format string ('man printf' for syntax)
-	 * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
-	 */
-	void log_warn(const char * const format, ...) {
-		if (!current_logger_.load()->shouldLog(warn))
-			return;
-		FILL_BUFFER
-		current_logger_.load()->log_str(warn, buffer);
-	}
-	/**
-	 * @brief Log info message
-	 * @param format format string ('man printf' for syntax)
-	 * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
-	 */
-	void log_info(const char * const format, ...) {
-		if (!current_logger_.load()->shouldLog(info))
-			return;
-		FILL_BUFFER
-		current_logger_.load()->log_str(info, buffer);
-	}
-	/**
-	 * @brief Log debug message
-	 * @param format format string ('man printf' for syntax)
-	 * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
-	 */
-	void log_debug(const char * const format, ...) {
-
-		if (!current_logger_.load()->shouldLog(debug))
-			return;
-		FILL_BUFFER
-		current_logger_.load()->log_str(debug, buffer);
-	}
-	/**
-	 * @brief Log trace message
-	 * @param format format string ('man printf' for syntax)
-	 * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
-	 */
-	void log_trace(const char * const format, ...) {
-
-		if (!current_logger_.load()->shouldLog(trace))
-			return;
-		FILL_BUFFER
-		current_logger_.load()->log_str(trace, buffer);
-	}
-
-	/**
-	 * @brief Log message
-	 * @param format format string ('man printf' for syntax)
-	 * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
-	 */
-	virtual void log_str(LOG_LEVEL_E level, const std::string &buffer) {
-		current_logger_.load()->log_str(level, buffer);
-	}
-
-	//! Destructor
-	~Logger() {
-	}
-
-	explicit Logger(const singleton &a) {
-
-		/**
-		 * flush on info to maintain current functionality
-		 */
-		std::shared_ptr<spdlog::logger> defaultsink = spdlog::rotating_logger_mt(LOG_NAME,
-				LOG_FILE_NAME,
-				DEFAULT_LOG_FILE_SIZE, DEFAULT_LOG_FILE_NUMBER);
-		defaultsink->flush_on(spdlog::level::level_enum::info);
-
-		std::unique_ptr<BaseLogger> new_logger_ = std::unique_ptr<BaseLogger>(
-				new BaseLogger("info", defaultsink));
-
-		new_logger_->setLogLevel(info);
-		current_logger_.store(new_logger_.release());
-	}
-
-	Logger(const Logger &parent) = delete;
-	Logger &operator=(const Logger &parent) = delete;
-
-protected:
-
-	/**
-	 * Allows for a null constructor above so that we can have a public constructor that
-	 * effectively limits us to being a singleton by having a protected argument in the constructor
-	 */
-	struct singleton {
-		explicit singleton(int) {
-		}
-	};
-
-	std::atomic<BaseLogger*> current_logger_;
-
-//! Singleton logger instance
-	static std::shared_ptr<Logger> singleton_logger_;
-};
-
-#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/ProcessContext.h
----------------------------------------------------------------------
diff --git a/libminifi/include/ProcessContext.h b/libminifi/include/ProcessContext.h
deleted file mode 100644
index 585e187..0000000
--- a/libminifi/include/ProcessContext.h
+++ /dev/null
@@ -1,113 +0,0 @@
-/**
- * @file ProcessContext.h
- * ProcessContext class declaration
- *
- * 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 __PROCESS_CONTEXT_H__
-#define __PROCESS_CONTEXT_H__
-
-#include <uuid/uuid.h>
-#include <vector>
-#include <queue>
-#include <map>
-#include <mutex>
-#include <atomic>
-#include <algorithm>
-
-#include "Logger.h"
-#include "Processor.h"
-
-//! ProcessContext Class
-class ProcessContext
-{
-public:
-	//! Constructor
-	/*!
-	 * Create a new process context associated with the processor/controller service/state manager
-	 */
-	ProcessContext(Processor *processor = NULL) : _processor(processor) {
-		logger_ = Logger::getLogger();
-	}
-	//! Destructor
-	virtual ~ProcessContext() {}
-	//! Get Processor associated with the Process Context
-	Processor *getProcessor() {
-		return _processor;
-	}
-	bool getProperty(std::string name, std::string &value) {
-		if (_processor)
-			return _processor->getProperty(name, value);
-		else
-			return false;
-	}
-	//! Sets the property value using the property's string name
-	bool setProperty(std::string name, std::string value)
-	{
-		if (_processor)
-			return _processor->setProperty(name, value);
-		else
-			return false;
-	}
-	//! Sets the property value using the Property object
-	bool setProperty(Property prop, std::string value) {
-		if (_processor)
-			return _processor->setProperty(prop, value);
-		else
-			return false;
-	}
-	//! Whether the relationship is supported
-	bool isSupportedRelationship(Relationship relationship) {
-		if (_processor)
-			return _processor->isSupportedRelationship(relationship);
-		else
-			return false;
-	}
-	//! Check whether the relationship is auto terminated
-	bool isAutoTerminated(Relationship relationship) {
-		if (_processor)
-			return _processor->isAutoTerminated(relationship);
-		else
-			return false;
-	}
-	//! Get ProcessContext Maximum Concurrent Tasks
-	uint8_t getMaxConcurrentTasks(void) {
-		if (_processor)
-			return _processor->getMaxConcurrentTasks();
-		else
-			return 0;
-	}
-	//! Yield based on the yield period
-	void yield() {
-		if (_processor)
-			_processor->yield();
-	}
-
-	// Prevent default copy constructor and assignment operation
-	// Only support pass by reference or pointer
-	ProcessContext(const ProcessContext &parent) = delete;
-	ProcessContext &operator=(const ProcessContext &parent) = delete;
-
-private:
-
-	//! Processor
-	Processor *_processor;
-	//! Logger
-	std::shared_ptr<Logger> logger_;
-
-};
-
-#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/ProcessGroup.h
----------------------------------------------------------------------
diff --git a/libminifi/include/ProcessGroup.h b/libminifi/include/ProcessGroup.h
deleted file mode 100644
index dfec6c5..0000000
--- a/libminifi/include/ProcessGroup.h
+++ /dev/null
@@ -1,187 +0,0 @@
-/**
- * @file ProcessGroup.h
- * ProcessGroup class declaration
- *
- * 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 __PROCESS_GROUP_H__
-#define __PROCESS_GROUP_H__
-
-#include <uuid/uuid.h>
-#include <vector>
-#include <queue>
-#include <map>
-#include <mutex>
-#include <atomic>
-#include <algorithm>
-#include <set>
-
-#include "Processor.h"
-#include "Exception.h"
-#include "TimerDrivenSchedulingAgent.h"
-#include "EventDrivenSchedulingAgent.h"
-#include "Logger.h"
-
-//! Process Group Type
-enum ProcessGroupType
-{
-	ROOT_PROCESS_GROUP = 0,
-	REMOTE_PROCESS_GROUP,
-	MAX_PROCESS_GROUP_TYPE
-};
-
-//! ProcessGroup Class
-class ProcessGroup
-{
-public:
-	//! Constructor
-	/*!
-	 * Create a new process group
-	 */
-	ProcessGroup(ProcessGroupType type, std::string name, uuid_t uuid = NULL, ProcessGroup *parent = NULL);
-	//! Destructor
-	virtual ~ProcessGroup();
-	//! Set Processor Name
-	void setName(std::string name) {
-		name_ = name;
-	}
-	//! Get Process Name
-	std::string getName(void) {
-		return (name_);
-	}
-	//! Set URL
-	void setURL(std::string url) {
-		url_ = url;
-	}
-	//! Get URL
-	std::string getURL(void) {
-		return (url_);
-	}
-	//! SetTransmitting
-	void setTransmitting(bool val)
-	{
-		transmitting_ = val;
-	}
-	//! Get Transmitting
-	bool getTransmitting()
-	{
-		return transmitting_;
-	}
-	//! setTimeOut
-	void setTimeOut(uint64_t time)
-	{
-		timeOut_ = time;
-	}
-	uint64_t getTimeOut()
-	{
-		return timeOut_;
-	}
-	//! Set Processor yield period in MilliSecond
-	void setYieldPeriodMsec(uint64_t period) {
-		yield_period_msec_ = period;
-	}
-	//! Get Processor yield period in MilliSecond
-	uint64_t getYieldPeriodMsec(void) {
-		return(yield_period_msec_);
-	}
-	//! Set UUID
-	void setUUID(uuid_t uuid) {
-		uuid_copy(uuid_, uuid);
-	}
-	//! Get UUID
-	bool getUUID(uuid_t uuid) {
-		if (uuid)
-		{
-			uuid_copy(uuid, uuid_);
-			return true;
-		}
-		else
-			return false;
-	}
-	//! Start Processing
-	void startProcessing(TimerDrivenSchedulingAgent *timeScheduler,
-			EventDrivenSchedulingAgent *eventScheduler);
-	//! Stop Processing
-	void stopProcessing(TimerDrivenSchedulingAgent *timeScheduler,
-			EventDrivenSchedulingAgent *eventScheduler);
-	//! Whether it is root process group
-	bool isRootProcessGroup();
-	//! set parent process group
-	void setParent(ProcessGroup *parent) {
-		std::lock_guard<std::mutex> lock(mtx_);
-		parent_process_group_ = parent;
-	}
-	//! get parent process group
-	ProcessGroup *getParent(void) {
-		std::lock_guard<std::mutex> lock(mtx_);
-		return parent_process_group_;
-	}
-	//! Add processor
-	void addProcessor(Processor *processor);
-	//! Remove processor
-	void removeProcessor(Processor *processor);
-	//! Add child processor group
-	void addProcessGroup(ProcessGroup *child);
-	//! Remove child processor group
-	void removeProcessGroup(ProcessGroup *child);
-	// ! Add connections
-	void addConnection(Connection *connection);
-	//! findProcessor based on UUID
-	Processor *findProcessor(uuid_t uuid);
-	//! findProcessor based on name
-	Processor *findProcessor(std::string processorName);
-	//! removeConnection
-	void removeConnection(Connection *connection);
-	//! update property value
-	void updatePropertyValue(std::string processorName, std::string propertyName, std::string propertyValue);
-	//! get connections under the process group
-	void getConnections(std::map<std::string, Connection*> *connectionMap);
-
-protected:
-	//! A global unique identifier
-	uuid_t uuid_;
-	//! Processor Group Name
-	std::string name_;
-	//! Process Group Type
-	ProcessGroupType type_;
-	//! Processors (ProcessNode) inside this process group which include Input/Output Port, Remote Process Group input/Output port
-	std::set<Processor *> processors_;
-	std::set<ProcessGroup *> child_process_groups_;
-	//! Connections between the processor inside the group;
-	std::set<Connection *> connections_;
-	//! Parent Process Group
-	ProcessGroup* parent_process_group_;
-	//! Yield Period in Milliseconds
-	std::atomic<uint64_t> yield_period_msec_;
-	std::atomic<uint64_t> timeOut_;
-	//! URL
-	std::string url_;
-	//! Transmitting
-	std::atomic<bool> transmitting_;
-
-private:
-
-	//! Mutex for protection
-	std::mutex mtx_;
-	//! Logger
-	std::shared_ptr<Logger> logger_;
-	// Prevent default copy constructor and assignment operation
-	// Only support pass by reference or pointer
-	ProcessGroup(const ProcessGroup &parent);
-	ProcessGroup &operator=(const ProcessGroup &parent);
-};
-
-#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/ProcessSession.h
----------------------------------------------------------------------
diff --git a/libminifi/include/ProcessSession.h b/libminifi/include/ProcessSession.h
deleted file mode 100644
index 4e26758..0000000
--- a/libminifi/include/ProcessSession.h
+++ /dev/null
@@ -1,125 +0,0 @@
-/**
- * @file ProcessSession.h
- * ProcessSession class declaration
- *
- * 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 __PROCESS_SESSION_H__
-#define __PROCESS_SESSION_H__
-
-#include <uuid/uuid.h>
-#include <vector>
-#include <queue>
-#include <map>
-#include <mutex>
-#include <atomic>
-#include <algorithm>
-#include <set>
-
-#include "Processor.h"
-#include "ProcessContext.h"
-#include "FlowFileRecord.h"
-#include "Exception.h"
-#include "Logger.h"
-#include "Provenance.h"
-
-//! ProcessSession Class
-class ProcessSession
-{
-public:
-	//! Constructor
-	/*!
-	 * Create a new process session
-	 */
-	ProcessSession(ProcessContext *processContext = NULL);
-	//! Destructor
-	virtual ~ProcessSession() {
-		if (_provenanceReport)
-			delete _provenanceReport;
-	}
-	//! Commit the session
-	void commit();
-	//! Roll Back the session
-	void rollback();
-	//! Get Provenance Report
-	ProvenanceReporter *getProvenanceReporter()
-	{
-		return _provenanceReport;
-	}
-	//!
-	//! Get the FlowFile from the highest priority queue
-	FlowFileRecord *get();
-	//! Create a new UUID FlowFile with no content resource claim and without parent
-	FlowFileRecord *create();
-	//! Create a new UUID FlowFile with no content resource claim and inherit all attributes from parent
-	FlowFileRecord *create(FlowFileRecord *parent);
-	//! Clone a new UUID FlowFile from parent both for content resource claim and attributes
-	FlowFileRecord *clone(FlowFileRecord *parent);
-	//! Clone a new UUID FlowFile from parent for attributes and sub set of parent content resource claim
-	FlowFileRecord *clone(FlowFileRecord *parent, long offset, long size);
-	//! Duplicate a FlowFile with the same UUID and all attributes and content resource claim for the roll back of the session
-	FlowFileRecord *duplicate(FlowFileRecord *orignal);
-	//! Transfer the FlowFile to the relationship
-	void transfer(FlowFileRecord *flow, Relationship relationship);
-	//! Put Attribute
-	void putAttribute(FlowFileRecord *flow, std::string key, std::string value);
-	//! Remove Attribute
-	void removeAttribute(FlowFileRecord *flow, std::string key);
-	//! Remove Flow File
-	void remove(FlowFileRecord *flow);
-	//! Execute the given read callback against the content
-	void read(FlowFileRecord *flow, InputStreamCallback *callback);
-	//! Execute the given write callback against the content
-	void write(FlowFileRecord *flow, OutputStreamCallback *callback);
-	//! Execute the given write/append callback against the content
-	void append(FlowFileRecord *flow, OutputStreamCallback *callback);
-	//! Penalize the flow
-	void penalize(FlowFileRecord *flow);
-	//! Import the existed file into the flow
-	void import(std::string source, FlowFileRecord *flow, bool keepSource = true, uint64_t offset = 0);
-
-	// Prevent default copy constructor and assignment operation
-	// Only support pass by reference or pointer
-	ProcessSession(const ProcessSession &parent) = delete;
-	ProcessSession &operator=(const ProcessSession &parent) = delete;
-
-protected:
-	//! FlowFiles being modified by current process session
-	std::map<std::string, FlowFileRecord *> _updatedFlowFiles;
-	//! Copy of the original FlowFiles being modified by current process session as above
-	std::map<std::string, FlowFileRecord *> _originalFlowFiles;
-	//! FlowFiles being added by current process session
-	std::map<std::string, FlowFileRecord *> _addedFlowFiles;
-	//! FlowFiles being deleted by current process session
-	std::map<std::string, FlowFileRecord *> _deletedFlowFiles;
-	//! FlowFiles being transfered to the relationship
-	std::map<std::string, Relationship> _transferRelationship;
-	//! FlowFiles being cloned for multiple connections per relationship
-	std::map<std::string, FlowFileRecord *> _clonedFlowFiles;
-
-private:
-	// Clone the flow file during transfer to multiple connections for a relationship
-	FlowFileRecord* cloneDuringTransfer(FlowFileRecord *parent);
-	//! ProcessContext
-	ProcessContext *_processContext;
-	//! Logger
-	std::shared_ptr<Logger> logger_;
-	//! Provenance Report
-	ProvenanceReporter *_provenanceReport;
-
-};
-
-#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/ProcessSessionFactory.h
----------------------------------------------------------------------
diff --git a/libminifi/include/ProcessSessionFactory.h b/libminifi/include/ProcessSessionFactory.h
deleted file mode 100644
index 7fc3410..0000000
--- a/libminifi/include/ProcessSessionFactory.h
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * @file ProcessSessionFactory.h
- * ProcessSessionFactory class declaration
- *
- * 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 __PROCESS_SESSION_FACTORY_H__
-#define __PROCESS_SESSION_FACTORY_H__
-
-#include <memory>
-
-#include "ProcessContext.h"
-#include "ProcessSession.h"
-
-//! ProcessSessionFactory Class
-class ProcessSessionFactory
-{
-public:
-	//! Constructor
-	/*!
-	 * Create a new process session factory
-	 */
-	explicit ProcessSessionFactory(ProcessContext *processContext) : _processContext(processContext) {}
-
-	//! Create the session
-	std::unique_ptr<ProcessSession> createSession();
-
-	// Prevent default copy constructor and assignment operation
-	// Only support pass by reference or pointer
-	ProcessSessionFactory(const ProcessSessionFactory &parent) = delete;
-	ProcessSessionFactory &operator=(const ProcessSessionFactory &parent) = delete;
-
-private:
-	//! ProcessContext
-	ProcessContext *_processContext;
-
-};
-
-#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/Processor.h
----------------------------------------------------------------------
diff --git a/libminifi/include/Processor.h b/libminifi/include/Processor.h
deleted file mode 100644
index 1634bc0..0000000
--- a/libminifi/include/Processor.h
+++ /dev/null
@@ -1,365 +0,0 @@
-/**
- * @file Processor.h
- * Processor class declaration
- *
- * 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 __PROCESSOR_H__
-#define __PROCESSOR_H__
-
-#include <uuid/uuid.h>
-#include <vector>
-#include <queue>
-#include <map>
-#include <mutex>
-#include <memory>
-#include <condition_variable>
-#include <atomic>
-#include <algorithm>
-#include <set>
-#include <chrono>
-#include <functional>
-
-#include "utils/TimeUtil.h"
-#include "Property.h"
-#include "Relationship.h"
-#include "Connection.h"
-
-//! Forwarder declaration
-class ProcessContext;
-class ProcessSession;
-class ProcessSessionFactory;
-
-//! Minimum scheduling period in Nano Second
-#define MINIMUM_SCHEDULING_NANOS 30000
-
-//! Default yield period in second
-#define DEFAULT_YIELD_PERIOD_SECONDS 1
-
-//! Default penalization period in second
-#define DEFAULT_PENALIZATION_PERIOD_SECONDS 30
-
-/*!
- * Indicates the valid values for the state of a entity
- * with respect to scheduling the entity to run.
- */
-enum ScheduledState {
-
-    /**
-     * Entity cannot be scheduled to run
-     */
-    DISABLED,
-    /**
-     * Entity can be scheduled to run but currently is not
-     */
-    STOPPED,
-    /**
-     * Entity is currently scheduled to run
-     */
-    RUNNING
-};
-
-/*!
- * Scheduling Strategy
- */
-enum SchedulingStrategy {
-	//! Event driven
-	EVENT_DRIVEN,
-	//! Timer driven
-	TIMER_DRIVEN,
-	//! Cron Driven
-	CRON_DRIVEN
-};
-
-//! Processor Class
-class Processor
-{
-	friend class ProcessContext;
-public:
-	//! Constructor
-	/*!
-	 * Create a new processor
-	 */
-	Processor(std::string name, uuid_t uuid = NULL);
-	//! Destructor
-	virtual ~Processor();
-	//! Set Processor Name
-	void setName(std::string name) {
-		_name = name;
-	}
-	//! Get Process Name
-	std::string getName(void) {
-		return (_name);
-	}
-	//! Set UUID
-	void setUUID(uuid_t uuid) {
-		uuid_copy(_uuid, uuid);
-		char uuidStr[37];
-		uuid_unparse_lower(_uuid, uuidStr);
-		_uuidStr = uuidStr;
-	}
-	//! Get UUID
-	bool getUUID(uuid_t uuid) {
-		if (uuid)
-		{
-			uuid_copy(uuid, _uuid);
-			return true;
-		}
-		else
-		{
-			return false;
-		}
-	}
-	//! Set the supported processor properties while the process is not running
-	bool setSupportedProperties(std::set<Property> properties);
-	//! Set the supported relationships while the process is not running
-	bool setSupportedRelationships(std::set<Relationship> relationships);
-	//! Get the supported property value by name
-	bool getProperty(std::string name, std::string &value);
-	//! Set the supported property value by name wile the process is not running
-	bool setProperty(std::string name, std::string value);
-	//! Set the supported property value by using the property itself.
-	bool setProperty(Property prop, std::string value);
-	//! Whether the relationship is supported
-	bool isSupportedRelationship(Relationship relationship);
-	//! Set the auto terminated relationships while the process is not running
-	bool setAutoTerminatedRelationships(std::set<Relationship> relationships);
-	//! Check whether the relationship is auto terminated
-	bool isAutoTerminated(Relationship relationship);
-	//! Check whether the processor is running
-	bool isRunning();
-	//! Set Processor Scheduled State
-	void setScheduledState(ScheduledState state);
-	//! Get Processor Scheduled State
-	ScheduledState getScheduledState(void) {
-		return _state;
-	}
-	//! Set Processor Scheduling Strategy
-	void setSchedulingStrategy(SchedulingStrategy strategy) {
-		_strategy = strategy;
-	}
-	//! Get Processor Scheduling Strategy
-	SchedulingStrategy getSchedulingStrategy(void) {
-		return _strategy;
-	}
-	//! Set Processor Loss Tolerant
-	void setlossTolerant(bool lossTolerant) {
-		_lossTolerant = lossTolerant;
-	}
-	//! Get Processor Loss Tolerant
-	bool getlossTolerant(void) {
-		return _lossTolerant;
-	}
-	//! Set Processor Scheduling Period in Nano Second
-	void setSchedulingPeriodNano(uint64_t period) {
-		uint64_t minPeriod = MINIMUM_SCHEDULING_NANOS;
-		_schedulingPeriodNano = std::max(period, minPeriod);
-	}
-	//! Get Processor Scheduling Period in Nano Second
-	uint64_t getSchedulingPeriodNano(void) {
-		return _schedulingPeriodNano;
-	}
-	//! Set Processor Run Duration in Nano Second
-	void setRunDurationNano(uint64_t period) {
-		_runDurantionNano = period;
-	}
-	//! Get Processor Run Duration in Nano Second
-	uint64_t getRunDurationNano(void) {
-		return(_runDurantionNano);
-	}
-	//! Set Processor yield period in MilliSecond
-	void setYieldPeriodMsec(uint64_t period) {
-		_yieldPeriodMsec = period;
-	}
-	//! Get Processor yield period in MilliSecond
-	uint64_t getYieldPeriodMsec(void) {
-		return(_yieldPeriodMsec);
-	}
-	//! Set Processor penalization period in MilliSecond
-	void setPenalizationPeriodMsec(uint64_t period) {
-		_penalizationPeriodMsec = period;
-	}
-	//! Get Processor penalization period in MilliSecond
-	uint64_t getPenalizationPeriodMsec(void) {
-		return(_penalizationPeriodMsec);
-	}
-	//! Set Processor Maximum Concurrent Tasks
-	void setMaxConcurrentTasks(uint8_t tasks) {
-		_maxConcurrentTasks = tasks;
-	}
-	//! Get Processor Maximum Concurrent Tasks
-	uint8_t getMaxConcurrentTasks(void) {
-		return(_maxConcurrentTasks);
-	}
-	//! Set Trigger when empty
-	void setTriggerWhenEmpty(bool value) {
-		_triggerWhenEmpty = value;
-	}
-	//! Get Trigger when empty
-	bool getTriggerWhenEmpty(void) {
-		return(_triggerWhenEmpty);
-	}
-	//! Get Active Task Counts
-	uint8_t getActiveTasks(void) {
-		return(_activeTasks);
-	}
-	//! Increment Active Task Counts
-	void incrementActiveTasks(void) {
-		_activeTasks++;
-	}
-	//! decrement Active Task Counts
-	void decrementActiveTask(void) {
-		_activeTasks--;
-	}
-	void clearActiveTask(void) {
-		_activeTasks = 0;
-	}
-	//! Yield based on the yield period
-	void yield()
-	{
-		_yieldExpiration = (getTimeMillis() + _yieldPeriodMsec);
-	}
-	//! Yield based on the input time
-	void yield(uint64_t time)
-	{
-		_yieldExpiration = (getTimeMillis() + time);
-	}
-	//! whether need be to yield
-	bool isYield()
-	{
-		if (_yieldExpiration > 0)
-			return (_yieldExpiration >= getTimeMillis());
-		else
-			return false;
-	}
-	// clear yield expiration
-	void clearYield()
-	{
-		_yieldExpiration = 0;
-	}
-	// get yield time
-	uint64_t getYieldTime()
-	{
-		uint64_t curTime = getTimeMillis();
-		if (_yieldExpiration > curTime)
-			return (_yieldExpiration - curTime);
-		else
-			return 0;;
-	}
-	//! Whether flow file queued in incoming connection
-	bool flowFilesQueued();
-	//! Whether flow file queue full in any of the outgoin connection
-	bool flowFilesOutGoingFull();
-	//! Get incoming connections
-	std::set<Connection *> getIncomingConnections() {
-		return _incomingConnections;
-	}
-	//! Has Incoming Connection
-	bool hasIncomingConnections() {
-		return (_incomingConnections.size() > 0);
-	}
-	//! Get outgoing connections based on relationship name
-	std::set<Connection *> getOutGoingConnections(std::string relationship);
-	//! Add connection
-	bool addConnection(Connection *connection);
-	//! Remove connection
-	void removeConnection(Connection *connection);
-	//! Get the UUID as string
-	std::string getUUIDStr() {
-		return _uuidStr;
-	}
-	//! Get the Next RoundRobin incoming connection
-	Connection *getNextIncomingConnection();
-	//! On Trigger
-	void onTrigger(ProcessContext *context, ProcessSessionFactory *sessionFactory);
-	//! Block until work is available on any input connection, or the given duration elapses
-	void waitForWork(uint64_t timeoutMs);
-	//! Notify this processor that work may be available
-	void notifyWork();
-
-public:
-	//! OnTrigger method, implemented by NiFi Processor Designer
-	virtual void onTrigger(ProcessContext *context, ProcessSession *session) = 0;
-	//! Initialize, overridden by NiFi Process Designer
-	virtual void initialize() {}
-	//! Scheduled event hook, overridden by NiFi Process Designer
-	virtual void onSchedule(ProcessContext *context, ProcessSessionFactory *sessionFactory) {}
-
-protected:
-
-	//! A global unique identifier
-	uuid_t _uuid;
-	//! Processor Name
-	std::string _name;
-	//! Supported properties
-	std::map<std::string, Property> _properties;
-	//! Supported relationships
-	std::map<std::string, Relationship> _relationships;
-	//! Autoterminated relationships
-	std::map<std::string, Relationship> _autoTerminatedRelationships;
-	//! Processor state
-	std::atomic<ScheduledState> _state;
-	//! Scheduling Strategy
-	std::atomic<SchedulingStrategy> _strategy;
-	//! lossTolerant
-	std::atomic<bool> _lossTolerant;
-	//! SchedulePeriod in Nano Seconds
-	std::atomic<uint64_t> _schedulingPeriodNano;
-	//! Run Duration in Nano Seconds
-	std::atomic<uint64_t> _runDurantionNano;
-	//! Yield Period in Milliseconds
-	std::atomic<uint64_t> _yieldPeriodMsec;
-	//! Penalization Period in MilliSecond
-	std::atomic<uint64_t> _penalizationPeriodMsec;
-	//! Maximum Concurrent Tasks
-	std::atomic<uint8_t> _maxConcurrentTasks;
-	//! Active Tasks
-	std::atomic<uint8_t> _activeTasks;
-	//! Trigger the Processor even if the incoming connection is empty
-	std::atomic<bool> _triggerWhenEmpty;
-	//! Incoming connections
-	std::set<Connection *> _incomingConnections;
-	//! Outgoing connections map based on Relationship name
-	std::map<std::string, std::set<Connection *>> _outGoingConnections;
-	//! UUID string
-	std::string _uuidStr;
-
-private:
-
-	//! Mutex for protection
-	std::mutex _mtx;
-	//! Yield Expiration
-	std::atomic<uint64_t> _yieldExpiration;
-	//! Incoming connection Iterator
-	std::set<Connection *>::iterator _incomingConnectionsIter;
-	//! Condition for whether there is incoming work to do
-	std::atomic<bool> _hasWork;
-	//! Concurrent condition mutex for whether there is incoming work to do
-	std::mutex _workAvailableMtx;
-	//! Concurrent condition variable for whether there is incoming work to do
-	std::condition_variable _hasWorkCondition;
-	//! Check all incoming connections for work
-	bool isWorkAvailable();
-	//! Logger
-	std::shared_ptr<Logger> logger_;
-	// Prevent default copy constructor and assignment operation
-	// Only support pass by reference or pointer
-	Processor(const Processor &parent);
-	Processor &operator=(const Processor &parent);
-
-};
-
-#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/Property.h
----------------------------------------------------------------------
diff --git a/libminifi/include/Property.h b/libminifi/include/Property.h
deleted file mode 100644
index bf33b35..0000000
--- a/libminifi/include/Property.h
+++ /dev/null
@@ -1,259 +0,0 @@
-/**
- * @file Property.h
- * Processor Property class declaration
- *
- * 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 __PROPERTY_H__
-#define __PROPERTY_H__
-
-#include <algorithm>
-#include <sstream>
-#include <string>
-#include <vector>
-#include <queue>
-#include <map>
-#include <mutex>
-#include <atomic>
-#include <functional>
-#include <set>
-#include <stdlib.h>
-#include <math.h>
-
-//! Time Unit
-enum TimeUnit {
-	DAY, HOUR, MINUTE, SECOND, MILLISECOND, NANOSECOND
-};
-
-//! Property Class
-class Property {
-
-public:
-	//! Constructor
-	/*!
-	 * Create a new property
-	 */
-	Property(const std::string name, const std::string description,
-			const std::string value) :
-			_name(name), _description(description), _value(value) {
-	}
-	Property() {
-	}
-	//! Destructor
-	virtual ~Property() {
-	}
-	//! Get Name for the property
-	std::string getName() {
-		return _name;
-	}
-	//! Get Description for the property
-	std::string getDescription() {
-		return _description;
-	}
-	//! Get value for the property
-	std::string getValue() {
-		return _value;
-	}
-	//! Set value for the property
-	void setValue(std::string value) {
-		_value = value;
-	}
-	//! Compare
-	bool operator <(const Property & right) const {
-		return _name < right._name;
-	}
-
-	//! Convert TimeUnit to MilliSecond
-	static bool ConvertTimeUnitToMS(int64_t input, TimeUnit unit,
-			int64_t &out) {
-		if (unit == MILLISECOND) {
-			out = input;
-			return true;
-		} else if (unit == SECOND) {
-			out = input * 1000;
-			return true;
-		} else if (unit == MINUTE) {
-			out = input * 60 * 1000;
-			return true;
-		} else if (unit == HOUR) {
-			out = input * 60 * 60 * 1000;
-			return true;
-		} else if (unit == DAY) {
-			out = 24 * 60 * 60 * 1000;
-			return true;
-		} else if (unit == NANOSECOND) {
-			out = input / 1000 / 1000;
-			return true;
-		} else {
-			return false;
-		}
-	}
-	//! Convert TimeUnit to NanoSecond
-	static bool ConvertTimeUnitToNS(int64_t input, TimeUnit unit,
-			int64_t &out) {
-		if (unit == MILLISECOND) {
-			out = input * 1000 * 1000;
-			return true;
-		} else if (unit == SECOND) {
-			out = input * 1000 * 1000 * 1000;
-			return true;
-		} else if (unit == MINUTE) {
-			out = input * 60 * 1000 * 1000 * 1000;
-			return true;
-		} else if (unit == HOUR) {
-			out = input * 60 * 60 * 1000 * 1000 * 1000;
-			return true;
-		} else if (unit == NANOSECOND) {
-			out = input;
-			return true;
-		} else {
-			return false;
-		}
-	}
-	//! Convert String
-	static bool StringToTime(std::string input, int64_t &output,
-			TimeUnit &timeunit) {
-		if (input.size() == 0) {
-			return false;
-		}
-
-		const char *cvalue = input.c_str();
-		char *pEnd;
-		long int ival = strtol(cvalue, &pEnd, 0);
-
-		if (pEnd[0] == '\0') {
-			return false;
-		}
-
-		while (*pEnd == ' ') {
-			// Skip the space
-			pEnd++;
-		}
-
-		std::string unit(pEnd);
-
-		if (unit == "sec" || unit == "s" || unit == "second"
-				|| unit == "seconds" || unit == "secs") {
-			timeunit = SECOND;
-			output = ival;
-			return true;
-		} else if (unit == "min" || unit == "m" || unit == "mins"
-				|| unit == "minute" || unit == "minutes") {
-			timeunit = MINUTE;
-			output = ival;
-			return true;
-		} else if (unit == "ns" || unit == "nano" || unit == "nanos"
-				|| unit == "nanoseconds") {
-			timeunit = NANOSECOND;
-			output = ival;
-			return true;
-		} else if (unit == "ms" || unit == "milli" || unit == "millis"
-				|| unit == "milliseconds") {
-			timeunit = MILLISECOND;
-			output = ival;
-			return true;
-		} else if (unit == "h" || unit == "hr" || unit == "hour"
-				|| unit == "hrs" || unit == "hours") {
-			timeunit = HOUR;
-			output = ival;
-			return true;
-		} else if (unit == "d" || unit == "day" || unit == "days") {
-			timeunit = DAY;
-			output = ival;
-			return true;
-		} else
-			return false;
-	}
-
-	//! Convert String to Integer
-	static bool StringToInt(std::string input, int64_t &output) {
-		if (input.size() == 0) {
-			return false;
-		}
-
-		const char *cvalue = input.c_str();
-		char *pEnd;
-		long int ival = strtol(cvalue, &pEnd, 0);
-
-		if (pEnd[0] == '\0') {
-			output = ival;
-			return true;
-		}
-
-		while (*pEnd == ' ') {
-			// Skip the space
-			pEnd++;
-		}
-
-		char end0 = toupper(pEnd[0]);
-		if ((end0 == 'K') || (end0 == 'M') || (end0 == 'G') || (end0 == 'T')
-				|| (end0 == 'P')) {
-			if (pEnd[1] == '\0') {
-				unsigned long int multiplier = 1000;
-
-				if ((end0 != 'K')) {
-					multiplier *= 1000;
-					if (end0 != 'M') {
-						multiplier *= 1000;
-						if (end0 != 'G') {
-							multiplier *= 1000;
-							if (end0 != 'T') {
-								multiplier *= 1000;
-							}
-						}
-					}
-				}
-				output = ival * multiplier;
-				return true;
-
-			} else if ((pEnd[1] == 'b' || pEnd[1] == 'B')
-					&& (pEnd[2] == '\0')) {
-
-				unsigned long int multiplier = 1024;
-
-				if ((end0 != 'K')) {
-					multiplier *= 1024;
-					if (end0 != 'M') {
-						multiplier *= 1024;
-						if (end0 != 'G') {
-							multiplier *= 1024;
-							if (end0 != 'T') {
-								multiplier *= 1024;
-							}
-						}
-					}
-				}
-				output = ival * multiplier;
-				return true;
-			}
-		}
-
-		return false;
-	}
-
-protected:
-	//! Name
-	std::string _name;
-	//! Description
-	std::string _description;
-	//! Value
-	std::string _value;
-
-private:
-
-};
-
-#endif


[14/16] nifi-minifi-cpp git commit: MINIFI-217: Updates namespaces and removes use of raw pointers for user facing API.

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/Provenance.h
----------------------------------------------------------------------
diff --git a/libminifi/include/Provenance.h b/libminifi/include/Provenance.h
deleted file mode 100644
index 3ba9792..0000000
--- a/libminifi/include/Provenance.h
+++ /dev/null
@@ -1,604 +0,0 @@
-/**
- * @file Provenance.h
- * Flow file record class declaration
- *
- * 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 __PROVENANCE_H__
-#define __PROVENANCE_H__
-
-#include <ftw.h>
-#include <uuid/uuid.h>
-#include <atomic>
-#include <cstdint>
-#include <cstring>
-#include <iostream>
-#include <map>
-#include <set>
-#include <string>
-#include <thread>
-#include <vector>
-
-#include "Configure.h"
-#include "Connection.h"
-#include "FlowFileRecord.h"
-#include "Logger.h"
-#include "Property.h"
-#include "ResourceClaim.h"
-#include "io/Serializable.h"
-#include "utils/TimeUtil.h"
-#include "Repository.h"
-
-class ProvenanceRepository;
-
-//! Provenance Event Record
-class ProvenanceEventRecord : protected Serializable
-{
-public:
-	enum ProvenanceEventType {
-
-	    /**
-	     * A CREATE event is used when a FlowFile is generated from data that was
-	     * not received from a remote system or external process
-	     */
-	    CREATE,
-
-	    /**
-	     * Indicates a provenance event for receiving data from an external process. This Event Type
-	     * is expected to be the first event for a FlowFile. As such, a Processor that receives data
-	     * from an external source and uses that data to replace the content of an existing FlowFile
-	     * should use the {@link #FETCH} event type, rather than the RECEIVE event type.
-	     */
-	    RECEIVE,
-
-	    /**
-	     * Indicates that the contents of a FlowFile were overwritten using the contents of some
-	     * external resource. This is similar to the {@link #RECEIVE} event but varies in that
-	     * RECEIVE events are intended to be used as the event that introduces the FlowFile into
-	     * the system, whereas FETCH is used to indicate that the contents of an existing FlowFile
-	     * were overwritten.
-	     */
-	    FETCH,
-
-	    /**
-	     * Indicates a provenance event for sending data to an external process
-	     */
-	    SEND,
-
-	    /**
-	     * Indicates that the contents of a FlowFile were downloaded by a user or external entity.
-	     */
-	    DOWNLOAD,
-
-	    /**
-	     * Indicates a provenance event for the conclusion of an object's life for
-	     * some reason other than object expiration
-	     */
-	    DROP,
-
-	    /**
-	     * Indicates a provenance event for the conclusion of an object's life due
-	     * to the fact that the object could not be processed in a timely manner
-	     */
-	    EXPIRE,
-
-	    /**
-	     * FORK is used to indicate that one or more FlowFile was derived from a
-	     * parent FlowFile.
-	     */
-	    FORK,
-
-	    /**
-	     * JOIN is used to indicate that a single FlowFile is derived from joining
-	     * together multiple parent FlowFiles.
-	     */
-	    JOIN,
-
-	    /**
-	     * CLONE is used to indicate that a FlowFile is an exact duplicate of its
-	     * parent FlowFile.
-	     */
-	    CLONE,
-
-	    /**
-	     * CONTENT_MODIFIED is used to indicate that a FlowFile's content was
-	     * modified in some way. When using this Event Type, it is advisable to
-	     * provide details about how the content is modified.
-	     */
-	    CONTENT_MODIFIED,
-
-	    /**
-	     * ATTRIBUTES_MODIFIED is used to indicate that a FlowFile's attributes were
-	     * modified in some way. This event is not needed when another event is
-	     * reported at the same time, as the other event will already contain all
-	     * FlowFile attributes.
-	     */
-	    ATTRIBUTES_MODIFIED,
-
-	    /**
-	     * ROUTE is used to show that a FlowFile was routed to a specified
-	     * {@link org.apache.nifi.processor.Relationship Relationship} and should provide
-	     * information about why the FlowFile was routed to this relationship.
-	     */
-	    ROUTE,
-
-	    /**
-	     * Indicates a provenance event for adding additional information such as a
-	     * new linkage to a new URI or UUID
-	     */
-	    ADDINFO,
-
-	    /**
-	     * Indicates a provenance event for replaying a FlowFile. The UUID of the
-	     * event will indicate the UUID of the original FlowFile that is being
-	     * replayed. The event will contain exactly one Parent UUID that is also the
-	     * UUID of the FlowFile that is being replayed and exactly one Child UUID
-	     * that is the UUID of the a newly created FlowFile that will be re-queued
-	     * for processing.
-	     */
-	    REPLAY
-	};
-	friend class ProcessSession;
-public:
-	//! Constructor
-	/*!
-	 * Create a new provenance event record
-	 */
-	ProvenanceEventRecord(ProvenanceEventType event, std::string componentId, std::string componentType) {
-		_eventType = event;
-		_componentId = componentId;
-		_componentType = componentType;
-		_eventTime = getTimeMillis();
-		char eventIdStr[37];
-		// Generate the global UUID for th event
-		uuid_generate(_eventId);
-		uuid_unparse_lower(_eventId, eventIdStr);
-		_eventIdStr = eventIdStr;
-		logger_ = Logger::getLogger();
-	}
-
-	ProvenanceEventRecord() {
-			_eventTime = getTimeMillis();
-			logger_ = Logger::getLogger();
-	}
-
-	//! Destructor
-	virtual ~ProvenanceEventRecord() {
-	}
-	//! Get the Event ID
-	std::string getEventId() {
-		return _eventIdStr;
-	}
-	//! Get Attributes
-	std::map<std::string, std::string> getAttributes() {
-		return _attributes;
-	}
-	//! Get Size
-	uint64_t getFileSize() {
-		return _size;
-	}
-	// ! Get Offset
-	uint64_t getFileOffset() {
-		return _offset;
-	}
-	// ! Get Entry Date
-	uint64_t getFlowFileEntryDate() {
-		return _entryDate;
-	}
-	// ! Get Lineage Start Date
-	uint64_t getlineageStartDate() {
-		return _lineageStartDate;
-	}
-	// ! Get Event Time
-	uint64_t getEventTime() {
-		return _eventTime;
-	}
-	// ! Get Event Duration
-	uint64_t getEventDuration() {
-		return _eventDuration;
-	}
-	//! Set Event Duration
-	void setEventDuration(uint64_t duration)
-	{
-		_eventDuration = duration;
-	}
-	// ! Get Event Type
-	ProvenanceEventType getEventType() {
-		return _eventType;
-	}
-	//! Get Component ID
-	std::string getComponentId()
-	{
-		return _componentId;
-	}
-	//! Get Component Type
-	std::string getComponentType()
-	{
-		return _componentType;
-	}
-	//! Get FlowFileUuid
-	std::string getFlowFileUuid()
-	{
-		return _uuid;
-	}
-	//! Get content full path
-	std::string getContentFullPath()
-	{
-		return _contentFullPath;
-	}
-	//! Get LineageIdentifiers
-	std::set<std::string> getLineageIdentifiers()
-	{
-		return _lineageIdentifiers;
-	}
-	//! Get Details
-	std::string getDetails()
-	{
-		return _details;
-	}
-	//! Set Details
-	void setDetails(std::string details)
-	{
-		_details = details;
-	}
-	//! Get TransitUri
-	std::string getTransitUri()
-	{
-		return _transitUri;
-	}
-	//! Set TransitUri
-	void setTransitUri(std::string uri)
-	{
-		_transitUri = uri;
-	}
-	//! Get SourceSystemFlowFileIdentifier
-	std::string getSourceSystemFlowFileIdentifier()
-	{
-		return _sourceSystemFlowFileIdentifier;
-	}
-	//! Set SourceSystemFlowFileIdentifier
-	void setSourceSystemFlowFileIdentifier(std::string identifier)
-	{
-		_sourceSystemFlowFileIdentifier = identifier;
-	}
-	//! Get Parent UUIDs
-	std::vector<std::string> getParentUuids()
-	{
-		return _parentUuids;
-	}
-	//! Add Parent UUID
-	void addParentUuid(std::string uuid)
-	{
-		if (std::find(_parentUuids.begin(), _parentUuids.end(), uuid) != _parentUuids.end())
-			return;
-		else
-			_parentUuids.push_back(uuid);
-	}
-	//! Add Parent Flow File
-	void addParentFlowFile(FlowFileRecord *flow)
-	{
-		addParentUuid(flow->getUUIDStr());
-		return;
-	}
-	//! Remove Parent UUID
-	void removeParentUuid(std::string uuid)
-	{
-		_parentUuids.erase(std::remove(_parentUuids.begin(), _parentUuids.end(), uuid), _parentUuids.end());
-	}
-	//! Remove Parent Flow File
-	void removeParentFlowFile(FlowFileRecord *flow)
-	{
-		removeParentUuid(flow->getUUIDStr());
-		return;
-	}
-	//! Get Children UUIDs
-	std::vector<std::string> getChildrenUuids()
-	{
-		return _childrenUuids;
-	}
-	//! Add Child UUID
-	void addChildUuid(std::string uuid)
-	{
-		if (std::find(_childrenUuids.begin(), _childrenUuids.end(), uuid) != _childrenUuids.end())
-			return;
-		else
-			_childrenUuids.push_back(uuid);
-	}
-	//! Add Child Flow File
-	void addChildFlowFile(FlowFileRecord *flow)
-	{
-		addChildUuid(flow->getUUIDStr());
-		return;
-	}
-	//! Remove Child UUID
-	void removeChildUuid(std::string uuid)
-	{
-		_childrenUuids.erase(std::remove(_childrenUuids.begin(), _childrenUuids.end(), uuid), _childrenUuids.end());
-	}
-	//! Remove Child Flow File
-	void removeChildFlowFile(FlowFileRecord *flow)
-	{
-		removeChildUuid(flow->getUUIDStr());
-		return;
-	}
-	//! Get AlternateIdentifierUri
-	std::string getAlternateIdentifierUri()
-	{
-		return _alternateIdentifierUri;
-	}
-	//! Set AlternateIdentifierUri
-	void setAlternateIdentifierUri(std::string uri)
-	{
-		_alternateIdentifierUri = uri;
-	}
-	//! Get Relationship
-	std::string getRelationship()
-	{
-		return _relationship;
-	}
-	//! Set Relationship
-	void setRelationship(std::string relation)
-	{
-		_relationship = relation;
-	}
-	//! Get sourceQueueIdentifier
-	std::string getSourceQueueIdentifier()
-	{
-		return _sourceQueueIdentifier;
-	}
-	//! Set sourceQueueIdentifier
-	void setSourceQueueIdentifier(std::string identifier)
-	{
-		_sourceQueueIdentifier = identifier;
-	}
-	//! fromFlowFile
-	void fromFlowFile(FlowFileRecord *flow)
-	{
-		_entryDate = flow->getEntryDate();
-		_lineageStartDate = flow->getlineageStartDate();
-		_lineageIdentifiers = flow->getlineageIdentifiers();
-		_uuid = flow->getUUIDStr();
-		_attributes = flow->getAttributes();
-		_size = flow->getSize();
-		_offset = flow->getOffset();
-		if (flow->getOriginalConnection())
-			_sourceQueueIdentifier = flow->getOriginalConnection()->getName();
-		if (flow->getResourceClaim())
-		{
-			_contentFullPath = flow->getResourceClaim()->getContentFullPath();
-		}
-	}
-	//! Serialize and Persistent to the repository
-	bool Serialize(ProvenanceRepository *repo);
-	//! DeSerialize
-	bool DeSerialize(const uint8_t *buffer, const int bufferSize);
-	//! DeSerialize
-	bool DeSerialize(DataStream &stream)
-	{
-		return DeSerialize(stream.getBuffer(),stream.getSize());
-	}
-	//! DeSerialize
-	bool DeSerialize(ProvenanceRepository *repo, std::string key);
-
-protected:
-
-	//! Event type
-	ProvenanceEventType _eventType;
-	//! Date at which the event was created
-	uint64_t _eventTime;
-	//! Date at which the flow file entered the flow
-	uint64_t _entryDate;
-	//! Date at which the origin of this flow file entered the flow
-	uint64_t _lineageStartDate;
-	//! Event Duration
-	uint64_t _eventDuration;
-	//! Component ID
-	std::string _componentId;
-	//! Component Type
-	std::string _componentType;
-	//! Size in bytes of the data corresponding to this flow file
-	uint64_t _size;
-	//! flow uuid
-	std::string _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
-	std::string _transitUri;
-	//! sourceSystemFlowFileIdentifier
-	std::string _sourceSystemFlowFileIdentifier;
-	//! parent UUID
-	std::vector<std::string> _parentUuids;
-	//! child UUID
-	std::vector<std::string> _childrenUuids;
-	//! detail
-	std::string _details;
-	//! sourceQueueIdentifier
-	std::string _sourceQueueIdentifier;
-	//! event ID Str
-	std::string _eventIdStr;
-	//! relationship
-	std::string _relationship;
-	//! alternateIdentifierUri;
-	std::string _alternateIdentifierUri;
-
-private:
-
-	//! Logger
-	std::shared_ptr<Logger> logger_;
-	
-	// Prevent default copy constructor and assignment operation
-	// Only support pass by reference or pointer
-	ProvenanceEventRecord(const ProvenanceEventRecord &parent);
-	ProvenanceEventRecord &operator=(const ProvenanceEventRecord &parent);
-
-};
-
-//! Provenance Reporter
-class ProvenanceReporter
-{
-	friend class ProcessSession;
-public:
-	//! Constructor
-	/*!
-	 * Create a new provenance reporter associated with the process session
-	 */
-	ProvenanceReporter(std::string componentId, std::string componentType) {
-		logger_ = Logger::getLogger();
-		_componentId = componentId;
-		_componentType = componentType;
-	}
-
-	//! Destructor
-	virtual ~ProvenanceReporter() {
-		clear();
-	}
-	//! Get events
-	std::set<ProvenanceEventRecord *> getEvents()
-	{
-		return _events;
-	}
-	//! Add event
-	void add(ProvenanceEventRecord *event)
-	{
-		_events.insert(event);
-	}
-	//! Remove event
-	void remove(ProvenanceEventRecord *event)
-	{
-		if (_events.find(event) != _events.end())
-		{
-			_events.erase(event);
-		}
-	}
-	//!
-	//! clear
-	void clear()
-	{
-		for (auto it : _events)
-		{
-			delete it;
-		}
-		_events.clear();
-	}
-	//! allocate
-	ProvenanceEventRecord *allocate(ProvenanceEventRecord::ProvenanceEventType eventType, FlowFileRecord *flow)
-	{
-		ProvenanceEventRecord *event = new ProvenanceEventRecord(eventType, _componentId, _componentType);
-		if (event)
-			event->fromFlowFile(flow);
-
-		return event;
-	}
-	//! commit
-	void commit();
-	//! create
-	void create(FlowFileRecord *flow, std::string detail);
-	//! route
-	void route(FlowFileRecord *flow, Relationship relation, std::string detail, uint64_t processingDuration);
-	//! modifyAttributes
-	void modifyAttributes(FlowFileRecord *flow, std::string detail);
-	//! modifyContent
-	void modifyContent(FlowFileRecord *flow, std::string detail, uint64_t processingDuration);
-	//! clone
-	void clone(FlowFileRecord *parent, FlowFileRecord *child);
-	//! join
-	void join(std::vector<FlowFileRecord *> parents, FlowFileRecord *child, std::string detail, uint64_t processingDuration);
-	//! fork
-	void fork(std::vector<FlowFileRecord *> child, FlowFileRecord *parent, std::string detail, uint64_t processingDuration);
-	//! expire
-	void expire(FlowFileRecord *flow, std::string detail);
-	//! drop
-	void drop(FlowFileRecord *flow, std::string reason);
-	//! send
-	void send(FlowFileRecord *flow, std::string transitUri, std::string detail, uint64_t processingDuration, bool force);
-	//! fetch
-	void fetch(FlowFileRecord *flow, std::string transitUri, std::string detail, uint64_t processingDuration);
-	//! receive
-	void receive(FlowFileRecord *flow, std::string transitUri, std::string sourceSystemFlowFileIdentifier, std::string detail, uint64_t processingDuration);
-
-protected:
-
-	//! Component ID
-	std::string _componentId;
-	//! Component Type
-	std::string _componentType;
-
-private:
-
-	//! Incoming connection Iterator
-	std::set<ProvenanceEventRecord *> _events;
-	//! Logger
-	std::shared_ptr<Logger> logger_;
-
-	// Prevent default copy constructor and assignment operation
-	// Only support pass by reference or pointer
-	ProvenanceReporter(const ProvenanceReporter &parent);
-	ProvenanceReporter &operator=(const ProvenanceReporter &parent);
-};
-
-#define PROVENANCE_DIRECTORY "./provenance_repository"
-#define MAX_PROVENANCE_STORAGE_SIZE (10*1024*1024) // 10M
-#define MAX_PROVENANCE_ENTRY_LIFE_TIME (60000) // 1 minute
-#define PROVENANCE_PURGE_PERIOD (2500) // 2500 msec
-
-//! Provenance Repository
-class ProvenanceRepository : public Repository
-{
-public:
-	//! Constructor
-	/*!
-	 * Create a new provenance repository
-	 */
-	ProvenanceRepository()
-	 : Repository(Repository::PROVENANCE, PROVENANCE_DIRECTORY,
-			MAX_PROVENANCE_ENTRY_LIFE_TIME, MAX_PROVENANCE_STORAGE_SIZE, PROVENANCE_PURGE_PERIOD)
-	{
-	}
-
-	//! Destructor
-	virtual ~ProvenanceRepository() {
-	}
-
-	//! Persistent event
-	void registerEvent(ProvenanceEventRecord *event)
-	{
-		event->Serialize(this);
-	}
-	//! Remove event
-	void removeEvent(ProvenanceEventRecord *event)
-	{
-		Delete(event->getEventId());
-	}
-
-protected:
-
-private:
-
-	// Prevent default copy constructor and assignment operation
-	// Only support pass by reference or pointer
-	ProvenanceRepository(const ProvenanceRepository &parent);
-	ProvenanceRepository &operator=(const ProvenanceRepository &parent);
-};
-
-#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/PutFile.h
----------------------------------------------------------------------
diff --git a/libminifi/include/PutFile.h b/libminifi/include/PutFile.h
deleted file mode 100644
index 015605e..0000000
--- a/libminifi/include/PutFile.h
+++ /dev/null
@@ -1,88 +0,0 @@
-/**
- * @file PutFile.h
- * PutFile class declaration
- *
- * 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 __PUT_FILE_H__
-#define __PUT_FILE_H__
-
-#include "FlowFileRecord.h"
-#include "Processor.h"
-#include "ProcessSession.h"
-
-//! PutFile Class
-class PutFile : public Processor
-{
-public:
-
-	static const std::string CONFLICT_RESOLUTION_STRATEGY_REPLACE;
-	static const std::string CONFLICT_RESOLUTION_STRATEGY_IGNORE;
-	static const std::string CONFLICT_RESOLUTION_STRATEGY_FAIL;
-
-	//! Constructor
-	/*!
-	 * Create a new processor
-	 */
-	PutFile(std::string name, uuid_t uuid = NULL)
-	: Processor(name, uuid)
-	{
-		logger_ = Logger::getLogger();
-	}
-	//! Destructor
-	virtual ~PutFile()
-	{
-	}
-	//! Processor Name
-	static const std::string ProcessorName;
-	//! Supported Properties
-	static Property Directory;
-	static Property ConflictResolution;
-	//! Supported Relationships
-	static Relationship Success;
-	static Relationship Failure;
-
-	//! OnTrigger method, implemented by NiFi PutFile
-	virtual void onTrigger(ProcessContext *context, ProcessSession *session);
-	//! Initialize, over write by NiFi PutFile
-	virtual void initialize(void);
-
-	class ReadCallback : public InputStreamCallback
-	{
-	public:
-		ReadCallback(const std::string &tmpFile, const std::string &destFile);
-		~ReadCallback();
-		virtual void process(std::ifstream *stream);
-		bool commit();
-
-	private:
-		std::shared_ptr<Logger> logger_;
-		std::ofstream _tmpFileOs;
-		bool _writeSucceeded = false;
-		std::string _tmpFile;
-		std::string _destFile;
-	};
-
-protected:
-
-private:
-	//! Logger
-	std::shared_ptr<Logger> logger_;
-
-	bool putFile(ProcessSession *session, FlowFileRecord *flowFile, const std::string &tmpFile, const std::string &destFile);
-};
-
-#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/RealTimeDataCollector.h
----------------------------------------------------------------------
diff --git a/libminifi/include/RealTimeDataCollector.h b/libminifi/include/RealTimeDataCollector.h
deleted file mode 100644
index 3b6d05f..0000000
--- a/libminifi/include/RealTimeDataCollector.h
+++ /dev/null
@@ -1,131 +0,0 @@
-/**
- * @file RealTimeDataCollector.h
- * RealTimeDataCollector class declaration
- *
- * 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 __REAL_TIME_DATA_COLLECTOR_H__
-#define __REAL_TIME_DATA_COLLECTOR_H__
-
-#include <stdio.h>
-#include <unistd.h>
-#include <sys/types.h>
-#include <sys/socket.h>
-#include <netinet/in.h>
-#include <arpa/inet.h>
-#include <fcntl.h>
-#include <netdb.h>
-#include <string>
-#include <errno.h>
-#include "FlowFileRecord.h"
-#include "Processor.h"
-#include "ProcessSession.h"
-
-//! RealTimeDataCollector Class
-class RealTimeDataCollector : public Processor
-{
-public:
-	//! Constructor
-	/*!
-	 * Create a new processor
-	 */
-	RealTimeDataCollector(std::string name, uuid_t uuid = NULL)
-	: Processor(name, uuid)
-	{
-		_realTimeSocket = 0;
-		_batchSocket = 0;
-		logger_ = Logger::getLogger();
-		_firstInvoking = false;
-		_realTimeAccumulated = 0;
-		_batchAcccumulated = 0;
-		_queuedDataSize = 0;
-	}
-	//! Destructor
-	virtual ~RealTimeDataCollector()
-	{
-		if (_realTimeSocket)
-			close(_realTimeSocket);
-		if (_batchSocket)
-			close(_batchSocket);
-		if (_fileStream.is_open())
-			_fileStream.close();
-	}
-	//! Processor Name
-	static const std::string ProcessorName;
-	//! Supported Properties
-	static Property REALTIMESERVERNAME;
-	static Property REALTIMESERVERPORT;
-	static Property BATCHSERVERNAME;
-	static Property BATCHSERVERPORT;
-	static Property FILENAME;
-	static Property ITERATION;
-	static Property REALTIMEMSGID;
-	static Property BATCHMSGID;
-	static Property REALTIMEINTERVAL;
-	static Property BATCHINTERVAL;
-	static Property BATCHMAXBUFFERSIZE;
-	//! Supported Relationships
-	static Relationship Success;
-	//! Connect to the socket
-	int connectServer(const char *host, uint16_t port);
-	int sendData(int socket, const char *buf, int buflen);
-	void onTriggerRealTime(ProcessContext *context, ProcessSession *session);
-	void onTriggerBatch(ProcessContext *context, ProcessSession *session);
-
-public:
-	//! OnTrigger method, implemented by NiFi RealTimeDataCollector
-	virtual void onTrigger(ProcessContext *context, ProcessSession *session);
-	//! Initialize, over write by NiFi RealTimeDataCollector
-	virtual void initialize(void);
-
-protected:
-
-private:
-	//! realtime server Name
-	std::string _realTimeServerName;
-	int64_t _realTimeServerPort;
-	std::string _batchServerName;
-	int64_t _batchServerPort;
-	int64_t _realTimeInterval;
-	int64_t _batchInterval;
-	int64_t _batchMaxBufferSize;
-	//! Match pattern for Real time Message ID
-	std::vector<std::string> _realTimeMsgID;
-	//! Match pattern for Batch Message ID
-	std::vector<std::string> _batchMsgID;
-	//! file for which the realTime collector will tail
-	std::string _fileName;
-	//! Whether we need to iterate from the beginning for demo
-	bool _iteration;
-	int _realTimeSocket;
-	int _batchSocket;
-	//! Logger
-	std::shared_ptr<Logger> logger_;
-	//! Mutex for protection
-	std::mutex _mtx;
-	//! Queued data size
-	uint64_t _queuedDataSize;
-	//! Queue for the batch process
-	std::queue<std::string> _queue;
-	std::thread::id _realTimeThreadId;
-	std::thread::id _batchThreadId;
-	std::atomic<bool> _firstInvoking;
-	int64_t _realTimeAccumulated;
-	int64_t _batchAcccumulated;
-	std::ifstream _fileStream;
-};
-
-#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/Relationship.h
----------------------------------------------------------------------
diff --git a/libminifi/include/Relationship.h b/libminifi/include/Relationship.h
deleted file mode 100644
index 3454ee5..0000000
--- a/libminifi/include/Relationship.h
+++ /dev/null
@@ -1,87 +0,0 @@
-/**
- * @file Relationship.h
- * Relationship class declaration
- *
- * 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 __RELATIONSHIP_H__
-#define __RELATIONSHIP_H__
-
-#include <string>
-#include <uuid/uuid.h>
-#include <vector>
-#include <queue>
-#include <map>
-#include <mutex>
-#include <atomic>
-
-//! undefined relationship for remote process group outgoing port and root process group incoming port
-#define UNDEFINED_RELATIONSHIP "undefined"
-
-inline bool isRelationshipNameUndefined(std::string name)
-{
-	if (name == UNDEFINED_RELATIONSHIP)
-		return true;
-	else
-		return false;
-}
-
-//! Relationship Class
-class Relationship {
-
-public:
-	//! Constructor
-	/*!
-	 * Create a new relationship 
-	 */
-	Relationship(const std::string name, const std::string description)
-		: _name(name), _description(description) {
-	}
-	Relationship()
-		: _name(UNDEFINED_RELATIONSHIP) {
-	}
-	//! Destructor
-	virtual ~Relationship() {
-	}
-	//! Get Name for the relationship
-	std::string getName() {
-		return _name;
-	}
-	//! Get Description for the relationship
-	std::string getDescription() {
-		return _description;
-	}
-	//! Compare
-	bool operator < (const Relationship & right) const {
-		return _name < right._name;
-	}
-	//! Whether it is a undefined relationship
-	bool isRelationshipUndefined()
-	{
-		return isRelationshipNameUndefined(_name);
-	}
-
-protected:
-
-	//! Name
-	std::string _name;
-	//! Description
-	std::string _description;
-
-private:
-};
-
-#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/RemoteProcessorGroupPort.h
----------------------------------------------------------------------
diff --git a/libminifi/include/RemoteProcessorGroupPort.h b/libminifi/include/RemoteProcessorGroupPort.h
index 05ecd17..e9a4228 100644
--- a/libminifi/include/RemoteProcessorGroupPort.h
+++ b/libminifi/include/RemoteProcessorGroupPort.h
@@ -20,71 +20,91 @@
 #ifndef __REMOTE_PROCESSOR_GROUP_PORT_H__
 #define __REMOTE_PROCESSOR_GROUP_PORT_H__
 
+#include <mutex>
 #include <memory>
+#include <stack>
 #include "FlowFileRecord.h"
-#include "Processor.h"
-#include "ProcessSession.h"
+#include "core/Processor.h"
+#include "core/ProcessSession.h"
 #include "Site2SiteClientProtocol.h"
 
-//! RemoteProcessorGroupPort Class
-class RemoteProcessorGroupPort: public Processor {
-public:
-	//! Constructor
-	/*!
-	 * Create a new processor
-	 */
-	RemoteProcessorGroupPort(std::string name, uuid_t uuid = NULL) :
-			Processor(name, uuid), direction_(SEND), transmitting_(false), peer_() {
-		logger_ = Logger::getLogger();
-		protocol_ = std::unique_ptr<Site2SiteClientProtocol>(
-				new Site2SiteClientProtocol(0));
-		protocol_->setPortId(uuid);
-	}
-	//! Destructor
-	virtual ~RemoteProcessorGroupPort() {
 
-	}
-	//! Processor Name
-	static const std::string ProcessorName;
-	//! Supported Properties
-	static Property hostName;
-	static Property port;
-	//! Supported Relationships
-	static Relationship relation;
-public:
-	//! OnTrigger method, implemented by NiFi RemoteProcessorGroupPort
-	virtual void onTrigger(ProcessContext *context, ProcessSession *session);
-	//! Initialize, over write by NiFi RemoteProcessorGroupPort
-	virtual void initialize(void);
-	//! Set Direction
-	void setDirection(TransferDirection direction) {
-		direction_ = direction;
-		if (direction_ == RECEIVE)
-			this->setTriggerWhenEmpty(true);
-	}
-	//! Set Timeout
-	void setTimeOut(uint64_t timeout) {
-		protocol_->setTimeOut(timeout);
-	}
-	//! SetTransmitting
-	void setTransmitting(bool val) {
-		transmitting_ = val;
-	}
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+// RemoteProcessorGroupPort Class
+class RemoteProcessorGroupPort :
+    public core::Processor {
+ public:
+  // Constructor
+  /*!
+   * Create a new processor
+   */
+  RemoteProcessorGroupPort(std::string name, uuid_t uuid = NULL)
+      : core::Processor(name, uuid),
+        direction_(SEND),
+        transmitting_(false){
+    logger_ = logging::Logger::getLogger();
+    uuid_copy(protocol_uuid_,uuid);
+  }
+  // Destructor
+  virtual ~RemoteProcessorGroupPort() {
 
-protected:
+  }
+  // Processor Name
+  static const std::string ProcessorName;
+  // Supported Properties
+  static core::Property hostName;
+  static core::Property port;
+  // Supported Relationships
+  static core::Relationship relation;
+ public:
+  // OnTrigger method, implemented by NiFi RemoteProcessorGroupPort
+  virtual void onTrigger(
+      core::ProcessContext *context,
+      core::ProcessSession *session);
+  // Initialize, over write by NiFi RemoteProcessorGroupPort
+  virtual void initialize(void);
+  // Set Direction
+  void setDirection(TransferDirection direction) {
+    direction_ = direction;
+    if (direction_ == RECEIVE)
+      this->setTriggerWhenEmpty(true);
+  }
+  // Set Timeout
+  void setTimeOut(uint64_t timeout) {
+    timeout_ = timeout;
+  }
+  // SetTransmitting
+  void setTransmitting(bool val) {
+    transmitting_ = val;
+  }
 
-private:
-	//! Logger
-	std::shared_ptr<Logger> logger_;
-	//! Peer Connection
-	Site2SitePeer peer_;
-	//! Peer Protocol
-	std::unique_ptr<Site2SiteClientProtocol> protocol_;
-	//! Transaction Direction
-	TransferDirection direction_;
-	//! Transmitting
-	bool transmitting_;
+ protected:
 
+ private:
+   
+  std::unique_ptr<Site2SiteClientProtocol> getNextProtocol();
+  void returnProtocol(std::unique_ptr<Site2SiteClientProtocol> protocol);
+   
+  std::stack<std::unique_ptr<Site2SiteClientProtocol>> available_protocols_;
+  std::mutex protocol_mutex_;
+  // Logger
+  std::shared_ptr<logging::Logger> logger_;
+  // Transaction Direction
+  TransferDirection direction_;
+  // Transmitting
+  bool transmitting_;
+  // timeout
+  uint64_t timeout_;
+  
+  uuid_t protocol_uuid_;
+ 
 };
 
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
 #endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/Repository.h
----------------------------------------------------------------------
diff --git a/libminifi/include/Repository.h b/libminifi/include/Repository.h
deleted file mode 100644
index 55fb442..0000000
--- a/libminifi/include/Repository.h
+++ /dev/null
@@ -1,318 +0,0 @@
-/**
- * @file Repository 
- * Repository class declaration
- *
- * 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 __REPOSITORY_H__
-#define __REPOSITORY_H__
-
-#include <ftw.h>
-#include <uuid/uuid.h>
-#include <atomic>
-#include <cstdint>
-#include <cstring>
-#include <iostream>
-#include <map>
-#include <set>
-#include <string>
-#include <thread>
-#include <vector>
-
-#ifdef LEVELDB_SUPPORT
-#include "leveldb/db.h"
-#include "leveldb/options.h"
-#include "leveldb/slice.h"
-#include "leveldb/status.h"
-#endif
-#include "Configure.h"
-#include "Connection.h"
-#include "FlowFileRecord.h"
-#include "Logger.h"
-#include "Property.h"
-#include "ResourceClaim.h"
-#include "io/Serializable.h"
-#include "utils/TimeUtil.h"
-#include "utils/StringUtils.h"
-
-//! Repository
-class Repository
-{
-public:
-	enum RepositoryType {
-		//! Provenance Repo Type
-		PROVENANCE,
-		//! FlowFile Repo Type
-		FLOWFILE,
-		MAX_REPO_TYPE
-	};
-	static const char *RepositoryTypeStr[MAX_REPO_TYPE];
-	//! Constructor
-	/*!
-	 * Create a new provenance repository
-	 */
-	Repository(RepositoryType type, std::string directory, 
-		int64_t maxPartitionMillis, int64_t maxPartitionBytes, uint64_t purgePeriod) {
-		_type = type;
-		_directory = directory;
-		_maxPartitionMillis = maxPartitionMillis;
-		_maxPartitionBytes = maxPartitionBytes;
-		_purgePeriod = purgePeriod;
-		logger_ = Logger::getLogger();
-		configure_ = Configure::getConfigure();
-#ifdef LEVELDB_SUPPORT
-		_db = NULL;
-#endif
-		_thread = NULL;
-		_running = false;
-		_repoFull = false;
-		_enable = true;
-	}
-
-	//! Destructor
-	virtual ~Repository() {
-		stop();
-		if (this->_thread)
-			delete this->_thread;
-		destroy();
-	}
-
-	//! initialize
-	virtual bool initialize()
-	{
-		std::string value;
-
-#ifdef LEVELDB_SUPPORT
-		if (_type == PROVENANCE)
-		{
-			if (!(configure_->get(Configure::nifi_provenance_repository_enable, value)
-					&& StringUtils::StringToBool(value, _enable))) {
-				_enable = true;
-			}
-			if (!_enable)
-				return false;
-			if (configure_->get(Configure::nifi_provenance_repository_directory_default, value))
-			{
-				_directory = value;
-			}
-			logger_->log_info("NiFi Provenance Repository Directory %s", _directory.c_str());
-			if (configure_->get(Configure::nifi_provenance_repository_max_storage_size, value))
-			{
-				Property::StringToInt(value, _maxPartitionBytes);
-			}
-			logger_->log_info("NiFi Provenance Max Partition Bytes %d", _maxPartitionBytes);
-			if (configure_->get(Configure::nifi_provenance_repository_max_storage_time, value))
-			{
-				TimeUnit unit;
-				if (Property::StringToTime(value, _maxPartitionMillis, unit) &&
-							Property::ConvertTimeUnitToMS(_maxPartitionMillis, unit, _maxPartitionMillis))
-				{
-				}
-			}
-			logger_->log_info("NiFi Provenance Max Storage Time: [%d] ms", _maxPartitionMillis);
-			leveldb::Options options;
-			options.create_if_missing = true;
-			leveldb::Status status = leveldb::DB::Open(options, _directory.c_str(), &_db);
-			if (status.ok())
-			{
-				logger_->log_info("NiFi Provenance Repository database open %s success", _directory.c_str());
-			}
-			else
-			{
-				logger_->log_error("NiFi Provenance Repository database open %s fail", _directory.c_str());
-				return false;
-			}
-		}
-
-		if (_type == FLOWFILE)
-		{
-			if (!(configure_->get(Configure::nifi_flowfile_repository_enable, value)
-					&& StringUtils::StringToBool(value, _enable))) {
-				_enable = true;
-			}
-			if (!_enable)
-				return false;
-			if (configure_->get(Configure::nifi_flowfile_repository_directory_default, value))
-			{
-				_directory = value;
-			}
-			logger_->log_info("NiFi FlowFile Repository Directory %s", _directory.c_str());
-			if (configure_->get(Configure::nifi_flowfile_repository_max_storage_size, value))
-			{
-				Property::StringToInt(value, _maxPartitionBytes);
-			}
-			logger_->log_info("NiFi FlowFile Max Partition Bytes %d", _maxPartitionBytes);
-			if (configure_->get(Configure::nifi_flowfile_repository_max_storage_time, value))
-			{
-				TimeUnit unit;
-				if (Property::StringToTime(value, _maxPartitionMillis, unit) &&
-							Property::ConvertTimeUnitToMS(_maxPartitionMillis, unit, _maxPartitionMillis))
-				{
-				}
-			}
-			logger_->log_info("NiFi FlowFile Max Storage Time: [%d] ms", _maxPartitionMillis);
-			leveldb::Options options;
-			options.create_if_missing = true;
-			leveldb::Status status = leveldb::DB::Open(options, _directory.c_str(), &_db);
-			if (status.ok())
-			{
-				logger_->log_info("NiFi FlowFile Repository database open %s success", _directory.c_str());
-			}
-			else
-			{
-				logger_->log_error("NiFi FlowFile Repository database open %s fail", _directory.c_str());
-				return false;
-			}
-		}
-
-		return true;
-#else
-		return false;
-#endif
-	}
-	//! Put
-	virtual bool Put(std::string key, uint8_t *buf, int bufLen)
-	{
-#ifdef LEVELDB_SUPPORT
-		if (!_enable)
-			return false;
-			
-		// persistent to the DB
-		leveldb::Slice value((const char *) buf, bufLen);
-		leveldb::Status status;
-		status = _db->Put(leveldb::WriteOptions(), key, value);
-		if (status.ok())
-			return true;
-		else
-			return false;
-#else
-		return false;
-#endif
-	}
-	//! Delete
-	virtual bool Delete(std::string key)
-	{
-#ifdef LEVELDB_SUPPORT
-		if (!_enable)
-			return false;
-		leveldb::Status status;
-		status = _db->Delete(leveldb::WriteOptions(), key);
-		if (status.ok())
-			return true;
-		else
-			return false;
-#else
-		return false;
-#endif
-	}
-	//! Get
-	virtual bool Get(std::string key, std::string &value)
-	{
-#ifdef LEVELDB_SUPPORT
-		if (!_enable)
-			return false;
-		leveldb::Status status;
-		status = _db->Get(leveldb::ReadOptions(), key, &value);
-		if (status.ok())
-			return true;
-		else
-			return false;
-#else
-		return false;
-#endif
-	}
-	//! Run function for the thread
-	static void run(Repository *repo);
-	//! Start the repository monitor thread
-	virtual void start();
-	//! Stop the repository monitor thread
-	virtual void stop();
-	//! whether the repo is full
-	virtual bool isFull()
-	{
-		return _repoFull;
-	}
-	//! whether the repo is enable 
-	virtual bool isEnable()
-	{
-		return _enable;
-	}
-
-protected:
-	//! Repo Type
-	RepositoryType _type;
-	//! Mutex for protection
-	std::mutex _mtx;
-	//! repository directory
-	std::string _directory;
-	//! Logger
-	std::shared_ptr<Logger> logger_;
-	//! Configure
-	//! max db entry life time
-	Configure *configure_;
-	int64_t _maxPartitionMillis;
-	//! max db size
-	int64_t _maxPartitionBytes;
-	//! purge period
-	uint64_t _purgePeriod;
-#ifdef LEVELDB_SUPPORT
-	//! level DB database
-	leveldb::DB* _db;
-#endif
-	//! thread
-	std::thread *_thread;
-	//! whether the monitoring thread is running for the repo while it was enabled 
-	bool _running;
-	//! whether it is enabled by minfi property for the repo 
-	bool _enable;
-	//! whether stop accepting provenace event
-	std::atomic<bool> _repoFull;
-	//! repoSize
-	uint64_t repoSize();
-	//! size of the directory
-	static uint64_t _repoSize[MAX_REPO_TYPE];
-	//! call back for directory size
-	static int repoSumProvenance(const char *fpath, const struct stat *sb, int typeflag)
-	{
-		_repoSize[PROVENANCE] += sb->st_size;
-		return 0;
-	}
-	//! call back for directory size
-	static int repoSumFlowFile(const char *fpath, const struct stat *sb, int typeflag)
-	{
-		_repoSize[FLOWFILE] += sb->st_size;
-		return 0;
-	}
-
-private:
-	//! destroy
-	void destroy()
-	{
-#ifdef LEVELDB_SUPPORT
-		if (_db)
-		{
-			delete _db;
-			_db = NULL;
-		}
-#endif
-	}
-	// Prevent default copy constructor and assignment operation
-	// Only support pass by reference or pointer
-	Repository(const Repository &parent);
-	Repository &operator=(const Repository &parent);
-};
-
-#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/ResourceClaim.h
----------------------------------------------------------------------
diff --git a/libminifi/include/ResourceClaim.h b/libminifi/include/ResourceClaim.h
index 7ca79a3..4c5438c 100644
--- a/libminifi/include/ResourceClaim.h
+++ b/libminifi/include/ResourceClaim.h
@@ -27,75 +27,85 @@
 #include <map>
 #include <mutex>
 #include <atomic>
-#include "Configure.h"
+#include "properties/Configure.h"
 
-//! Default content directory
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+
+// Default content directory
 #define DEFAULT_CONTENT_DIRECTORY "./content_repository"
 
 
 
-//! ResourceClaim Class
+// ResourceClaim Class
 class ResourceClaim {
 
 public:
   
 	static std::string default_directory_path;
-	//! Constructor
+	// Constructor
 	/*!
 	 * Create a new resource claim
 	 */
 	ResourceClaim(const std::string contentDirectory = default_directory_path);
-	//! Destructor
+	// Destructor
 	virtual ~ResourceClaim() {}
-	//! increaseFlowFileRecordOwnedCount
+	// increaseFlowFileRecordOwnedCount
 	void increaseFlowFileRecordOwnedCount()
 	{
 		++_flowFileRecordOwnedCount;
 	}
-	//! decreaseFlowFileRecordOwenedCount
+	// decreaseFlowFileRecordOwenedCount
 	void decreaseFlowFileRecordOwnedCount()
 	{
 		--_flowFileRecordOwnedCount;
 	}
-	//! getFlowFileRecordOwenedCount
+	// getFlowFileRecordOwenedCount
 	uint64_t getFlowFileRecordOwnedCount()
 	{
 		return _flowFileRecordOwnedCount;
 	}
-	//! Get the content full path
+	// Get the content full path
 	std::string getContentFullPath()
 	{
 		return _contentFullPath;
 	}
-	//! Set the content full path
+	// Set the content full path
 	void setContentFullPath(std::string path)
 	{
 		_contentFullPath = path;
 	}
 
 protected:
-	//! A global unique identifier
+	// A global unique identifier
 	uuid_t _uuid;
-	//! A local unique identifier
+	// A local unique identifier
 	uint64_t _id;
-	//! Full path to the content
+	// Full path to the content
 	std::string _contentFullPath;
 
-	//! How many FlowFileRecord Own this cliam
+	// How many FlowFileRecord Own this cliam
 	std::atomic<uint64_t> _flowFileRecordOwnedCount;
 
 private:
-	//! Configure
+	// Configure
 	Configure *configure_;
-	//! Logger
-	std::shared_ptr<Logger> logger_;
+	// Logger
+	std::shared_ptr<logging::Logger> logger_;
 	// Prevent default copy constructor and assignment operation
 	// Only support pass by reference or pointer
 	ResourceClaim(const ResourceClaim &parent);
 	ResourceClaim &operator=(const ResourceClaim &parent);
 
-	//! Local resource claim number
+	// Local resource claim number
 	static std::atomic<uint64_t> _localResourceClaimNumber;
 };
 
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
 #endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/SchedulingAgent.h
----------------------------------------------------------------------
diff --git a/libminifi/include/SchedulingAgent.h b/libminifi/include/SchedulingAgent.h
index f6d5a1c..0493640 100644
--- a/libminifi/include/SchedulingAgent.h
+++ b/libminifi/include/SchedulingAgent.h
@@ -28,72 +28,92 @@
 #include <algorithm>
 #include <thread>
 #include "utils/TimeUtil.h"
-#include "Logger.h"
-#include "Configure.h"
+#include "core/core.h"
+#include "core/logging/Logger.h"
+#include "properties/Configure.h"
 #include "FlowFileRecord.h"
-#include "Logger.h"
-#include "Processor.h"
-#include "ProcessContext.h"
+#include "core/logging/Logger.h"
+#include "core/Processor.h"
+#include "core/ProcessContext.h"
+#include "provenance/ProvenanceRepository.h"
 
-//! SchedulingAgent Class
-class SchedulingAgent
-{
-public:
-	//! Constructor
-	/*!
-	 * Create a new processor
-	 */
-	SchedulingAgent() {
-		configure_ = Configure::getConfigure();
-		logger_ = Logger::getLogger();
-		_running = false;
-	}
-	//! Destructor
-	virtual ~SchedulingAgent()
-	{
 
-	}
-	//! onTrigger, return whether the yield is need
-	bool onTrigger(Processor *processor, ProcessContext *processContext, ProcessSessionFactory *sessionFactory);
-	//! Whether agent has work to do
-	bool hasWorkToDo(Processor *processor);
-	//! Whether the outgoing need to be backpressure
-	bool hasTooMuchOutGoing(Processor *processor);
-	//! start
-	void start() {
-		_running = true;
-	}
-	//! stop
-	void stop() {
-		_running = false;
-	}
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
 
-public:
-	//! schedule, overwritten by different DrivenSchedulingAgent
-	virtual void schedule(Processor *processor) = 0;
-	//! unschedule, overwritten by different DrivenSchedulingAgent
-	virtual void unschedule(Processor *processor) = 0;
 
-protected:
-	//! Logger
-	std::shared_ptr<Logger> logger_;
-	//! Configure
-	Configure *configure_;
-	//! Mutex for protection
-	std::mutex _mtx;
-	//! Whether it is running
-	std::atomic<bool> _running;
-	//! AdministrativeYieldDuration
-	int64_t _administrativeYieldDuration;
-	//! BoredYieldDuration
-	int64_t _boredYieldDuration;
+// SchedulingAgent Class
+class SchedulingAgent {
+ public:
+  // Constructor
+  /*!
+   * Create a new processor
+   */
+  SchedulingAgent(std::shared_ptr<core::Repository> repo) {
+    configure_ = Configure::getConfigure();
+    logger_ = logging::Logger::getLogger();
+    running_ = false;
+    repo_ = repo;
+  }
+  // Destructor
+  virtual ~SchedulingAgent() {
 
-private:
-	// Prevent default copy constructor and assignment operation
-	// Only support pass by reference or pointer
-	SchedulingAgent(const SchedulingAgent &parent);
-	SchedulingAgent &operator=(const SchedulingAgent &parent);
+  }
+  // onTrigger, return whether the yield is need
+  bool onTrigger(
+      std::shared_ptr<core::Processor> processor,
+      core::ProcessContext *processContext,
+      core::ProcessSessionFactory *sessionFactory);
+  // Whether agent has work to do
+  bool hasWorkToDo(std::shared_ptr<core::Processor> processor);
+  // Whether the outgoing need to be backpressure
+  bool hasTooMuchOutGoing(
+      std::shared_ptr<core::Processor> processor);
+  // start
+  void start() {
+    running_ = true;
+  }
+  // stop
+  void stop() {
+    running_ = false;
+  }
+
+ public:
+  // schedule, overwritten by different DrivenSchedulingAgent
+  virtual void schedule(
+      std::shared_ptr<core::Processor> processor) = 0;
+  // unschedule, overwritten by different DrivenSchedulingAgent
+  virtual void unschedule(
+      std::shared_ptr<core::Processor> processor) = 0;
+
+  SchedulingAgent(const SchedulingAgent &parent) = delete;
+  SchedulingAgent &operator=(const SchedulingAgent &parent) = delete;
+ protected:
+  // Logger
+  std::shared_ptr<logging::Logger> logger_;
+  // Configure
+  Configure *configure_;
+  // Mutex for protection
+  std::mutex mutex_;
+  // Whether it is running
+  std::atomic<bool> running_;
+  // AdministrativeYieldDuration
+  int64_t _administrativeYieldDuration;
+  // BoredYieldDuration
+  int64_t _boredYieldDuration;
+
+  std::shared_ptr<core::Repository> repo_;
+
+ private:
+  // Prevent default copy constructor and assignment operation
+  // Only support pass by reference or pointer
 
 };
 
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
 #endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/Site2SiteClientProtocol.h
----------------------------------------------------------------------
diff --git a/libminifi/include/Site2SiteClientProtocol.h b/libminifi/include/Site2SiteClientProtocol.h
index 444eed5..6120e3e 100644
--- a/libminifi/include/Site2SiteClientProtocol.h
+++ b/libminifi/include/Site2SiteClientProtocol.h
@@ -34,16 +34,23 @@
 #include <thread>
 #include <algorithm>
 #include <uuid/uuid.h>
-#include "Configure.h"
-#include "Property.h"
+
+#include "core/Property.h"
+#include "properties/Configure.h"
 #include "Site2SitePeer.h"
 #include "FlowFileRecord.h"
-#include "Logger.h"
-#include "ProcessContext.h"
-#include "ProcessSession.h"
+#include "core/logging/Logger.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
 #include "io/CRCStream.h"
 
-//! Resource Negotiated Status Code
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+// Resource Negotiated Status Code
 #define RESOURCE_OK 20
 #define DIFFERENT_RESOURCE_VERSION 21
 #define NEGOTIATED_ABORT 255
@@ -55,143 +62,144 @@
  * transferred between a client and a remote NiFi instance.
  */
 typedef enum {
-	/**
-	 * * The client is to send data to the remote instance.
-	 * */
-	SEND,
-	/**
-	 * * The client is to receive data from the remote instance.
-	 * */
-	RECEIVE
+  /**
+   * * The client is to send data to the remote instance.
+   * */
+  SEND,
+  /**
+   * * The client is to receive data from the remote instance.
+   * */
+  RECEIVE
 } TransferDirection;
 
-//! Peer State
+// Peer State
 typedef enum {
-	/**
-	 * * IDLE
-	 * */
-	IDLE = 0,
-	/**
-	 * * Socket Established
-	 * */
-	ESTABLISHED,
-	/**
-	 * * HandShake Done
-	 * */
-	HANDSHAKED,
-	/**
-	 * * After CodeDec Completion
-	 * */
-	READY
+  /**
+   * * IDLE
+   * */
+  IDLE = 0,
+  /**
+   * * Socket Established
+   * */
+  ESTABLISHED,
+  /**
+   * * HandShake Done
+   * */
+  HANDSHAKED,
+  /**
+   * * After CodeDec Completion
+   * */
+  READY
 } PeerState;
 
-//! Transaction State
+// Transaction State
 typedef enum {
-	/**
-	 * * Transaction has been started but no data has been sent or received.
-	 * */
-	TRANSACTION_STARTED,
-	/**
-	 * * Transaction has been started and data has been sent or received.
-	 * */
-	DATA_EXCHANGED,
-	/**
-	 * * Data that has been transferred has been confirmed via its CRC.
-	 * * Transaction is ready to be completed.
-	 * */
-	TRANSACTION_CONFIRMED,
-	/**
-	 * * Transaction has been successfully completed.
-	 * */
-	TRANSACTION_COMPLETED,
-	/**
-	 * * The Transaction has been canceled.
-	 * */
-	TRANSACTION_CANCELED,
-	/**
-	 * * The Transaction ended in an error.
-	 * */
-	TRANSACTION_ERROR
+  /**
+   * * Transaction has been started but no data has been sent or received.
+   * */
+  TRANSACTION_STARTED,
+  /**
+   * * Transaction has been started and data has been sent or received.
+   * */
+  DATA_EXCHANGED,
+  /**
+   * * Data that has been transferred has been confirmed via its CRC.
+   * * Transaction is ready to be completed.
+   * */
+  TRANSACTION_CONFIRMED,
+  /**
+   * * Transaction has been successfully completed.
+   * */
+  TRANSACTION_COMPLETED,
+  /**
+   * * The Transaction has been canceled.
+   * */
+  TRANSACTION_CANCELED,
+  /**
+   * * The Transaction ended in an error.
+   * */
+  TRANSACTION_ERROR
 } TransactionState;
 
-//! Request Type
+// Request Type
 typedef enum {
-	NEGOTIATE_FLOWFILE_CODEC = 0,
-	REQUEST_PEER_LIST,
-	SEND_FLOWFILES,
-	RECEIVE_FLOWFILES,
-	SHUTDOWN,
-	MAX_REQUEST_TYPE
+  NEGOTIATE_FLOWFILE_CODEC = 0,
+  REQUEST_PEER_LIST,
+  SEND_FLOWFILES,
+  RECEIVE_FLOWFILES,
+  SHUTDOWN,
+  MAX_REQUEST_TYPE
 } RequestType;
 
-//! Request Type Str
+// Request Type Str
 static const char *RequestTypeStr[MAX_REQUEST_TYPE] = {
-		"NEGOTIATE_FLOWFILE_CODEC", "REQUEST_PEER_LIST", "SEND_FLOWFILES",
-		"RECEIVE_FLOWFILES", "SHUTDOWN" };
+    "NEGOTIATE_FLOWFILE_CODEC", "REQUEST_PEER_LIST", "SEND_FLOWFILES",
+    "RECEIVE_FLOWFILES", "SHUTDOWN" };
 
-//! Respond Code
+// Respond Code
 typedef enum {
-	RESERVED = 0,
-	// ResponseCode, so that we can indicate a 0 followed by some other bytes
-
-	// handshaking properties
-	PROPERTIES_OK = 1,
-	UNKNOWN_PROPERTY_NAME = 230,
-	ILLEGAL_PROPERTY_VALUE = 231,
-	MISSING_PROPERTY = 232,
-	// transaction indicators
-	CONTINUE_TRANSACTION = 10,
-	FINISH_TRANSACTION = 11,
-	CONFIRM_TRANSACTION = 12, // "Explanation" of this code is the checksum
-	TRANSACTION_FINISHED = 13,
-	TRANSACTION_FINISHED_BUT_DESTINATION_FULL = 14,
-	CANCEL_TRANSACTION = 15,
-	BAD_CHECKSUM = 19,
-	// data availability indicators
-	MORE_DATA = 20,
-	NO_MORE_DATA = 21,
-	// port state indicators
-	UNKNOWN_PORT = 200,
-	PORT_NOT_IN_VALID_STATE = 201,
-	PORTS_DESTINATION_FULL = 202,
-	// authorization
-	UNAUTHORIZED = 240,
-	// error indicators
-	ABORT = 250,
-	UNRECOGNIZED_RESPONSE_CODE = 254,
-	END_OF_STREAM = 255
+  RESERVED = 0,
+  // ResponseCode, so that we can indicate a 0 followed by some other bytes
+
+  // handshaking properties
+  PROPERTIES_OK = 1,
+  UNKNOWN_PROPERTY_NAME = 230,
+  ILLEGAL_PROPERTY_VALUE = 231,
+  MISSING_PROPERTY = 232,
+  // transaction indicators
+  CONTINUE_TRANSACTION = 10,
+  FINISH_TRANSACTION = 11,
+  CONFIRM_TRANSACTION = 12,  // "Explanation" of this code is the checksum
+  TRANSACTION_FINISHED = 13,
+  TRANSACTION_FINISHED_BUT_DESTINATION_FULL = 14,
+  CANCEL_TRANSACTION = 15,
+  BAD_CHECKSUM = 19,
+  // data availability indicators
+  MORE_DATA = 20,
+  NO_MORE_DATA = 21,
+  // port state indicators
+  UNKNOWN_PORT = 200,
+  PORT_NOT_IN_VALID_STATE = 201,
+  PORTS_DESTINATION_FULL = 202,
+  // authorization
+  UNAUTHORIZED = 240,
+  // error indicators
+  ABORT = 250,
+  UNRECOGNIZED_RESPONSE_CODE = 254,
+  END_OF_STREAM = 255
 } RespondCode;
 
-//! Respond Code Class
+// Respond Code Class
 typedef struct {
-	RespondCode code;
-	const char *description;
-	bool hasDescription;
+  RespondCode code;
+  const char *description;
+  bool hasDescription;
 } RespondCodeContext;
 
-//! Respond Code Context
+// Respond Code Context
 static RespondCodeContext respondCodeContext[] = { { RESERVED,
-		"Reserved for Future Use", false }, { PROPERTIES_OK, "Properties OK",
-		false }, { UNKNOWN_PROPERTY_NAME, "Unknown Property Name", true }, {
-		ILLEGAL_PROPERTY_VALUE, "Illegal Property Value", true }, {
-		MISSING_PROPERTY, "Missing Property", true }, { CONTINUE_TRANSACTION,
-		"Continue Transaction", false }, { FINISH_TRANSACTION,
-		"Finish Transaction", false }, { CONFIRM_TRANSACTION,
-		"Confirm Transaction", true }, { TRANSACTION_FINISHED,
-		"Transaction Finished", false }, {
-		TRANSACTION_FINISHED_BUT_DESTINATION_FULL,
-		"Transaction Finished But Destination is Full", false }, {
-		CANCEL_TRANSACTION, "Cancel Transaction", true }, { BAD_CHECKSUM,
-		"Bad Checksum", false }, { MORE_DATA, "More Data Exists", false }, {
-		NO_MORE_DATA, "No More Data Exists", false }, { UNKNOWN_PORT,
-		"Unknown Port", false }, { PORT_NOT_IN_VALID_STATE,
-		"Port Not in a Valid State", true }, { PORTS_DESTINATION_FULL,
-		"Port's Destination is Full", false }, { UNAUTHORIZED,
-		"User Not Authorized", true }, { ABORT, "Abort", true }, {
-		UNRECOGNIZED_RESPONSE_CODE, "Unrecognized Response Code", false }, {
-		END_OF_STREAM, "End of Stream", false } };
-
-//! Respond Code Sequence Pattern
+    "Reserved for Future Use", false },
+    { PROPERTIES_OK, "Properties OK", false }, { UNKNOWN_PROPERTY_NAME,
+        "Unknown Property Name", true }, { ILLEGAL_PROPERTY_VALUE,
+        "Illegal Property Value", true }, { MISSING_PROPERTY,
+        "Missing Property", true }, { CONTINUE_TRANSACTION,
+        "Continue Transaction", false }, { FINISH_TRANSACTION,
+        "Finish Transaction", false }, { CONFIRM_TRANSACTION,
+        "Confirm Transaction", true }, { TRANSACTION_FINISHED,
+        "Transaction Finished", false }, {
+        TRANSACTION_FINISHED_BUT_DESTINATION_FULL,
+        "Transaction Finished But Destination is Full", false }, {
+        CANCEL_TRANSACTION, "Cancel Transaction", true }, { BAD_CHECKSUM,
+        "Bad Checksum", false }, { MORE_DATA, "More Data Exists", false }, {
+        NO_MORE_DATA, "No More Data Exists", false }, { UNKNOWN_PORT,
+        "Unknown Port", false }, { PORT_NOT_IN_VALID_STATE,
+        "Port Not in a Valid State", true }, { PORTS_DESTINATION_FULL,
+        "Port's Destination is Full", false }, { UNAUTHORIZED,
+        "User Not Authorized", true }, { ABORT, "Abort", true }, {
+        UNRECOGNIZED_RESPONSE_CODE, "Unrecognized Response Code", false }, {
+        END_OF_STREAM, "End of Stream", false } };
+
+// Respond Code Sequence Pattern
 static const uint8_t CODE_SEQUENCE_VALUE_1 = (uint8_t) 'R';
 static const uint8_t CODE_SEQUENCE_VALUE_2 = (uint8_t) 'C';
 
@@ -200,43 +208,44 @@ static const uint8_t CODE_SEQUENCE_VALUE_2 = (uint8_t) 'C';
  * Protocol.
  */
 typedef enum {
-	/**
-	 * Boolean value indicating whether or not the contents of a FlowFile should
-	 * be GZipped when transferred.
-	 */
-	GZIP,
-	/**
-	 * The unique identifier of the port to communicate with
-	 */
-	PORT_IDENTIFIER,
-	/**
-	 * Indicates the number of milliseconds after the request was made that the
-	 * client will wait for a response. If no response has been received by the
-	 * time this value expires, the server can move on without attempting to
-	 * service the request because the client will have already disconnected.
-	 */
-	REQUEST_EXPIRATION_MILLIS,
-	/**
-	 * The preferred number of FlowFiles that the server should send to the
-	 * client when pulling data. This property was introduced in version 5 of
-	 * the protocol.
-	 */
-	BATCH_COUNT,
-	/**
-	 * The preferred number of bytes that the server should send to the client
-	 * when pulling data. This property was introduced in version 5 of the
-	 * protocol.
-	 */
-	BATCH_SIZE,
-	/**
-	 * The preferred amount of time that the server should send data to the
-	 * client when pulling data. This property was introduced in version 5 of
-	 * the protocol. Value is in milliseconds.
-	 */
-	BATCH_DURATION, MAX_HANDSHAKE_PROPERTY
+  /**
+   * Boolean value indicating whether or not the contents of a FlowFile should
+   * be GZipped when transferred.
+   */
+  GZIP,
+  /**
+   * The unique identifier of the port to communicate with
+   */
+  PORT_IDENTIFIER,
+  /**
+   * Indicates the number of milliseconds after the request was made that the
+   * client will wait for a response. If no response has been received by the
+   * time this value expires, the server can move on without attempting to
+   * service the request because the client will have already disconnected.
+   */
+  REQUEST_EXPIRATION_MILLIS,
+  /**
+   * The preferred number of FlowFiles that the server should send to the
+   * client when pulling data. This property was introduced in version 5 of
+   * the protocol.
+   */
+  BATCH_COUNT,
+  /**
+   * The preferred number of bytes that the server should send to the client
+   * when pulling data. This property was introduced in version 5 of the
+   * protocol.
+   */
+  BATCH_SIZE,
+  /**
+   * The preferred amount of time that the server should send data to the
+   * client when pulling data. This property was introduced in version 5 of
+   * the protocol. Value is in milliseconds.
+   */
+  BATCH_DURATION,
+  MAX_HANDSHAKE_PROPERTY
 } HandshakeProperty;
 
-//! HandShakeProperty Str
+// HandShakeProperty Str
 static const char *HandShakePropertyStr[MAX_HANDSHAKE_PROPERTY] = {
 /**
  * Boolean value indicating whether or not the contents of a FlowFile should
@@ -275,89 +284,89 @@ static const char *HandShakePropertyStr[MAX_HANDSHAKE_PROPERTY] = {
 
 class Site2SiteClientProtocol;
 
-//! Transaction Class
+// Transaction Class
 class Transaction {
-	friend class Site2SiteClientProtocol;
-public:
-	//! Constructor
-	/*!
-	 * Create a new transaction
-	 */
-	explicit Transaction(TransferDirection direction,
-			CRCStream<Site2SitePeer> &stream) :
-			crcStream(std::move(stream)) {
-		_state = TRANSACTION_STARTED;
-		_direction = direction;
-		_dataAvailable = false;
-		_transfers = 0;
-		_bytes = 0;
-
-		char uuidStr[37];
-
-		// Generate the global UUID for the transaction
-		uuid_generate(_uuid);
-		uuid_unparse_lower(_uuid, uuidStr);
-		_uuidStr = uuidStr;
-	}
-	//! Destructor
-	virtual ~Transaction() {
-	}
-	//! getUUIDStr
-	std::string getUUIDStr() {
-		return _uuidStr;
-	}
-	//! getState
-	TransactionState getState() {
-		return _state;
-	}
-	//! isDataAvailable
-	bool isDataAvailable() {
-		return _dataAvailable;
-	}
-	//! setDataAvailable()
-	void setDataAvailable(bool value) {
-		_dataAvailable = value;
-	}
-	//! getDirection
-	TransferDirection getDirection() {
-		return _direction;
-	}
-	//! getCRC
-	long getCRC() {
-		return crcStream.getCRC();
-	}
-	//! updateCRC
-	void updateCRC(uint8_t *buffer, uint32_t length) {
-		crcStream.updateCRC(buffer, length);
-	}
-
-	CRCStream<Site2SitePeer> &getStream() {
-		return crcStream;
-	}
-	
-	
-	Transaction(const Transaction &parent) = delete;
-	Transaction &operator=(const Transaction &parent) = delete;
-
-protected:
-
-private:
-
-	CRCStream<Site2SitePeer> crcStream;
-	//! Transaction State
-	TransactionState _state;
-	//! Transaction Direction
-	TransferDirection _direction;
-	//! Whether received data is available
-	bool _dataAvailable;
-	//! A global unique identifier
-	uuid_t _uuid;
-	//! UUID string
-	std::string _uuidStr;
-	//! Number of transfer
-	int _transfers;
-	//! Number of content bytes
-	uint64_t _bytes;
+  friend class Site2SiteClientProtocol;
+ public:
+  // Constructor
+  /*!
+   * Create a new transaction
+   */
+  explicit Transaction(
+      TransferDirection direction,
+      org::apache::nifi::minifi::io::CRCStream<Site2SitePeer> &stream)
+      : crcStream(std::move(stream)) {
+    _state = TRANSACTION_STARTED;
+    _direction = direction;
+    _dataAvailable = false;
+    _transfers = 0;
+    _bytes = 0;
+
+    char uuidStr[37];
+
+    // Generate the global UUID for the transaction
+    uuid_generate(_uuid);
+    uuid_unparse_lower(_uuid, uuidStr);
+    _uuidStr = uuidStr;
+  }
+  // Destructor
+  virtual ~Transaction() {
+  }
+  // getUUIDStr
+  std::string getUUIDStr() {
+    return _uuidStr;
+  }
+  // getState
+  TransactionState getState() {
+    return _state;
+  }
+  // isDataAvailable
+  bool isDataAvailable() {
+    return _dataAvailable;
+  }
+  // setDataAvailable()
+  void setDataAvailable(bool value) {
+    _dataAvailable = value;
+  }
+  // getDirection
+  TransferDirection getDirection() {
+    return _direction;
+  }
+  // getCRC
+  long getCRC() {
+    return crcStream.getCRC();
+  }
+  // updateCRC
+  void updateCRC(uint8_t *buffer, uint32_t length) {
+    crcStream.updateCRC(buffer, length);
+  }
+
+  org::apache::nifi::minifi::io::CRCStream<Site2SitePeer> &getStream() {
+    return crcStream;
+  }
+
+  Transaction(const Transaction &parent) = delete;
+  Transaction &operator=(const Transaction &parent) = delete;
+
+ protected:
+
+ private:
+
+  org::apache::nifi::minifi::io::CRCStream<Site2SitePeer> crcStream;
+  // Transaction State
+  TransactionState _state;
+  // Transaction Direction
+  TransferDirection _direction;
+  // Whether received data is available
+  bool _dataAvailable;
+  // A global unique identifier
+  uuid_t _uuid;
+  // UUID string
+  std::string _uuidStr;
+  // Number of transfer
+  int _transfers;
+  // Number of content bytes
+  uint64_t _bytes;
 
 };
 
@@ -366,264 +375,268 @@ private:
  * NiFi instance.
  */
 class DataPacket {
-public:
-	DataPacket(Site2SiteClientProtocol *protocol, Transaction *transaction,
-			std::map<std::string, std::string> attributes) {
-		_protocol = protocol;
-		_size = 0;
-		_transaction = transaction;
-		_attributes = attributes;
-	}
-	std::map<std::string, std::string> _attributes;
-	uint64_t _size;
-	Site2SiteClientProtocol *_protocol;
-	Transaction *_transaction;
+ public:
+  DataPacket(Site2SiteClientProtocol *protocol, Transaction *transaction,
+             std::map<std::string, std::string> attributes) {
+    _protocol = protocol;
+    _size = 0;
+    _transaction = transaction;
+    _attributes = attributes;
+  }
+  std::map<std::string, std::string> _attributes;
+  uint64_t _size;
+  Site2SiteClientProtocol *_protocol;
+  Transaction *_transaction;
 };
 
-//! Site2SiteClientProtocol Class
+// Site2SiteClientProtocol Class
 class Site2SiteClientProtocol {
-public:
-	//! Constructor
-	/*!
-	 * Create a new control protocol
-	 */
-	Site2SiteClientProtocol(Site2SitePeer *peer) {
-		logger_ = Logger::getLogger();
-		configure_ = Configure::getConfigure();
-		peer_ = peer;
-		_batchSize = 0;
-		_batchCount = 0;
-		_batchDuration = 0;
-		_batchSendNanos = 5000000000; // 5 seconds
-		_timeOut = 30000; // 30 seconds
-		_peerState = IDLE;
-		_supportedVersion[0] = 5;
-		_supportedVersion[1] = 4;
-		_supportedVersion[2] = 3;
-		_supportedVersion[3] = 2;
-		_supportedVersion[4] = 1;
-		_currentVersion = _supportedVersion[0];
-		_currentVersionIndex = 0;
-		_supportedCodecVersion[0] = 1;
-		_currentCodecVersion = _supportedCodecVersion[0];
-		_currentCodecVersionIndex = 0;
-	}
-	//! Destructor
-	virtual ~Site2SiteClientProtocol() {
-	}
-
-public:
-	//! setBatchSize
-	void setBatchSize(uint64_t size) {
-		_batchSize = size;
-	}
-	//! setBatchCount
-	void setBatchCount(uint64_t count) {
-		_batchCount = count;
-	}
-	//! setBatchDuration
-	void setBatchDuration(uint64_t duration) {
-		_batchDuration = duration;
-	}
-	//! setTimeOut
-	void setTimeOut(uint64_t time) {
-		_timeOut = time;
-		if (peer_)
-			peer_->setTimeOut(time);
-
-	}
-	
-	void setPeer(Site2SitePeer *peer)
-	{
-	  peer_ = peer;
-	}
-	/**
-	 * Provides a reference to the time out
-	 * @returns timeout
-	 */
-	const uint64_t getTimeOut() const {
-		return _timeOut;
-	}
-
-	/**
-	 * Provides a reference to the port identifier
-	 * @returns port identifier
-	 */
-	const std::string getPortId() const {
-		return _portIdStr;
-	}
-	//! setPortId
-	void setPortId(uuid_t id) {
-		uuid_copy(_portId, id);
-		char idStr[37];
-		uuid_unparse_lower(id, idStr);
-		_portIdStr = idStr;
-	}
-	//! getResourceName
-	std::string getResourceName() {
-		return "SocketFlowFileProtocol";
-	}
-	//! getCodecResourceName
-	std::string getCodecResourceName() {
-		return "StandardFlowFileCodec";
-	}
-	//! bootstrap the protocol to the ready for transaction state by going through the state machine
-	bool bootstrap();
-	//! establish
-	bool establish();
-	//! handShake
-	bool handShake();
-	//! negotiateCodec
-	bool negotiateCodec();
-	//! initiateResourceNegotiation
-	bool initiateResourceNegotiation();
-	//! initiateCodecResourceNegotiation
-	bool initiateCodecResourceNegotiation();
-	//! tearDown
-	void tearDown();
-	//! write Request Type
-	int writeRequestType(RequestType type);
-	//! read Request Type
-	int readRequestType(RequestType &type);
-	//! read Respond
-	int readRespond(RespondCode &code, std::string &message);
-	//! write respond
-	int writeRespond(RespondCode code, std::string message);
-	//! getRespondCodeContext
-	RespondCodeContext *getRespondCodeContext(RespondCode code) {
-		for (unsigned int i = 0;
-				i < sizeof(respondCodeContext) / sizeof(RespondCodeContext);
-				i++) {
-			if (respondCodeContext[i].code == code) {
-				return &respondCodeContext[i];
-			}
-		}
-		return NULL;
-	}
-	//! getPeer
-	Site2SitePeer *getPeer() {
-		return peer_;
-	}
-	//! Creation of a new transaction, return the transaction ID if success,
-	//! Return NULL when any error occurs
-	Transaction *createTransaction(std::string &transactionID,
-			TransferDirection direction);
-	//! Receive the data packet from the transaction
-	//! Return false when any error occurs
-	bool receive(std::string transactionID, DataPacket *packet, bool &eof);
-	//! Send the data packet from the transaction
-	//! Return false when any error occurs
-	bool send(std::string transactionID, DataPacket *packet,
-			FlowFileRecord *flowFile, ProcessSession *session);
-	//! Confirm the data that was sent or received by comparing CRC32's of the data sent and the data received.
-	bool confirm(std::string transactionID);
-	//! Cancel the transaction
-	void cancel(std::string transactionID);
-	//! Complete the transaction
-	bool complete(std::string transactionID);
-	//! Error the transaction
-	void error(std::string transactionID);
-	//! Receive flow files for the process session
-	void receiveFlowFiles(ProcessContext *context, ProcessSession *session);
-	//! Transfer flow files for the process session
-	void transferFlowFiles(ProcessContext *context, ProcessSession *session);
-	//! deleteTransaction
-	void deleteTransaction(std::string transactionID);
-	//! Nest Callback Class for write stream
-	class WriteCallback: public OutputStreamCallback {
-	public:
-		WriteCallback(DataPacket *packet) :
-				_packet(packet) {
-		}
-		DataPacket *_packet;
-		void process(std::ofstream *stream) {
-			uint8_t buffer[8192];
-			int len = _packet->_size;
-			while (len > 0) {
-				int size = std::min(len, (int) sizeof(buffer));
-				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;
-				}
-				stream->write((const char *) buffer, size);
-				len -= size;
-			}
-		}
-	};
-	//! Nest Callback Class for read stream
-	class ReadCallback: public InputStreamCallback {
-	public:
-		ReadCallback(DataPacket *packet) :
-				_packet(packet) {
-		}
-		DataPacket *_packet;
-		void process(std::ifstream *stream) {
-			_packet->_size = 0;
-			uint8_t buffer[8192];
-			int readSize;
-			while (stream->good()) {
-				if (!stream->read((char *) buffer, 8192))
-					readSize = stream->gcount();
-				else
-					readSize = 8192;
-				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;
-				}
-				_packet->_size += readSize;
-			}
-		}
-	};
-
-protected:
-
-private:
-
-	//! Mutex for protection
-	std::mutex _mtx;
-	//! Logger
-	std::shared_ptr<Logger> logger_;
-	//! Configure
-	Configure *configure_;
-	//! Batch Count
-	std::atomic<uint64_t> _batchCount;
-	//! Batch Size
-	std::atomic<uint64_t> _batchSize;
-	//! Batch Duration in msec
-	std::atomic<uint64_t> _batchDuration;
-	//! Timeout in msec
-	std::atomic<uint64_t> _timeOut;
-	//! Peer Connection
-	Site2SitePeer *peer_;
-	//! portId
-	uuid_t _portId;
-	//! portIDStr
-	std::string _portIdStr;
-	//! BATCH_SEND_NANOS
-	uint64_t _batchSendNanos;
-	//! Peer State
-	PeerState _peerState;
-	uint32_t _supportedVersion[5];
-	uint32_t _currentVersion;
-	int _currentVersionIndex;
-	uint32_t _supportedCodecVersion[1];
-	uint32_t _currentCodecVersion;
-	int _currentCodecVersionIndex;
-	//! commsIdentifier
-	std::string _commsIdentifier;
-	//! transaction map
-	std::map<std::string, Transaction *> _transactionMap;
-
-	// Prevent default copy constructor and assignment operation
-	// Only support pass by reference or pointer
-	Site2SiteClientProtocol(const Site2SiteClientProtocol &parent);
-	Site2SiteClientProtocol &operator=(const Site2SiteClientProtocol &parent);
+ public:
+  // Constructor
+  /*!
+   * Create a new control protocol
+   */
+  Site2SiteClientProtocol(std::unique_ptr<Site2SitePeer> peer) {
+    logger_ = logging::Logger::getLogger();
+    configure_ = Configure::getConfigure();
+    peer_ = std::move(peer);
+    _batchSize = 0;
+    _batchCount = 0;
+    _batchDuration = 0;
+    _batchSendNanos = 5000000000;  // 5 seconds
+    _timeOut = 30000;  // 30 seconds
+    _peerState = IDLE;
+    _supportedVersion[0] = 5;
+    _supportedVersion[1] = 4;
+    _supportedVersion[2] = 3;
+    _supportedVersion[3] = 2;
+    _supportedVersion[4] = 1;
+    _currentVersion = _supportedVersion[0];
+    _currentVersionIndex = 0;
+    _supportedCodecVersion[0] = 1;
+    _currentCodecVersion = _supportedCodecVersion[0];
+    _currentCodecVersionIndex = 0;
+  }
+  // Destructor
+  virtual ~Site2SiteClientProtocol() {
+    tearDown();
+  }
+
+ public:
+  // setBatchSize
+  void setBatchSize(uint64_t size) {
+    _batchSize = size;
+  }
+  // setBatchCount
+  void setBatchCount(uint64_t count) {
+    _batchCount = count;
+  }
+  // setBatchDuration
+  void setBatchDuration(uint64_t duration) {
+    _batchDuration = duration;
+  }
+  // setTimeOut
+  void setTimeOut(uint64_t time) {
+    _timeOut = time;
+    if (peer_)
+      peer_->setTimeOut(time);
+
+  }
+
+  void setPeer(std::unique_ptr<Site2SitePeer> peer) {
+    peer_ = std::move(peer);
+  }
+  /**
+   * Provides a reference to the time out
+   * @returns timeout
+   */
+  const uint64_t getTimeOut() const {
+    return _timeOut;
+  }
+
+  /**
+   * Provides a reference to the port identifier
+   * @returns port identifier
+   */
+  const std::string getPortId() const {
+    return _portIdStr;
+  }
+  // setPortId
+  void setPortId(uuid_t id) {
+    uuid_copy(_portId, id);
+    char idStr[37];
+    uuid_unparse_lower(id, idStr);
+    _portIdStr = idStr;
+  }
+  // getResourceName
+  std::string getResourceName() {
+    return "SocketFlowFileProtocol";
+  }
+  // getCodecResourceName
+  std::string getCodecResourceName() {
+    return "StandardFlowFileCodec";
+  }
+  // bootstrap the protocol to the ready for transaction state by going through the state machine
+  bool bootstrap();
+  // establish
+  bool establish();
+  // handShake
+  bool handShake();
+  // negotiateCodec
+  bool negotiateCodec();
+  // initiateResourceNegotiation
+  bool initiateResourceNegotiation();
+  // initiateCodecResourceNegotiation
+  bool initiateCodecResourceNegotiation();
+  // tearDown
+  void tearDown();
+  // write Request Type
+  int writeRequestType(RequestType type);
+  // read Request Type
+  int readRequestType(RequestType &type);
+  // read Respond
+  int readRespond(RespondCode &code, std::string &message);
+  // write respond
+  int writeRespond(RespondCode code, std::string message);
+  // getRespondCodeContext
+  RespondCodeContext *getRespondCodeContext(RespondCode code) {
+    for (unsigned int i = 0;
+        i < sizeof(respondCodeContext) / sizeof(RespondCodeContext); i++) {
+      if (respondCodeContext[i].code == code) {
+        return &respondCodeContext[i];
+      }
+    }
+    return NULL;
+  }
+
+  // Creation of a new transaction, return the transaction ID if success,
+  // Return NULL when any error occurs
+  Transaction *createTransaction(std::string &transactionID,
+                                 TransferDirection direction);
+  // Receive the data packet from the transaction
+  // Return false when any error occurs
+  bool receive(std::string transactionID, DataPacket *packet, bool &eof);
+  // Send the data packet from the transaction
+  // Return false when any error occurs
+  bool send(std::string transactionID, DataPacket *packet,
+            std::shared_ptr<FlowFileRecord> flowFile,
+            core::ProcessSession *session);
+  // Confirm the data that was sent or received by comparing CRC32's of the data sent and the data received.
+  bool confirm(std::string transactionID);
+  // Cancel the transaction
+  void cancel(std::string transactionID);
+  // Complete the transaction
+  bool complete(std::string transactionID);
+  // Error the transaction
+  void error(std::string transactionID);
+  // Receive flow files for the process session
+  void receiveFlowFiles(
+      core::ProcessContext *context,
+      core::ProcessSession *session);
+  // Transfer flow files for the process session
+  void transferFlowFiles(
+      core::ProcessContext *context,
+      core::ProcessSession *session);
+  // deleteTransaction
+  void deleteTransaction(std::string transactionID);
+  // Nest Callback Class for write stream
+  class WriteCallback : public OutputStreamCallback {
+   public:
+    WriteCallback(DataPacket *packet)
+        : _packet(packet) {
+    }
+    DataPacket *_packet;
+    void process(std::ofstream *stream) {
+      uint8_t buffer[8192];
+      int len = _packet->_size;
+      while (len > 0) {
+        int size = std::min(len, (int) sizeof(buffer));
+        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;
+        }
+        stream->write((const char *) buffer, size);
+        len -= size;
+      }
+    }
+  };
+  // Nest Callback Class for read stream
+  class ReadCallback : public InputStreamCallback {
+   public:
+    ReadCallback(DataPacket *packet)
+        : _packet(packet) {
+    }
+    DataPacket *_packet;
+    void process(std::ifstream *stream) {
+      _packet->_size = 0;
+      uint8_t buffer[8192];
+      int readSize;
+      while (stream->good()) {
+        if (!stream->read((char *) buffer, 8192))
+          readSize = stream->gcount();
+        else
+          readSize = 8192;
+        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;
+        }
+        _packet->_size += readSize;
+      }
+    }
+  };
+
+ protected:
+
+ private:
+
+  // Mutex for protection
+  std::mutex mutex_;
+  // Logger
+  std::shared_ptr<logging::Logger> logger_;
+  // Configure
+  Configure *configure_;
+  // Batch Count
+  std::atomic<uint64_t> _batchCount;
+  // Batch Size
+  std::atomic<uint64_t> _batchSize;
+  // Batch Duration in msec
+  std::atomic<uint64_t> _batchDuration;
+  // Timeout in msec
+  std::atomic<uint64_t> _timeOut;
+  // Peer Connection
+  std::unique_ptr<Site2SitePeer> peer_;
+  // portId
+  uuid_t _portId;
+  // portIDStr
+  std::string _portIdStr;
+  // BATCH_SEND_NANOS
+  uint64_t _batchSendNanos;
+  // Peer State
+  PeerState _peerState;
+  uint32_t _supportedVersion[5];
+  uint32_t _currentVersion;
+  int _currentVersionIndex;
+  uint32_t _supportedCodecVersion[1];
+  uint32_t _currentCodecVersion;
+  int _currentCodecVersionIndex;
+  // commsIdentifier
+  std::string _commsIdentifier;
+  // transaction map
+  std::map<std::string, Transaction *> _transactionMap;
+
+  // Prevent default copy constructor and assignment operation
+  // Only support pass by reference or pointer
+  Site2SiteClientProtocol(const Site2SiteClientProtocol &parent);
+  Site2SiteClientProtocol &operator=(const Site2SiteClientProtocol &parent);
 };
 
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
 #endif


[03/16] nifi-minifi-cpp git commit: MINIFI-217: Updates namespaces and removes use of raw pointers for user facing API.

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/processors/ExecuteProcess.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/ExecuteProcess.cpp b/libminifi/src/processors/ExecuteProcess.cpp
new file mode 100644
index 0000000..3cbbc1b
--- /dev/null
+++ b/libminifi/src/processors/ExecuteProcess.cpp
@@ -0,0 +1,255 @@
+/**
+ * @file ExecuteProcess.cpp
+ * ExecuteProcess class implementation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include "processors/ExecuteProcess.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include <cstring>
+#include "utils/StringUtils.h"
+#include "utils/TimeUtil.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const std::string ExecuteProcess::ProcessorName("ExecuteProcess");
+core::Property ExecuteProcess::Command(
+    "Command",
+    "Specifies the command to be executed; if just the name of an executable is provided, it must be in the user's environment PATH.",
+    "");
+core::Property ExecuteProcess::CommandArguments(
+    "Command Arguments",
+    "The arguments to supply to the executable delimited by white space. White space can be escaped by enclosing it in double-quotes.",
+    "");
+core::Property ExecuteProcess::WorkingDir(
+    "Working Directory",
+    "The directory to use as the current working directory when executing the command",
+    "");
+core::Property ExecuteProcess::BatchDuration(
+    "Batch Duration",
+    "If the process is expected to be long-running and produce textual output, a batch duration can be specified.",
+    "0");
+core::Property ExecuteProcess::RedirectErrorStream(
+    "Redirect Error Stream",
+    "If true will redirect any error stream output of the process to the output stream.",
+    "false");
+core::Relationship ExecuteProcess::Success(
+    "success", "All created FlowFiles are routed to this relationship.");
+
+void ExecuteProcess::initialize() {
+  // Set the supported properties
+  std::set<core::Property> properties;
+  properties.insert(Command);
+  properties.insert(CommandArguments);
+  properties.insert(WorkingDir);
+  properties.insert(BatchDuration);
+  properties.insert(RedirectErrorStream);
+  setSupportedProperties(properties);
+  // Set the supported relationships
+  std::set<core::Relationship> relationships;
+  relationships.insert(Success);
+  setSupportedRelationships(relationships);
+}
+
+void ExecuteProcess::onTrigger(
+    core::ProcessContext *context,
+    core::ProcessSession *session) {
+  std::string value;
+  if (context->getProperty(Command.getName(), value)) {
+    this->_command = value;
+  }
+  if (context->getProperty(CommandArguments.getName(), value)) {
+    this->_commandArgument = value;
+  }
+  if (context->getProperty(WorkingDir.getName(), value)) {
+    this->_workingDir = value;
+  }
+  if (context->getProperty(BatchDuration.getName(), value)) {
+    core::TimeUnit unit;
+    if (core::Property::StringToTime(value,
+                                                                _batchDuration,
+                                                                unit)
+        && core::Property::ConvertTimeUnitToMS(
+            _batchDuration, unit, _batchDuration)) {
+
+    }
+  }
+  if (context->getProperty(RedirectErrorStream.getName(), value)) {
+    org::apache::nifi::minifi::utils::StringUtils::StringToBool(
+        value, _redirectErrorStream);
+  }
+  this->_fullCommand = _command + " " + _commandArgument;
+  if (_fullCommand.length() == 0) {
+    yield();
+    return;
+  }
+  if (_workingDir.length() > 0 && _workingDir != ".") {
+    // change to working directory
+    if (chdir(_workingDir.c_str()) != 0) {
+      logger_->log_error("Execute Command can not chdir %s",
+                         _workingDir.c_str());
+      yield();
+      return;
+    }
+  }
+  logger_->log_info("Execute Command %s", _fullCommand.c_str());
+  // split the command into array
+  char cstr[_fullCommand.length() + 1];
+  std::strcpy(cstr, _fullCommand.c_str());
+  char *p = std::strtok(cstr, " ");
+  int argc = 0;
+  char *argv[64];
+  while (p != 0 && argc < 64) {
+    argv[argc] = p;
+    p = std::strtok(NULL, " ");
+    argc++;
+  }
+  argv[argc] = NULL;
+  int status, died;
+  if (!_processRunning) {
+    _processRunning = true;
+    // if the process has not launched yet
+    // create the pipe
+    if (pipe(_pipefd) == -1) {
+      _processRunning = false;
+      yield();
+      return;
+    }
+    switch (_pid = fork()) {
+      case -1:
+        logger_->log_error("Execute Process fork failed");
+        _processRunning = false;
+        close(_pipefd[0]);
+        close(_pipefd[1]);
+        yield();
+        break;
+      case 0:  // this is the code the child runs
+        close(1);      // close stdout
+        dup(_pipefd[1]);  // points pipefd at file descriptor
+        if (_redirectErrorStream)
+          // redirect stderr
+          dup2(_pipefd[1], 2);
+        close(_pipefd[0]);
+        execvp(argv[0], argv);
+        exit(1);
+        break;
+      default:  // this is the code the parent runs
+        // the parent isn't going to write to the pipe
+        close(_pipefd[1]);
+        if (_batchDuration > 0) {
+          while (1) {
+            std::this_thread::sleep_for(
+                std::chrono::milliseconds(_batchDuration));
+            char buffer[4096];
+            int numRead = read(_pipefd[0], buffer, sizeof(buffer));
+            if (numRead <= 0)
+              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());
+            if (!flowFile)
+              continue;
+            flowFile->addAttribute("command", _command.c_str());
+            flowFile->addAttribute("command.arguments",
+                                   _commandArgument.c_str());
+            session->write(flowFile, &callback);
+            session->transfer(flowFile, Success);
+            session->commit();
+          }
+        } else {
+          char buffer[4096];
+          char *bufPtr = buffer;
+          int totalRead = 0;
+          std::shared_ptr<FlowFileRecord> flowFile = nullptr;
+          while (1) {
+            int numRead = read(_pipefd[0], bufPtr,
+                               (sizeof(buffer) - totalRead));
+            if (numRead <= 0) {
+              if (totalRead > 0) {
+                logger_->log_info("Execute Command Respond %d", totalRead);
+                // child exits and close the pipe
+                ExecuteProcess::WriteCallback callback(buffer, totalRead);
+                if (!flowFile) {
+                  flowFile = std::static_pointer_cast<FlowFileRecord>(
+                      session->create());
+                  if (!flowFile)
+                    break;
+                  flowFile->addAttribute("command", _command.c_str());
+                  flowFile->addAttribute("command.arguments",
+                                         _commandArgument.c_str());
+                  session->write(flowFile, &callback);
+                } else {
+                  session->append(flowFile, &callback);
+                }
+                session->transfer(flowFile, Success);
+              }
+              break;
+            } else {
+              if (numRead == (sizeof(buffer) - totalRead)) {
+                // we reach the max buffer size
+                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());
+                  if (!flowFile)
+                    continue;
+                  flowFile->addAttribute("command", _command.c_str());
+                  flowFile->addAttribute("command.arguments",
+                                         _commandArgument.c_str());
+                  session->write(flowFile, &callback);
+                } else {
+                  session->append(flowFile, &callback);
+                }
+                // Rewind
+                totalRead = 0;
+                bufPtr = buffer;
+              } else {
+                totalRead += numRead;
+                bufPtr += numRead;
+              }
+            }
+          }
+        }
+
+        died = wait(&status);
+        if (WIFEXITED(status)) {
+          logger_->log_info("Execute Command Complete %s status %d pid %d",
+                            _fullCommand.c_str(), WEXITSTATUS(status), _pid);
+        } else {
+          logger_->log_info("Execute Command Complete %s status %d pid %d",
+                            _fullCommand.c_str(), WTERMSIG(status), _pid);
+        }
+
+        close(_pipefd[0]);
+        _processRunning = false;
+        break;
+    }
+  }
+}
+
+} /* namespace processors */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/processors/GenerateFlowFile.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/GenerateFlowFile.cpp b/libminifi/src/processors/GenerateFlowFile.cpp
new file mode 100644
index 0000000..ebdaaa3
--- /dev/null
+++ b/libminifi/src/processors/GenerateFlowFile.cpp
@@ -0,0 +1,145 @@
+/**
+ * @file GenerateFlowFile.cpp
+ * GenerateFlowFile class implementation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <vector>
+#include <queue>
+#include <map>
+#include <set>
+#include <sys/time.h>
+#include <time.h>
+#include <chrono>
+#include <thread>
+#include <random>
+#include "utils/StringUtils.h"
+
+#include "processors/GenerateFlowFile.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+const char *GenerateFlowFile::DATA_FORMAT_BINARY = "Binary";
+const char *GenerateFlowFile::DATA_FORMAT_TEXT = "Text";
+const std::string GenerateFlowFile::ProcessorName("GenerateFlowFile");
+core::Property GenerateFlowFile::FileSize("File Size", "The size of the file that will be used", "1 kB");
+core::Property GenerateFlowFile::BatchSize("Batch Size", "The number of FlowFiles to be transferred in each invocation", "1");
+core::Property GenerateFlowFile::DataFormat("Data Format", "Specifies whether the data should be Text or Binary", GenerateFlowFile::DATA_FORMAT_BINARY);
+core::Property GenerateFlowFile::UniqueFlowFiles("Unique FlowFiles",
+		"If true, each FlowFile that is generated will be unique. If false, a random value will be generated and all FlowFiles", "true");
+core::Relationship GenerateFlowFile::Success("success", "success operational on the flow record");
+
+void GenerateFlowFile::initialize()
+{
+	// Set the supported properties
+	std::set<core::Property> properties;
+	properties.insert(FileSize);
+	properties.insert(BatchSize);
+	properties.insert(DataFormat);
+	properties.insert(UniqueFlowFiles);
+	setSupportedProperties(properties);
+	// Set the supported relationships
+	std::set<core::Relationship> relationships;
+	relationships.insert(Success);
+	setSupportedRelationships(relationships);
+}
+
+void GenerateFlowFile::onTrigger(core::ProcessContext *context, core::ProcessSession *session)
+{
+	int64_t batchSize = 1;
+	bool uniqueFlowFile = true;
+	int64_t fileSize = 1024;
+
+	std::string value;
+	if (context->getProperty(FileSize.getName(), value))
+	{
+	  core::Property::StringToInt(value, fileSize);
+	}
+	if (context->getProperty(BatchSize.getName(), value))
+	{
+	  core::Property::StringToInt(value, batchSize);
+	}
+	if (context->getProperty(UniqueFlowFiles.getName(), value))
+	{
+		org::apache::nifi::minifi::utils::StringUtils::StringToBool(value, uniqueFlowFile);
+	}
+
+	if (!uniqueFlowFile)
+	{
+		char *data;
+		data = new char[fileSize];
+		if (!data)
+			return;
+		uint64_t dataSize = fileSize;
+		GenerateFlowFile::WriteCallback callback(data, dataSize);
+		char *current = data;
+		for (int i = 0; i < fileSize; i+= sizeof(int))
+		{
+			int randValue = random();
+			*((int *) current) = randValue;
+			current += sizeof(int);
+		}
+		for (int i = 0; i < batchSize; i++)
+		{
+			// For each batch
+			std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast<FlowFileRecord>(session->create());
+			if (!flowFile)
+				return;
+			if (fileSize > 0)
+				session->write(flowFile, &callback);
+			session->transfer(flowFile, Success);
+		}
+		delete[] data;
+	}
+	else
+	{
+		if (!_data)
+		{
+			// We have not create the unique data yet
+			_data = new char[fileSize];
+			_dataSize = fileSize;
+			char *current = _data;
+			for (int i = 0; i < fileSize; i+= sizeof(int))
+			{
+				int randValue = random();
+				*((int *) current) = randValue;
+				// *((int *) current) = (0xFFFFFFFF & i);
+				current += sizeof(int);
+			}
+		}
+		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());
+			if (!flowFile)
+				return;
+			if (fileSize > 0)
+				session->write(flowFile, &callback);
+			session->transfer(flowFile, Success);
+		}
+	}
+}
+} /* namespace processors */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/processors/GetFile.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/GetFile.cpp b/libminifi/src/processors/GetFile.cpp
new file mode 100644
index 0000000..cf05657
--- /dev/null
+++ b/libminifi/src/processors/GetFile.cpp
@@ -0,0 +1,340 @@
+/**
+ * @file GetFile.cpp
+ * GetFile class implementation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <vector>
+#include <queue>
+#include <map>
+#include <set>
+#include <sys/time.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <time.h>
+#include <sstream>
+#include <stdio.h>
+#include <string>
+#include <iostream>
+#include <dirent.h>
+#include <limits.h>
+#include <unistd.h>
+#if  (__GNUC__ >= 4) 
+#if (__GNUC_MINOR__ < 9)
+#include <regex.h>
+#endif
+#endif
+#include "utils/StringUtils.h"
+#include <regex>
+#include "utils/TimeUtil.h"
+#include "processors/GetFile.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+const std::string GetFile::ProcessorName("GetFile");
+core::Property GetFile::BatchSize(
+    "Batch Size", "The maximum number of files to pull in each iteration",
+    "10");
+core::Property GetFile::Directory(
+    "Input Directory", "The input directory from which to pull files", ".");
+core::Property GetFile::IgnoreHiddenFile(
+    "Ignore Hidden Files",
+    "Indicates whether or not hidden files should be ignored", "true");
+core::Property GetFile::KeepSourceFile(
+    "Keep Source File",
+    "If true, the file is not deleted after it has been copied to the Content Repository",
+    "false");
+core::Property GetFile::MaxAge(
+    "Maximum File Age",
+    "The minimum age that a file must be in order to be pulled; any file younger than this amount of time (according to last modification date) will be ignored",
+    "0 sec");
+core::Property GetFile::MinAge(
+    "Minimum File Age",
+    "The maximum age that a file must be in order to be pulled; any file older than this amount of time (according to last modification date) will be ignored",
+    "0 sec");
+core::Property GetFile::MaxSize(
+    "Maximum File Size",
+    "The maximum size that a file can be in order to be pulled", "0 B");
+core::Property GetFile::MinSize(
+    "Minimum File Size",
+    "The minimum size that a file must be in order to be pulled", "0 B");
+core::Property GetFile::PollInterval(
+    "Polling Interval",
+    "Indicates how long to wait before performing a directory listing",
+    "0 sec");
+core::Property GetFile::Recurse(
+    "Recurse Subdirectories",
+    "Indicates whether or not to pull files from subdirectories", "true");
+core::Property GetFile::FileFilter(
+    "File Filter",
+    "Only files whose names match the given regular expression will be picked up",
+    "[^\\.].*");
+core::Relationship GetFile::Success(
+    "success", "All files are routed to success");
+
+void GetFile::initialize() {
+  // Set the supported properties
+  std::set<core::Property> properties;
+  properties.insert(BatchSize);
+  properties.insert(Directory);
+  properties.insert(IgnoreHiddenFile);
+  properties.insert(KeepSourceFile);
+  properties.insert(MaxAge);
+  properties.insert(MinAge);
+  properties.insert(MaxSize);
+  properties.insert(MinSize);
+  properties.insert(PollInterval);
+  properties.insert(Recurse);
+  properties.insert(FileFilter);
+  setSupportedProperties(properties);
+  // Set the supported relationships
+  std::set<core::Relationship> relationships;
+  relationships.insert(Success);
+  setSupportedRelationships(relationships);
+}
+
+void GetFile::onTrigger(
+    core::ProcessContext *context,
+    core::ProcessSession *session) {
+  std::string value;
+
+  logger_->log_info("onTrigger GetFile");
+  if (context->getProperty(Directory.getName(), value)) {
+    _directory = value;
+  }
+  if (context->getProperty(BatchSize.getName(), value)) {
+    core::Property::StringToInt(value, _batchSize);
+  }
+  if (context->getProperty(IgnoreHiddenFile.getName(), value)) {
+    org::apache::nifi::minifi::utils::StringUtils::StringToBool(
+        value, _ignoreHiddenFile);
+  }
+  if (context->getProperty(KeepSourceFile.getName(), value)) {
+    org::apache::nifi::minifi::utils::StringUtils::StringToBool(
+        value, _keepSourceFile);
+  }
+
+  logger_->log_info("onTrigger GetFile");
+  if (context->getProperty(MaxAge.getName(), value)) {
+    core::TimeUnit unit;
+    if (core::Property::StringToTime(value, _maxAge,
+                                                                unit)
+        && core::Property::ConvertTimeUnitToMS(
+            _maxAge, unit, _maxAge)) {
+
+    }
+  }
+  if (context->getProperty(MinAge.getName(), value)) {
+    core::TimeUnit unit;
+    if (core::Property::StringToTime(value, _minAge,
+                                                                unit)
+        && core::Property::ConvertTimeUnitToMS(
+            _minAge, unit, _minAge)) {
+
+    }
+  }
+  if (context->getProperty(MaxSize.getName(), value)) {
+    core::Property::StringToInt(value, _maxSize);
+  }
+  if (context->getProperty(MinSize.getName(), value)) {
+    core::Property::StringToInt(value, _minSize);
+  }
+  if (context->getProperty(PollInterval.getName(), value)) {
+    core::TimeUnit unit;
+    if (core::Property::StringToTime(value,
+                                                                _pollInterval,
+                                                                unit)
+        && core::Property::ConvertTimeUnitToMS(
+            _pollInterval, unit, _pollInterval)) {
+
+    }
+  }
+  if (context->getProperty(Recurse.getName(), value)) {
+    org::apache::nifi::minifi::utils::StringUtils::StringToBool(value,
+                                                                _recursive);
+  }
+
+  if (context->getProperty(FileFilter.getName(), value)) {
+    _fileFilter = value;
+  }
+
+  // Perform directory list
+  logger_->log_info("Is listing empty %i", isListingEmpty());
+  if (isListingEmpty()) {
+    if (_pollInterval == 0
+        || (getTimeMillis() - _lastDirectoryListingTime) > _pollInterval) {
+      performListing(_directory);
+    }
+  }
+  logger_->log_info("Is listing empty %i", isListingEmpty());
+
+  if (!isListingEmpty()) {
+    try {
+      std::queue<std::string> list;
+      pollListing(list, _batchSize);
+      while (!list.empty()) {
+
+        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());
+        if (flowFile == nullptr)
+          return;
+        std::size_t found = fileName.find_last_of("/\\");
+        std::string path = fileName.substr(0, found);
+        std::string name = fileName.substr(found + 1);
+        flowFile->updateKeyedAttribute(FILENAME, name);
+        flowFile->updateKeyedAttribute(PATH, path);
+        flowFile->addKeyedAttribute(ABSOLUTE_PATH, fileName);
+        session->import(fileName, flowFile, _keepSourceFile);
+        session->transfer(flowFile, Success);
+      }
+    } catch (std::exception &exception) {
+      logger_->log_debug("GetFile Caught Exception %s", exception.what());
+      throw;
+    } catch (...) {
+      throw;
+    }
+  }
+
+}
+
+bool GetFile::isListingEmpty() {
+  std::lock_guard<std::mutex> lock(mutex_);
+
+  return _dirList.empty();
+}
+
+void GetFile::putListing(std::string fileName) {
+  std::lock_guard<std::mutex> lock(mutex_);
+
+  _dirList.push(fileName);
+}
+
+void GetFile::pollListing(std::queue<std::string> &list, int maxSize) {
+  std::lock_guard<std::mutex> lock(mutex_);
+
+  while (!_dirList.empty() && (maxSize == 0 || list.size() < maxSize)) {
+    std::string fileName = _dirList.front();
+    _dirList.pop();
+    list.push(fileName);
+  }
+
+  return;
+}
+
+bool GetFile::acceptFile(std::string fullName, std::string name) {
+  struct stat statbuf;
+
+  if (stat(fullName.c_str(), &statbuf) == 0) {
+    if (_minSize > 0 && statbuf.st_size < _minSize)
+      return false;
+
+    if (_maxSize > 0 && statbuf.st_size > _maxSize)
+      return false;
+
+    uint64_t modifiedTime = ((uint64_t) (statbuf.st_mtime) * 1000);
+    uint64_t fileAge = getTimeMillis() - modifiedTime;
+    if (_minAge > 0 && fileAge < _minAge)
+      return false;
+    if (_maxAge > 0 && fileAge > _maxAge)
+      return false;
+
+    if (_ignoreHiddenFile && fullName.c_str()[0] == '.')
+      return false;
+
+    if (access(fullName.c_str(), R_OK) != 0)
+      return false;
+
+    if (_keepSourceFile == false && access(fullName.c_str(), W_OK) != 0)
+      return false;
+
+#ifdef __GNUC__
+#if (__GNUC__ >= 4)
+#if (__GNUC_MINOR__ < 9)
+    regex_t regex;
+    int ret = regcomp(&regex, _fileFilter.c_str(), 0);
+    if (ret)
+      return false;
+    ret = regexec(&regex, name.c_str(), (size_t) 0, NULL, 0);
+    regfree(&regex);
+    if (ret)
+      return false;
+#else
+    try {
+      std::regex re(_fileFilter);
+
+      if (!std::regex_match(name, re)) {
+        return false;
+      }
+    } catch (std::regex_error e) {
+      logger_->log_error("Invalid File Filter regex: %s.", e.what());
+      return false;
+    }
+#endif
+#endif
+#else
+    logger_->log_info("Cannot support regex filtering");
+#endif
+    return true;
+  }
+
+  return false;
+}
+
+void GetFile::performListing(std::string dir) {
+  logger_->log_info("Performing file listing against %s", dir.c_str());
+  DIR *d;
+  d = opendir(dir.c_str());
+  if (!d)
+    return;
+  // only perform a listing while we are not empty
+  logger_->log_info("Performing file listing against %s", dir.c_str());
+  while (isRunning()) {
+    struct dirent *entry;
+    entry = readdir(d);
+    if (!entry)
+      break;
+    std::string d_name = entry->d_name;
+    if ((entry->d_type & DT_DIR)) {
+      // if this is a directory
+      if (_recursive && strcmp(d_name.c_str(), "..") != 0
+          && strcmp(d_name.c_str(), ".") != 0) {
+        std::string path = dir + "/" + d_name;
+        performListing(path);
+      }
+    } else {
+      std::string fileName = dir + "/" + d_name;
+      if (acceptFile(fileName, d_name)) {
+        // check whether we can take this file
+        putListing(fileName);
+      }
+    }
+  }
+  closedir(d);
+}
+
+} /* namespace processors */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/processors/ListenHTTP.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/ListenHTTP.cpp b/libminifi/src/processors/ListenHTTP.cpp
new file mode 100644
index 0000000..36c743e
--- /dev/null
+++ b/libminifi/src/processors/ListenHTTP.cpp
@@ -0,0 +1,380 @@
+/**
+ * @file ListenHTTP.cpp
+ * ListenHTTP class implementation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <sstream>
+#include <stdio.h>
+#include <string>
+#include <iostream>
+#include <fstream>
+#include <uuid/uuid.h>
+
+#include <CivetServer.h>
+
+#include "processors/ListenHTTP.h"
+
+#include "utils/TimeUtil.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/ProcessSessionFactory.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const std::string ListenHTTP::ProcessorName("ListenHTTP");
+
+core::Property ListenHTTP::BasePath(
+    "Base Path", "Base path for incoming connections", "contentListener");
+core::Property ListenHTTP::Port(
+    "Listening Port", "The Port to listen on for incoming connections", "");
+core::Property ListenHTTP::AuthorizedDNPattern(
+    "Authorized DN Pattern",
+    "A Regular Expression to apply against the Distinguished Name of incoming connections. If the Pattern does not match the DN, the connection will be refused.",
+    ".*");
+core::Property ListenHTTP::SSLCertificate(
+    "SSL Certificate",
+    "File containing PEM-formatted file including TLS/SSL certificate and key",
+    "");
+core::Property ListenHTTP::SSLCertificateAuthority(
+    "SSL Certificate Authority",
+    "File containing trusted PEM-formatted certificates", "");
+core::Property ListenHTTP::SSLVerifyPeer(
+    "SSL Verify Peer",
+    "Whether or not to verify the client's certificate (yes/no)", "no");
+core::Property ListenHTTP::SSLMinimumVersion(
+    "SSL Minimum Version",
+    "Minimum TLS/SSL version allowed (SSL2, SSL3, TLS1.0, TLS1.1, TLS1.2)",
+    "SSL2");
+core::Property ListenHTTP::HeadersAsAttributesRegex(
+    "HTTP Headers to receive as Attributes (Regex)",
+    "Specifies the Regular Expression that determines the names of HTTP Headers that should be passed along as FlowFile attributes",
+    "");
+
+core::Relationship ListenHTTP::Success(
+    "success", "All files are routed to success");
+
+void ListenHTTP::initialize() {
+  _logger->log_info("Initializing ListenHTTP");
+
+  // Set the supported properties
+  std::set < core::Property > properties;
+  properties.insert(BasePath);
+  properties.insert(Port);
+  properties.insert(AuthorizedDNPattern);
+  properties.insert(SSLCertificate);
+  properties.insert(SSLCertificateAuthority);
+  properties.insert(SSLVerifyPeer);
+  properties.insert(SSLMinimumVersion);
+  properties.insert(HeadersAsAttributesRegex);
+  setSupportedProperties (properties);
+  // Set the supported relationships
+  std::set < core::Relationship > relationships;
+  relationships.insert(Success);
+  setSupportedRelationships (relationships);
+}
+
+void ListenHTTP::onSchedule(
+    core::ProcessContext *context,
+    core::ProcessSessionFactory *sessionFactory) {
+
+  std::string basePath;
+
+  if (!context->getProperty(BasePath.getName(), basePath)) {
+    _logger->log_info(
+        "%s attribute is missing, so default value of %s will be used",
+        BasePath.getName().c_str(), BasePath.getValue().c_str());
+    basePath = BasePath.getValue();
+  }
+
+  basePath.insert(0, "/");
+
+  std::string listeningPort;
+
+  if (!context->getProperty(Port.getName(), listeningPort)) {
+    _logger->log_error("%s attribute is missing or invalid",
+                       Port.getName().c_str());
+    return;
+  }
+
+  std::string authDNPattern;
+
+  if (context->getProperty(AuthorizedDNPattern.getName(), authDNPattern)
+      && !authDNPattern.empty()) {
+    _logger->log_info("ListenHTTP using %s: %s",
+                      AuthorizedDNPattern.getName().c_str(),
+                      authDNPattern.c_str());
+  }
+
+  std::string sslCertFile;
+
+  if (context->getProperty(SSLCertificate.getName(), sslCertFile)
+      && !sslCertFile.empty()) {
+    _logger->log_info("ListenHTTP using %s: %s",
+                      SSLCertificate.getName().c_str(), sslCertFile.c_str());
+  }
+
+  // Read further TLS/SSL options only if TLS/SSL usage is implied by virtue of certificate value being set
+  std::string sslCertAuthorityFile;
+  std::string sslVerifyPeer;
+  std::string sslMinVer;
+
+  if (!sslCertFile.empty()) {
+    if (context->getProperty(SSLCertificateAuthority.getName(),
+                             sslCertAuthorityFile)
+        && !sslCertAuthorityFile.empty()) {
+      _logger->log_info("ListenHTTP using %s: %s",
+                        SSLCertificateAuthority.getName().c_str(),
+                        sslCertAuthorityFile.c_str());
+    }
+
+    if (context->getProperty(SSLVerifyPeer.getName(), sslVerifyPeer)) {
+      if (sslVerifyPeer.empty() || sslVerifyPeer.compare("no") == 0) {
+        _logger->log_info("ListenHTTP will not verify peers");
+      } else {
+        _logger->log_info("ListenHTTP will verify peers");
+      }
+    } else {
+      _logger->log_info("ListenHTTP will not verify peers");
+    }
+
+    if (context->getProperty(SSLMinimumVersion.getName(), sslMinVer)) {
+      _logger->log_info("ListenHTTP using %s: %s",
+                        SSLMinimumVersion.getName().c_str(), sslMinVer.c_str());
+    }
+  }
+
+  std::string headersAsAttributesPattern;
+
+  if (context->getProperty(HeadersAsAttributesRegex.getName(),
+                           headersAsAttributesPattern)
+      && !headersAsAttributesPattern.empty()) {
+    _logger->log_info("ListenHTTP using %s: %s",
+                      HeadersAsAttributesRegex.getName().c_str(),
+                      headersAsAttributesPattern.c_str());
+  }
+
+  auto numThreads = getMaxConcurrentTasks();
+
+  _logger->log_info(
+      "ListenHTTP starting HTTP server on port %s and path %s with %d threads",
+      listeningPort.c_str(), basePath.c_str(), numThreads);
+
+  // Initialize web server
+  std::vector < std::string > options;
+  options.push_back("enable_keep_alive");
+  options.push_back("yes");
+  options.push_back("keep_alive_timeout_ms");
+  options.push_back("15000");
+  options.push_back("num_threads");
+  options.push_back(std::to_string(numThreads));
+
+  if (sslCertFile.empty()) {
+    options.push_back("listening_ports");
+    options.push_back(listeningPort);
+  } else {
+    listeningPort += "s";
+    options.push_back("listening_ports");
+    options.push_back(listeningPort);
+
+    options.push_back("ssl_certificate");
+    options.push_back(sslCertFile);
+
+    if (!sslCertAuthorityFile.empty()) {
+      options.push_back("ssl_ca_file");
+      options.push_back(sslCertAuthorityFile);
+    }
+
+    if (sslVerifyPeer.empty() || sslVerifyPeer.compare("no") == 0) {
+      options.push_back("ssl_verify_peer");
+      options.push_back("no");
+    } else {
+      options.push_back("ssl_verify_peer");
+      options.push_back("yes");
+    }
+
+    if (sslMinVer.compare("SSL2") == 0) {
+      options.push_back("ssl_protocol_version");
+      options.push_back(std::to_string(0));
+    } else if (sslMinVer.compare("SSL3") == 0) {
+      options.push_back("ssl_protocol_version");
+      options.push_back(std::to_string(1));
+    } else if (sslMinVer.compare("TLS1.0") == 0) {
+      options.push_back("ssl_protocol_version");
+      options.push_back(std::to_string(2));
+    } else if (sslMinVer.compare("TLS1.1") == 0) {
+      options.push_back("ssl_protocol_version");
+      options.push_back(std::to_string(3));
+    } else {
+      options.push_back("ssl_protocol_version");
+      options.push_back(std::to_string(4));
+    }
+  }
+
+  _server.reset(new CivetServer(options));
+  _handler.reset(
+      new Handler(context, sessionFactory, std::move(authDNPattern),
+                  std::move(headersAsAttributesPattern)));
+  _server->addHandler(basePath, _handler.get());
+}
+
+ListenHTTP::~ListenHTTP() {
+}
+
+void ListenHTTP::onTrigger(
+    core::ProcessContext *context,
+    core::ProcessSession *session) {
+
+  std::shared_ptr < FlowFileRecord > flowFile = std::static_pointer_cast
+      < FlowFileRecord > (session->get());
+
+  // Do nothing if there are no incoming files
+  if (!flowFile) {
+    return;
+  }
+}
+
+ListenHTTP::Handler::Handler(
+    core::ProcessContext *context,
+    core::ProcessSessionFactory *sessionFactory,
+    std::string &&authDNPattern, std::string &&headersAsAttributesPattern)
+    : _authDNRegex(std::move(authDNPattern)),
+      _headersAsAttributesRegex(std::move(headersAsAttributesPattern)) {
+  _processContext = context;
+  _processSessionFactory = sessionFactory;
+}
+
+void ListenHTTP::Handler::sendErrorResponse(struct mg_connection *conn) {
+  mg_printf(conn, "HTTP/1.1 500 Internal Server Error\r\n"
+            "Content-Type: text/html\r\n"
+            "Content-Length: 0\r\n\r\n");
+}
+
+bool ListenHTTP::Handler::handlePost(CivetServer *server,
+                                     struct mg_connection *conn) {
+  _logger = logging::Logger::getLogger();
+
+  auto req_info = mg_get_request_info(conn);
+  _logger->log_info("ListenHTTP handling POST request of length %d",
+                    req_info->content_length);
+
+  // If this is a two-way TLS connection, authorize the peer against the configured pattern
+  if (req_info->is_ssl && req_info->client_cert != nullptr) {
+    if (!std::regex_match(req_info->client_cert->subject, _authDNRegex)) {
+      mg_printf(conn, "HTTP/1.1 403 Forbidden\r\n"
+                "Content-Type: text/html\r\n"
+                "Content-Length: 0\r\n\r\n");
+      _logger->log_warn("ListenHTTP client DN not authorized: %s",
+                        req_info->client_cert->subject);
+      return true;
+    }
+  }
+
+  // Always send 100 Continue, as allowed per standard to minimize client delay (https://www.w3.org/Protocols/rfc2616/rfc2616-sec8.html)
+  mg_printf(conn, "HTTP/1.1 100 Continue\r\n\r\n");
+
+  auto session = _processSessionFactory->createSession();
+  ListenHTTP::WriteCallback callback(conn, req_info);
+  auto flowFile = std::static_pointer_cast < FlowFileRecord
+      > (session->create());
+
+  if (!flowFile) {
+    sendErrorResponse(conn);
+    return true;
+  }
+
+  try {
+    session->write(flowFile, &callback);
+
+    // Add filename from "filename" header value (and pattern headers)
+    for (int i = 0; i < req_info->num_headers; i++) {
+      auto header = &req_info->http_headers[i];
+
+      if (strcmp("filename", header->name) == 0) {
+        if (!flowFile->updateAttribute("filename", header->value)) {
+          flowFile->addAttribute("filename", header->value);
+        }
+      } else if (std::regex_match(header->name, _headersAsAttributesRegex)) {
+        if (!flowFile->updateAttribute(header->name, header->value)) {
+          flowFile->addAttribute(header->name, header->value);
+        }
+      }
+    }
+
+    session->transfer(flowFile, Success);
+    session->commit();
+  } catch (std::exception &exception) {
+    _logger->log_debug("ListenHTTP Caught Exception %s", exception.what());
+    sendErrorResponse(conn);
+    session->rollback();
+    throw;
+  } catch (...) {
+    _logger->log_debug("ListenHTTP Caught Exception Processor::onTrigger");
+    sendErrorResponse(conn);
+    session->rollback();
+    throw;
+  }
+
+  mg_printf(conn, "HTTP/1.1 200 OK\r\n"
+            "Content-Type: text/html\r\n"
+            "Content-Length: 0\r\n\r\n");
+
+  return true;
+}
+
+ListenHTTP::WriteCallback::WriteCallback(
+    struct mg_connection *conn, const struct mg_request_info *reqInfo) {
+  _logger = logging::Logger::getLogger();
+  _conn = conn;
+  _reqInfo = reqInfo;
+}
+
+void ListenHTTP::WriteCallback::process(std::ofstream *stream) {
+  long long rlen;
+  long long nlen = 0;
+  long long tlen = _reqInfo->content_length;
+  char buf[16384];
+
+  while (nlen < tlen) {
+    rlen = tlen - nlen;
+
+    if (rlen > sizeof(buf)) {
+      rlen = sizeof(buf);
+    }
+
+    // Read a buffer of data from client
+    rlen = mg_read(_conn, &buf[0], (size_t) rlen);
+
+    if (rlen <= 0) {
+      break;
+    }
+
+    // Transfer buffer data to the output stream
+    stream->write(&buf[0], rlen);
+
+    nlen += rlen;
+  }
+}
+
+} /* namespace processors */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/processors/ListenSyslog.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/ListenSyslog.cpp b/libminifi/src/processors/ListenSyslog.cpp
new file mode 100644
index 0000000..2dd223c
--- /dev/null
+++ b/libminifi/src/processors/ListenSyslog.cpp
@@ -0,0 +1,331 @@
+/**
+ * @file ListenSyslog.cpp
+ * ListenSyslog class implementation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <queue>
+#include <stdio.h>
+#include <string>
+#include "utils/TimeUtil.h"
+#include "utils/StringUtils.h"
+#include "processors/ListenSyslog.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const std::string ListenSyslog::ProcessorName("ListenSyslog");
+core::Property ListenSyslog::RecvBufSize(
+    "Receive Buffer Size",
+    "The size of each buffer used to receive Syslog messages.", "65507 B");
+core::Property ListenSyslog::MaxSocketBufSize(
+    "Max Size of Socket Buffer",
+    "The maximum size of the socket buffer that should be used.", "1 MB");
+core::Property ListenSyslog::MaxConnections(
+    "Max Number of TCP Connections",
+    "The maximum number of concurrent connections to accept Syslog messages in TCP mode.",
+    "2");
+core::Property ListenSyslog::MaxBatchSize(
+    "Max Batch Size",
+    "The maximum number of Syslog events to add to a single FlowFile.", "1");
+core::Property ListenSyslog::MessageDelimiter(
+    "Message Delimiter",
+    "Specifies the delimiter to place between Syslog messages when multiple messages are bundled together (see <Max Batch Size> core::Property).",
+    "\n");
+core::Property ListenSyslog::ParseMessages(
+    "Parse Messages",
+    "Indicates if the processor should parse the Syslog messages. If set to false, each outgoing FlowFile will only.",
+    "false");
+core::Property ListenSyslog::Protocol(
+    "Protocol", "The protocol for Syslog communication.", "UDP");
+core::Property ListenSyslog::Port(
+    "Port", "The port for Syslog communication.", "514");
+core::Relationship ListenSyslog::Success(
+    "success", "All files are routed to success");
+core::Relationship ListenSyslog::Invalid(
+    "invalid", "SysLog message format invalid");
+
+void ListenSyslog::initialize() {
+  // Set the supported properties
+  std::set<core::Property> properties;
+  properties.insert(RecvBufSize);
+  properties.insert(MaxSocketBufSize);
+  properties.insert(MaxConnections);
+  properties.insert(MaxBatchSize);
+  properties.insert(MessageDelimiter);
+  properties.insert(ParseMessages);
+  properties.insert(Protocol);
+  properties.insert(Port);
+  setSupportedProperties(properties);
+  // Set the supported relationships
+  std::set<core::Relationship> relationships;
+  relationships.insert(Success);
+  relationships.insert(Invalid);
+  setSupportedRelationships(relationships);
+}
+
+void ListenSyslog::startSocketThread() {
+  if (_thread != NULL)
+    return;
+
+  logger_->log_info("ListenSysLog Socket Thread Start");
+  _serverTheadRunning = true;
+  _thread = new std::thread(run, this);
+  _thread->detach();
+}
+
+void ListenSyslog::run(ListenSyslog *process) {
+  process->runThread();
+}
+
+void ListenSyslog::runThread() {
+  while (_serverTheadRunning) {
+    if (_resetServerSocket) {
+      _resetServerSocket = false;
+      // need to reset the socket
+      std::vector<int>::iterator it;
+      for (it = _clientSockets.begin(); it != _clientSockets.end(); ++it) {
+        int clientSocket = *it;
+        close(clientSocket);
+      }
+      _clientSockets.clear();
+      if (_serverSocket > 0) {
+        close(_serverSocket);
+        _serverSocket = 0;
+      }
+    }
+
+    if (_serverSocket <= 0) {
+      uint16_t portno = _port;
+      struct sockaddr_in serv_addr;
+      int sockfd;
+      if (_protocol == "TCP")
+        sockfd = socket(AF_INET, SOCK_STREAM, 0);
+      else
+        sockfd = socket(AF_INET, SOCK_DGRAM, 0);
+      if (sockfd < 0) {
+        logger_->log_info("ListenSysLog Server socket creation failed");
+        break;
+      }
+      bzero((char *) &serv_addr, sizeof(serv_addr));
+      serv_addr.sin_family = AF_INET;
+      serv_addr.sin_addr.s_addr = INADDR_ANY;
+      serv_addr.sin_port = htons(portno);
+      if (bind(sockfd, (struct sockaddr *) &serv_addr, sizeof(serv_addr)) < 0) {
+        logger_->log_error("ListenSysLog Server socket bind failed");
+        break;
+      }
+      if (_protocol == "TCP")
+        listen(sockfd, 5);
+      _serverSocket = sockfd;
+      logger_->log_error("ListenSysLog Server socket %d bind OK to port %d",
+                         _serverSocket, portno);
+    }
+    FD_ZERO(&_readfds);
+    FD_SET(_serverSocket, &_readfds);
+    _maxFds = _serverSocket;
+    std::vector<int>::iterator it;
+    for (it = _clientSockets.begin(); it != _clientSockets.end(); ++it) {
+      int clientSocket = *it;
+      if (clientSocket >= _maxFds)
+        _maxFds = clientSocket;
+      FD_SET(clientSocket, &_readfds);
+    }
+    fd_set fds;
+    struct timeval tv;
+    int retval;
+    fds = _readfds;
+    tv.tv_sec = 0;
+    // 100 msec
+    tv.tv_usec = 100000;
+    retval = select(_maxFds + 1, &fds, NULL, NULL, &tv);
+    if (retval < 0)
+      break;
+    if (retval == 0)
+      continue;
+    if (FD_ISSET(_serverSocket, &fds)) {
+      // server socket, either we have UDP datagram or TCP connection request
+      if (_protocol == "TCP") {
+        socklen_t clilen;
+        struct sockaddr_in cli_addr;
+        clilen = sizeof(cli_addr);
+        int newsockfd = accept(_serverSocket, (struct sockaddr *) &cli_addr,
+                               &clilen);
+        if (newsockfd > 0) {
+          if (_clientSockets.size() < _maxConnections) {
+            _clientSockets.push_back(newsockfd);
+            logger_->log_info("ListenSysLog new client socket %d connection",
+                              newsockfd);
+            continue;
+          } else {
+            close(newsockfd);
+          }
+        }
+      } else {
+        socklen_t clilen;
+        struct sockaddr_in cli_addr;
+        clilen = sizeof(cli_addr);
+        int recvlen = recvfrom(_serverSocket, _buffer, sizeof(_buffer), 0,
+                               (struct sockaddr *) &cli_addr, &clilen);
+        if (recvlen > 0
+            && (recvlen + getEventQueueByteSize()) <= _recvBufSize) {
+          uint8_t *payload = new uint8_t[recvlen];
+          memcpy(payload, _buffer, recvlen);
+          putEvent(payload, recvlen);
+        }
+      }
+    }
+    it = _clientSockets.begin();
+    while (it != _clientSockets.end()) {
+      int clientSocket = *it;
+      if (FD_ISSET(clientSocket, &fds)) {
+        int recvlen = readline(clientSocket, (char *) _buffer, sizeof(_buffer));
+        if (recvlen <= 0) {
+          close(clientSocket);
+          logger_->log_info("ListenSysLog client socket %d close",
+                            clientSocket);
+          it = _clientSockets.erase(it);
+        } else {
+          if ((recvlen + getEventQueueByteSize()) <= _recvBufSize) {
+            uint8_t *payload = new uint8_t[recvlen];
+            memcpy(payload, _buffer, recvlen);
+            putEvent(payload, recvlen);
+          }
+          ++it;
+        }
+      }
+    }
+  }
+  return;
+}
+
+int ListenSyslog::readline(int fd, char *bufptr, size_t len) {
+  char *bufx = bufptr;
+  static char *bp;
+  static int cnt = 0;
+  static char b[2048];
+  char c;
+
+  while (--len > 0) {
+    if (--cnt <= 0) {
+      cnt = recv(fd, b, sizeof(b), 0);
+      if (cnt < 0) {
+        if ( errno == EINTR) {
+          len++; /* the while will decrement */
+          continue;
+        }
+        return -1;
+      }
+      if (cnt == 0)
+        return 0;
+      bp = b;
+    }
+    c = *bp++;
+    *bufptr++ = c;
+    if (c == '\n') {
+      *bufptr = '\n';
+      return bufptr - bufx + 1;
+    }
+  }
+  return -1;
+}
+
+void ListenSyslog::onTrigger(
+    core::ProcessContext *context,
+    core::ProcessSession *session) {
+  std::string value;
+  bool needResetServerSocket = false;
+  if (context->getProperty(Protocol.getName(), value)) {
+    if (_protocol != value)
+      needResetServerSocket = true;
+    _protocol = value;
+  }
+  if (context->getProperty(RecvBufSize.getName(), value)) {
+    core::Property::StringToInt(value, _recvBufSize);
+  }
+  if (context->getProperty(MaxSocketBufSize.getName(), value)) {
+    core::Property::StringToInt(value,
+                                                           _maxSocketBufSize);
+  }
+  if (context->getProperty(MaxConnections.getName(), value)) {
+    core::Property::StringToInt(value,
+                                                           _maxConnections);
+  }
+  if (context->getProperty(MessageDelimiter.getName(), value)) {
+    _messageDelimiter = value;
+  }
+  if (context->getProperty(ParseMessages.getName(), value)) {
+    org::apache::nifi::minifi::utils::StringUtils::StringToBool(value,
+                                                                _parseMessages);
+  }
+  if (context->getProperty(Port.getName(), value)) {
+    int64_t oldPort = _port;
+    core::Property::StringToInt(value, _port);
+    if (_port != oldPort)
+      needResetServerSocket = true;
+  }
+  if (context->getProperty(MaxBatchSize.getName(), value)) {
+    core::Property::StringToInt(value,
+                                                           _maxBatchSize);
+  }
+
+  if (needResetServerSocket)
+    _resetServerSocket = true;
+
+  startSocketThread();
+
+  // read from the event queue
+  if (isEventQueueEmpty()) {
+    context->yield();
+    return;
+  }
+
+  std::queue<SysLogEvent> eventQueue;
+  pollEvent(eventQueue, _maxBatchSize);
+  bool firstEvent = true;
+  std::shared_ptr<FlowFileRecord> flowFile = NULL;
+  while (!eventQueue.empty()) {
+    SysLogEvent event = eventQueue.front();
+    eventQueue.pop();
+    if (firstEvent) {
+      flowFile = std::static_pointer_cast<FlowFileRecord>(session->create());
+      if (!flowFile)
+        return;
+      ListenSyslog::WriteCallback callback((char *) event.payload, event.len);
+      session->write(flowFile, &callback);
+      delete[] event.payload;
+      firstEvent = false;
+    } else {
+      ListenSyslog::WriteCallback callback((char *) event.payload, event.len);
+      session->append(flowFile, &callback);
+      delete[] event.payload;
+    }
+  }
+  flowFile->addAttribute("syslog.protocol", _protocol);
+  flowFile->addAttribute("syslog.port", std::to_string(_port));
+  session->transfer(flowFile, Success);
+}
+
+} /* namespace processors */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/processors/LogAttribute.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/LogAttribute.cpp b/libminifi/src/processors/LogAttribute.cpp
new file mode 100644
index 0000000..e2cf16c
--- /dev/null
+++ b/libminifi/src/processors/LogAttribute.cpp
@@ -0,0 +1,176 @@
+/**
+ * @file LogAttribute.cpp
+ * LogAttribute class implementation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <vector>
+#include <queue>
+#include <map>
+#include <set>
+#include <sys/time.h>
+#include <time.h>
+#include <sstream>
+#include <string.h>
+#include <iostream>
+
+#include "utils/TimeUtil.h"
+#include "utils/StringUtils.h"
+#include "processors/LogAttribute.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+const std::string LogAttribute::ProcessorName("LogAttribute");
+core::Property LogAttribute::LogLevel(
+    "Log Level", "The Log Level to use when logging the Attributes", "info");
+core::Property LogAttribute::AttributesToLog(
+    "Attributes to Log",
+    "A comma-separated list of Attributes to Log. If not specified, all attributes will be logged.",
+    "");
+core::Property LogAttribute::AttributesToIgnore(
+    "Attributes to Ignore",
+    "A comma-separated list of Attributes to ignore. If not specified, no attributes will be ignored.",
+    "");
+core::Property LogAttribute::LogPayload(
+    "Log Payload",
+    "If true, the FlowFile's payload will be logged, in addition to its attributes; otherwise, just the Attributes will be logged.",
+    "false");
+core::Property LogAttribute::LogPrefix(
+    "Log prefix",
+    "Log prefix appended to the log lines. It helps to distinguish the output of multiple LogAttribute processors.",
+    "");
+core::Relationship LogAttribute::Success(
+    "success", "success operational on the flow record");
+
+void LogAttribute::initialize() {
+  // Set the supported properties
+  std::set<core::Property> properties;
+  properties.insert(LogLevel);
+  properties.insert(AttributesToLog);
+  properties.insert(AttributesToIgnore);
+  properties.insert(LogPayload);
+  properties.insert(LogPrefix);
+  setSupportedProperties(properties);
+  // Set the supported relationships
+  std::set<core::Relationship> relationships;
+  relationships.insert(Success);
+  setSupportedRelationships(relationships);
+}
+
+void LogAttribute::onTrigger(
+    core::ProcessContext *context,
+    core::ProcessSession *session) {
+  std::string dashLine = "--------------------------------------------------";
+  LogAttrLevel level = LogAttrLevelInfo;
+  bool logPayload = false;
+  std::ostringstream message;
+
+  std::shared_ptr<core::FlowFile> flow =
+      session->get();
+
+  if (!flow)
+    return;
+
+  std::string value;
+  if (context->getProperty(LogLevel.getName(), value)) {
+    logLevelStringToEnum(value, level);
+  }
+  if (context->getProperty(LogPrefix.getName(), value)) {
+    dashLine = "-----" + value + "-----";
+  }
+  if (context->getProperty(LogPayload.getName(), value)) {
+    org::apache::nifi::minifi::utils::StringUtils::StringToBool(value, logPayload);
+  }
+
+  message << "Logging for flow file " << "\n";
+  message << dashLine;
+  message << "\nStandard FlowFile Attributes";
+  message << "\n" << "UUID:" << flow->getUUIDStr();
+  message << "\n" << "EntryDate:" << getTimeStr(flow->getEntryDate());
+  message << "\n" << "lineageStartDate:"
+          << getTimeStr(flow->getlineageStartDate());
+  message << "\n" << "Size:" << flow->getSize() << " Offset:"
+          << flow->getOffset();
+  message << "\nFlowFile Attributes Map Content";
+  std::map<std::string, std::string> attrs = flow->getAttributes();
+  std::map<std::string, std::string>::iterator it;
+  for (it = attrs.begin(); it != attrs.end(); it++) {
+    message << "\n" << "key:" << it->first << " value:" << it->second;
+  }
+  message << "\nFlowFile Resource Claim Content";
+  std::shared_ptr<ResourceClaim> claim = flow->getResourceClaim();
+  if (claim) {
+    message << "\n" << "Content Claim:" << claim->getContentFullPath();
+  }
+  if (logPayload && flow->getSize() <= 1024 * 1024) {
+    message << "\n" << "Payload:" << "\n";
+    ReadCallback callback(flow->getSize());
+    session->read(flow, &callback);
+    for (unsigned int i = 0, j = 0; i < callback._readSize; i++) {
+      char temp[8];
+      sprintf(temp, "%02x ", (unsigned char) (callback._buffer[i]));
+      message << temp;
+      j++;
+      if (j == 16) {
+        message << '\n';
+        j = 0;
+      }
+    }
+  }
+  message << "\n" << dashLine << std::ends;
+  std::string output = message.str();
+
+  switch (level) {
+    case LogAttrLevelInfo:
+      logger_->log_info("%s", output.c_str());
+      break;
+    case LogAttrLevelDebug:
+      logger_->log_debug("%s", output.c_str());
+      break;
+    case LogAttrLevelError:
+      logger_->log_error("%s", output.c_str());
+      break;
+    case LogAttrLevelTrace:
+      logger_->log_trace("%s", output.c_str());
+      break;
+    case LogAttrLevelWarn:
+      logger_->log_warn("%s", output.c_str());
+      break;
+    default:
+      break;
+  }
+
+  // Test Import
+  /*
+   std::shared_ptr<FlowFileRecord> importRecord = session->create();
+   session->import(claim->getContentFullPath(), importRecord);
+   session->transfer(importRecord, Success); */
+
+  // Transfer to the relationship
+  session->transfer(flow, Success);
+}
+
+
+} /* namespace processors */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/processors/PutFile.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/PutFile.cpp b/libminifi/src/processors/PutFile.cpp
new file mode 100644
index 0000000..85cf09b
--- /dev/null
+++ b/libminifi/src/processors/PutFile.cpp
@@ -0,0 +1,213 @@
+/**
+ * @file PutFile.cpp
+ * PutFile class implementation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <sstream>
+#include <stdio.h>
+#include <string>
+#include <iostream>
+#include <fstream>
+#include <uuid/uuid.h>
+
+#include "utils/StringUtils.h"
+#include "utils/TimeUtil.h"
+#include "processors/PutFile.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const std::string PutFile::CONFLICT_RESOLUTION_STRATEGY_REPLACE("replace");
+const std::string PutFile::CONFLICT_RESOLUTION_STRATEGY_IGNORE("ignore");
+const std::string PutFile::CONFLICT_RESOLUTION_STRATEGY_FAIL("fail");
+
+const std::string PutFile::ProcessorName("PutFile");
+
+core::Property PutFile::Directory(
+    "Output Directory", "The output directory to which to put files", ".");
+core::Property PutFile::ConflictResolution(
+    "Conflict Resolution Strategy",
+    "Indicates what should happen when a file with the same name already exists in the output directory",
+    CONFLICT_RESOLUTION_STRATEGY_FAIL);
+
+core::Relationship PutFile::Success(
+    "success", "All files are routed to success");
+core::Relationship PutFile::Failure(
+    "failure",
+    "Failed files (conflict, write failure, etc.) are transferred to failure");
+
+void PutFile::initialize() {
+  // Set the supported properties
+  std::set<core::Property> properties;
+  properties.insert(Directory);
+  properties.insert(ConflictResolution);
+  setSupportedProperties(properties);
+  // Set the supported relationships
+  std::set<core::Relationship> relationships;
+  relationships.insert(Success);
+  relationships.insert(Failure);
+  setSupportedRelationships(relationships);
+}
+
+void PutFile::onTrigger(
+    core::ProcessContext *context,
+    core::ProcessSession *session) {
+  std::string directory;
+
+  if (!context->getProperty(Directory.getName(), directory)) {
+    logger_->log_error("Directory attribute is missing or invalid");
+    return;
+  }
+
+  std::string conflictResolution;
+
+  if (!context->getProperty(ConflictResolution.getName(), conflictResolution)) {
+    logger_->log_error(
+        "Conflict Resolution Strategy attribute is missing or invalid");
+    return;
+  }
+
+  std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast<FlowFileRecord>(session->get());
+
+  // Do nothing if there are no incoming files
+  if (!flowFile) {
+    return;
+  }
+
+  std::string filename;
+  flowFile->getKeyedAttribute(FILENAME, filename);
+
+  // Generate a safe (universally-unique) temporary filename on the same partition
+  char tmpFileUuidStr[37];
+  uuid_t tmpFileUuid;
+  uuid_generate(tmpFileUuid);
+  uuid_unparse_lower(tmpFileUuid, tmpFileUuidStr);
+  std::stringstream tmpFileSs;
+  tmpFileSs << directory << "/." << filename << "." << tmpFileUuidStr;
+  std::string tmpFile = tmpFileSs.str();
+  logger_->log_info("PutFile using temporary file %s", tmpFile.c_str());
+
+  // Determine dest full file paths
+  std::stringstream destFileSs;
+  destFileSs << directory << "/" << filename;
+  std::string destFile = destFileSs.str();
+
+  logger_->log_info("PutFile writing file %s into directory %s",
+                    filename.c_str(), directory.c_str());
+
+  // If file exists, apply conflict resolution strategy
+  struct stat statResult;
+
+  if (stat(destFile.c_str(), &statResult) == 0) {
+    logger_->log_info(
+        "Destination file %s exists; applying Conflict Resolution Strategy: %s",
+        destFile.c_str(), conflictResolution.c_str());
+
+    if (conflictResolution == CONFLICT_RESOLUTION_STRATEGY_REPLACE) {
+      putFile(session, flowFile, tmpFile, destFile);
+    } else if (conflictResolution == CONFLICT_RESOLUTION_STRATEGY_IGNORE) {
+      session->transfer(flowFile, Success);
+    } else {
+      session->transfer(flowFile, Failure);
+    }
+  } else {
+    putFile(session, flowFile, tmpFile, destFile);
+  }
+}
+
+bool PutFile::putFile(core::ProcessSession *session,
+                      std::shared_ptr<FlowFileRecord> flowFile, const std::string &tmpFile,
+                      const std::string &destFile) {
+
+  ReadCallback cb(tmpFile, destFile);
+  session->read(flowFile, &cb);
+
+  if (cb.commit()) {
+    session->transfer(flowFile, Success);
+    return true;
+  } else {
+    session->transfer(flowFile, Failure);
+  }
+  return false;
+}
+
+PutFile::ReadCallback::ReadCallback(const std::string &tmpFile,
+                                    const std::string &destFile)
+    : _tmpFile(tmpFile),
+      _tmpFileOs(tmpFile),
+      _destFile(destFile) {
+  logger_ = logging::Logger::getLogger();
+}
+
+// Copy the entire file contents to the temporary file
+void PutFile::ReadCallback::process(std::ifstream *stream) {
+  // Copy file contents into tmp file
+  _writeSucceeded = false;
+  _tmpFileOs << stream->rdbuf();
+  _writeSucceeded = true;
+}
+
+// Renames tmp file to final destination
+// Returns true if commit succeeded
+bool PutFile::ReadCallback::commit() {
+  bool success = false;
+
+  logger_->log_info("PutFile committing put file operation to %s",
+                    _destFile.c_str());
+
+  if (_writeSucceeded) {
+    _tmpFileOs.close();
+
+    if (rename(_tmpFile.c_str(), _destFile.c_str())) {
+      logger_->log_info(
+          "PutFile commit put file operation to %s failed because rename() call failed",
+          _destFile.c_str());
+    } else {
+      success = true;
+      logger_->log_info("PutFile commit put file operation to %s succeeded",
+                        _destFile.c_str());
+    }
+  } else {
+    logger_->log_error(
+        "PutFile commit put file operation to %s failed because write failed",
+        _destFile.c_str());
+  }
+
+  return success;
+}
+
+// Clean up resources
+PutFile::ReadCallback::~ReadCallback() {
+  // Close tmp file
+  _tmpFileOs.close();
+
+  // Clean up tmp file, if necessary
+  unlink(_tmpFile.c_str());
+}
+
+
+} /* namespace processors */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/processors/RealTimeDataCollector.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/RealTimeDataCollector.cpp b/libminifi/src/processors/RealTimeDataCollector.cpp
new file mode 100644
index 0000000..922835d
--- /dev/null
+++ b/libminifi/src/processors/RealTimeDataCollector.cpp
@@ -0,0 +1,480 @@
+/**
+ * @file RealTimeDataCollector.cpp
+ * RealTimeDataCollector class implementation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <vector>
+#include <queue>
+#include <map>
+#include <set>
+#include <sys/time.h>
+#include <time.h>
+#include <chrono>
+#include <thread>
+#include <memory>
+#include <random>
+#include <netinet/tcp.h>
+
+#include "utils/StringUtils.h"
+#include "processors/RealTimeDataCollector.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+const std::string RealTimeDataCollector::ProcessorName("RealTimeDataCollector");
+core::Property RealTimeDataCollector::FILENAME(
+    "File Name", "File Name for the real time processor to process",
+    "data.osp");
+core::Property RealTimeDataCollector::REALTIMESERVERNAME(
+    "Real Time Server Name", "Real Time Server Name", "localhost");
+core::Property RealTimeDataCollector::REALTIMESERVERPORT(
+    "Real Time Server Port", "Real Time Server Port", "10000");
+core::Property RealTimeDataCollector::BATCHSERVERNAME(
+    "Batch Server Name", "Batch Server Name", "localhost");
+core::Property RealTimeDataCollector::BATCHSERVERPORT(
+    "Batch Server Port", "Batch Server Port", "10001");
+core::Property RealTimeDataCollector::ITERATION(
+    "Iteration", "If true, sample osp file will be iterated", "true");
+core::Property RealTimeDataCollector::REALTIMEMSGID(
+    "Real Time Message ID", "Real Time Message ID", "41");
+core::Property RealTimeDataCollector::BATCHMSGID(
+    "Batch Message ID", "Batch Message ID", "172, 30, 48");
+core::Property RealTimeDataCollector::REALTIMEINTERVAL(
+    "Real Time Interval", "Real Time Data Collection Interval in msec",
+    "10 ms");
+core::Property RealTimeDataCollector::BATCHINTERVAL(
+    "Batch Time Interval", "Batch Processing Interval in msec", "100 ms");
+core::Property RealTimeDataCollector::BATCHMAXBUFFERSIZE(
+    "Batch Max Buffer Size", "Batch Buffer Maximum size in bytes", "262144");
+core::Relationship RealTimeDataCollector::Success(
+    "success", "success operational on the flow record");
+
+void RealTimeDataCollector::initialize() {
+  // Set the supported properties
+  std::set<core::Property> properties;
+  properties.insert(FILENAME);
+  properties.insert(REALTIMESERVERNAME);
+  properties.insert(REALTIMESERVERPORT);
+  properties.insert(BATCHSERVERNAME);
+  properties.insert(BATCHSERVERPORT);
+  properties.insert(ITERATION);
+  properties.insert(REALTIMEMSGID);
+  properties.insert(BATCHMSGID);
+  properties.insert(REALTIMEINTERVAL);
+  properties.insert(BATCHINTERVAL);
+  properties.insert(BATCHMAXBUFFERSIZE);
+
+  setSupportedProperties(properties);
+  // Set the supported relationships
+  std::set<core::Relationship> relationships;
+  relationships.insert(Success);
+  setSupportedRelationships(relationships);
+
+}
+
+int RealTimeDataCollector::connectServer(const char *host, uint16_t port) {
+  in_addr_t addr;
+  int sock = 0;
+  struct hostent *h;
+#ifdef __MACH__
+  h = gethostbyname(host);
+#else
+  char buf[1024];
+  struct hostent he;
+  int hh_errno;
+  gethostbyname_r(host, &he, buf, sizeof(buf), &h, &hh_errno);
+#endif
+  memcpy((char *) &addr, h->h_addr_list[0], h->h_length);
+  sock = socket(AF_INET, SOCK_STREAM, 0);
+  if (sock < 0) {
+    logger_->log_error("Could not create socket to hostName %s", host);
+    return 0;
+  }
+
+#ifndef __MACH__
+  int opt = 1;
+  bool nagle_off = true;
+
+  if (nagle_off)
+  {
+    if (setsockopt(sock, SOL_TCP, TCP_NODELAY, (void *)&opt, sizeof(opt)) < 0)
+    {
+      logger_->log_error("setsockopt() TCP_NODELAY failed");
+      close(sock);
+      return 0;
+    }
+    if (setsockopt(sock, SOL_SOCKET, SO_REUSEADDR,
+            (char *)&opt, sizeof(opt)) < 0)
+    {
+      logger_->log_error("setsockopt() SO_REUSEADDR failed");
+      close(sock);
+      return 0;
+    }
+  }
+
+  int sndsize = 256*1024;
+  if (setsockopt(sock, SOL_SOCKET, SO_SNDBUF, (char *)&sndsize, (int)sizeof(sndsize)) < 0)
+  {
+    logger_->log_error("setsockopt() SO_SNDBUF failed");
+    close(sock);
+    return 0;
+  }
+#endif
+
+  struct sockaddr_in sa;
+  socklen_t socklen;
+  int status;
+
+  //TODO bind socket to the interface
+  memset(&sa, 0, sizeof(sa));
+  sa.sin_family = AF_INET;
+  sa.sin_addr.s_addr = htonl(INADDR_ANY);
+  sa.sin_port = htons(0);
+  socklen = sizeof(sa);
+  if (bind(sock, (struct sockaddr *) &sa, socklen) < 0) {
+    logger_->log_error("socket bind failed");
+    close(sock);
+    return 0;
+  }
+
+  memset(&sa, 0, sizeof(sa));
+  sa.sin_family = AF_INET;
+  sa.sin_addr.s_addr = addr;
+  sa.sin_port = htons(port);
+  socklen = sizeof(sa);
+
+  status = connect(sock, (struct sockaddr *) &sa, socklen);
+
+  if (status < 0) {
+    logger_->log_error("socket connect failed to %s %d", host, port);
+    close(sock);
+    return 0;
+  }
+
+  logger_->log_info("socket %d connect to server %s port %d success", sock,
+                    host, port);
+
+  return sock;
+}
+
+int RealTimeDataCollector::sendData(int socket, const char *buf, int buflen) {
+  int ret = 0, bytes = 0;
+
+  while (bytes < buflen) {
+    ret = send(socket, buf + bytes, buflen - bytes, 0);
+    //check for errors
+    if (ret == -1) {
+      return ret;
+    }
+    bytes += ret;
+  }
+
+  if (ret)
+    logger_->log_debug("Send data size %d over socket %d", buflen, socket);
+
+  return ret;
+}
+
+void RealTimeDataCollector::onTriggerRealTime(
+    core::ProcessContext *context,
+    core::ProcessSession *session) {
+  if (_realTimeAccumulated >= this->_realTimeInterval) {
+    std::string value;
+    if (this->getProperty(REALTIMEMSGID.getName(), value)) {
+      this->_realTimeMsgID.clear();
+      this->logger_->log_info("Real Time Msg IDs %s", value.c_str());
+      std::stringstream lineStream(value);
+      std::string cell;
+
+      while (std::getline(lineStream, cell, ',')) {
+        this->_realTimeMsgID.push_back(cell);
+        // this->logger_->log_debug("Real Time Msg ID %s", cell.c_str());
+      }
+    }
+    if (this->getProperty(BATCHMSGID.getName(), value)) {
+      this->_batchMsgID.clear();
+      this->logger_->log_info("Batch Msg IDs %s", value.c_str());
+      std::stringstream lineStream(value);
+      std::string cell;
+
+      while (std::getline(lineStream, cell, ',')) {
+        cell = org::apache::nifi::minifi::utils::StringUtils::trim(cell);
+        this->_batchMsgID.push_back(cell);
+      }
+    }
+    // Open the file
+    if (!this->_fileStream.is_open()) {
+      _fileStream.open(this->_fileName.c_str(), std::ifstream::in);
+      if (this->_fileStream.is_open())
+        logger_->log_debug("open %s", _fileName.c_str());
+    }
+    if (!_fileStream.good()) {
+      logger_->log_error("load data file failed %s", _fileName.c_str());
+      return;
+    }
+    if (this->_fileStream.is_open()) {
+      std::string line;
+
+      while (std::getline(_fileStream, line)) {
+        line += "\n";
+        std::stringstream lineStream(line);
+        std::string cell;
+        if (std::getline(lineStream, cell, ',')) {
+          cell = org::apache::nifi::minifi::utils::StringUtils::trim(cell);
+          // Check whether it match to the batch traffic
+          for (std::vector<std::string>::iterator it = _batchMsgID.begin();
+              it != _batchMsgID.end(); ++it) {
+            if (cell == *it) {
+              // push the batch data to the queue
+              std::lock_guard<std::mutex> lock(mutex_);
+              while ((_queuedDataSize + line.size()) > _batchMaxBufferSize) {
+                std::string item = _queue.front();
+                _queuedDataSize -= item.size();
+                logger_->log_debug(
+                    "Pop item size %d from batch queue, queue buffer size %d",
+                    item.size(), _queuedDataSize);
+                _queue.pop();
+              }
+              _queue.push(line);
+              _queuedDataSize += line.size();
+              logger_->log_debug(
+                  "Push batch msg ID %s into batch queue, queue buffer size %d",
+                  cell.c_str(), _queuedDataSize);
+            }
+          }
+          bool findRealTime = false;
+          // Check whether it match to the real time traffic
+          for (std::vector<std::string>::iterator it = _realTimeMsgID.begin();
+              it != _realTimeMsgID.end(); ++it) {
+            if (cell == *it) {
+              int status = 0;
+              if (this->_realTimeSocket <= 0) {
+                // Connect the LTE socket
+                uint16_t port = _realTimeServerPort;
+                this->_realTimeSocket = connectServer(
+                    _realTimeServerName.c_str(), port);
+              }
+              if (this->_realTimeSocket) {
+                // try to send the data
+                status = sendData(_realTimeSocket, line.data(), line.size());
+                if (status < 0) {
+                  close(_realTimeSocket);
+                  _realTimeSocket = 0;
+                }
+              }
+              if (this->_realTimeSocket <= 0 || status < 0) {
+                // push the batch data to the queue
+                std::lock_guard<std::mutex> lock(mutex_);
+                while ((_queuedDataSize + line.size()) > _batchMaxBufferSize) {
+                  std::string item = _queue.front();
+                  _queuedDataSize -= item.size();
+                  logger_->log_debug(
+                      "Pop item size %d from batch queue, queue buffer size %d",
+                      item.size(), _queuedDataSize);
+                  _queue.pop();
+                }
+                _queue.push(line);
+                _queuedDataSize += line.size();
+                logger_->log_debug(
+                    "Push real time msg ID %s into batch queue, queue buffer size %d",
+                    cell.c_str(), _queuedDataSize);
+              }
+              // find real time
+              findRealTime = true;
+            }  // cell
+          }  // for real time pattern
+          if (findRealTime)
+            // we break the while once we find the first real time
+            break;
+        }  // if get line
+      }  // while
+      if (_fileStream.eof()) {
+        _fileStream.close();
+      }
+    }  // if open
+    _realTimeAccumulated = 0;
+  }
+  std::shared_ptr<Processor> processor = std::static_pointer_cast<Processor>(
+      context->getProcessorNode().getProcessor());
+  _realTimeAccumulated += processor->getSchedulingPeriodNano();
+}
+
+void RealTimeDataCollector::onTriggerBatch(
+    core::ProcessContext *context,
+    core::ProcessSession *session) {
+  if (_batchAcccumulated >= this->_batchInterval) {
+    // logger_->log_info("onTriggerBatch");
+    // dequeue the batch and send over WIFI
+    int status = 0;
+    if (this->_batchSocket <= 0) {
+      // Connect the WIFI socket
+      uint16_t port = _batchServerPort;
+      this->_batchSocket = connectServer(_batchServerName.c_str(), port);
+    }
+    if (this->_batchSocket) {
+      std::lock_guard<std::mutex> lock(mutex_);
+
+      while (!_queue.empty()) {
+        std::string line = _queue.front();
+        status = sendData(_batchSocket, line.data(), line.size());
+        _queue.pop();
+        _queuedDataSize -= line.size();
+        if (status < 0) {
+          close(_batchSocket);
+          _batchSocket = 0;
+          break;
+        }
+      }
+    }
+    _batchAcccumulated = 0;
+  }
+  std::shared_ptr<Processor> processor = std::static_pointer_cast<Processor>(
+      context->getProcessorNode().getProcessor());
+  _batchAcccumulated += processor->getSchedulingPeriodNano();
+}
+
+void RealTimeDataCollector::onTrigger(
+    core::ProcessContext *context,
+    core::ProcessSession *session) {
+  std::thread::id id = std::this_thread::get_id();
+
+  if (id == _realTimeThreadId)
+    return onTriggerRealTime(context, session);
+  else if (id == _batchThreadId)
+    return onTriggerBatch(context, session);
+  else {
+    std::lock_guard<std::mutex> lock(mutex_);
+    if (!this->_firstInvoking) {
+      this->_fileName = "data.osp";
+      std::string value;
+      if (this->getProperty(FILENAME.getName(), value)) {
+        this->_fileName = value;
+        this->logger_->log_info("Data Collector File Name %s",
+                                _fileName.c_str());
+      }
+      this->_realTimeServerName = "localhost";
+      if (this->getProperty(REALTIMESERVERNAME.getName(), value)) {
+        this->_realTimeServerName = value;
+        this->logger_->log_info("Real Time Server Name %s",
+                                this->_realTimeServerName.c_str());
+      }
+      this->_realTimeServerPort = 10000;
+      if (this->getProperty(REALTIMESERVERPORT.getName(), value)) {
+        core::Property::StringToInt(
+            value, _realTimeServerPort);
+        this->logger_->log_info("Real Time Server Port %d",
+                                _realTimeServerPort);
+      }
+      if (this->getProperty(BATCHSERVERNAME.getName(), value)) {
+        this->_batchServerName = value;
+        this->logger_->log_info("Batch Server Name %s",
+                                this->_batchServerName.c_str());
+      }
+      this->_batchServerPort = 10001;
+      if (this->getProperty(BATCHSERVERPORT.getName(), value)) {
+        core::Property::StringToInt(
+            value, _batchServerPort);
+        this->logger_->log_info("Batch Server Port %d", _batchServerPort);
+      }
+      if (this->getProperty(ITERATION.getName(), value)) {
+        org::apache::nifi::minifi::utils::StringUtils::StringToBool(value, this->_iteration);
+        logger_->log_info("Iteration %d", _iteration);
+      }
+      this->_realTimeInterval = 10000000;  //10 msec
+      if (this->getProperty(REALTIMEINTERVAL.getName(), value)) {
+        core::TimeUnit unit;
+        if (core::Property::StringToTime(
+            value, _realTimeInterval, unit)
+            && core::Property::ConvertTimeUnitToNS(
+                _realTimeInterval, unit, _realTimeInterval)) {
+          logger_->log_info("Real Time Interval: [%d] ns", _realTimeInterval);
+        }
+      }
+      this->_batchInterval = 100000000;  //100 msec
+      if (this->getProperty(BATCHINTERVAL.getName(), value)) {
+        core::TimeUnit unit;
+        if (core::Property::StringToTime(
+            value, _batchInterval, unit)
+            && core::Property::ConvertTimeUnitToNS(
+                _batchInterval, unit, _batchInterval)) {
+          logger_->log_info("Batch Time Interval: [%d] ns", _batchInterval);
+        }
+      }
+      this->_batchMaxBufferSize = 256 * 1024;
+      if (this->getProperty(BATCHMAXBUFFERSIZE.getName(), value)) {
+        core::Property::StringToInt(
+            value, _batchMaxBufferSize);
+        this->logger_->log_info("Batch Max Buffer Size %d",
+                                _batchMaxBufferSize);
+      }
+      if (this->getProperty(REALTIMEMSGID.getName(), value)) {
+        this->logger_->log_info("Real Time Msg IDs %s", value.c_str());
+        std::stringstream lineStream(value);
+        std::string cell;
+
+        while (std::getline(lineStream, cell, ',')) {
+          this->_realTimeMsgID.push_back(cell);
+          this->logger_->log_info("Real Time Msg ID %s", cell.c_str());
+        }
+      }
+      if (this->getProperty(BATCHMSGID.getName(), value)) {
+        this->logger_->log_info("Batch Msg IDs %s", value.c_str());
+        std::stringstream lineStream(value);
+        std::string cell;
+
+        while (std::getline(lineStream, cell, ',')) {
+          cell = org::apache::nifi::minifi::utils::StringUtils::trim(cell);
+          this->_batchMsgID.push_back(cell);
+          this->logger_->log_info("Batch Msg ID %s", cell.c_str());
+        }
+      }
+      // Connect the LTE socket
+      uint16_t port = _realTimeServerPort;
+
+      this->_realTimeSocket = connectServer(_realTimeServerName.c_str(), port);
+
+      // Connect the WIFI socket
+      port = _batchServerPort;
+
+      this->_batchSocket = connectServer(_batchServerName.c_str(), port);
+
+      // Open the file
+      _fileStream.open(this->_fileName.c_str(), std::ifstream::in);
+      if (!_fileStream.good()) {
+        logger_->log_error("load data file failed %s", _fileName.c_str());
+        return;
+      } else {
+        logger_->log_debug("open %s", _fileName.c_str());
+      }
+      _realTimeThreadId = id;
+      this->_firstInvoking = true;
+    } else {
+      if (id != _realTimeThreadId)
+        _batchThreadId = id;
+      this->_firstInvoking = false;
+    }
+  }
+}
+} /* namespace processors */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */


[11/16] nifi-minifi-cpp git commit: MINIFI-217: Updates namespaces and removes use of raw pointers for user facing API.

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/io/DataStream.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/DataStream.h b/libminifi/include/io/DataStream.h
index 2f793ff..e37dbf7 100644
--- a/libminifi/include/io/DataStream.h
+++ b/libminifi/include/io/DataStream.h
@@ -22,99 +22,111 @@
 #include <cstdint>
 #include <vector>
 #include "EndianCheck.h"
-
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace io {
 /**
  * DataStream defines the mechanism through which
  * binary data will be written to a sink
  */
 class DataStream {
-public:
-
-	DataStream() :
-			readBuffer(0) {
-
-	}
-
-	/**
-	 * Constructor
-	 **/
-	explicit DataStream(const uint8_t *buf, const uint32_t buflen) :
-			DataStream() {
-		writeData((uint8_t*) buf, buflen);
-
-	}
-
-	virtual short initialize() {
-		buffer.clear();
-		readBuffer = 0;
-		return 0;
-	}
-
-	virtual void closeStream()
-	{
-
-	}
-	/**
-	 * 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);
-
-	/**
-	 * writes valiue to buffer
-	 * @param value value to write
-	 * @param size size of value
-	 */
-	virtual int writeData(uint8_t *value, int size);
-
-	/**
-	 * Reads a system word
-	 * @param value value to write
-	 */
-	virtual int read(uint64_t &value, bool is_little_endian =
-			EndiannessCheck::IS_LITTLE);
-
-	/**
-	 * Reads a uint32_t
-	 * @param value value to write
-	 */
-	virtual int read(uint32_t &value, bool is_little_endian =
-			EndiannessCheck::IS_LITTLE);
-
-	/**
-	 * Reads a system short
-	 * @param value value to write
-	 */
-	virtual int read(uint16_t &value, bool is_little_endian =
-			EndiannessCheck::IS_LITTLE);
-
-	/**
-	 * Returns the underlying buffer
-	 * @return vector's array
-	 **/
-	const uint8_t *getBuffer() const {
-		return &buffer[0];
-	}
-
-	/**
-	 * Retrieve size of data stream
-	 * @return size of data stream
-	 **/
-	const uint32_t getSize() const {
-		return buffer.size();
-	}
-
-protected:
-	// All serialization related method and internal buf
-	std::vector<uint8_t> buffer;
-	uint32_t readBuffer;
+ public:
+
+  DataStream()
+      : readBuffer(0) {
+
+  }
+
+  ~DataStream() {
+
+  }
+
+  /**
+   * Constructor
+   **/
+  explicit DataStream(const uint8_t *buf, const uint32_t buflen)
+      : DataStream() {
+    writeData((uint8_t*) buf, buflen);
+
+  }
+
+  virtual short initialize() {
+    buffer.clear();
+    readBuffer = 0;
+    return 0;
+  }
+
+  virtual void closeStream() {
+
+  }
+  /**
+   * 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);
+
+  /**
+   * writes valiue to buffer
+   * @param value value to write
+   * @param size size of value
+   */
+  virtual int writeData(uint8_t *value, int size);
+
+  /**
+   * Reads a system word
+   * @param value value to write
+   */
+  virtual int read(uint64_t &value, bool is_little_endian =
+                       EndiannessCheck::IS_LITTLE);
+
+  /**
+   * Reads a uint32_t
+   * @param value value to write
+   */
+  virtual int read(uint32_t &value, bool is_little_endian =
+                       EndiannessCheck::IS_LITTLE);
+
+  /**
+   * Reads a system short
+   * @param value value to write
+   */
+  virtual int read(uint16_t &value, bool is_little_endian =
+                       EndiannessCheck::IS_LITTLE);
+
+  /**
+   * Returns the underlying buffer
+   * @return vector's array
+   **/
+  const uint8_t *getBuffer() const {
+    return &buffer[0];
+  }
+
+  /**
+   * Retrieve size of data stream
+   * @return size of data stream
+   **/
+  const uint32_t getSize() const {
+    return buffer.size();
+  }
+
+ protected:
+  // All serialization related method and internal buf
+  std::vector<uint8_t> buffer;
+  uint32_t readBuffer;
 };
 
+} /* namespace io */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
 #endif /* LIBMINIFI_INCLUDE_IO_DATASTREAM_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/io/EndianCheck.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/EndianCheck.h b/libminifi/include/io/EndianCheck.h
index ef900e0..3ceb19c 100644
--- a/libminifi/include/io/EndianCheck.h
+++ b/libminifi/include/io/EndianCheck.h
@@ -1,5 +1,5 @@
 /**
-  *
+ *
  * 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.
@@ -16,32 +16,34 @@
  * limitations under the License.
  */
 
-
 #ifndef LIBMINIFI_INCLUDE_IO_ENDIANCHECK_H_
 #define LIBMINIFI_INCLUDE_IO_ENDIANCHECK_H_
-
-
-
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace io {
 /**
  * Mechanism to determine endianness of host.
  * Accounts for only BIG/LITTLE/BIENDIAN
  **/
-class EndiannessCheck
-{
-public:
-    static bool IS_LITTLE;
-private:
-
-    static bool is_little_endian() {
-        /* do whatever is needed at static init time */
-        unsigned int x = 1;
-        char *c = (char*) &x;
-        IS_LITTLE=*c==1;
-        return IS_LITTLE;
-    }
+class EndiannessCheck {
+ public:
+  static bool IS_LITTLE;
+ private:
+
+  static bool is_little_endian() {
+    /* do whatever is needed at static init time */
+    unsigned int x = 1;
+    char *c = (char*) &x;
+    IS_LITTLE = *c == 1;
+    return IS_LITTLE;
+  }
 };
 
-
-
-
+} /* namespace io */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
 #endif /* LIBMINIFI_INCLUDE_IO_ENDIANCHECK_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/io/Serializable.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/Serializable.h b/libminifi/include/io/Serializable.h
index 59d3a73..5ee886b 100644
--- a/libminifi/include/io/Serializable.h
+++ b/libminifi/include/io/Serializable.h
@@ -1,5 +1,5 @@
 /**
-  *
+ *
  * 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.
@@ -22,9 +22,11 @@
 #include <string>
 #include "EndianCheck.h"
 #include "DataStream.h"
-
-
-
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace io {
 /**
  * Serializable instances provide base functionality to
  * write certain objects/primitives to a data stream.
@@ -32,155 +34,158 @@
  */
 class Serializable {
 
-public:
-
-    /**
-     * Inline function to write T to stream
-     **/
-    template<typename T>
-    inline int writeData(const T &t,DataStream *stream);
-
-    /**
-     * Inline function to write T to to_vec
-     **/
-    template<typename T>
-    inline int writeData(const T &t, uint8_t *to_vec);
-
-    /**
-     * Inline function to write T to to_vec
-     **/
-    template<typename T>
-    inline int writeData(const T &t, std::vector<uint8_t> &to_vec);
-
-
-    /**
-     * write byte to stream
-     * @return resulting write size
-     **/
-    int write(uint8_t value,DataStream *stream);
-
-    /**
-     * write byte to stream
-     * @return resulting write size
-     **/
-    int write(char value,DataStream *stream);
-
-    /**
-     * write 4 bytes to stream
-     * @param base_value non encoded value
-     * @param stream output stream
-     * @param is_little_endian endianness determination
-     * @return resulting write size
-     **/
-    int write(uint32_t base_value,DataStream *stream, bool is_little_endian =
-                  EndiannessCheck::IS_LITTLE);
-
-    /**
-     * write 2 bytes to stream
-     * @param base_value non encoded value
-     * @param stream output stream
-     * @param is_little_endian endianness determination
-     * @return resulting write size
-     **/
-    int write(uint16_t base_value,DataStream *stream, bool is_little_endian =
-                  EndiannessCheck::IS_LITTLE);
-
-    /**
-     * write valueto stream
-     * @param value non encoded value
-     * @param len length of value
-     * @param strema output stream
-     * @return resulting write size
-     **/
-    int write(uint8_t *value, int len,DataStream *stream);
-
-    /**
-     * write 8 bytes to stream
-     * @param base_value non encoded value
-     * @param stream output stream
-     * @param is_little_endian endianness determination
-     * @return resulting write size
-     **/
-    int write(uint64_t base_value,DataStream *stream, bool is_little_endian =
-                  EndiannessCheck::IS_LITTLE);
-
-    /**
-    * write bool to stream
-     * @param value non encoded value
-     * @return resulting write size
-     **/
-    int write(bool value);
-
-    /**
-     * write UTF string to stream
-     * @param str string to write
-     * @return resulting write size
-     **/
-    int writeUTF(std::string str,DataStream *stream, bool widen = false);
-
-    /**
-     * reads a byte from the stream
-     * @param value reference in which will set the result
-     * @param stream stream from which we will read
-     * @return resulting read size
-     **/
-    int read(uint8_t &value,DataStream *stream);
-
-    /**
-     * reads two bytes from the stream
-     * @param value reference in which will set the result
-     * @param stream stream from which we will read
-     * @return resulting read size
-     **/
-    int read(uint16_t &base_value,DataStream *stream, bool is_little_endian =
-                 EndiannessCheck::IS_LITTLE);
-
-    /**
-     * reads a byte from the stream
-     * @param value reference in which will set the result
-     * @param stream stream from which we will read
-     * @return resulting read size
-     **/
-    int read(char &value,DataStream *stream);
-
-    /**
-     * reads a byte array from the stream
-     * @param value reference in which will set the result
-     * @param len length to read
-     * @param stream stream from which we will read
-     * @return resulting read size
-     **/
-    int read(uint8_t *value, int len,DataStream *stream);
-
-    /**
-     * reads four bytes from the stream
-     * @param value reference in which will set the result
-     * @param stream stream from which we will read
-     * @return resulting read size
-     **/
-    int read(uint32_t &value,DataStream *stream,
-             bool is_little_endian = EndiannessCheck::IS_LITTLE);
-
-    /**
-     * reads eight byte from the stream
-     * @param value reference in which will set the result
-     * @param stream stream from which we will read
-     * @return resulting read size
-     **/
-    int read(uint64_t &value,DataStream *stream,
-             bool is_little_endian = EndiannessCheck::IS_LITTLE);
-
-    /**
-     * read UTF from stream
-     * @param str reference string
-     * @param stream stream from which we will read
-     * @return resulting read size
-     **/
-    int readUTF(std::string &str,DataStream *stream, bool widen = false);
-
-protected:
+ public:
+
+  /**
+   * Inline function to write T to stream
+   **/
+  template<typename T>
+  inline int writeData(const T &t, DataStream *stream);
+
+  /**
+   * Inline function to write T to to_vec
+   **/
+  template<typename T>
+  inline int writeData(const T &t, uint8_t *to_vec);
+
+  /**
+   * Inline function to write T to to_vec
+   **/
+  template<typename T>
+  inline int writeData(const T &t, std::vector<uint8_t> &to_vec);
+
+  /**
+   * write byte to stream
+   * @return resulting write size
+   **/
+  int write(uint8_t value, DataStream *stream);
+
+  /**
+   * write byte to stream
+   * @return resulting write size
+   **/
+  int write(char value, DataStream *stream);
+
+  /**
+   * write 4 bytes to stream
+   * @param base_value non encoded value
+   * @param stream output stream
+   * @param is_little_endian endianness determination
+   * @return resulting write size
+   **/
+  int write(uint32_t base_value, DataStream *stream, bool is_little_endian =
+                EndiannessCheck::IS_LITTLE);
+
+  /**
+   * write 2 bytes to stream
+   * @param base_value non encoded value
+   * @param stream output stream
+   * @param is_little_endian endianness determination
+   * @return resulting write size
+   **/
+  int write(uint16_t base_value, DataStream *stream, bool is_little_endian =
+                EndiannessCheck::IS_LITTLE);
+
+  /**
+   * write valueto stream
+   * @param value non encoded value
+   * @param len length of value
+   * @param strema output stream
+   * @return resulting write size
+   **/
+  int write(uint8_t *value, int len, DataStream *stream);
+
+  /**
+   * write 8 bytes to stream
+   * @param base_value non encoded value
+   * @param stream output stream
+   * @param is_little_endian endianness determination
+   * @return resulting write size
+   **/
+  int write(uint64_t base_value, DataStream *stream, bool is_little_endian =
+                EndiannessCheck::IS_LITTLE);
+
+  /**
+   * write bool to stream
+   * @param value non encoded value
+   * @return resulting write size
+   **/
+  int write(bool value);
+
+  /**
+   * write UTF string to stream
+   * @param str string to write
+   * @return resulting write size
+   **/
+  int writeUTF(std::string str, DataStream *stream, bool widen = false);
+
+  /**
+   * reads a byte from the stream
+   * @param value reference in which will set the result
+   * @param stream stream from which we will read
+   * @return resulting read size
+   **/
+  int read(uint8_t &value, DataStream *stream);
+
+  /**
+   * reads two bytes from the stream
+   * @param value reference in which will set the result
+   * @param stream stream from which we will read
+   * @return resulting read size
+   **/
+  int read(uint16_t &base_value, DataStream *stream, bool is_little_endian =
+               EndiannessCheck::IS_LITTLE);
+
+  /**
+   * reads a byte from the stream
+   * @param value reference in which will set the result
+   * @param stream stream from which we will read
+   * @return resulting read size
+   **/
+  int read(char &value, DataStream *stream);
+
+  /**
+   * reads a byte array from the stream
+   * @param value reference in which will set the result
+   * @param len length to read
+   * @param stream stream from which we will read
+   * @return resulting read size
+   **/
+  int read(uint8_t *value, int len, DataStream *stream);
+
+  /**
+   * reads four bytes from the stream
+   * @param value reference in which will set the result
+   * @param stream stream from which we will read
+   * @return resulting read size
+   **/
+  int read(uint32_t &value, DataStream *stream, bool is_little_endian =
+               EndiannessCheck::IS_LITTLE);
+
+  /**
+   * reads eight byte from the stream
+   * @param value reference in which will set the result
+   * @param stream stream from which we will read
+   * @return resulting read size
+   **/
+  int read(uint64_t &value, DataStream *stream, bool is_little_endian =
+               EndiannessCheck::IS_LITTLE);
+
+  /**
+   * read UTF from stream
+   * @param str reference string
+   * @param stream stream from which we will read
+   * @return resulting read size
+   **/
+  int readUTF(std::string &str, DataStream *stream, bool widen = false);
+
+ protected:
 
 };
 
-
+} /* namespace io */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
 #endif /* LIBMINIFI_INCLUDE_IO_SERIALIZABLE_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/io/SocketFactory.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/SocketFactory.h b/libminifi/include/io/SocketFactory.h
deleted file mode 100644
index c8cbcb1..0000000
--- a/libminifi/include/io/SocketFactory.h
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#ifndef SOCKET_FACTORY_H
-#define SOCKET_FACTORY_H
-
-#include "ClientSocket.h"
-#include "TLSSocket.h"
-#include "ClientSocket.h"
-#include "Configure.h"
-#include "utils/StringUtils.h"
-
-/**
-  Purpose: Due to the current design this is the only mechanism by which we can
-  inject different socket types
-  
-**/
-class SocketFactory{
-public:
-
-	/**
-	 * Build an instance, creating a memory fence, which
-	 * allows us to avoid locking. This is tantamount to double checked locking.
-	 * @returns new SocketFactory;
-	 */
-	static SocketFactory *getInstance() {
-		SocketFactory* atomic_context = context_instance_.load(
-				std::memory_order_relaxed);
-		std::atomic_thread_fence(std::memory_order_acquire);
-		if (atomic_context == nullptr) {
-			std::lock_guard<std::mutex> lock(context_mutex_);
-			atomic_context = context_instance_.load(std::memory_order_relaxed);
-			if (atomic_context == nullptr) {
-				atomic_context = new SocketFactory();
-				std::atomic_thread_fence(std::memory_order_release);
-				context_instance_.store(atomic_context,
-						std::memory_order_relaxed);
-			}
-		}
-		return atomic_context;
-	}
-
-	/**
-	 * Creates a socket and returns a unique ptr
-	 *
-	 */
-	std::unique_ptr<Socket> createSocket(const std::string &host, const uint16_t port) {
-		Socket *socket = 0;
-		if (is_secure_) {
-#ifdef OPENSSL_SUPPORT
-			socket = new TLSSocket(host, port);
-#else
-			socket = 0;
-#endif
-		} else {
-			socket = new Socket(host, port);
-		}
-		return std::unique_ptr<Socket>(socket);
-	}
-protected:
-	SocketFactory() :
-			configure_(Configure::getConfigure()) {
-		std::string secureStr;
-		is_secure_ = false;
-		if (configure_->get(Configure::nifi_remote_input_secure, secureStr)) {
-			StringUtils::StringToBool(secureStr, is_secure_);
-		}
-	}
-
-	bool is_secure_;
-	static std::atomic<SocketFactory*> context_instance_;
-	static std::mutex context_mutex_;
-
-	Configure *configure_;
-};
-
-#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/io/Sockets.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/Sockets.h b/libminifi/include/io/Sockets.h
new file mode 100644
index 0000000..2c0b163
--- /dev/null
+++ b/libminifi/include/io/Sockets.h
@@ -0,0 +1,27 @@
+/**
+ *
+ * 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_SOCKET_H_
+#define LIBMINIFI_INCLUDE_IO_SOCKET_H_
+
+#include "ClientSocket.h"
+
+#ifdef OPENSSL_SUPPORT
+#include "tls/TLSSocket.h"
+#endif
+
+#endif /* LIBMINIFI_INCLUDE_IO_TLS_SECURESOCKET_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/io/StreamFactory.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/StreamFactory.h b/libminifi/include/io/StreamFactory.h
new file mode 100644
index 0000000..faa10b5
--- /dev/null
+++ b/libminifi/include/io/StreamFactory.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 SOCKET_FACTORY_H
+#define SOCKET_FACTORY_H
+
+#include "properties/Configure.h"
+#include "Sockets.h"
+#include "utils/StringUtils.h"
+#include "validation.h"
+
+#ifdef OPENSSL_SUPPORT
+#include "tls/TLSSocket.h"
+#endif
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace io {
+
+/**
+ * Purpose: Socket Creator is a class that will determine if the provided socket type
+ * exists per the compilation parameters
+ */
+template<typename T>
+class SocketCreator {
+
+	template<bool cond, typename U>
+	using TypeCheck = typename std::enable_if< cond, U >::type;
+
+public:
+	template<typename U = T>
+	TypeCheck<true, U> *create(const std::string &host, const uint16_t port) {
+		return new T(host, port);
+	}
+	template<typename U = T>
+	TypeCheck<false, U> *create(const std::string &host, const uint16_t port) {
+		return new Socket(host, port);
+	}
+
+};
+
+/**
+ Purpose: Due to the current design this is the only mechanism by which we can
+ inject different socket types
+ 
+ **/
+class StreamFactory {
+public:
+
+	/**
+	 * Build an instance, creating a memory fence, which
+	 * allows us to avoid locking. This is tantamount to double checked locking.
+	 * @returns new StreamFactory;
+	 */
+	static StreamFactory *getInstance() {
+		StreamFactory* atomic_context = context_instance_.load(
+				std::memory_order_relaxed);
+		std::atomic_thread_fence(std::memory_order_acquire);
+		if (atomic_context == nullptr) {
+			std::lock_guard < std::mutex > lock(context_mutex_);
+			atomic_context = context_instance_.load(std::memory_order_relaxed);
+			if (atomic_context == nullptr) {
+				atomic_context = new StreamFactory();
+				std::atomic_thread_fence(std::memory_order_release);
+				context_instance_.store(atomic_context,
+						std::memory_order_relaxed);
+			}
+		}
+		return atomic_context;
+	}
+
+	/**
+	 * Creates a socket and returns a unique ptr
+	 *
+	 */
+	std::unique_ptr<Socket> createSocket(const std::string &host,
+			const uint16_t port) {
+		Socket *socket = 0;
+
+		if (is_secure_) {
+			socket = createSocket<TLSSocket>(host, port);
+		} else {
+			socket = createSocket<Socket>(host, port);
+		}
+		return std::unique_ptr < Socket > (socket);
+	}
+
+protected:
+
+	/**
+	 * Creates a socket and returns a unique ptr
+	 *
+	 */
+	template<typename T>
+	Socket *createSocket(const std::string &host, const uint16_t port) {
+		SocketCreator<T> creator;
+		return creator.create(host, port);
+	}
+
+	StreamFactory() :
+			configure_(Configure::getConfigure()) {
+		std::string secureStr;
+		is_secure_ = false;
+		if (configure_->get(Configure::nifi_remote_input_secure, secureStr)) {
+			org::apache::nifi::minifi::utils::StringUtils::StringToBool(
+					secureStr, is_secure_);
+		}
+	}
+
+	bool is_secure_;
+	static std::atomic<StreamFactory*> context_instance_;
+	static std::mutex context_mutex_;
+
+	Configure *configure_;
+};
+
+} /* namespace io */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/io/TLSSocket.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/TLSSocket.h b/libminifi/include/io/TLSSocket.h
deleted file mode 100644
index 32645ca..0000000
--- a/libminifi/include/io/TLSSocket.h
+++ /dev/null
@@ -1,187 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#ifndef LIBMINIFI_INCLUDE_IO_TLSSOCKET_H_
-#define LIBMINIFI_INCLUDE_IO_TLSSOCKET_H_
-
-#ifdef OPENSSL_SUPPORT
-#include <cstdint>
-#include "ClientSocket.h"
-#include <atomic>
-#include <mutex>
-
-#include "Configure.h"
-#include <openssl/ssl.h>
-#include <openssl/err.h>
-
-#define TLS_ERROR_CONTEXT 1
-#define TLS_ERROR_PEM_MISSING 2
-#define TLS_ERROR_CERT_MISSING 3
-#define TLS_ERROR_KEY_ERROR 4
-#define TLS_ERROR_CERT_ERROR 5
-
-class TLSContext {
-
-public:
-
-	/**
-	 * Build an instance, creating a memory fence, which
-	 * allows us to avoid locking. This is tantamount to double checked locking.
-	 * @returns new TLSContext;
-	 */
-	static TLSContext *getInstance() {
-		TLSContext* atomic_context = context_instance.load(
-				std::memory_order_relaxed);
-		std::atomic_thread_fence(std::memory_order_acquire);
-		if (atomic_context == nullptr) {
-			std::lock_guard<std::mutex> lock(context_mutex);
-			atomic_context = context_instance.load(std::memory_order_relaxed);
-			if (atomic_context == nullptr) {
-				atomic_context = new TLSContext();
-				atomic_context->initialize();
-				std::atomic_thread_fence(std::memory_order_release);
-				context_instance.store(atomic_context,
-						std::memory_order_relaxed);
-			}
-		}
-		return atomic_context;
-	}
-
-	virtual ~TLSContext() {
-		if (0 != ctx)
-			SSL_CTX_free(ctx);
-	}
-
-	SSL_CTX *getContext() {
-		return ctx;
-	}
-
-	short getError() {
-		return error_value;
-	}
-
-	short initialize();
-
-private:
-
-	static int pemPassWordCb(char *buf, int size, int rwflag, void *userdata) {
-		std::string passphrase;
-
-		if (Configure::getConfigure()->get(
-				Configure::nifi_security_client_pass_phrase, passphrase)) {
-
-			std::ifstream file(passphrase.c_str(), std::ifstream::in);
-			if (!file.good()) {
-				memset(buf, 0x00, size);
-				return 0;
-			}
-
-			std::string password;
-			password.assign((std::istreambuf_iterator<char>(file)),
-					std::istreambuf_iterator<char>());
-			file.close();
-			memset(buf,0x00,size);
-			memcpy(buf, password.c_str(), password.length()-1);
-
-			return password.length()-1;
-		}
-		return 0;
-	}
-
-	TLSContext();
-
-	std::shared_ptr<Logger> logger_;
-	Configure *configuration;
-	SSL_CTX *ctx;
-
-	short error_value;
-	static std::atomic<TLSContext*> context_instance;
-
-	static std::mutex context_mutex;
-};
-
-class TLSSocket: public Socket {
-public:
-
-	/**
-	 * Constructor that accepts host name, port and listeners. With this
-	 * contructor we will be creating a server socket
-	 * @param hostname our host name
-	 * @param port connecting port
-	 * @param listeners number of listeners in the queue
-	 */
-	explicit TLSSocket(const std::string &hostname, const uint16_t port,
-			const uint16_t listeners);
-
-	/**
-	 * Constructor that creates a client socket.
-	 * @param hostname hostname we are connecting to.
-	 * @param port port we are connecting to.
-	 */
-	explicit TLSSocket(const std::string &hostname, const uint16_t port);
-
-	/**
-	 * Move constructor.
-	 */
-	explicit TLSSocket(const TLSSocket &&);
-
-	virtual ~TLSSocket();
-
-	/**
-	 * Initializes the socket
-	 * @return result of the creation operation.
-	 */
-	short initialize();
-
-	/**
-	 * Attempt to select the socket file descriptor
-	 * @param msec timeout interval to wait
-	 * @returns file descriptor
-	 */
-	virtual short select_descriptor(const uint16_t msec);
-
-	/**
-	 * Reads data and places it into buf
-	 * @param buf buffer in which we extract data
-	 * @param buflen
-	 */
-	virtual int readData(uint8_t *buf, int buflen);
-
-	/**
-	 * Write value to the stream using std::vector
-	 * @param buf incoming buffer
-	 * @param buflen buffer to write
-	 *
-	 */
-	int writeData(std::vector<uint8_t> &buf, int buflen);
-
-	/**
-	 * Write value to the stream using uint8_t ptr
-	 * @param buf incoming buffer
-	 * @param buflen buffer to write
-	 *
-	 */
-	int writeData(uint8_t *value, int size);
-
-protected:
-
-	SSL* ssl;
-
-};
-#endif
-
-#endif /* LIBMINIFI_INCLUDE_IO_TLSSOCKET_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/io/tls/TLSSocket.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/tls/TLSSocket.h b/libminifi/include/io/tls/TLSSocket.h
new file mode 100644
index 0000000..f86f8bc
--- /dev/null
+++ b/libminifi/include/io/tls/TLSSocket.h
@@ -0,0 +1,198 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef LIBMINIFI_INCLUDE_IO_TLSSOCKET_H_
+#define LIBMINIFI_INCLUDE_IO_TLSSOCKET_H_
+
+#include <cstdint>
+#include "../ClientSocket.h"
+#include <atomic>
+#include <mutex>
+
+#include "properties/Configure.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace io {
+
+#include <openssl/ssl.h>
+#include <openssl/err.h>
+
+#define TLS_ERROR_CONTEXT 1
+#define TLS_ERROR_PEM_MISSING 2
+#define TLS_ERROR_CERT_MISSING 3
+#define TLS_ERROR_KEY_ERROR 4
+#define TLS_ERROR_CERT_ERROR 5
+
+class TLSContext {
+
+ public:
+
+  /**
+   * Build an instance, creating a memory fence, which
+   * allows us to avoid locking. This is tantamount to double checked locking.
+   * @returns new TLSContext;
+   */
+  static TLSContext *getInstance() {
+    TLSContext* atomic_context = context_instance.load(
+        std::memory_order_relaxed);
+    std::atomic_thread_fence(std::memory_order_acquire);
+    if (atomic_context == nullptr) {
+      std::lock_guard<std::mutex> lock(context_mutex);
+      atomic_context = context_instance.load(std::memory_order_relaxed);
+      if (atomic_context == nullptr) {
+        atomic_context = new TLSContext();
+        atomic_context->initialize();
+        std::atomic_thread_fence(std::memory_order_release);
+        context_instance.store(atomic_context, std::memory_order_relaxed);
+      }
+    }
+    return atomic_context;
+  }
+
+  virtual ~TLSContext() {
+    if (0 != ctx)
+      SSL_CTX_free(ctx);
+  }
+
+  SSL_CTX *getContext() {
+    return ctx;
+  }
+
+  short getError() {
+    return error_value;
+  }
+
+  short initialize();
+
+ private:
+
+  static int pemPassWordCb(char *buf, int size, int rwflag, void *userdata) {
+    std::string passphrase;
+
+    if (Configure::getConfigure()->get(
+        Configure::nifi_security_client_pass_phrase, passphrase)) {
+
+      std::ifstream file(passphrase.c_str(), std::ifstream::in);
+      if (!file.good()) {
+        memset(buf, 0x00, size);
+        return 0;
+      }
+
+      std::string password;
+      password.assign((std::istreambuf_iterator<char>(file)),
+                      std::istreambuf_iterator<char>());
+      file.close();
+      memset(buf, 0x00, size);
+      memcpy(buf, password.c_str(), password.length() - 1);
+
+      return password.length() - 1;
+    }
+    return 0;
+  }
+
+  TLSContext();
+
+  std::shared_ptr<logging::Logger> logger_;
+  Configure *configuration;
+  SSL_CTX *ctx;
+
+  short error_value;
+
+  static std::atomic<TLSContext*> context_instance;
+  static std::mutex context_mutex;
+
+};
+
+class TLSSocket : public Socket {
+ public:
+
+  /**
+   * Constructor that accepts host name, port and listeners. With this
+   * contructor we will be creating a server socket
+   * @param hostname our host name
+   * @param port connecting port
+   * @param listeners number of listeners in the queue
+   */
+  explicit TLSSocket(const std::string &hostname, const uint16_t port,
+                     const uint16_t listeners);
+
+  /**
+   * Constructor that creates a client socket.
+   * @param hostname hostname we are connecting to.
+   * @param port port we are connecting to.
+   */
+  explicit TLSSocket(const std::string &hostname, const uint16_t port);
+
+  /**
+   * Move constructor.
+   */
+  explicit TLSSocket(const TLSSocket &&);
+
+  virtual ~TLSSocket();
+
+  /**
+   * Initializes the socket
+   * @return result of the creation operation.
+   */
+  short initialize();
+
+  /**
+   * Attempt to select the socket file descriptor
+   * @param msec timeout interval to wait
+   * @returns file descriptor
+   */
+  virtual short select_descriptor(const uint16_t msec);
+
+  /**
+   * Reads data and places it into buf
+   * @param buf buffer in which we extract data
+   * @param buflen
+   */
+  virtual int readData(uint8_t *buf, int buflen);
+
+  /**
+   * Write value to the stream using std::vector
+   * @param buf incoming buffer
+   * @param buflen buffer to write
+   *
+   */
+  int writeData(std::vector<uint8_t> &buf, int buflen);
+
+  /**
+   * Write value to the stream using uint8_t ptr
+   * @param buf incoming buffer
+   * @param buflen buffer to write
+   *
+   */
+  int writeData(uint8_t *value, int size);
+
+ protected:
+
+  SSL* ssl;
+
+};
+
+} /* namespace io */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_IO_TLSSOCKET_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/io/validation.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/validation.h b/libminifi/include/io/validation.h
index e1b4bb6..c66c412 100644
--- a/libminifi/include/io/validation.h
+++ b/libminifi/include/io/validation.h
@@ -18,35 +18,38 @@
 
 #ifndef VALIDATION_H
 #define VALIDATION_H
+#include <type_traits>
 #include <string>
 #include <cstring>
 
+
 /**
  * A checker that will, at compile time, tell us
  * if the declared type has a size method.
  */
 template<typename T>
 class size_function_functor_checker {
-    typedef char hasit;
-    typedef long doesnothaveit;
+  typedef char hasit;
+  typedef long doesnothaveit;
 
-    // look for the declared type
-    template<typename O> static hasit test(decltype(&O::size));
-    template<typename O> static doesnothaveit test(...);
+  // look for the declared type
+  template<typename O> static hasit test(decltype(&O::size));
+  template<typename O> static doesnothaveit test(...);
 
-public:
-    enum {
-        has_size_function = sizeof(test<T>(0)) == sizeof(char)
-    };
+ public:
+  enum {
+    has_size_function = sizeof(test<T>(0)) == sizeof(char)
+  };
 };
 
+
 /**
  * Determines if the variable is null or ::size() == 0
  */
 template<typename T>
 static auto IsNullOrEmpty(
     T &object) -> typename std::enable_if<size_function_functor_checker<T>::has_size_function==1, bool>::type {
-    return object.size() == 0;
+  return object.size() == 0;
 }
 
 /**
@@ -55,7 +58,7 @@ static auto IsNullOrEmpty(
 template<typename T>
 static auto IsNullOrEmpty(
     T *object) -> typename std::enable_if<size_function_functor_checker<T>::has_size_function==1, bool>::type {
-    return (0 == object || object->size() == 0);
+  return (0 == object || object->size() == 0);
 }
 
 /**
@@ -64,13 +67,13 @@ static auto IsNullOrEmpty(
 template<typename T>
 static auto IsNullOrEmpty(
     T *object) -> typename std::enable_if<not size_function_functor_checker<T>::has_size_function , bool>::type {
-    return (0 == object);
+  return (0 == object);
 }
 /**
  * Determines if the variable is null or strlen(str) == 0
  */
 static auto IsNullOrEmpty(char *str)-> decltype(NULL !=str, bool()) {
-    return (NULL == str || strlen(str) == 0);
+  return (NULL == str || strlen(str) == 0);
 }
 
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/processors/AppendHostInfo.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/AppendHostInfo.h b/libminifi/include/processors/AppendHostInfo.h
new file mode 100644
index 0000000..6515918
--- /dev/null
+++ b/libminifi/include/processors/AppendHostInfo.h
@@ -0,0 +1,80 @@
+/**
+ * @file AppendHostInfo.h
+ * AppendHostInfo class declaration
+ *
+ * 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 __APPEND_HOSTINFO_H__
+#define __APPEND_HOSTINFO_H__
+
+#include "core/Property.h"
+#include "FlowFileRecord.h"
+#include "core/Processor.h"
+#include "core/ProcessSession.h"
+#include "core/core.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+// AppendHostInfo Class
+class AppendHostInfo : public core::Processor {
+ public:
+  // Constructor
+  /*!
+   * Create a new processor
+   */
+  AppendHostInfo(std::string name, uuid_t uuid = NULL)
+      : core::Processor(name, uuid) {
+    logger_ = logging::Logger::getLogger();
+  }
+  // Destructor
+  virtual ~AppendHostInfo() {
+  }
+  // Processor Name
+  static const std::string ProcessorName;
+  // Supported Properties
+  static core::Property InterfaceName;
+  static core::Property HostAttribute;
+  static core::Property IPAttribute;
+
+  // Supported Relationships
+  static core::Relationship Success;
+
+ public:
+  // OnTrigger method, implemented by NiFi AppendHostInfo
+  virtual void onTrigger(
+      core::ProcessContext *context,
+      core::ProcessSession *session);
+  // Initialize, over write by NiFi AppendHostInfo
+  virtual void initialize(void);
+
+ protected:
+
+ private:
+  // Logger
+  std::shared_ptr<logging::Logger> logger_;
+};
+
+} /* namespace processors */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/processors/ExecuteProcess.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/ExecuteProcess.h b/libminifi/include/processors/ExecuteProcess.h
new file mode 100644
index 0000000..123eed3
--- /dev/null
+++ b/libminifi/include/processors/ExecuteProcess.h
@@ -0,0 +1,125 @@
+/**
+ * @file ExecuteProcess.h
+ * ExecuteProcess class declaration
+ *
+ * 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 __EXECUTE_PROCESS_H__
+#define __EXECUTE_PROCESS_H__
+
+#include <stdio.h>
+#include <unistd.h>
+#include <string>
+#include <errno.h>
+#include <chrono>
+#include <thread>
+#include <unistd.h>
+#include <sys/wait.h>
+#include <iostream>
+#include <sys/types.h>
+#include <signal.h>
+#include "FlowFileRecord.h"
+#include "core/Processor.h"
+#include "core/ProcessSession.h"
+#include "core/core.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+// ExecuteProcess Class
+class ExecuteProcess : public core::Processor {
+ public:
+  // Constructor
+  /*!
+   * Create a new processor
+   */
+  ExecuteProcess(std::string name, uuid_t uuid = NULL)
+      : Processor(name, uuid) {
+    logger_ = logging::Logger::getLogger();
+    _redirectErrorStream = false;
+    _batchDuration = 0;
+    _workingDir = ".";
+    _processRunning = false;
+    _pid = 0;
+  }
+  // Destructor
+  virtual ~ExecuteProcess() {
+    if (_processRunning && _pid > 0)
+      kill(_pid, SIGTERM);
+  }
+  // Processor Name
+  static const std::string ProcessorName;
+  // Supported Properties
+  static core::Property Command;
+  static core::Property CommandArguments;
+  static core::Property WorkingDir;
+  static core::Property BatchDuration;
+  static core::Property RedirectErrorStream;
+  // Supported Relationships
+  static core::Relationship Success;
+
+  // Nest Callback Class for write stream
+  class WriteCallback : public OutputStreamCallback {
+   public:
+    WriteCallback(char *data, uint64_t size)
+        : _data(data),
+          _dataSize(size) {
+    }
+    char *_data;
+    uint64_t _dataSize;
+    void process(std::ofstream *stream) {
+      if (_data && _dataSize > 0)
+        stream->write(_data, _dataSize);
+    }
+  };
+
+ public:
+  // OnTrigger method, implemented by NiFi ExecuteProcess
+  virtual void onTrigger(
+      core::ProcessContext *context,
+      core::ProcessSession *session);
+  // Initialize, over write by NiFi ExecuteProcess
+  virtual void initialize(void);
+
+ protected:
+
+ private:
+  // Logger
+  std::shared_ptr<logging::Logger> logger_;
+  // Property
+  std::string _command;
+  std::string _commandArgument;
+  std::string _workingDir;
+  int64_t _batchDuration;
+  bool _redirectErrorStream;
+  // Full command
+  std::string _fullCommand;
+  // whether the process is running
+  bool _processRunning;
+  int _pipefd[2];
+  pid_t _pid;
+};
+
+} /* namespace processors */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/processors/GenerateFlowFile.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/GenerateFlowFile.h b/libminifi/include/processors/GenerateFlowFile.h
new file mode 100644
index 0000000..c4ab6fe
--- /dev/null
+++ b/libminifi/include/processors/GenerateFlowFile.h
@@ -0,0 +1,98 @@
+/**
+ * @file GenerateFlowFile.h
+ * GenerateFlowFile class declaration
+ *
+ * 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 __GENERATE_FLOW_FILE_H__
+#define __GENERATE_FLOW_FILE_H__
+
+#include "FlowFileRecord.h"
+#include "core/Processor.h"
+#include "core/ProcessSession.h"
+#include "core/core.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+// GenerateFlowFile Class
+class GenerateFlowFile : public core::Processor {
+ public:
+  // Constructor
+  /*!
+   * Create a new processor
+   */
+  GenerateFlowFile(std::string name, uuid_t uuid = NULL)
+      : Processor(name, uuid) {
+    _data = NULL;
+    _dataSize = 0;
+  }
+  // Destructor
+  virtual ~GenerateFlowFile() {
+    if (_data)
+      delete[] _data;
+  }
+  // Processor Name
+  static const std::string ProcessorName;
+  // Supported Properties
+  static core::Property FileSize;
+  static core::Property BatchSize;
+  static core::Property DataFormat;
+  static core::Property UniqueFlowFiles;
+  static const char *DATA_FORMAT_BINARY;
+  static const char *DATA_FORMAT_TEXT;
+  // Supported Relationships
+  static core::Relationship Success;
+  // Nest Callback Class for write stream
+  class WriteCallback : public OutputStreamCallback {
+   public:
+    WriteCallback(char *data, uint64_t size)
+        : _data(data),
+          _dataSize(size) {
+    }
+    char *_data;
+    uint64_t _dataSize;
+    void process(std::ofstream *stream) {
+      if (_data && _dataSize > 0)
+        stream->write(_data, _dataSize);
+    }
+  };
+
+ public:
+  // OnTrigger method, implemented by NiFi GenerateFlowFile
+  virtual void onTrigger(
+      core::ProcessContext *context,
+      core::ProcessSession *session);
+  // Initialize, over write by NiFi GenerateFlowFile
+  virtual void initialize(void);
+
+ protected:
+
+ private:
+  // Generated data
+  char * _data;
+  // Size of the generate data
+  uint64_t _dataSize;
+};
+
+} /* namespace processors */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/processors/GetFile.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/GetFile.h b/libminifi/include/processors/GetFile.h
new file mode 100644
index 0000000..f1f0694
--- /dev/null
+++ b/libminifi/include/processors/GetFile.h
@@ -0,0 +1,129 @@
+/**
+ * @file GetFile.h
+ * GetFile class declaration
+ *
+ * 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 __GET_FILE_H__
+#define __GET_FILE_H__
+
+#include "FlowFileRecord.h"
+#include "core/Processor.h"
+#include "core/ProcessSession.h"
+#include "core/core.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+// GetFile Class
+class GetFile : public core::Processor {
+ public:
+  // Constructor
+  /*!
+   * Create a new processor
+   */
+  GetFile(std::string name, uuid_t uuid = NULL)
+      : Processor(name, uuid) {
+    logger_ = logging::Logger::getLogger();
+    _directory = ".";
+    _recursive = true;
+    _keepSourceFile = false;
+    _minAge = 0;
+    _maxAge = 0;
+    _minSize = 0;
+    _maxSize = 0;
+    _ignoreHiddenFile = true;
+    _pollInterval = 0;
+    _batchSize = 10;
+    _lastDirectoryListingTime = getTimeMillis();
+    _fileFilter = "[^\\.].*";
+  }
+  // Destructor
+  virtual ~GetFile() {
+  }
+  // Processor Name
+  static const std::string ProcessorName;
+  // Supported Properties
+  static core::Property Directory;
+  static core::Property Recurse;
+  static core::Property KeepSourceFile;
+  static core::Property MinAge;
+  static core::Property MaxAge;
+  static core::Property MinSize;
+  static core::Property MaxSize;
+  static core::Property IgnoreHiddenFile;
+  static core::Property PollInterval;
+  static core::Property BatchSize;
+  static core::Property FileFilter;
+  // Supported Relationships
+  static core::Relationship Success;
+
+ public:
+  // OnTrigger method, implemented by NiFi GetFile
+  virtual void onTrigger(
+      core::ProcessContext *context,
+      core::ProcessSession *session);
+  // Initialize, over write by NiFi GetFile
+  virtual void initialize(void);
+  // perform directory listing
+  void performListing(std::string dir);
+
+ protected:
+
+ private:
+  // Logger
+  std::shared_ptr<logging::Logger> logger_;
+  // Queue for store directory list
+  std::queue<std::string> _dirList;
+  // Get Listing size
+  uint64_t getListingSize() {
+    std::lock_guard<std::mutex> lock(mutex_);
+    return _dirList.size();
+  }
+  // Whether the directory listing is empty
+  bool isListingEmpty();
+  // Put full path file name into directory listing
+  void putListing(std::string fileName);
+  // Poll directory listing for files
+  void pollListing(std::queue<std::string> &list, int maxSize);
+  // Check whether file can be added to the directory listing
+  bool acceptFile(std::string fullName, std::string name);
+  // Mutex for protection of the directory listing
+  std::mutex mutex_;
+  std::string _directory;
+  bool _recursive;
+  bool _keepSourceFile;
+  int64_t _minAge;
+  int64_t _maxAge;
+  int64_t _minSize;
+  int64_t _maxSize;
+  bool _ignoreHiddenFile;
+  int64_t _pollInterval;
+  int64_t _batchSize;
+  uint64_t _lastDirectoryListingTime;
+  std::string _fileFilter;
+};
+
+} /* namespace processors */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/processors/ListenHTTP.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/ListenHTTP.h b/libminifi/include/processors/ListenHTTP.h
new file mode 100644
index 0000000..adaefb1
--- /dev/null
+++ b/libminifi/include/processors/ListenHTTP.h
@@ -0,0 +1,126 @@
+/**
+ * @file ListenHTTP.h
+ * ListenHTTP class declaration
+ *
+ * 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 __LISTEN_HTTP_H__
+#define __LISTEN_HTTP_H__
+
+#include <memory>
+#include <regex>
+
+#include <CivetServer.h>
+
+#include "FlowFileRecord.h"
+#include "core/Processor.h"
+#include "core/ProcessSession.h"
+#include "core/core.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+// ListenHTTP Class
+class ListenHTTP : public core::Processor {
+ public:
+
+  // Constructor
+  /*!
+   * Create a new processor
+   */
+  ListenHTTP(std::string name, uuid_t uuid = NULL)
+      : Processor(name, uuid) {
+    _logger = logging::Logger::getLogger();
+  }
+  // Destructor
+  virtual ~ListenHTTP();
+  // Processor Name
+  static const std::string ProcessorName;
+  // Supported Properties
+  static core::Property BasePath;
+  static core::Property Port;
+  static core::Property AuthorizedDNPattern;
+  static core::Property SSLCertificate;
+  static core::Property SSLCertificateAuthority;
+  static core::Property SSLVerifyPeer;
+  static core::Property SSLMinimumVersion;
+  static core::Property HeadersAsAttributesRegex;
+  // Supported Relationships
+  static core::Relationship Success;
+
+  void onTrigger(core::ProcessContext *context,
+                 core::ProcessSession *session);
+  void initialize();
+  void onSchedule(
+      core::ProcessContext *context,
+      core::ProcessSessionFactory *sessionFactory);
+
+  // HTTP request handler
+  class Handler : public CivetHandler {
+   public:
+    Handler(
+        core::ProcessContext *context,
+        core::ProcessSessionFactory *sessionFactory,
+        std::string &&authDNPattern, std::string &&headersAsAttributesPattern);
+    bool handlePost(CivetServer *server, struct mg_connection *conn);
+
+   private:
+    // Send HTTP 500 error response to client
+    void sendErrorResponse(struct mg_connection *conn);
+    // Logger
+    std::shared_ptr<logging::Logger> _logger;
+
+    std::regex _authDNRegex;
+    std::regex _headersAsAttributesRegex;
+    core::ProcessContext *_processContext;
+    core::ProcessSessionFactory *_processSessionFactory;
+  };
+
+  // Write callback for transferring data from HTTP request to content repo
+  class WriteCallback : public OutputStreamCallback {
+   public:
+    WriteCallback(struct mg_connection *conn,
+                  const struct mg_request_info *reqInfo);
+    void process(std::ofstream *stream);
+
+   private:
+    // Logger
+    std::shared_ptr<logging::Logger> _logger;
+
+    struct mg_connection *_conn;
+    const struct mg_request_info *_reqInfo;
+  };
+
+ protected:
+
+ private:
+  // Logger
+  std::shared_ptr<logging::Logger> _logger;
+
+  std::unique_ptr<CivetServer> _server;
+  std::unique_ptr<Handler> _handler;
+};
+
+} /* namespace processors */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/processors/ListenSyslog.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/ListenSyslog.h b/libminifi/include/processors/ListenSyslog.h
new file mode 100644
index 0000000..1e1e11f
--- /dev/null
+++ b/libminifi/include/processors/ListenSyslog.h
@@ -0,0 +1,216 @@
+/**
+ * @file ListenSyslog.h
+ * ListenSyslog class declaration
+ *
+ * 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 __LISTEN_SYSLOG_H__
+#define __LISTEN_SYSLOG_H__
+
+#include <stdio.h>
+#include <unistd.h>
+#include <sys/types.h>
+#include <sys/socket.h>
+#include <netinet/in.h>
+#include <arpa/inet.h>
+#include <errno.h>
+#include <sys/select.h>
+#include <sys/time.h>
+#include <sys/types.h>
+#include <chrono>
+#include <thread>
+#include "FlowFileRecord.h"
+#include "core/Processor.h"
+#include "core/ProcessSession.h"
+#include "core/core.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+// SyslogEvent
+typedef struct {
+  uint8_t *payload;
+  uint64_t len;
+} SysLogEvent;
+
+// ListenSyslog Class
+class ListenSyslog : public core::Processor {
+ public:
+  // Constructor
+  /*!
+   * Create a new processor
+   */
+  ListenSyslog(std::string name, uuid_t uuid = NULL)
+      : Processor(name, uuid) {
+    logger_ = logging::Logger::getLogger();
+    _eventQueueByteSize = 0;
+    _serverSocket = 0;
+    _recvBufSize = 65507;
+    _maxSocketBufSize = 1024 * 1024;
+    _maxConnections = 2;
+    _maxBatchSize = 1;
+    _messageDelimiter = "\n";
+    _protocol = "UDP";
+    _port = 514;
+    _parseMessages = false;
+    _serverSocket = 0;
+    _maxFds = 0;
+    FD_ZERO(&_readfds);
+    _thread = NULL;
+    _resetServerSocket = false;
+    _serverTheadRunning = false;
+  }
+  // Destructor
+  virtual ~ListenSyslog() {
+    _serverTheadRunning = false;
+    if (this->_thread)
+      delete this->_thread;
+    // need to reset the socket
+    std::vector<int>::iterator it;
+    for (it = _clientSockets.begin(); it != _clientSockets.end(); ++it) {
+      int clientSocket = *it;
+      close(clientSocket);
+    }
+    _clientSockets.clear();
+    if (_serverSocket > 0) {
+      logger_->log_info("ListenSysLog Server socket %d close", _serverSocket);
+      close(_serverSocket);
+      _serverSocket = 0;
+    }
+  }
+  // Processor Name
+  static const std::string ProcessorName;
+  // Supported Properties
+  static core::Property RecvBufSize;
+  static core::Property MaxSocketBufSize;
+  static core::Property MaxConnections;
+  static core::Property MaxBatchSize;
+  static core::Property MessageDelimiter;
+  static core::Property ParseMessages;
+  static core::Property Protocol;
+  static core::Property Port;
+  // Supported Relationships
+  static core::Relationship Success;
+  static core::Relationship Invalid;
+  // Nest Callback Class for write stream
+  class WriteCallback : public OutputStreamCallback {
+   public:
+    WriteCallback(char *data, uint64_t size)
+        : _data(data),
+          _dataSize(size) {
+    }
+    char *_data;
+    uint64_t _dataSize;
+    void process(std::ofstream *stream) {
+      if (_data && _dataSize > 0)
+        stream->write(_data, _dataSize);
+    }
+  };
+
+ public:
+  // OnTrigger method, implemented by NiFi ListenSyslog
+  virtual void onTrigger(
+      core::ProcessContext *context,
+      core::ProcessSession *session);
+  // Initialize, over write by NiFi ListenSyslog
+  virtual void initialize(void);
+
+ protected:
+
+ private:
+  // Logger
+  std::shared_ptr<logging::Logger> logger_;
+  // Run function for the thread
+  static void run(ListenSyslog *process);
+  // Run Thread
+  void runThread();
+  // Queue for store syslog event
+  std::queue<SysLogEvent> _eventQueue;
+  // Size of Event queue in bytes
+  uint64_t _eventQueueByteSize;
+  // Get event queue size
+  uint64_t getEventQueueSize() {
+    std::lock_guard<std::mutex> lock(mutex_);
+    return _eventQueue.size();
+  }
+  // Get event queue byte size
+  uint64_t getEventQueueByteSize() {
+    std::lock_guard<std::mutex> lock(mutex_);
+    return _eventQueueByteSize;
+  }
+  // Whether the event queue  is empty
+  bool isEventQueueEmpty() {
+    std::lock_guard<std::mutex> lock(mutex_);
+    return _eventQueue.empty();
+  }
+  // Put event into directory listing
+  void putEvent(uint8_t *payload, uint64_t len) {
+    std::lock_guard<std::mutex> lock(mutex_);
+    SysLogEvent event;
+    event.payload = payload;
+    event.len = len;
+    _eventQueue.push(event);
+    _eventQueueByteSize += len;
+  }
+  // Read \n terminated line from TCP socket
+  int readline(int fd, char *bufptr, size_t len);
+  // start server socket and handling client socket
+  void startSocketThread();
+  // Poll event
+  void pollEvent(std::queue<SysLogEvent> &list, int maxSize) {
+    std::lock_guard<std::mutex> lock(mutex_);
+
+    while (!_eventQueue.empty() && (maxSize == 0 || list.size() < maxSize)) {
+      SysLogEvent event = _eventQueue.front();
+      _eventQueue.pop();
+      _eventQueueByteSize -= event.len;
+      list.push(event);
+    }
+    return;
+  }
+  // Mutex for protection of the directory listing
+  std::mutex mutex_;
+  int64_t _recvBufSize;
+  int64_t _maxSocketBufSize;
+  int64_t _maxConnections;
+  int64_t _maxBatchSize;
+  std::string _messageDelimiter;
+  std::string _protocol;
+  int64_t _port;
+  bool _parseMessages;
+  int _serverSocket;
+  std::vector<int> _clientSockets;
+  int _maxFds;
+  fd_set _readfds;
+  // thread
+  std::thread *_thread;
+  // whether to reset the server socket
+  bool _resetServerSocket;
+  bool _serverTheadRunning;
+  // buffer for read socket
+  uint8_t _buffer[2048];
+};
+
+} /* namespace processors */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/processors/LogAttribute.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/LogAttribute.h b/libminifi/include/processors/LogAttribute.h
new file mode 100644
index 0000000..37c0ec3
--- /dev/null
+++ b/libminifi/include/processors/LogAttribute.h
@@ -0,0 +1,130 @@
+/**
+ * @file LogAttribute.h
+ * LogAttribute class declaration
+ *
+ * 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 __LOG_ATTRIBUTE_H__
+#define __LOG_ATTRIBUTE_H__
+
+#include "FlowFileRecord.h"
+#include "core/Processor.h"
+#include "core/ProcessSession.h"
+#include "core/core.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+// LogAttribute Class
+class LogAttribute : public core::Processor {
+ public:
+  // Constructor
+  /*!
+   * Create a new processor
+   */
+  LogAttribute(std::string name, uuid_t uuid = NULL)
+      : Processor(name, uuid) {
+    logger_ = logging::Logger::getLogger();
+  }
+  // Destructor
+  virtual ~LogAttribute() {
+  }
+  // Processor Name
+  static const std::string ProcessorName;
+  // Supported Properties
+  static core::Property LogLevel;
+  static core::Property AttributesToLog;
+  static core::Property AttributesToIgnore;
+  static core::Property LogPayload;
+  static core::Property LogPrefix;
+  // Supported Relationships
+  static core::Relationship Success;
+  enum LogAttrLevel {
+    LogAttrLevelTrace,
+    LogAttrLevelDebug,
+    LogAttrLevelInfo,
+    LogAttrLevelWarn,
+    LogAttrLevelError
+  };
+  // Convert log level from string to enum
+  bool logLevelStringToEnum(std::string logStr, LogAttrLevel &level) {
+    if (logStr == "trace") {
+      level = LogAttrLevelTrace;
+      return true;
+    } else if (logStr == "debug") {
+      level = LogAttrLevelDebug;
+      return true;
+    } else if (logStr == "info") {
+      level = LogAttrLevelInfo;
+      return true;
+    } else if (logStr == "warn") {
+      level = LogAttrLevelWarn;
+      return true;
+    } else if (logStr == "error") {
+      level = LogAttrLevelError;
+      return true;
+    } else
+      return false;
+  }
+  // Nest Callback Class for read stream
+  class ReadCallback : public InputStreamCallback {
+   public:
+    ReadCallback(uint64_t size) {
+      _bufferSize = size;
+      _buffer = new char[_bufferSize];
+    }
+    ~ReadCallback() {
+      if (_buffer)
+        delete[] _buffer;
+    }
+    void process(std::ifstream *stream) {
+
+      stream->read(_buffer, _bufferSize);
+      if (!stream)
+        _readSize = stream->gcount();
+      else
+        _readSize = _bufferSize;
+    }
+    char *_buffer;
+    uint64_t _bufferSize;
+    uint64_t _readSize;
+  };
+
+ public:
+  // OnTrigger method, implemented by NiFi LogAttribute
+  virtual void onTrigger(
+      core::ProcessContext *context,
+      core::ProcessSession *session);
+  // Initialize, over write by NiFi LogAttribute
+  virtual void initialize(void);
+
+ protected:
+
+ private:
+  // Logger
+  std::shared_ptr<logging::Logger> logger_;
+};
+
+} /* namespace processors */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/processors/PutFile.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/PutFile.h b/libminifi/include/processors/PutFile.h
new file mode 100644
index 0000000..7653fac
--- /dev/null
+++ b/libminifi/include/processors/PutFile.h
@@ -0,0 +1,101 @@
+/**
+ * @file PutFile.h
+ * PutFile class declaration
+ *
+ * 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 __PUT_FILE_H__
+#define __PUT_FILE_H__
+
+#include "FlowFileRecord.h"
+#include "core/Processor.h"
+#include "core/ProcessSession.h"
+#include "core/core.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+// PutFile Class
+class PutFile : public core::Processor {
+ public:
+
+  static const std::string CONFLICT_RESOLUTION_STRATEGY_REPLACE;
+  static const std::string CONFLICT_RESOLUTION_STRATEGY_IGNORE;
+  static const std::string CONFLICT_RESOLUTION_STRATEGY_FAIL;
+
+  // Constructor
+  /*!
+   * Create a new processor
+   */
+  PutFile(std::string name, uuid_t uuid = NULL)
+      : core::Processor(name, uuid) {
+    logger_ = logging::Logger::getLogger();
+  }
+  // Destructor
+  virtual ~PutFile() {
+  }
+  // Processor Name
+  static const std::string ProcessorName;
+  // Supported Properties
+  static core::Property Directory;
+  static core::Property ConflictResolution;
+  // Supported Relationships
+  static core::Relationship Success;
+  static core::Relationship Failure;
+
+  // OnTrigger method, implemented by NiFi PutFile
+  virtual void onTrigger(
+      core::ProcessContext *context,
+      core::ProcessSession *session);
+  // Initialize, over write by NiFi PutFile
+  virtual void initialize(void);
+
+  class ReadCallback : public InputStreamCallback {
+   public:
+    ReadCallback(const std::string &tmpFile, const std::string &destFile);
+    ~ReadCallback();
+    virtual void process(std::ifstream *stream);
+    bool commit();
+
+   private:
+    std::shared_ptr<logging::Logger> logger_;
+    std::ofstream _tmpFileOs;
+    bool _writeSucceeded = false;
+    std::string _tmpFile;
+    std::string _destFile;
+  };
+
+ protected:
+
+ private:
+  // Logger
+  std::shared_ptr<logging::Logger> logger_;
+
+  bool putFile(core::ProcessSession *session,
+               std::shared_ptr<FlowFileRecord> flowFile,
+               const std::string &tmpFile, const std::string &destFile);
+};
+
+} /* namespace processors */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/processors/RealTimeDataCollector.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/RealTimeDataCollector.h b/libminifi/include/processors/RealTimeDataCollector.h
new file mode 100644
index 0000000..41bd814
--- /dev/null
+++ b/libminifi/include/processors/RealTimeDataCollector.h
@@ -0,0 +1,145 @@
+/**
+ * @file RealTimeDataCollector.h
+ * RealTimeDataCollector class declaration
+ *
+ * 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 __REAL_TIME_DATA_COLLECTOR_H__
+#define __REAL_TIME_DATA_COLLECTOR_H__
+
+#include <stdio.h>
+#include <unistd.h>
+#include <sys/types.h>
+#include <sys/socket.h>
+#include <netinet/in.h>
+#include <arpa/inet.h>
+#include <fcntl.h>
+#include <netdb.h>
+#include <string>
+#include <errno.h>
+#include "FlowFileRecord.h"
+#include "core/Processor.h"
+#include "core/ProcessSession.h"
+#include "core/core.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+// RealTimeDataCollector Class
+class RealTimeDataCollector : public core::Processor {
+ public:
+  // Constructor
+  /*!
+   * Create a new processor
+   */
+  explicit RealTimeDataCollector(std::string name, uuid_t uuid = NULL)
+      : core::Processor(name, uuid) {
+    _realTimeSocket = 0;
+    _batchSocket = 0;
+    logger_ = logging::Logger::getLogger();
+    _firstInvoking = false;
+    _realTimeAccumulated = 0;
+    _batchAcccumulated = 0;
+    _queuedDataSize = 0;
+  }
+  // Destructor
+  virtual ~RealTimeDataCollector() {
+    if (_realTimeSocket)
+      close(_realTimeSocket);
+    if (_batchSocket)
+      close(_batchSocket);
+    if (_fileStream.is_open())
+      _fileStream.close();
+  }
+  // Processor Name
+  static const std::string ProcessorName;
+  // Supported Properties
+  static core::Property REALTIMESERVERNAME;
+  static core::Property REALTIMESERVERPORT;
+  static core::Property BATCHSERVERNAME;
+  static core::Property BATCHSERVERPORT;
+  static core::Property FILENAME;
+  static core::Property ITERATION;
+  static core::Property REALTIMEMSGID;
+  static core::Property BATCHMSGID;
+  static core::Property REALTIMEINTERVAL;
+  static core::Property BATCHINTERVAL;
+  static core::Property BATCHMAXBUFFERSIZE;
+  // Supported Relationships
+  static core::Relationship Success;
+  // Connect to the socket
+  int connectServer(const char *host, uint16_t port);
+  int sendData(int socket, const char *buf, int buflen);
+  void onTriggerRealTime(
+      core::ProcessContext *context,
+      core::ProcessSession *session);
+  void onTriggerBatch(core::ProcessContext *context,
+                      core::ProcessSession *session);
+
+ public:
+  // OnTrigger method, implemented by NiFi RealTimeDataCollector
+  virtual void onTrigger(
+      core::ProcessContext *context,
+      core::ProcessSession *session);
+  // Initialize, over write by NiFi RealTimeDataCollector
+  virtual void initialize(void);
+
+ protected:
+
+ private:
+  // realtime server Name
+  std::string _realTimeServerName;
+  int64_t _realTimeServerPort;
+  std::string _batchServerName;
+  int64_t _batchServerPort;
+  int64_t _realTimeInterval;
+  int64_t _batchInterval;
+  int64_t _batchMaxBufferSize;
+  // Match pattern for Real time Message ID
+  std::vector<std::string> _realTimeMsgID;
+  // Match pattern for Batch Message ID
+  std::vector<std::string> _batchMsgID;
+  // file for which the realTime collector will tail
+  std::string _fileName;
+  // Whether we need to iterate from the beginning for demo
+  bool _iteration;
+  int _realTimeSocket;
+  int _batchSocket;
+  // Logger
+  std::shared_ptr<logging::Logger> logger_;
+  // Mutex for protection
+  std::mutex mutex_;
+  // Queued data size
+  uint64_t _queuedDataSize;
+  // Queue for the batch process
+  std::queue<std::string> _queue;
+  std::thread::id _realTimeThreadId;
+  std::thread::id _batchThreadId;
+  std::atomic<bool> _firstInvoking;
+  int64_t _realTimeAccumulated;
+  int64_t _batchAcccumulated;
+  std::ifstream _fileStream;
+};
+
+} /* namespace processors */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/processors/TailFile.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/TailFile.h b/libminifi/include/processors/TailFile.h
new file mode 100644
index 0000000..5be76e4
--- /dev/null
+++ b/libminifi/include/processors/TailFile.h
@@ -0,0 +1,105 @@
+/**
+ * @file TailFile.h
+ * TailFile class declaration
+ *
+ * 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 __TAIL_FILE_H__
+#define __TAIL_FILE_H__
+
+#include "FlowFileRecord.h"
+#include "core/Processor.h"
+#include "core/ProcessSession.h"
+#include "core/core.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+// TailFile Class
+class TailFile : public core::Processor {
+ public:
+  // Constructor
+  /*!
+   * Create a new processor
+   */
+  TailFile(std::string name, uuid_t uuid = NULL)
+      : core::Processor(name, uuid) {
+    logger_ = logging::Logger::getLogger();
+    _stateRecovered = false;
+  }
+  // Destructor
+  virtual ~TailFile() {
+    storeState();
+  }
+  // Processor Name
+  static const std::string ProcessorName;
+  // Supported Properties
+  static core::Property FileName;
+  static core::Property StateFile;
+  // Supported Relationships
+  static core::Relationship Success;
+
+ public:
+  // OnTrigger method, implemented by NiFi TailFile
+  virtual void onTrigger(
+      core::ProcessContext *context,
+      core::ProcessSession *session);
+  // Initialize, over write by NiFi TailFile
+  virtual void initialize(void);
+  // recoverState
+  void recoverState();
+  // storeState
+  void storeState();
+
+ protected:
+
+ private:
+  // Logger
+  std::shared_ptr<logging::Logger> logger_;
+  std::string _fileLocation;
+  // Property Specified Tailed File Name
+  std::string _fileName;
+  // File to save state
+  std::string _stateFile;
+  // State related to the tailed file
+  std::string _currentTailFileName;
+  uint64_t _currentTailFilePosition;
+  bool _stateRecovered;
+  uint64_t _currentTailFileCreatedTime;
+  // Utils functions for parse state file
+  std::string trimLeft(const std::string& s);
+  std::string trimRight(const std::string& s);
+  void parseStateFileLine(char *buf);
+  void checkRollOver();
+
+};
+
+// Matched File Item for Roll over check
+typedef struct {
+  std::string fileName;
+  uint64_t modifiedTime;
+} TailMatchedFileItem;
+
+} /* namespace processors */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/properties/Configure.h
----------------------------------------------------------------------
diff --git a/libminifi/include/properties/Configure.h b/libminifi/include/properties/Configure.h
new file mode 100644
index 0000000..c0d9bd4
--- /dev/null
+++ b/libminifi/include/properties/Configure.h
@@ -0,0 +1,131 @@
+/**
+ * @file Configure.h
+ * Configure class declaration
+ *
+ * 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 __CONFIGURE_H__
+#define __CONFIGURE_H__
+
+#include <stdio.h>
+#include <string>
+#include <map>
+#include <stdlib.h>
+#include <errno.h>
+#include <iostream>
+#include <fstream>
+#include "core/core.h"
+#include "core/logging/Logger.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+class Configure {
+ public:
+  // Get the singleton logger instance
+  static Configure * getConfigure() {
+    if (!configure_) {
+      configure_ = new Configure();
+    }
+    return configure_;
+  }
+  // nifi.flow.configuration.file
+  static const char *nifi_flow_configuration_file;
+  static const char *nifi_administrative_yield_duration;
+  static const char *nifi_bored_yield_duration;
+  static const char *nifi_graceful_shutdown_seconds;
+  static const char *nifi_log_level;
+  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_provenance_repository_class_name;
+  static const char *nifi_server_port;
+  static const char *nifi_server_report_interval;
+  static const char *nifi_provenance_repository_max_storage_time;
+  static const char *nifi_provenance_repository_max_storage_size;
+  static const char *nifi_provenance_repository_directory_default;
+  static const char *nifi_provenance_repository_enable;
+  static const char *nifi_flowfile_repository_max_storage_time;
+  static const char *nifi_flowfile_repository_max_storage_size;
+  static const char *nifi_flowfile_repository_directory_default;
+  static const char *nifi_flowfile_repository_enable;
+  static const char *nifi_remote_input_secure;
+  static const char *nifi_security_need_ClientAuth;
+  static const char *nifi_security_client_certificate;
+  static const char *nifi_security_client_private_key;
+  static const char *nifi_security_client_pass_phrase;
+  static const char *nifi_security_client_ca_certificate;
+
+  // Clear the load config
+  void clear() {
+    std::lock_guard<std::mutex> lock(mutex_);
+    properties_.clear();
+  }
+  // Set the config value
+  void set(std::string key, std::string value) {
+    std::lock_guard<std::mutex> lock(mutex_);
+    properties_[key] = value;
+  }
+  // Check whether the config value existed
+  bool has(std::string key) {
+    std::lock_guard<std::mutex> lock(mutex_);
+    return (properties_.find(key) != properties_.end());
+  }
+  // Get the config value
+  bool get(std::string key, std::string &value);
+  // Parse one line in configure file like key=value
+  void parseConfigureFileLine(char *buf);
+  // Load Configure File
+  void loadConfigureFile(const char *fileName);
+  // Set the determined MINIFI_HOME
+  void setHome(std::string minifiHome) {
+    minifi_home_ = minifiHome;
+  }
+
+  // Get the determined MINIFI_HOME
+  std::string getHome() {
+    return minifi_home_;
+  }
+  // Parse Command Line
+  void parseCommandLine(int argc, char **argv);
+
+ private:
+  // Mutex for protection
+  std::mutex mutex_;
+  // Logger
+  std::shared_ptr<logging::Logger> logger_;
+  // Home location for this executable
+  std::string minifi_home_;
+
+  Configure() {
+    logger_ = logging::Logger::getLogger();
+  }
+  virtual ~Configure() {
+
+  }
+  static Configure *configure_;
+
+ protected:
+  std::map<std::string, std::string> properties_;
+};
+
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+#endif


[07/16] nifi-minifi-cpp git commit: MINIFI-217: Updates namespaces and removes use of raw pointers for user facing API.

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/RealTimeDataCollector.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/RealTimeDataCollector.cpp b/libminifi/src/RealTimeDataCollector.cpp
deleted file mode 100644
index 7dd6469..0000000
--- a/libminifi/src/RealTimeDataCollector.cpp
+++ /dev/null
@@ -1,481 +0,0 @@
-/**
- * @file RealTimeDataCollector.cpp
- * RealTimeDataCollector class implementation
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#include <vector>
-#include <queue>
-#include <map>
-#include <set>
-#include <sys/time.h>
-#include <time.h>
-#include <chrono>
-#include <thread>
-#include <random>
-#include <netinet/tcp.h>
-
-#include "utils/StringUtils.h"
-#include "RealTimeDataCollector.h"
-#include "ProcessContext.h"
-#include "ProcessSession.h"
-
-const std::string RealTimeDataCollector::ProcessorName("RealTimeDataCollector");
-Property RealTimeDataCollector::FILENAME("File Name", "File Name for the real time processor to process", "data.osp");
-Property RealTimeDataCollector::REALTIMESERVERNAME("Real Time Server Name", "Real Time Server Name", "localhost");
-Property RealTimeDataCollector::REALTIMESERVERPORT("Real Time Server Port", "Real Time Server Port", "10000");
-Property RealTimeDataCollector::BATCHSERVERNAME("Batch Server Name", "Batch Server Name", "localhost");
-Property RealTimeDataCollector::BATCHSERVERPORT("Batch Server Port", "Batch Server Port", "10001");
-Property RealTimeDataCollector::ITERATION("Iteration",
-		"If true, sample osp file will be iterated", "true");
-Property RealTimeDataCollector::REALTIMEMSGID("Real Time Message ID", "Real Time Message ID", "41");
-Property RealTimeDataCollector::BATCHMSGID("Batch Message ID", "Batch Message ID", "172, 30, 48");
-Property RealTimeDataCollector::REALTIMEINTERVAL("Real Time Interval", "Real Time Data Collection Interval in msec", "10 ms");
-Property RealTimeDataCollector::BATCHINTERVAL("Batch Time Interval", "Batch Processing Interval in msec", "100 ms");
-Property RealTimeDataCollector::BATCHMAXBUFFERSIZE("Batch Max Buffer Size", "Batch Buffer Maximum size in bytes", "262144");
-Relationship RealTimeDataCollector::Success("success", "success operational on the flow record");
-
-void RealTimeDataCollector::initialize()
-{
-	//! Set the supported properties
-	std::set<Property> properties;
-	properties.insert(FILENAME);
-	properties.insert(REALTIMESERVERNAME);
-	properties.insert(REALTIMESERVERPORT);
-	properties.insert(BATCHSERVERNAME);
-	properties.insert(BATCHSERVERPORT);
-	properties.insert(ITERATION);
-	properties.insert(REALTIMEMSGID);
-	properties.insert(BATCHMSGID);
-	properties.insert(REALTIMEINTERVAL);
-	properties.insert(BATCHINTERVAL);
-	properties.insert(BATCHMAXBUFFERSIZE);
-
-	setSupportedProperties(properties);
-	//! Set the supported relationships
-	std::set<Relationship> relationships;
-	relationships.insert(Success);
-	setSupportedRelationships(relationships);
-
-}
-
-int RealTimeDataCollector::connectServer(const char *host, uint16_t port)
-{
-	in_addr_t addr;
-	int sock = 0;
-	struct hostent *h;
-#ifdef __MACH__
-	h = gethostbyname(host);
-#else
-	char buf[1024];
-	struct hostent he;
-	int hh_errno;
-	gethostbyname_r(host, &he, buf, sizeof(buf), &h, &hh_errno);
-#endif
-	memcpy((char *) &addr, h->h_addr_list[0], h->h_length);
-	sock = socket(AF_INET, SOCK_STREAM, 0);
-	if (sock < 0)
-	{
-		logger_->log_error("Could not create socket to hostName %s", host);
-		return 0;
-	}
-
-#ifndef __MACH__
-	int opt = 1;
-	bool nagle_off = true;
-
-	if (nagle_off)
-	{
-		if (setsockopt(sock, SOL_TCP, TCP_NODELAY, (void *)&opt, sizeof(opt)) < 0)
-		{
-			logger_->log_error("setsockopt() TCP_NODELAY failed");
-			close(sock);
-			return 0;
-		}
-		if (setsockopt(sock, SOL_SOCKET, SO_REUSEADDR,
-				(char *)&opt, sizeof(opt)) < 0)
-		{
-			logger_->log_error("setsockopt() SO_REUSEADDR failed");
-			close(sock);
-			return 0;
-		}
-	}
-
-	int sndsize = 256*1024;
-	if (setsockopt(sock, SOL_SOCKET, SO_SNDBUF, (char *)&sndsize, (int)sizeof(sndsize)) < 0)
-	{
-		logger_->log_error("setsockopt() SO_SNDBUF failed");
-		close(sock);
-		return 0;
-	}
-#endif
-
-	struct sockaddr_in sa;
-	socklen_t socklen;
-	int status;
-
-	//TODO bind socket to the interface
-	memset(&sa, 0, sizeof(sa));
-	sa.sin_family = AF_INET;
-	sa.sin_addr.s_addr = htonl(INADDR_ANY);
-	sa.sin_port = htons(0);
-	socklen = sizeof(sa);
-	if (bind(sock, (struct sockaddr *)&sa, socklen) < 0)
-	{
-		logger_->log_error("socket bind failed");
-		close(sock);
-		return 0;
-	}
-
-	memset(&sa, 0, sizeof(sa));
-	sa.sin_family = AF_INET;
-	sa.sin_addr.s_addr = addr;
-	sa.sin_port = htons(port);
-	socklen = sizeof(sa);
-
-	status = connect(sock, (struct sockaddr *)&sa, socklen);
-
-	if (status < 0)
-	{
-		logger_->log_error("socket connect failed to %s %d", host, port);
-		close(sock);
-		return 0;
-	}
-
-	logger_->log_info("socket %d connect to server %s port %d success", sock, host, port);
-
-	return sock;
-}
-
-int RealTimeDataCollector::sendData(int socket, const char *buf, int buflen)
-{
-	int ret = 0, bytes = 0;
-
-	while (bytes < buflen)
-	{
-		ret = send(socket, buf+bytes, buflen-bytes, 0);
-		//check for errors
-		if (ret == -1)
-		{
-			return ret;
-		}
-		bytes+=ret;
-	}
-
-	if (ret)
-		logger_->log_debug("Send data size %d over socket %d", buflen, socket);
-
-	return ret;
-}
-
-void RealTimeDataCollector::onTriggerRealTime(ProcessContext *context, ProcessSession *session)
-{
-	if (_realTimeAccumulated >= this->_realTimeInterval)
-	{
-		std::string value;
-		if (this->getProperty(REALTIMEMSGID.getName(), value))
-		{
-			this->_realTimeMsgID.clear();
-			this->logger_->log_info("Real Time Msg IDs %s", value.c_str());
-			std::stringstream lineStream(value);
-			std::string cell;
-
-			while(std::getline(lineStream, cell, ','))
-		    {
-		        this->_realTimeMsgID.push_back(cell);
-		        // this->logger_->log_debug("Real Time Msg ID %s", cell.c_str());
-		    }
-		}
-		if (this->getProperty(BATCHMSGID.getName(), value))
-		{
-			this->_batchMsgID.clear();
-			this->logger_->log_info("Batch Msg IDs %s", value.c_str());
-			std::stringstream lineStream(value);
-			std::string cell;
-
-			while(std::getline(lineStream, cell, ','))
-		    {
-				cell = StringUtils::trim(cell);
-		        this->_batchMsgID.push_back(cell);
-		    }
-		}
-		// Open the file
-		if (!this->_fileStream.is_open())
-		{
-			_fileStream.open(this->_fileName.c_str(), std::ifstream::in);
-			if (this->_fileStream.is_open())
-				logger_->log_debug("open %s", _fileName.c_str());
-		}
-		if (!_fileStream.good())
-		{
-			logger_->log_error("load data file failed %s", _fileName.c_str());
-			return;
-		}
-		if (this->_fileStream.is_open())
-		{
-			std::string line;
-
-			while (std::getline(_fileStream, line))
-			{
-				line += "\n";
-				std::stringstream lineStream(line);
-				std::string cell;
-				if (std::getline(lineStream, cell, ','))
-				{
-					cell = StringUtils::trim(cell);
-					// Check whether it match to the batch traffic
-					for (std::vector<std::string>::iterator it = _batchMsgID.begin(); it != _batchMsgID.end(); ++it)
-					{
-						if (cell == *it)
-						{
-							// push the batch data to the queue
-							std::lock_guard<std::mutex> lock(_mtx);
-							while ((_queuedDataSize + line.size()) > _batchMaxBufferSize)
-							{
-								std::string item = _queue.front();
-								_queuedDataSize -= item.size();
-								logger_->log_debug("Pop item size %d from batch queue, queue buffer size %d", item.size(), _queuedDataSize);
-								_queue.pop();
-							}
-							_queue.push(line);
-							_queuedDataSize += line.size();
-							logger_->log_debug("Push batch msg ID %s into batch queue, queue buffer size %d", cell.c_str(), _queuedDataSize);
-						}
-					}
-					bool findRealTime = false;
-					// Check whether it match to the real time traffic
-					for (std::vector<std::string>::iterator it = _realTimeMsgID.begin(); it != _realTimeMsgID.end(); ++it)
-					{
-						if (cell == *it)
-						{
-							int status = 0;
-							if (this->_realTimeSocket <= 0)
-							{
-								// Connect the LTE socket
-								uint16_t port = _realTimeServerPort;
-								this->_realTimeSocket = connectServer(_realTimeServerName.c_str(), port);
-							}
-							if (this->_realTimeSocket)
-							{
-								// try to send the data
-								status = sendData(_realTimeSocket, line.data(), line.size());
-								if (status < 0)
-								{
-									close(_realTimeSocket);
-									_realTimeSocket = 0;
-								}
-							}
-							if (this->_realTimeSocket <= 0 || status < 0)
-							{
-								// push the batch data to the queue
-								std::lock_guard<std::mutex> lock(_mtx);
-								while ((_queuedDataSize + line.size()) > _batchMaxBufferSize)
-								{
-									std::string item = _queue.front();
-									_queuedDataSize -= item.size();
-									logger_->log_debug("Pop item size %d from batch queue, queue buffer size %d", item.size(), _queuedDataSize);
-									_queue.pop();
-								}
-								_queue.push(line);
-								_queuedDataSize += line.size();
-								logger_->log_debug("Push real time msg ID %s into batch queue, queue buffer size %d", cell.c_str(), _queuedDataSize);
-							}
-							// find real time
-							findRealTime = true;
-						} // cell
-					} // for real time pattern
-					if (findRealTime)
-						// we break the while once we find the first real time
-						break;
-				}  // if get line
-			} // while
-			if (_fileStream.eof())
-			{
-				_fileStream.close();
-			}
-		} // if open
-		_realTimeAccumulated = 0;
-	}
-	_realTimeAccumulated += context->getProcessor()->getSchedulingPeriodNano();
-}
-
-void RealTimeDataCollector::onTriggerBatch(ProcessContext *context, ProcessSession *session)
-{
-	if (_batchAcccumulated >= this->_batchInterval)
-	{
-		// logger_->log_info("onTriggerBatch");
-		// dequeue the batch and send over WIFI
-		int status = 0;
-		if (this->_batchSocket <= 0)
-		{
-			// Connect the WIFI socket
-			uint16_t port = _batchServerPort;
-			this->_batchSocket = connectServer(_batchServerName.c_str(), port);
-		}
-		if (this->_batchSocket)
-		{
-			std::lock_guard<std::mutex> lock(_mtx);
-
-			while (!_queue.empty())
-			{
-				std::string line = _queue.front();
-				status = sendData(_batchSocket, line.data(), line.size());
-				_queue.pop();
-				_queuedDataSize -= line.size();
-				if (status < 0)
-				{
-					close(_batchSocket);
-					_batchSocket = 0;
-					break;
-				}
-			}
-		}
-		_batchAcccumulated = 0;
-	}
-	_batchAcccumulated += context->getProcessor()->getSchedulingPeriodNano();
-}
-
-void RealTimeDataCollector::onTrigger(ProcessContext *context, ProcessSession *session)
-{
-	std::thread::id id = std::this_thread::get_id();
-
-	if (id == _realTimeThreadId)
-		return onTriggerRealTime(context, session);
-	else if (id == _batchThreadId)
-		return onTriggerBatch(context, session);
-	else
-	{
-		std::lock_guard<std::mutex> lock(_mtx);
-		if (!this->_firstInvoking)
-		{
-			this->_fileName = "data.osp";
-			std::string value;
-			if (this->getProperty(FILENAME.getName(), value))
-			{
-				this->_fileName = value;
-				this->logger_->log_info("Data Collector File Name %s", _fileName.c_str());
-			}
-			this->_realTimeServerName = "localhost";
-			if (this->getProperty(REALTIMESERVERNAME.getName(), value))
-			{
-				this->_realTimeServerName = value;
-				this->logger_->log_info("Real Time Server Name %s", this->_realTimeServerName.c_str());
-			}
-			this->_realTimeServerPort = 10000;
-			if (this->getProperty(REALTIMESERVERPORT.getName(), value))
-			{
-				Property::StringToInt(value, _realTimeServerPort);
-				this->logger_->log_info("Real Time Server Port %d", _realTimeServerPort);
-			}
-			if (this->getProperty(BATCHSERVERNAME.getName(), value))
-			{
-				this->_batchServerName = value;
-				this->logger_->log_info("Batch Server Name %s", this->_batchServerName.c_str());
-			}
-			this->_batchServerPort = 10001;
-			if (this->getProperty(BATCHSERVERPORT.getName(), value))
-			{
-				Property::StringToInt(value, _batchServerPort);
-				this->logger_->log_info("Batch Server Port %d", _batchServerPort);
-			}
-			if (this->getProperty(ITERATION.getName(), value))
-			{
-				StringUtils::StringToBool(value, this->_iteration);
-				logger_->log_info("Iteration %d", _iteration);
-			}
-			this->_realTimeInterval = 10000000; //10 msec
-			if (this->getProperty(REALTIMEINTERVAL.getName(), value))
-			{
-				TimeUnit unit;
-				if (Property::StringToTime(value, _realTimeInterval, unit) &&
-								Property::ConvertTimeUnitToNS(_realTimeInterval, unit, _realTimeInterval))
-				{
-					logger_->log_info("Real Time Interval: [%d] ns", _realTimeInterval);
-				}
-			}
-			this->_batchInterval = 100000000; //100 msec
-			if (this->getProperty(BATCHINTERVAL.getName(), value))
-			{
-				TimeUnit unit;
-				if (Property::StringToTime(value, _batchInterval, unit) &&
-								Property::ConvertTimeUnitToNS(_batchInterval, unit, _batchInterval))
-				{
-					logger_->log_info("Batch Time Interval: [%d] ns", _batchInterval);
-				}
-			}
-			this->_batchMaxBufferSize = 256*1024;
-			if (this->getProperty(BATCHMAXBUFFERSIZE.getName(), value))
-			{
-				Property::StringToInt(value, _batchMaxBufferSize);
-				this->logger_->log_info("Batch Max Buffer Size %d", _batchMaxBufferSize);
-			}
-			if (this->getProperty(REALTIMEMSGID.getName(), value))
-			{
-				this->logger_->log_info("Real Time Msg IDs %s", value.c_str());
-				std::stringstream lineStream(value);
-				std::string cell;
-
-				while(std::getline(lineStream, cell, ','))
-			    {
-			        this->_realTimeMsgID.push_back(cell);
-			        this->logger_->log_info("Real Time Msg ID %s", cell.c_str());
-			    }
-			}
-			if (this->getProperty(BATCHMSGID.getName(), value))
-			{
-				this->logger_->log_info("Batch Msg IDs %s", value.c_str());
-				std::stringstream lineStream(value);
-				std::string cell;
-
-				while(std::getline(lineStream, cell, ','))
-			    {
-					cell = StringUtils::trim(cell);
-			        this->_batchMsgID.push_back(cell);
-			        this->logger_->log_info("Batch Msg ID %s", cell.c_str());
-			    }
-			}
-			// Connect the LTE socket
-			uint16_t port = _realTimeServerPort;
-
-			this->_realTimeSocket = connectServer(_realTimeServerName.c_str(), port);
-
-			// Connect the WIFI socket
-			port = _batchServerPort;
-
-			this->_batchSocket = connectServer(_batchServerName.c_str(), port);
-
-			// Open the file
-			_fileStream.open(this->_fileName.c_str(), std::ifstream::in);
-			if (!_fileStream.good())
-			{
-				logger_->log_error("load data file failed %s", _fileName.c_str());
-				return;
-			}
-			else
-			{
-				logger_->log_debug("open %s", _fileName.c_str());
-			}
-			_realTimeThreadId = id;
-			this->_firstInvoking = true;
-		}
-		else
-		{
-			if (id != _realTimeThreadId)
-				_batchThreadId = id;
-			this->_firstInvoking = false;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/RemoteProcessorGroupPort.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/RemoteProcessorGroupPort.cpp b/libminifi/src/RemoteProcessorGroupPort.cpp
index dd1d035..9790256 100644
--- a/libminifi/src/RemoteProcessorGroupPort.cpp
+++ b/libminifi/src/RemoteProcessorGroupPort.cpp
@@ -29,93 +29,116 @@
 
 #include "RemoteProcessorGroupPort.h"
 
+#include "../include/io/StreamFactory.h"
 #include "io/ClientSocket.h"
-#include "io/SocketFactory.h"
-
 #include "utils/TimeUtil.h"
-#include "ProcessContext.h"
-#include "ProcessSession.h"
-
-const std::string RemoteProcessorGroupPort::ProcessorName("RemoteProcessorGroupPort");
-Property RemoteProcessorGroupPort::hostName("Host Name", "Remote Host Name.", "localhost");
-Property RemoteProcessorGroupPort::port("Port", "Remote Port", "9999");
-Relationship RemoteProcessorGroupPort::relation;
-
-void RemoteProcessorGroupPort::initialize()
-{
-
-	//! Set the supported properties
-	std::set<Property> properties;
-	properties.insert(hostName);
-	properties.insert(port);
-	setSupportedProperties(properties);
-	//! Set the supported relationships
-	std::set<Relationship> relationships;
-	relationships.insert(relation);
-	setSupportedRelationships(relationships);
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+const std::string RemoteProcessorGroupPort::ProcessorName(
+    "RemoteProcessorGroupPort");
+core::Property RemoteProcessorGroupPort::hostName("Host Name",
+                                                  "Remote Host Name.",
+                                                  "localhost");
+core::Property RemoteProcessorGroupPort::port("Port", "Remote Port", "9999");
+core::Relationship RemoteProcessorGroupPort::relation;
+
+
+std::unique_ptr<Site2SiteClientProtocol> RemoteProcessorGroupPort::getNextProtocol() {
+  std::lock_guard<std::mutex> protocol_lock_(protocol_mutex_);
+  if (available_protocols_.empty())
+    return nullptr;
+
+  std::unique_ptr<Site2SiteClientProtocol> return_pointer = std::move(available_protocols_.top());
+  available_protocols_.pop();
+  return std::move(return_pointer);
+}
+
+void RemoteProcessorGroupPort::returnProtocol(
+    std::unique_ptr<Site2SiteClientProtocol> return_protocol) {
+  std::lock_guard<std::mutex> protocol_lock_(protocol_mutex_);
+  available_protocols_.push(std::move(return_protocol));
+}
+
+void RemoteProcessorGroupPort::initialize() {
+
+  // Set the supported properties
+  std::set<core::Property> properties;
+  properties.insert(hostName);
+  properties.insert(port);
+  setSupportedProperties(properties);
+  // Set the supported relationships
+  std::set<core::Relationship> relationships;
+  relationships.insert(relation);
+  setSupportedRelationships(relationships);
 
 }
 
-void RemoteProcessorGroupPort::onTrigger(ProcessContext *context, ProcessSession *session)
-{
-	std::string value;
-
-	if (!transmitting_)
-		return;
-	
-	std::string host = peer_.getHostName();
-	uint16_t sport = peer_.getPort();
-	int64_t lvalue;
-	
-	if (context->getProperty(hostName.getName(), value))
-	{
-		host = value;
-	}
-	if (context->getProperty(port.getName(), value) && Property::StringToInt(value, lvalue))
-	{
-		sport = (uint16_t) lvalue;
-	}
-	
-	if (host != peer_.getHostName() || sport != peer_.getPort())
-	
-	{
-	  
-	      std::unique_ptr<DataStream> str = std::unique_ptr<DataStream>(SocketFactory::getInstance()->createSocket(host,sport));
-	      peer_ = std::move(Site2SitePeer (std::move(str), host, sport));
-	      protocol_->setPeer(&peer_);
-	  
-	}
-		
-	
-	
-	bool needReset = false;
-
-	
-	if (host != peer_.getHostName())
-	{
-		peer_.setHostName(host);
-		needReset= true;
-	}
-	if (sport != peer_.getPort())
-	{
-		peer_.setPort(sport);
-		needReset = true;
-	}
-	if (needReset)
-		protocol_->tearDown();
-
-	if (!protocol_->bootstrap())
-	{
-		// bootstrap the client protocol if needeed
-		context->yield();
-		logger_->log_error("Site2Site bootstrap failed yield period %d peer ", context->getProcessor()->getYieldPeriodMsec());
-		return;
-	}
-
-	if (direction_ == RECEIVE)
-		protocol_->receiveFlowFiles(context, session);
-	else
-		protocol_->transferFlowFiles(context, session);
-
-	return;
+void RemoteProcessorGroupPort::onTrigger(core::ProcessContext *context,
+                                         core::ProcessSession *session) {
+  std::string value;
+
+  if (!transmitting_)
+    return;
+
+  std::unique_ptr<Site2SiteClientProtocol> protocol_ = getNextProtocol();
+
+  // Peer Connection
+  if (protocol_ == nullptr) {
+
+    protocol_ = std::unique_ptr<Site2SiteClientProtocol>(
+        new Site2SiteClientProtocol(0));
+    protocol_->setPortId(protocol_uuid_);
+    protocol_->setTimeOut(timeout_);
+
+    std::string host = "";
+    uint16_t sport = 0;
+    int64_t lvalue;
+
+    if (context->getProperty(hostName.getName(), value)) {
+      host = value;
+    }
+    if (context->getProperty(port.getName(), value)
+        && core::Property::StringToInt(value, lvalue)) {
+      sport = (uint16_t) lvalue;
+    }
+    std::unique_ptr<org::apache::nifi::minifi::io::DataStream> str =
+        std::unique_ptr<org::apache::nifi::minifi::io::DataStream>(
+            org::apache::nifi::minifi::io::StreamFactory::getInstance()
+                ->createSocket(host, sport));
+
+    std::unique_ptr<Site2SitePeer> peer_ = std::unique_ptr<Site2SitePeer>(
+        new Site2SitePeer(std::move(str), host, sport));
+
+    protocol_->setPeer(std::move(peer_));
+  }
+
+  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);
+
+  returnProtocol(std::move(protocol_));
+
+  return;
 }
+
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/Repository.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/Repository.cpp b/libminifi/src/Repository.cpp
deleted file mode 100644
index 1a3c7b0..0000000
--- a/libminifi/src/Repository.cpp
+++ /dev/null
@@ -1,140 +0,0 @@
-/**
- * @file Repository.cpp
- * Repository implemenatation 
- *
- * 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 <cstdint>
-#include <vector>
-#include <arpa/inet.h>
-#include "io/DataStream.h"
-#include "io/Serializable.h"
-#include "Relationship.h"
-#include "Logger.h"
-#include "FlowController.h"
-#include "Repository.h"
-#include "Provenance.h"
-#include "FlowFileRepository.h"
-
-const char *Repository::RepositoryTypeStr[MAX_REPO_TYPE] = {"Provenace Repository", "FlowFile Repository"};
-uint64_t Repository::_repoSize[MAX_REPO_TYPE] = {0, 0}; 
-
-void Repository::start() {
-	if (!_enable)
-		return;
-	if (this->_purgePeriod <= 0)
-		return;
-	if (_running)
-		return;
-	_running = true;
-	logger_->log_info("%s Repository Monitor Thread Start", RepositoryTypeStr[_type]);
-	_thread = new std::thread(run, this);
-	_thread->detach();
-}
-
-void Repository::stop() {
-	if (!_running)
-		return;
-	_running = false;
-	logger_->log_info("%s Repository Monitor Thread Stop", RepositoryTypeStr[_type]);
-}
-
-void Repository::run(Repository *repo) {
-#ifdef LEVELDB_SUPPORT
-	// threshold for purge
-	uint64_t purgeThreshold = repo->_maxPartitionBytes * 3 / 4;
-	while (repo->_running) {
-		std::this_thread::sleep_for(
-				std::chrono::milliseconds(repo->_purgePeriod));
-		uint64_t curTime = getTimeMillis();
-		uint64_t size = repo->repoSize();
-		if (size >= purgeThreshold) {
-			std::vector<std::string> purgeList;
-			leveldb::Iterator* it = repo->_db->NewIterator(
-					leveldb::ReadOptions());
-			if (repo->_type == PROVENANCE)
-			{
-				for (it->SeekToFirst(); it->Valid(); it->Next()) {
-					ProvenanceEventRecord eventRead;
-					std::string key = it->key().ToString();
-					if (eventRead.DeSerialize((uint8_t *) it->value().data(),
-						(int) it->value().size())) {
-						if ((curTime - eventRead.getEventTime())
-							> repo->_maxPartitionMillis)
-							purgeList.push_back(key);
-					} else {
-						repo->logger_->log_debug(
-							"NiFi %s retrieve event %s fail",
-							RepositoryTypeStr[repo->_type],
-							key.c_str());
-						purgeList.push_back(key);
-					}
-				}
-			}
-			if (repo->_type == FLOWFILE)
-			{
-				for (it->SeekToFirst(); it->Valid(); it->Next()) {
-					FlowFileEventRecord eventRead;
-					std::string key = it->key().ToString();
-					if (eventRead.DeSerialize((uint8_t *) it->value().data(),
-						(int) it->value().size())) {
-						if ((curTime - eventRead.getEventTime())
-							> repo->_maxPartitionMillis)
-							purgeList.push_back(key);
-					} else {
-						repo->logger_->log_debug(
-							"NiFi %s retrieve event %s fail",
-							RepositoryTypeStr[repo->_type],
-							key.c_str());
-						purgeList.push_back(key);
-					}
-				}
-			}
-			delete it;
-			for (auto eventId : purgeList)
-			{
-				repo->logger_->log_info("Repository Repo %s Purge %s",
-						RepositoryTypeStr[repo->_type],
-						eventId.c_str());
-				repo->Delete(eventId);
-			}
-		}
-		if (size > repo->_maxPartitionBytes)
-			repo->_repoFull = true;
-		else
-			repo->_repoFull = false;
-	}
-#endif
-	return;
-}
-
-//! repoSize
-uint64_t Repository::repoSize()
-{
-	_repoSize[_type] = 0;
-	if (_type == PROVENANCE)
-        {
-		if (ftw(_directory.c_str(), repoSumProvenance, 1) != 0)
-			_repoSize[_type] = 0;
-	}
-	if (_type == FLOWFILE)
-        {
-		if (ftw(_directory.c_str(), repoSumFlowFile, 1) != 0)
-			_repoSize[_type] = 0;
-	}
-	return _repoSize[_type];
-}
-

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/ResourceClaim.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/ResourceClaim.cpp b/libminifi/src/ResourceClaim.cpp
index be52b49..826ca1d 100644
--- a/libminifi/src/ResourceClaim.cpp
+++ b/libminifi/src/ResourceClaim.cpp
@@ -23,27 +23,36 @@
 
 #include "ResourceClaim.h"
 
-std::atomic<uint64_t> ResourceClaim::_localResourceClaimNumber(0);
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
 
+std::atomic<uint64_t> ResourceClaim::_localResourceClaimNumber(0);
 
-std::string ResourceClaim::default_directory_path=DEFAULT_CONTENT_DIRECTORY;
+std::string ResourceClaim::default_directory_path = DEFAULT_CONTENT_DIRECTORY;
 
 ResourceClaim::ResourceClaim(const std::string contentDirectory)
-: _id(_localResourceClaimNumber.load()),
-  _flowFileRecordOwnedCount(0)
-{
-  
-	char uuidStr[37];
-
-	// Generate the global UUID for the resource claim
-	uuid_generate(_uuid);
-	// Increase the local ID for the resource claim
-	++_localResourceClaimNumber;
-	uuid_unparse_lower(_uuid, uuidStr);
-	// Create the full content path for the content
-	_contentFullPath = contentDirectory + "/" + uuidStr;
-
-	configure_ = Configure::getConfigure();
-	logger_ = Logger::getLogger();
-	logger_->log_debug("Resource Claim created %s", uuidStr);
+    : _id(_localResourceClaimNumber.load()),
+      _flowFileRecordOwnedCount(0) {
+
+  char uuidStr[37];
+
+  // Generate the global UUID for the resource claim
+  uuid_generate(_uuid);
+  // Increase the local ID for the resource claim
+  ++_localResourceClaimNumber;
+  uuid_unparse_lower(_uuid, uuidStr);
+  // Create the full content path for the content
+  _contentFullPath = contentDirectory + "/" + uuidStr;
+
+  configure_ = Configure::getConfigure();
+
+  logger_ = logging::Logger::getLogger();
+  logger_->log_debug("Resource Claim created %s", _contentFullPath.c_str());
 }
+
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/SchedulingAgent.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/SchedulingAgent.cpp b/libminifi/src/SchedulingAgent.cpp
index 984abdc..8cb88e0 100644
--- a/libminifi/src/SchedulingAgent.cpp
+++ b/libminifi/src/SchedulingAgent.cpp
@@ -22,64 +22,71 @@
 #include <iostream>
 #include "Exception.h"
 #include "SchedulingAgent.h"
+#include "core/Processor.h"
 
-bool SchedulingAgent::hasWorkToDo(Processor *processor)
-{
-	// Whether it has work to do
-	if (processor->getTriggerWhenEmpty() || !processor->hasIncomingConnections() ||
-			processor->flowFilesQueued())
-		return true;
-	else
-		return false;
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+bool SchedulingAgent::hasWorkToDo(
+    std::shared_ptr<core::Processor> processor) {
+  // Whether it has work to do
+  if (processor->getTriggerWhenEmpty() || !processor->hasIncomingConnections()
+      || processor->flowFilesQueued())
+    return true;
+  else
+    return false;
 }
 
-bool SchedulingAgent::hasTooMuchOutGoing(Processor *processor)
-{
-	return processor->flowFilesOutGoingFull();
+bool SchedulingAgent::hasTooMuchOutGoing(
+    std::shared_ptr<core::Processor> processor) {
+  return processor->flowFilesOutGoingFull();
 }
 
-bool SchedulingAgent::onTrigger(Processor *processor, ProcessContext *processContext, ProcessSessionFactory *sessionFactory)
-{
-	if (processor->isYield())
-		return false;
+bool SchedulingAgent::onTrigger(
+    std::shared_ptr<core::Processor> processor,
+    core::ProcessContext *processContext,
+    core::ProcessSessionFactory *sessionFactory) {
+  if (processor->isYield())
+    return false;
+
+  // No need to yield, reset yield expiration to 0
+  processor->clearYield();
 
-	// No need to yield, reset yield expiration to 0
-	processor->clearYield();
+  if (!hasWorkToDo(processor))
+    // No work to do, yield
+    return true;
 
-	if (!hasWorkToDo(processor))
-		// No work to do, yield
-		return true;
+  if (hasTooMuchOutGoing(processor))
+    // need to apply backpressure
+    return true;
 
-	if(hasTooMuchOutGoing(processor))
-		// need to apply backpressure
-		return true;
+  //TODO runDuration
 
-	//TODO runDuration
+  processor->incrementActiveTasks();
+  try {
+    processor->onTrigger(processContext, sessionFactory);
+    processor->decrementActiveTask();
+  } catch (Exception &exception) {
+    // Normal exception
+    logger_->log_debug("Caught Exception %s", exception.what());
+    processor->decrementActiveTask();
+  } catch (std::exception &exception) {
+    logger_->log_debug("Caught Exception %s", exception.what());
+    processor->yield(_administrativeYieldDuration);
+    processor->decrementActiveTask();
+  } catch (...) {
+    logger_->log_debug("Caught Exception during SchedulingAgent::onTrigger");
+    processor->yield(_administrativeYieldDuration);
+    processor->decrementActiveTask();
+  }
+
+  return false;
+}
 
-	processor->incrementActiveTasks();
-	try
-	{
-		processor->onTrigger(processContext, sessionFactory);
-		processor->decrementActiveTask();
-	}
-	catch (Exception &exception)
-	{
-		// Normal exception
-		logger_->log_debug("Caught Exception %s", exception.what());
-		processor->decrementActiveTask();
-	}
-	catch (std::exception &exception)
-	{
-		logger_->log_debug("Caught Exception %s", exception.what());
-		processor->yield(_administrativeYieldDuration);
-		processor->decrementActiveTask();
-	}
-	catch (...)
-	{
-		logger_->log_debug("Caught Exception during SchedulingAgent::onTrigger");
-		processor->yield(_administrativeYieldDuration);
-		processor->decrementActiveTask();
-	}
 
-	return false;
-}
\ No newline at end of file
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */


[06/16] nifi-minifi-cpp git commit: MINIFI-217: Updates namespaces and removes use of raw pointers for user facing API.

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/Site2SiteClientProtocol.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/Site2SiteClientProtocol.cpp b/libminifi/src/Site2SiteClientProtocol.cpp
index bd4de97..e0265bb 100644
--- a/libminifi/src/Site2SiteClientProtocol.cpp
+++ b/libminifi/src/Site2SiteClientProtocol.cpp
@@ -29,1308 +29,1219 @@
 #include "Site2SitePeer.h"
 #include "Site2SiteClientProtocol.h"
 
-bool Site2SiteClientProtocol::establish()
-{
-	if (_peerState != IDLE)
-	{
-		logger_->log_error("Site2Site peer state is not idle while try to establish");
-		return false;
-	}
-
-	bool ret = peer_->Open();
-
-	if (!ret)
-	{
-		logger_->log_error("Site2Site peer socket open failed");
-		return false;
-	}
-
-	// Negotiate the version
-	ret = initiateResourceNegotiation();
-
-	if (!ret)
-	{
-		logger_->log_error("Site2Site Protocol Version Negotiation failed");
-		/*
-		peer_->yield();
-		tearDown(); */
-		return false;
-	}
-
-	logger_->log_info("Site2Site socket established");
-	_peerState = ESTABLISHED;
-
-	return true;
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+bool Site2SiteClientProtocol::establish() {
+  if (_peerState != IDLE) {
+    logger_->log_error(
+        "Site2Site peer state is not idle while try to establish");
+    return false;
+  }
+
+  bool ret = peer_->Open();
+
+  if (!ret) {
+    logger_->log_error("Site2Site peer socket open failed");
+    return false;
+  }
+
+  // Negotiate the version
+  ret = initiateResourceNegotiation();
+
+  if (!ret) {
+    logger_->log_error("Site2Site Protocol Version Negotiation failed");
+    /*
+     peer_->yield();
+     tearDown(); */
+    return false;
+  }
+
+  logger_->log_info("Site2Site socket established");
+  _peerState = ESTABLISHED;
+
+  return true;
 }
 
-bool Site2SiteClientProtocol::initiateResourceNegotiation()
-{
-	// Negotiate the version
-	if (_peerState != IDLE)
-	{
-		logger_->log_error("Site2Site peer state is not idle while initiateResourceNegotiation");
-		return false;
-	}
-
-	logger_->log_info("Negotiate protocol version with destination port %s current version %d", _portIdStr.c_str(), _currentVersion);
-
-	int ret = peer_->writeUTF(this->getResourceName());
-
-	logger_->log_info("result of writing resource name is %i",ret);
-	if (ret <= 0)
-	{
-		logger_->log_debug("result of writing resource name is %i",ret);
-		// tearDown();
-		return false;
-	}
-
-	ret = peer_->write(_currentVersion);
-
-	if (ret <= 0)
-	{
-	  logger_->log_info("result of writing version is %i",ret);
-		// tearDown();
-		return false;
-	}
-
-	uint8_t statusCode;
-	ret = peer_->read(statusCode);
-
-	if (ret <= 0)
-	{
-	  logger_->log_info("result of writing version status code  %i",ret);
-		// tearDown();
-		return false;
-	}
-      logger_->log_info("status code is %i",statusCode);
-	switch (statusCode)
-	{
-	case RESOURCE_OK:
-		logger_->log_info("Site2Site Protocol Negotiate protocol version OK");
-		return true;
-	case DIFFERENT_RESOURCE_VERSION:
-		uint32_t serverVersion;
-		ret = peer_->read(serverVersion);
-		if (ret <= 0)
-		{
-			// tearDown();
-			return false;
-		}
-		logger_->log_info("Site2Site Server Response asked for a different protocol version %d", serverVersion);
-		for (unsigned int i = (_currentVersionIndex + 1); i < sizeof(_supportedVersion)/sizeof(uint32_t); i++)
-		{
-			if (serverVersion >= _supportedVersion[i])
-			{
-				_currentVersion = _supportedVersion[i];
-				_currentVersionIndex = i;
-				return initiateResourceNegotiation();
-			}
-		}
-		ret = -1;
-		// tearDown();
-		return false;
-	case NEGOTIATED_ABORT:
-		logger_->log_info("Site2Site Negotiate protocol response ABORT");
-		ret = -1;
-		// tearDown();
-		return false;
-	default:
-		logger_->log_info("Negotiate protocol response unknown code %d", statusCode);
-		return true;
-	}
-
-	return true;
+bool Site2SiteClientProtocol::initiateResourceNegotiation() {
+  // Negotiate the version
+  if (_peerState != IDLE) {
+    logger_->log_error(
+        "Site2Site peer state is not idle while initiateResourceNegotiation");
+    return false;
+  }
+
+  logger_->log_info(
+      "Negotiate protocol version with destination port %s current version %d",
+      _portIdStr.c_str(), _currentVersion);
+
+  int ret = peer_->writeUTF(this->getResourceName());
+
+  logger_->log_info("result of writing resource name is %i", ret);
+  if (ret <= 0) {
+    logger_->log_debug("result of writing resource name is %i", ret);
+    // tearDown();
+    return false;
+  }
+
+  ret = peer_->write(_currentVersion);
+
+  if (ret <= 0) {
+    logger_->log_info("result of writing version is %i", ret);
+    // tearDown();
+    return false;
+  }
+
+  uint8_t statusCode;
+  ret = peer_->read(statusCode);
+
+  if (ret <= 0) {
+    logger_->log_info("result of writing version status code  %i", ret);
+    // tearDown();
+    return false;
+  }
+  logger_->log_info("status code is %i", statusCode);
+  switch (statusCode) {
+    case RESOURCE_OK:
+      logger_->log_info("Site2Site Protocol Negotiate protocol version OK");
+      return true;
+    case DIFFERENT_RESOURCE_VERSION:
+      uint32_t serverVersion;
+      ret = peer_->read(serverVersion);
+      if (ret <= 0) {
+        // tearDown();
+        return false;
+      }
+      logger_->log_info(
+          "Site2Site Server Response asked for a different protocol version %d",
+          serverVersion);
+      for (unsigned int i = (_currentVersionIndex + 1);
+          i < sizeof(_supportedVersion) / sizeof(uint32_t); i++) {
+        if (serverVersion >= _supportedVersion[i]) {
+          _currentVersion = _supportedVersion[i];
+          _currentVersionIndex = i;
+          return initiateResourceNegotiation();
+        }
+      }
+      ret = -1;
+      // tearDown();
+      return false;
+    case NEGOTIATED_ABORT:
+      logger_->log_info("Site2Site Negotiate protocol response ABORT");
+      ret = -1;
+      // tearDown();
+      return false;
+    default:
+      logger_->log_info("Negotiate protocol response unknown code %d",
+                        statusCode);
+      return true;
+  }
+
+  return true;
 }
 
-bool Site2SiteClientProtocol::initiateCodecResourceNegotiation()
-{
-	// Negotiate the version
-	if (_peerState != HANDSHAKED)
-	{
-		logger_->log_error("Site2Site peer state is not handshaked while initiateCodecResourceNegotiation");
-		return false;
-	}
-
-	logger_->log_info("Negotiate Codec version with destination port %s current version %d", _portIdStr.c_str(), _currentCodecVersion);
-
-	int ret = peer_->writeUTF(this->getCodecResourceName());
-
-	if (ret <= 0)
-	{
-	  logger_->log_debug("result of getCodecResourceName is %i",ret);
-		// tearDown();
-		return false;
-	}
-
-	ret = peer_->write(_currentCodecVersion);
-
-	if (ret <= 0)
-	{
-	  logger_->log_debug("result of _currentCodecVersion is %i",ret);
-		// tearDown();
-		return false;
-	}
-
-	uint8_t statusCode;
-	ret = peer_->read(statusCode);
-
-	if (ret <= 0)
-	{
-		// tearDown();
-		return false;
-	}
-
-	switch (statusCode)
-	{
-	case RESOURCE_OK:
-		logger_->log_info("Site2Site Codec Negotiate version OK");
-		return true;
-	case DIFFERENT_RESOURCE_VERSION:
-		uint32_t serverVersion;
-		ret = peer_->read(serverVersion);
-		if (ret <= 0)
-		{
-			// tearDown();
-			return false;
-		}
-		logger_->log_info("Site2Site Server Response asked for a different codec version %d", serverVersion);
-		for (unsigned int i = (_currentCodecVersionIndex + 1); i < sizeof(_supportedCodecVersion)/sizeof(uint32_t); i++)
-		{
-			if (serverVersion >= _supportedCodecVersion[i])
-			{
-				_currentCodecVersion = _supportedCodecVersion[i];
-				_currentCodecVersionIndex = i;
-				return initiateCodecResourceNegotiation();
-			}
-		}
-		ret = -1;
-		// tearDown();
-		return false;
-	case NEGOTIATED_ABORT:
-		logger_->log_info("Site2Site Codec Negotiate response ABORT");
-		ret = -1;
-		// tearDown();
-		return false;
-	default:
-		logger_->log_info("Negotiate Codec response unknown code %d", statusCode);
-		return true;
-	}
-
-	return true;
+bool Site2SiteClientProtocol::initiateCodecResourceNegotiation() {
+  // Negotiate the version
+  if (_peerState != HANDSHAKED) {
+    logger_->log_error(
+        "Site2Site peer state is not handshaked while initiateCodecResourceNegotiation");
+    return false;
+  }
+
+  logger_->log_info(
+      "Negotiate Codec version with destination port %s current version %d",
+      _portIdStr.c_str(), _currentCodecVersion);
+
+  int ret = peer_->writeUTF(this->getCodecResourceName());
+
+  if (ret <= 0) {
+    logger_->log_debug("result of getCodecResourceName is %i", ret);
+    // tearDown();
+    return false;
+  }
+
+  ret = peer_->write(_currentCodecVersion);
+
+  if (ret <= 0) {
+    logger_->log_debug("result of _currentCodecVersion is %i", ret);
+    // tearDown();
+    return false;
+  }
+
+  uint8_t statusCode;
+  ret = peer_->read(statusCode);
+
+  if (ret <= 0) {
+    // tearDown();
+    return false;
+  }
+
+  switch (statusCode) {
+    case RESOURCE_OK:
+      logger_->log_info("Site2Site Codec Negotiate version OK");
+      return true;
+    case DIFFERENT_RESOURCE_VERSION:
+      uint32_t serverVersion;
+      ret = peer_->read(serverVersion);
+      if (ret <= 0) {
+        // tearDown();
+        return false;
+      }
+      logger_->log_info(
+          "Site2Site Server Response asked for a different codec version %d",
+          serverVersion);
+      for (unsigned int i = (_currentCodecVersionIndex + 1);
+          i < sizeof(_supportedCodecVersion) / sizeof(uint32_t); i++) {
+        if (serverVersion >= _supportedCodecVersion[i]) {
+          _currentCodecVersion = _supportedCodecVersion[i];
+          _currentCodecVersionIndex = i;
+          return initiateCodecResourceNegotiation();
+        }
+      }
+      ret = -1;
+      // tearDown();
+      return false;
+    case NEGOTIATED_ABORT:
+      logger_->log_info("Site2Site Codec Negotiate response ABORT");
+      ret = -1;
+      // tearDown();
+      return false;
+    default:
+      logger_->log_info("Negotiate Codec response unknown code %d", statusCode);
+      return true;
+  }
+
+  return true;
 }
 
-bool Site2SiteClientProtocol::handShake()
-{
-	if (_peerState != ESTABLISHED)
-	{
-		logger_->log_error("Site2Site peer state is not established while handshake");
-		return false;
-	}
-	logger_->log_info("Site2Site Protocol Perform hand shake with destination port %s", _portIdStr.c_str());
-	uuid_t uuid;
-	// Generate the global UUID for the com identify
-	uuid_generate(uuid);
-	char uuidStr[37];
-	uuid_unparse_lower(uuid, uuidStr);
-	_commsIdentifier = uuidStr;
-
-	int ret = peer_->writeUTF(_commsIdentifier);
-
-	if (ret <= 0)
-	{
-		// tearDown();
-		return false;
-	}
-
-	std::map<std::string, std::string> properties;
-	properties[HandShakePropertyStr[GZIP]] = "false";
-	properties[HandShakePropertyStr[PORT_IDENTIFIER]] = _portIdStr;
-	properties[HandShakePropertyStr[REQUEST_EXPIRATION_MILLIS]] = std::to_string(this->_timeOut);
-	if (this->_currentVersion >= 5)
-	{
-		if (this->_batchCount > 0)
-			properties[HandShakePropertyStr[BATCH_COUNT]] = std::to_string(this->_batchCount);
-		if (this->_batchSize > 0)
-			properties[HandShakePropertyStr[BATCH_SIZE]] = std::to_string(this->_batchSize);
-		if (this->_batchDuration > 0)
-			properties[HandShakePropertyStr[BATCH_DURATION]] = std::to_string(this->_batchDuration);
-	}
-
-	if (_currentVersion >= 3)
-	{
-		ret = peer_->writeUTF(peer_->getURL());
-		if (ret <= 0)
-		{
-			// tearDown();
-			return false;
-		}
-	}
-
-	uint32_t size = properties.size();
-	ret = peer_->write(size);
-	if (ret <= 0)
-	{
-		// tearDown();
-		return false;
-	}
-
-	std::map<std::string, std::string>::iterator it;
-	for (it = properties.begin(); it!= properties.end(); it++)
-	{
-		ret = peer_->writeUTF(it->first);
-		if (ret <= 0)
-		{
-			// tearDown();
-			return false;
-		}
-		ret = peer_->writeUTF(it->second);
-		if (ret <= 0)
-		{
-			// tearDown();
-			return false;
-		}
-		logger_->log_info("Site2Site Protocol Send handshake properties %s %s", it->first.c_str(), it->second.c_str());
-	}
-
-	RespondCode code;
-	std::string message;
-
-	ret = this->readRespond(code, message);
-
-	if (ret <= 0)
-	{
-		// tearDown();
-		return false;
-	}
-
-	switch (code)
-	{
-	case PROPERTIES_OK:
-		logger_->log_info("Site2Site HandShake Completed");
-		_peerState = HANDSHAKED;
-		return true;
-	case PORT_NOT_IN_VALID_STATE:
+bool Site2SiteClientProtocol::handShake() {
+  if (_peerState != ESTABLISHED) {
+    logger_->log_error(
+        "Site2Site peer state is not established while handshake");
+    return false;
+  }
+  logger_->log_info(
+      "Site2Site Protocol Perform hand shake with destination port %s",
+      _portIdStr.c_str());
+  uuid_t uuid;
+  // Generate the global UUID for the com identify
+  uuid_generate(uuid);
+  char uuidStr[37];
+  uuid_unparse_lower(uuid, uuidStr);
+  _commsIdentifier = uuidStr;
+
+  int ret = peer_->writeUTF(_commsIdentifier);
+
+  if (ret <= 0) {
+    // tearDown();
+    return false;
+  }
+
+  std::map<std::string, std::string> properties;
+  properties[HandShakePropertyStr[GZIP]] = "false";
+  properties[HandShakePropertyStr[PORT_IDENTIFIER]] = _portIdStr;
+  properties[HandShakePropertyStr[REQUEST_EXPIRATION_MILLIS]] = std::to_string(
+      this->_timeOut);
+  if (this->_currentVersion >= 5) {
+    if (this->_batchCount > 0)
+      properties[HandShakePropertyStr[BATCH_COUNT]] = std::to_string(
+          this->_batchCount);
+    if (this->_batchSize > 0)
+      properties[HandShakePropertyStr[BATCH_SIZE]] = std::to_string(
+          this->_batchSize);
+    if (this->_batchDuration > 0)
+      properties[HandShakePropertyStr[BATCH_DURATION]] = std::to_string(
+          this->_batchDuration);
+  }
+
+  if (_currentVersion >= 3) {
+    ret = peer_->writeUTF(peer_->getURL());
+    if (ret <= 0) {
+      // tearDown();
+      return false;
+    }
+  }
+
+  uint32_t size = properties.size();
+  ret = peer_->write(size);
+  if (ret <= 0) {
+    // tearDown();
+    return false;
+  }
+
+  std::map<std::string, std::string>::iterator it;
+  for (it = properties.begin(); it != properties.end(); it++) {
+    ret = peer_->writeUTF(it->first);
+    if (ret <= 0) {
+      // tearDown();
+      return false;
+    }
+    ret = peer_->writeUTF(it->second);
+    if (ret <= 0) {
+      // tearDown();
+      return false;
+    }
+    logger_->log_info("Site2Site Protocol Send handshake properties %s %s",
+                      it->first.c_str(), it->second.c_str());
+  }
+
+  RespondCode code;
+  std::string message;
+
+  ret = this->readRespond(code, message);
+
+  if (ret <= 0) {
+    // tearDown();
+    return false;
+  }
+
+  switch (code) {
+    case PROPERTIES_OK:
+      logger_->log_info("Site2Site HandShake Completed");
+      _peerState = HANDSHAKED;
+      return true;
+    case PORT_NOT_IN_VALID_STATE:
     case UNKNOWN_PORT:
     case PORTS_DESTINATION_FULL:
-    	logger_->log_error("Site2Site HandShake Failed because destination port is either invalid or full");
-		ret = -1;
-		/*
-		peer_->yield();
-		tearDown(); */
-		return false;
-	default:
-		logger_->log_info("HandShake Failed because of unknown respond code %d", code);
-		ret = -1;
-		/*
-		peer_->yield();
-		tearDown(); */
-		return false;
-	}
-
-	return false;
+      logger_->log_error(
+          "Site2Site HandShake Failed because destination port is either invalid or full");
+      ret = -1;
+      /*
+       peer_->yield();
+       tearDown(); */
+      return false;
+    default:
+      logger_->log_info("HandShake Failed because of unknown respond code %d",
+                        code);
+      ret = -1;
+      /*
+       peer_->yield();
+       tearDown(); */
+      return false;
+  }
+
+  return false;
 }
 
-void Site2SiteClientProtocol::tearDown()
-{
-	if (_peerState >= ESTABLISHED)
-	{
-		logger_->log_info("Site2Site Protocol tearDown");
-		// need to write shutdown request
-		writeRequestType(SHUTDOWN);
-	}
-
-	std::map<std::string, Transaction *>::iterator it;
-	for (it = _transactionMap.begin(); it!= _transactionMap.end(); it++)
-	{
-		delete it->second;
-	}
-	_transactionMap.clear();
-	peer_->Close();
-	_peerState = IDLE;
+void Site2SiteClientProtocol::tearDown() {
+  if (_peerState >= ESTABLISHED) {
+    logger_->log_info("Site2Site Protocol tearDown");
+    // need to write shutdown request
+    writeRequestType(SHUTDOWN);
+  }
+
+  std::map<std::string, Transaction *>::iterator it;
+  for (it = _transactionMap.begin(); it != _transactionMap.end(); it++) {
+    delete it->second;
+  }
+  _transactionMap.clear();
+  peer_->Close();
+  _peerState = IDLE;
 }
 
-int Site2SiteClientProtocol::writeRequestType(RequestType type)
-{
-	if (type >= MAX_REQUEST_TYPE)
-		return -1;
+int Site2SiteClientProtocol::writeRequestType(RequestType type) {
+  if (type >= MAX_REQUEST_TYPE)
+    return -1;
 
-	return peer_->writeUTF(RequestTypeStr[type]);
+  return peer_->writeUTF(RequestTypeStr[type]);
 }
 
-int Site2SiteClientProtocol::readRequestType(RequestType &type)
-{
-	std::string requestTypeStr;
+int Site2SiteClientProtocol::readRequestType(RequestType &type) {
+  std::string requestTypeStr;
 
-	int ret = peer_->readUTF(requestTypeStr);
+  int ret = peer_->readUTF(requestTypeStr);
 
-	if (ret <= 0)
-		return ret;
+  if (ret <= 0)
+    return ret;
 
-	for (int i = (int) NEGOTIATE_FLOWFILE_CODEC; i <= (int) SHUTDOWN; i++)
-	{
-		if (RequestTypeStr[i] == requestTypeStr)
-		{
-			type = (RequestType) i;
-			return ret;
-		}
-	}
+  for (int i = (int) NEGOTIATE_FLOWFILE_CODEC; i <= (int) SHUTDOWN; i++) {
+    if (RequestTypeStr[i] == requestTypeStr) {
+      type = (RequestType) i;
+      return ret;
+    }
+  }
 
-	return -1;
+  return -1;
 }
 
-int Site2SiteClientProtocol::readRespond(RespondCode &code, std::string &message)
-{
-	uint8_t firstByte;
+int Site2SiteClientProtocol::readRespond(RespondCode &code,
+                                         std::string &message) {
+  uint8_t firstByte;
 
-	int ret = peer_->read(firstByte);
+  int ret = peer_->read(firstByte);
 
-	if (ret <= 0 || firstByte != CODE_SEQUENCE_VALUE_1)
-		return -1;
+  if (ret <= 0 || firstByte != CODE_SEQUENCE_VALUE_1)
+    return -1;
 
-	uint8_t secondByte;
+  uint8_t secondByte;
 
-	ret = peer_->read(secondByte);
+  ret = peer_->read(secondByte);
 
-	if (ret <= 0 || secondByte != CODE_SEQUENCE_VALUE_2)
-		return -1;
+  if (ret <= 0 || secondByte != CODE_SEQUENCE_VALUE_2)
+    return -1;
 
-	uint8_t thirdByte;
+  uint8_t thirdByte;
 
-	ret = peer_->read(thirdByte);
+  ret = peer_->read(thirdByte);
 
-	if (ret <= 0)
-		return ret;
+  if (ret <= 0)
+    return ret;
 
-	code = (RespondCode) thirdByte;
+  code = (RespondCode) thirdByte;
 
-	RespondCodeContext *resCode = this->getRespondCodeContext(code);
+  RespondCodeContext *resCode = this->getRespondCodeContext(code);
 
-	if ( resCode == NULL)
-	{
-		// Not a valid respond code
-		return -1;
-	}
-	if (resCode->hasDescription)
-	{
-		ret = peer_->readUTF(message);
-		if (ret <= 0)
-			return -1;
-	}
-	return 3 + message.size();
+  if (resCode == NULL) {
+    // Not a valid respond code
+    return -1;
+  }
+  if (resCode->hasDescription) {
+    ret = peer_->readUTF(message);
+    if (ret <= 0)
+      return -1;
+  }
+  return 3 + message.size();
 }
 
-int Site2SiteClientProtocol::writeRespond(RespondCode code, std::string message)
-{
-	RespondCodeContext *resCode = this->getRespondCodeContext(code);
-
-	if (resCode == NULL)
-	{
-		// Not a valid respond code
-		return -1;
-	}
-
-	uint8_t codeSeq[3];
-	codeSeq[0] = CODE_SEQUENCE_VALUE_1;
-	codeSeq[1] = CODE_SEQUENCE_VALUE_2;
-	codeSeq[2] = (uint8_t) code;
-
-	int ret = peer_->write(codeSeq, 3);
-
-	if (ret != 3)
-		return -1;
-
-	if (resCode->hasDescription)
-	{
-		ret = peer_->writeUTF(message);
-		if (ret > 0)
-			return (3 + ret);
-		else
-			return ret;
-	}
-	else
-		return 3;
+int Site2SiteClientProtocol::writeRespond(RespondCode code,
+                                          std::string message) {
+  RespondCodeContext *resCode = this->getRespondCodeContext(code);
+
+  if (resCode == NULL) {
+    // Not a valid respond code
+    return -1;
+  }
+
+  uint8_t codeSeq[3];
+  codeSeq[0] = CODE_SEQUENCE_VALUE_1;
+  codeSeq[1] = CODE_SEQUENCE_VALUE_2;
+  codeSeq[2] = (uint8_t) code;
+
+  int ret = peer_->write(codeSeq, 3);
+
+  if (ret != 3)
+    return -1;
+
+  if (resCode->hasDescription) {
+    ret = peer_->writeUTF(message);
+    if (ret > 0)
+      return (3 + ret);
+    else
+      return ret;
+  } else
+    return 3;
 }
 
-bool Site2SiteClientProtocol::negotiateCodec()
-{
-	if (_peerState != HANDSHAKED)
-	{
-		logger_->log_error("Site2Site peer state is not handshaked while negotiate codec");
-		return false;
-	}
+bool Site2SiteClientProtocol::negotiateCodec() {
+  if (_peerState != HANDSHAKED) {
+    logger_->log_error(
+        "Site2Site peer state is not handshaked while negotiate codec");
+    return false;
+  }
 
-	logger_->log_info("Site2Site Protocol Negotiate Codec with destination port %s", _portIdStr.c_str());
+  logger_->log_info(
+      "Site2Site Protocol Negotiate Codec with destination port %s",
+      _portIdStr.c_str());
 
-	int status = this->writeRequestType(NEGOTIATE_FLOWFILE_CODEC);
+  int status = this->writeRequestType(NEGOTIATE_FLOWFILE_CODEC);
 
-	if (status <= 0)
-	{
-		// tearDown();
-		return false;
-	}
+  if (status <= 0) {
+    // tearDown();
+    return false;
+  }
 
-	// Negotiate the codec version
-	bool ret = initiateCodecResourceNegotiation();
+  // Negotiate the codec version
+  bool ret = initiateCodecResourceNegotiation();
 
-	if (!ret)
-	{
-		logger_->log_error("Site2Site Codec Version Negotiation failed");
-		/*
-		peer_->yield();
-		tearDown(); */
-		return false;
-	}
+  if (!ret) {
+    logger_->log_error("Site2Site Codec Version Negotiation failed");
+    /*
+     peer_->yield();
+     tearDown(); */
+    return false;
+  }
 
-	logger_->log_info("Site2Site Codec Completed and move to READY state for data transfer");
-	_peerState = READY;
+  logger_->log_info(
+      "Site2Site Codec Completed and move to READY state for data transfer");
+  _peerState = READY;
 
-	return true;
+  return true;
 }
 
-bool Site2SiteClientProtocol::bootstrap()
-{
-	if (_peerState == READY)
-		return true;
-
-	tearDown();
-
-	if (establish() && handShake() && negotiateCodec())
-	{
-		logger_->log_info("Site2Site Ready For data transaction");
-		return true;
-	}
-	else
-	{
-		peer_->yield();
-		tearDown();
-		return false;
-	}
+bool Site2SiteClientProtocol::bootstrap() {
+  if (_peerState == READY)
+    return true;
+
+  tearDown();
+
+  if (establish() && handShake() && negotiateCodec()) {
+    logger_->log_info("Site2Site Ready For data transaction");
+    return true;
+  } else {
+    peer_->yield();
+    tearDown();
+    return false;
+  }
 }
 
-Transaction* Site2SiteClientProtocol::createTransaction(std::string &transactionID, TransferDirection direction)
-{
-	int ret;
-	bool dataAvailable;
-	Transaction *transaction = NULL;
-
-	if (_peerState != READY)
-	{
-		bootstrap();
-	}
-
-	if (_peerState != READY)
-	{
-		return NULL;
-	}
-
-	if (direction == RECEIVE)
-	{
-		ret = writeRequestType(RECEIVE_FLOWFILES);
-
-		if (ret <= 0)
-		{
-			// tearDown();
-			return NULL;
-		}
-
-		RespondCode code;
-		std::string message;
-
-		ret = readRespond(code, message);
-
-		if (ret <= 0)
-		{
-			// tearDown();
-			return NULL;
-		}
-
-		 CRCStream<Site2SitePeer> crcstream(peer_);
-		switch (code)
-		{
-		case MORE_DATA:
-			dataAvailable = true;
-			logger_->log_info("Site2Site peer indicates that data is available");
-			transaction = new Transaction(direction,crcstream);
-			_transactionMap[transaction->getUUIDStr()] = transaction;
-			transactionID = transaction->getUUIDStr();
-			transaction->setDataAvailable(dataAvailable);
-			logger_->log_info("Site2Site create transaction %s", transaction->getUUIDStr().c_str());
-			return transaction;
-		case NO_MORE_DATA:
-			dataAvailable = false;
-			logger_->log_info("Site2Site peer indicates that no data is available");
-   			transaction = new Transaction(direction,crcstream);
-			_transactionMap[transaction->getUUIDStr()] = transaction;
-			transactionID = transaction->getUUIDStr();
-			transaction->setDataAvailable(dataAvailable);
-			logger_->log_info("Site2Site create transaction %s", transaction->getUUIDStr().c_str());
-			return transaction;
-		default:
-			logger_->log_info("Site2Site got unexpected response %d when asking for data", code);
-			// tearDown();
-			return NULL;
-		}
-	}
-	else
-	{
-		ret = writeRequestType(SEND_FLOWFILES);
-
-		if (ret <= 0)
-		{
-			// tearDown();
-			return NULL;
-		}
-		else
-		{
-			CRCStream<Site2SitePeer> crcstream(peer_);
-			transaction = new Transaction(direction,crcstream);
-			_transactionMap[transaction->getUUIDStr()] = transaction;
-			transactionID = transaction->getUUIDStr();
-			logger_->log_info("Site2Site create transaction %s", transaction->getUUIDStr().c_str());
-			return transaction;
-		}
-	}
+Transaction* Site2SiteClientProtocol::createTransaction(
+    std::string &transactionID, TransferDirection direction) {
+  int ret;
+  bool dataAvailable;
+  Transaction *transaction = NULL;
+
+  if (_peerState != READY) {
+    bootstrap();
+  }
+
+  if (_peerState != READY) {
+    return NULL;
+  }
+
+  if (direction == RECEIVE) {
+    ret = writeRequestType(RECEIVE_FLOWFILES);
+
+    if (ret <= 0) {
+      // tearDown();
+      return NULL;
+    }
+
+    RespondCode code;
+    std::string message;
+
+    ret = readRespond(code, message);
+
+    if (ret <= 0) {
+      // tearDown();
+      return NULL;
+    }
+
+    org::apache::nifi::minifi::io::CRCStream<Site2SitePeer> crcstream(peer_.get());
+    switch (code) {
+      case MORE_DATA:
+        dataAvailable = true;
+        logger_->log_info("Site2Site peer indicates that data is available");
+        transaction = new Transaction(direction, crcstream);
+        _transactionMap[transaction->getUUIDStr()] = transaction;
+        transactionID = transaction->getUUIDStr();
+        transaction->setDataAvailable(dataAvailable);
+        logger_->log_info("Site2Site create transaction %s",
+                          transaction->getUUIDStr().c_str());
+        return transaction;
+      case NO_MORE_DATA:
+        dataAvailable = false;
+        logger_->log_info("Site2Site peer indicates that no data is available");
+        transaction = new Transaction(direction, crcstream);
+        _transactionMap[transaction->getUUIDStr()] = transaction;
+        transactionID = transaction->getUUIDStr();
+        transaction->setDataAvailable(dataAvailable);
+        logger_->log_info("Site2Site create transaction %s",
+                          transaction->getUUIDStr().c_str());
+        return transaction;
+      default:
+        logger_->log_info(
+            "Site2Site got unexpected response %d when asking for data", code);
+        // tearDown();
+        return NULL;
+    }
+  } else {
+    ret = writeRequestType(SEND_FLOWFILES);
+
+    if (ret <= 0) {
+      // tearDown();
+      return NULL;
+    } else {
+      org::apache::nifi::minifi::io::CRCStream<Site2SitePeer> crcstream(peer_.get());
+      transaction = new Transaction(direction, crcstream);
+      _transactionMap[transaction->getUUIDStr()] = transaction;
+      transactionID = transaction->getUUIDStr();
+      logger_->log_info("Site2Site create transaction %s",
+                        transaction->getUUIDStr().c_str());
+      return transaction;
+    }
+  }
 }
 
-bool Site2SiteClientProtocol::receive(std::string transactionID, DataPacket *packet, bool &eof)
-{
-	int ret;
-	Transaction *transaction = NULL;
-
-	if (_peerState != READY)
-	{
-		bootstrap();
-	}
-
-	if (_peerState != READY)
-	{
-		return false;
-	}
-
-	std::map<std::string, Transaction *>::iterator it = this->_transactionMap.find(transactionID);
-
-	if (it == _transactionMap.end())
-	{
-		return false;
-	}
-	else
-	{
-		transaction = it->second;
-	}
-
-	if (transaction->getState() != TRANSACTION_STARTED && transaction->getState() != DATA_EXCHANGED)
-	{
-		logger_->log_info("Site2Site transaction %s is not at started or exchanged state", transactionID.c_str());
-		return false;
-	}
-
-	if (transaction->getDirection() != RECEIVE)
-	{
-		logger_->log_info("Site2Site transaction %s direction is wrong", transactionID.c_str());
-		return false;
-	}
-
-	if (!transaction->isDataAvailable())
-	{
-		eof = true;
-		return true;
-	}
-
-	if (transaction->_transfers > 0)
-	{
-		// if we already has transfer before, check to see whether another one is available
-		RespondCode code;
-		std::string message;
-
-		ret = readRespond(code, message);
-
-		if (ret <= 0)
-		{
-			return false;
-		}
-		if (code == CONTINUE_TRANSACTION)
-		{
-			logger_->log_info("Site2Site transaction %s peer indicate continue transaction", transactionID.c_str());
-			transaction->_dataAvailable = true;
-		}
-		else if (code == FINISH_TRANSACTION)
-		{
-			logger_->log_info("Site2Site transaction %s peer indicate finish transaction", transactionID.c_str());
-			transaction->_dataAvailable = false;
-		}
-		else
-		{
-			logger_->log_info("Site2Site transaction %s peer indicate wrong respond code %d", transactionID.c_str(), code);
-			return false;
-		}
-	}
-
-	if (!transaction->isDataAvailable())
-	{
-		eof = true;
-		return true;
-	}
-
-	// start to read the packet
-	uint32_t numAttributes;
-	ret = transaction->getStream().read(numAttributes);
-	if (ret <= 0 || numAttributes > MAX_NUM_ATTRIBUTES)
-	{
-		return false;
-	}
-
-	// read the attributes
-	for (unsigned int i = 0; i < numAttributes; i++)
-	{
-		std::string key;
-		std::string value;
-		ret = transaction->getStream().readUTF(key,true);
-		if (ret <= 0)
-		{
-			return false;
-		}
-		ret = transaction->getStream().readUTF(value,true);
-		if (ret <= 0)
-		{
-			return false;
-		}
-		packet->_attributes[key] = value;
-		logger_->log_info("Site2Site transaction %s receives attribute key %s value %s", transactionID.c_str(), key.c_str(), value.c_str());
-	}
-
-	uint64_t len;
-	ret = transaction->getStream().read(len);
-	if (ret <= 0)
-	{
-		return false;
-	}
-
-	packet->_size = len;
-	transaction->_transfers++;
-	transaction->_state = DATA_EXCHANGED;
-	transaction->_bytes += len;
-	logger_->log_info("Site2Site transaction %s receives flow record %d, total length %d", transactionID.c_str(),
-			transaction->_transfers, transaction->_bytes);
-
-	return true;
+bool Site2SiteClientProtocol::receive(std::string transactionID,
+                                      DataPacket *packet, bool &eof) {
+  int ret;
+  Transaction *transaction = NULL;
+
+  if (_peerState != READY) {
+    bootstrap();
+  }
+
+  if (_peerState != READY) {
+    return false;
+  }
+
+  std::map<std::string, Transaction *>::iterator it =
+      this->_transactionMap.find(transactionID);
+
+  if (it == _transactionMap.end()) {
+    return false;
+  } else {
+    transaction = it->second;
+  }
+
+  if (transaction->getState() != TRANSACTION_STARTED
+      && transaction->getState() != DATA_EXCHANGED) {
+    logger_->log_info(
+        "Site2Site transaction %s is not at started or exchanged state",
+        transactionID.c_str());
+    return false;
+  }
+
+  if (transaction->getDirection() != RECEIVE) {
+    logger_->log_info("Site2Site transaction %s direction is wrong",
+                      transactionID.c_str());
+    return false;
+  }
+
+  if (!transaction->isDataAvailable()) {
+    eof = true;
+    return true;
+  }
+
+  if (transaction->_transfers > 0) {
+    // if we already has transfer before, check to see whether another one is available
+    RespondCode code;
+    std::string message;
+
+    ret = readRespond(code, message);
+
+    if (ret <= 0) {
+      return false;
+    }
+    if (code == CONTINUE_TRANSACTION) {
+      logger_->log_info(
+          "Site2Site transaction %s peer indicate continue transaction",
+          transactionID.c_str());
+      transaction->_dataAvailable = true;
+    } else if (code == FINISH_TRANSACTION) {
+      logger_->log_info(
+          "Site2Site transaction %s peer indicate finish transaction",
+          transactionID.c_str());
+      transaction->_dataAvailable = false;
+    } else {
+      logger_->log_info(
+          "Site2Site transaction %s peer indicate wrong respond code %d",
+          transactionID.c_str(), code);
+      return false;
+    }
+  }
+
+  if (!transaction->isDataAvailable()) {
+    eof = true;
+    return true;
+  }
+
+  // start to read the packet
+  uint32_t numAttributes;
+  ret = transaction->getStream().read(numAttributes);
+  if (ret <= 0 || numAttributes > MAX_NUM_ATTRIBUTES) {
+    return false;
+  }
+
+  // read the attributes
+  for (unsigned int i = 0; i < numAttributes; i++) {
+    std::string key;
+    std::string value;
+    ret = transaction->getStream().readUTF(key, true);
+    if (ret <= 0) {
+      return false;
+    }
+    ret = transaction->getStream().readUTF(value, true);
+    if (ret <= 0) {
+      return false;
+    }
+    packet->_attributes[key] = value;
+    logger_->log_info(
+        "Site2Site transaction %s receives attribute key %s value %s",
+        transactionID.c_str(), key.c_str(), value.c_str());
+  }
+
+  uint64_t len;
+  ret = transaction->getStream().read(len);
+  if (ret <= 0) {
+    return false;
+  }
+
+  packet->_size = len;
+  transaction->_transfers++;
+  transaction->_state = DATA_EXCHANGED;
+  transaction->_bytes += len;
+  logger_->log_info(
+      "Site2Site transaction %s receives flow record %d, total length %d",
+      transactionID.c_str(), transaction->_transfers, transaction->_bytes);
+
+  return true;
 }
 
-bool Site2SiteClientProtocol::send(std::string transactionID, DataPacket *packet, FlowFileRecord *flowFile, ProcessSession *session)
-{
-	int ret;
-	Transaction *transaction = NULL;
-
-	if (_peerState != READY)
-	{
-		bootstrap();
-	}
-
-	if (_peerState != READY)
-	{
-		return false;
-	}
-
-	std::map<std::string, Transaction *>::iterator it = this->_transactionMap.find(transactionID);
-
-	if (it == _transactionMap.end())
-	{
-		return false;
-	}
-	else
-	{
-		transaction = it->second;
-	}
-
-	if (transaction->getState() != TRANSACTION_STARTED && transaction->getState() != DATA_EXCHANGED)
-	{
-		logger_->log_info("Site2Site transaction %s is not at started or exchanged state", transactionID.c_str());
-		return false;
-	}
-
-	if (transaction->getDirection() != SEND)
-	{
-		logger_->log_info("Site2Site transaction %s direction is wrong", transactionID.c_str());
-		return false;
-	}
-
-	if (transaction->_transfers > 0)
-	{
-		ret = writeRespond(CONTINUE_TRANSACTION, "CONTINUE_TRANSACTION");
-		if (ret <= 0)
-		{
-			return false;
-		}
-	}
-
-	// start to read the packet
-	uint32_t numAttributes = packet->_attributes.size();
-	ret = transaction->getStream().write(numAttributes);
-	if (ret != 4)
-	{
-		return false;
-	}
-
-	std::map<std::string, std::string>::iterator itAttribute;
-	for (itAttribute = packet->_attributes.begin(); itAttribute!= packet->_attributes.end(); itAttribute++)
-	{
-		ret = transaction->getStream().writeUTF(itAttribute->first, true);
-
-		if (ret <= 0)
-		{
-			return false;
-		}
-		ret = transaction->getStream().writeUTF(itAttribute->second, true);
-		if (ret <= 0)
-		{
-			return false;
-		}
-		logger_->log_info("Site2Site transaction %s send attribute key %s value %s", transactionID.c_str(),
-				itAttribute->first.c_str(), itAttribute->second.c_str());
-	}
-
-	uint64_t len = flowFile->getSize() ;
-	ret = transaction->getStream().write(len);
-	if (ret != 8)
-	{
-		return false;
-	}
-
-	if (flowFile->getSize())
-	{
-		Site2SiteClientProtocol::ReadCallback callback(packet);
-		session->read(flowFile, &callback);
-		if (flowFile->getSize() != packet->_size)
-		{
-			return false;
-		}
-	}
-
-	transaction->_transfers++;
-	transaction->_state = DATA_EXCHANGED;
-	transaction->_bytes += len;
-	logger_->log_info("Site2Site transaction %s send flow record %d, total length %d", transactionID.c_str(),
-				transaction->_transfers, transaction->_bytes);
-
-	return true;
+bool Site2SiteClientProtocol::send(
+    std::string transactionID, DataPacket *packet, std::shared_ptr<FlowFileRecord> flowFile,
+    core::ProcessSession *session) {
+  int ret;
+  Transaction *transaction = NULL;
+
+  if (_peerState != READY) {
+    bootstrap();
+  }
+
+  if (_peerState != READY) {
+    return false;
+  }
+
+  std::map<std::string, Transaction *>::iterator it =
+      this->_transactionMap.find(transactionID);
+
+  if (it == _transactionMap.end()) {
+    return false;
+  } else {
+    transaction = it->second;
+  }
+
+  if (transaction->getState() != TRANSACTION_STARTED
+      && transaction->getState() != DATA_EXCHANGED) {
+    logger_->log_info(
+        "Site2Site transaction %s is not at started or exchanged state",
+        transactionID.c_str());
+    return false;
+  }
+
+  if (transaction->getDirection() != SEND) {
+    logger_->log_info("Site2Site transaction %s direction is wrong",
+                      transactionID.c_str());
+    return false;
+  }
+
+  if (transaction->_transfers > 0) {
+    ret = writeRespond(CONTINUE_TRANSACTION, "CONTINUE_TRANSACTION");
+    if (ret <= 0) {
+      return false;
+    }
+  }
+
+  // start to read the packet
+  uint32_t numAttributes = packet->_attributes.size();
+  ret = transaction->getStream().write(numAttributes);
+  if (ret != 4) {
+    return false;
+  }
+
+  std::map<std::string, std::string>::iterator itAttribute;
+  for (itAttribute = packet->_attributes.begin();
+      itAttribute != packet->_attributes.end(); itAttribute++) {
+    ret = transaction->getStream().writeUTF(itAttribute->first, true);
+
+    if (ret <= 0) {
+      return false;
+    }
+    ret = transaction->getStream().writeUTF(itAttribute->second, true);
+    if (ret <= 0) {
+      return false;
+    }
+    logger_->log_info("Site2Site transaction %s send attribute key %s value %s",
+                      transactionID.c_str(), itAttribute->first.c_str(),
+                      itAttribute->second.c_str());
+  }
+
+  uint64_t len = flowFile->getSize();
+  ret = transaction->getStream().write(len);
+  if (ret != 8) {
+    return false;
+  }
+
+  if (flowFile->getSize()) {
+    Site2SiteClientProtocol::ReadCallback callback(packet);
+    session->read(flowFile, &callback);
+    if (flowFile->getSize() != packet->_size) {
+      return false;
+    }
+  }
+
+  transaction->_transfers++;
+  transaction->_state = DATA_EXCHANGED;
+  transaction->_bytes += len;
+  logger_->log_info(
+      "Site2Site transaction %s send flow record %d, total length %d",
+      transactionID.c_str(), transaction->_transfers, transaction->_bytes);
+
+  return true;
 }
 
-void Site2SiteClientProtocol::receiveFlowFiles(ProcessContext *context, ProcessSession *session)
-{
-	uint64_t bytes = 0;
-	int transfers = 0;
-	Transaction *transaction = NULL;
-
-	if (_peerState != READY)
-	{
-		bootstrap();
-	}
-
-	if (_peerState != READY)
-	{
-		context->yield();
-		tearDown();
-		throw Exception(SITE2SITE_EXCEPTION, "Can not establish handshake with peer");
-		return;
-	}
-
-	// Create the transaction
-	std::string transactionID;
-	transaction = createTransaction(transactionID, RECEIVE);
-
-	if (transaction == NULL)
-	{
-		context->yield();
-		tearDown();
-		throw Exception(SITE2SITE_EXCEPTION, "Can not create transaction");
-		return;
-	}
-
-	try
-	{
-		while (true)
-		{
-			std::map<std::string, std::string> empty;
-			uint64_t startTime = getTimeMillis();
-			DataPacket packet(this, transaction, empty);
-			bool eof = false;
-
-			if (!receive(transactionID, &packet, eof))
-			{
-				throw Exception(SITE2SITE_EXCEPTION, "Receive Failed");
-				return;
-			}
-			if (eof)
-			{
-				// transaction done
-				break;
-			}
-			FlowFileRecord *flowFile = session->create();
-			if (!flowFile)
-			{
-				throw Exception(SITE2SITE_EXCEPTION, "Flow File Creation Failed");
-				return;
-			}
-			std::map<std::string, std::string>::iterator it;
-			std::string sourceIdentifier;
-			for (it = packet._attributes.begin(); it!= packet._attributes.end(); it++)
-			{
-				if (it->first == FlowAttributeKey(UUID))
-					sourceIdentifier = it->second;
-				flowFile->addAttribute(it->first, it->second);
-			}
-
-			if (packet._size > 0)
-			{
-				Site2SiteClientProtocol::WriteCallback callback(&packet);
-				session->write(flowFile, &callback);
-				if (flowFile->getSize() != packet._size)
-				{
-					throw Exception(SITE2SITE_EXCEPTION, "Receive Size Not Right");
-					return;
-				}
-			}
-			Relationship relation; // undefined relationship
-			uint64_t endTime = getTimeMillis();
-			std::string transitUri = peer_->getURL() + "/" + sourceIdentifier;
-			std::string details = "urn:nifi:" + sourceIdentifier + "Remote Host=" + peer_->getHostName();
-			session->getProvenanceReporter()->receive(flowFile, transitUri, sourceIdentifier, details, endTime - startTime);
-			session->transfer(flowFile, relation);
-			// receive the transfer for the flow record
-			bytes += packet._size;
-			transfers++;
-		} // while true
-
-		if (!confirm(transactionID))
-		{
-			throw Exception(SITE2SITE_EXCEPTION, "Confirm Transaction Failed");
-			return;
-		}
-		if (!complete(transactionID))
-		{
-			throw Exception(SITE2SITE_EXCEPTION, "Complete Transaction Failed");
-			return;
-		}
-		logger_->log_info("Site2Site transaction %s successfully receive flow record %d, content bytes %d",
-				transactionID.c_str(), transfers, bytes);
-		// we yield the receive if we did not get anything
-		if (transfers == 0)
-			context->yield();
-	}
-	catch (std::exception &exception)
-	{
-		if (transaction)
-			deleteTransaction(transactionID);
-		context->yield();
-		tearDown();
-		logger_->log_debug("Caught Exception %s", exception.what());
-		throw;
-	}
-	catch (...)
-	{
-		if (transaction)
-			deleteTransaction(transactionID);
-		context->yield();
-		tearDown();
-		logger_->log_debug("Caught Exception during Site2SiteClientProtocol::receiveFlowFiles");
-		throw;
-	}
-
-	deleteTransaction(transactionID);
-
-	return;
+void Site2SiteClientProtocol::receiveFlowFiles(
+    core::ProcessContext *context,
+    core::ProcessSession *session) {
+  uint64_t bytes = 0;
+  int transfers = 0;
+  Transaction *transaction = NULL;
+
+  if (_peerState != READY) {
+    bootstrap();
+  }
+
+  if (_peerState != READY) {
+    context->yield();
+    tearDown();
+    throw Exception(SITE2SITE_EXCEPTION,
+                    "Can not establish handshake with peer");
+    return;
+  }
+
+  // Create the transaction
+  std::string transactionID;
+  transaction = createTransaction(transactionID, RECEIVE);
+
+  if (transaction == NULL) {
+    context->yield();
+    tearDown();
+    throw Exception(SITE2SITE_EXCEPTION, "Can not create transaction");
+    return;
+  }
+
+  try {
+    while (true) {
+      std::map<std::string, std::string> empty;
+      uint64_t startTime = getTimeMillis();
+      DataPacket packet(this, transaction, empty);
+      bool eof = false;
+
+      if (!receive(transactionID, &packet, eof)) {
+        throw Exception(SITE2SITE_EXCEPTION, "Receive Failed");
+        return;
+      }
+      if (eof) {
+        // transaction done
+        break;
+      }
+      std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast<FlowFileRecord>(session->create());;
+      if (!flowFile) {
+        throw Exception(SITE2SITE_EXCEPTION, "Flow File Creation Failed");
+        return;
+      }
+      std::map<std::string, std::string>::iterator it;
+      std::string sourceIdentifier;
+      for (it = packet._attributes.begin(); it != packet._attributes.end();
+          it++) {
+        if (it->first == FlowAttributeKey(UUID))
+          sourceIdentifier = it->second;
+        flowFile->addAttribute(it->first, it->second);
+      }
+
+      if (packet._size > 0) {
+        Site2SiteClientProtocol::WriteCallback callback(&packet);
+        session->write(flowFile, &callback);
+        if (flowFile->getSize() != packet._size) {
+          throw Exception(SITE2SITE_EXCEPTION, "Receive Size Not Right");
+          return;
+        }
+      }
+      core::Relationship relation;  // undefined relationship
+      uint64_t endTime = getTimeMillis();
+      std::string transitUri = peer_->getURL() + "/" + sourceIdentifier;
+      std::string details = "urn:nifi:" + sourceIdentifier + "Remote Host="
+          + peer_->getHostName();
+      session->getProvenanceReporter()->receive(flowFile, transitUri,
+                                                sourceIdentifier, details,
+                                                endTime - startTime);
+      session->transfer(flowFile, relation);
+      // receive the transfer for the flow record
+      bytes += packet._size;
+      transfers++;
+    }  // while true
+
+    if (!confirm(transactionID)) {
+      throw Exception(SITE2SITE_EXCEPTION, "Confirm Transaction Failed");
+      return;
+    }
+    if (!complete(transactionID)) {
+      throw Exception(SITE2SITE_EXCEPTION, "Complete Transaction Failed");
+      return;
+    }
+    logger_->log_info(
+        "Site2Site transaction %s successfully receive flow record %d, content bytes %d",
+        transactionID.c_str(), transfers, bytes);
+    // we yield the receive if we did not get anything
+    if (transfers == 0)
+      context->yield();
+  } catch (std::exception &exception) {
+    if (transaction)
+      deleteTransaction(transactionID);
+    context->yield();
+    tearDown();
+    logger_->log_debug("Caught Exception %s", exception.what());
+    throw;
+  } catch (...) {
+    if (transaction)
+      deleteTransaction(transactionID);
+    context->yield();
+    tearDown();
+    logger_->log_debug(
+        "Caught Exception during Site2SiteClientProtocol::receiveFlowFiles");
+    throw;
+  }
+
+  deleteTransaction(transactionID);
+
+  return;
 }
 
-bool Site2SiteClientProtocol::confirm(std::string transactionID)
-{
-	int ret;
-	Transaction *transaction = NULL;
-
-	if (_peerState != READY)
-	{
-		bootstrap();
-	}
-
-	if (_peerState != READY)
-	{
-		return false;
-	}
-
-	std::map<std::string, Transaction *>::iterator it = this->_transactionMap.find(transactionID);
-
-	if (it == _transactionMap.end())
-	{
-		return false;
-	}
-	else
-	{
-		transaction = it->second;
-	}
-
-	if (transaction->getState() == TRANSACTION_STARTED && !transaction->isDataAvailable() &&
-			transaction->getDirection() == RECEIVE)
-	{
-		transaction->_state = TRANSACTION_CONFIRMED;
-		return true;
-	}
-
-	if (transaction->getState() != DATA_EXCHANGED)
-		return false;
-
-	if (transaction->getDirection() == RECEIVE)
-	{
-		if (transaction->isDataAvailable())
-			return false;
-		// we received a FINISH_TRANSACTION indicator. Send back a CONFIRM_TRANSACTION message
-		// to peer so that we can verify that the connection is still open. This is a two-phase commit,
-		// which helps to prevent the chances of data duplication. Without doing this, we may commit the
-		// session and then when we send the response back to the peer, the peer may have timed out and may not
-		// be listening. As a result, it will re-send the data. By doing this two-phase commit, we narrow the
-		// Critical Section involved in this transaction so that rather than the Critical Section being the
-		// time window involved in the entire transaction, it is reduced to a simple round-trip conversation.
-		long crcValue = transaction->getCRC();
-		std::string crc = std::to_string(crcValue);
-		logger_->log_info("Site2Site Send confirm with CRC %d to transaction %s", transaction->getCRC(),
-						transactionID.c_str());
-		ret = writeRespond(CONFIRM_TRANSACTION, crc);
-		if (ret <= 0)
-			return false;
-		RespondCode code;
-		std::string message;
-		readRespond(code, message);
-		if (ret <= 0)
-			return false;
-
-		if (code == CONFIRM_TRANSACTION)
-		{
-			logger_->log_info("Site2Site transaction %s peer confirm transaction", transactionID.c_str());
-			transaction->_state = TRANSACTION_CONFIRMED;
-			return true;
-		}
-		else if (code == BAD_CHECKSUM)
-		{
-			logger_->log_info("Site2Site transaction %s peer indicate bad checksum", transactionID.c_str());
-			/*
-			transaction->_state = TRANSACTION_CONFIRMED;
-			return true; */
-			return false;
-		}
-		else
-		{
-			logger_->log_info("Site2Site transaction %s peer unknown respond code %d",
-					transactionID.c_str(), code);
-			return false;
-		}
-	}
-	else
-	{
-		logger_->log_info("Site2Site Send FINISH TRANSACTION for transaction %s",
-								transactionID.c_str());
-		ret = writeRespond(FINISH_TRANSACTION, "FINISH_TRANSACTION");
-		if (ret <= 0)
-			return false;
-		RespondCode code;
-		std::string message;
-		readRespond(code, message);
-		if (ret <= 0)
-			return false;
-
-		// we've sent a FINISH_TRANSACTION. Now we'll wait for the peer to send a 'Confirm Transaction' response
-		if (code == CONFIRM_TRANSACTION)
-		{
-			logger_->log_info("Site2Site transaction %s peer confirm transaction with CRC %s", transactionID.c_str(), message.c_str());
-			if (this->_currentVersion > 3)
-			{
-				long crcValue = transaction->getCRC();
-				std::string crc = std::to_string(crcValue);
-				if (message == crc)
-				{
-					logger_->log_info("Site2Site transaction %s CRC matched", transactionID.c_str());
-					ret = writeRespond(CONFIRM_TRANSACTION, "CONFIRM_TRANSACTION");
-					if (ret <= 0)
-						return false;
-					transaction->_state = TRANSACTION_CONFIRMED;
-					return true;
-				}
-				else
-				{
-					logger_->log_info("Site2Site transaction %s CRC not matched %s", transactionID.c_str(), crc.c_str());
-					ret = writeRespond(BAD_CHECKSUM, "BAD_CHECKSUM");
-					/*
-					ret = writeRespond(CONFIRM_TRANSACTION, "CONFIRM_TRANSACTION");
-										if (ret <= 0)
-											return false;
-										transaction->_state = TRANSACTION_CONFIRMED;
-					return true; */
-					return false;
-				}
-			}
-			ret = writeRespond(CONFIRM_TRANSACTION, "CONFIRM_TRANSACTION");
-			if (ret <= 0)
-				return false;
-			transaction->_state = TRANSACTION_CONFIRMED;
-			return true;
-		}
-		else
-		{
-			logger_->log_info("Site2Site transaction %s peer unknown respond code %d",
-					transactionID.c_str(), code);
-			return false;
-		}
-		return false;
-	}
+bool Site2SiteClientProtocol::confirm(std::string transactionID) {
+  int ret;
+  Transaction *transaction = NULL;
+
+  if (_peerState != READY) {
+    bootstrap();
+  }
+
+  if (_peerState != READY) {
+    return false;
+  }
+
+  std::map<std::string, Transaction *>::iterator it =
+      this->_transactionMap.find(transactionID);
+
+  if (it == _transactionMap.end()) {
+    return false;
+  } else {
+    transaction = it->second;
+  }
+
+  if (transaction->getState() == TRANSACTION_STARTED
+      && !transaction->isDataAvailable()
+      && transaction->getDirection() == RECEIVE) {
+    transaction->_state = TRANSACTION_CONFIRMED;
+    return true;
+  }
+
+  if (transaction->getState() != DATA_EXCHANGED)
+    return false;
+
+  if (transaction->getDirection() == RECEIVE) {
+    if (transaction->isDataAvailable())
+      return false;
+    // we received a FINISH_TRANSACTION indicator. Send back a CONFIRM_TRANSACTION message
+    // to peer so that we can verify that the connection is still open. This is a two-phase commit,
+    // which helps to prevent the chances of data duplication. Without doing this, we may commit the
+    // session and then when we send the response back to the peer, the peer may have timed out and may not
+    // be listening. As a result, it will re-send the data. By doing this two-phase commit, we narrow the
+    // Critical Section involved in this transaction so that rather than the Critical Section being the
+    // time window involved in the entire transaction, it is reduced to a simple round-trip conversation.
+    long crcValue = transaction->getCRC();
+    std::string crc = std::to_string(crcValue);
+    logger_->log_info("Site2Site Send confirm with CRC %d to transaction %s",
+                      transaction->getCRC(), transactionID.c_str());
+    ret = writeRespond(CONFIRM_TRANSACTION, crc);
+    if (ret <= 0)
+      return false;
+    RespondCode code;
+    std::string message;
+    readRespond(code, message);
+    if (ret <= 0)
+      return false;
+
+    if (code == CONFIRM_TRANSACTION) {
+      logger_->log_info("Site2Site transaction %s peer confirm transaction",
+                        transactionID.c_str());
+      transaction->_state = TRANSACTION_CONFIRMED;
+      return true;
+    } else if (code == BAD_CHECKSUM) {
+      logger_->log_info("Site2Site transaction %s peer indicate bad checksum",
+                        transactionID.c_str());
+      /*
+       transaction->_state = TRANSACTION_CONFIRMED;
+       return true; */
+      return false;
+    } else {
+      logger_->log_info("Site2Site transaction %s peer unknown respond code %d",
+                        transactionID.c_str(), code);
+      return false;
+    }
+  } else {
+    logger_->log_info("Site2Site Send FINISH TRANSACTION for transaction %s",
+                      transactionID.c_str());
+    ret = writeRespond(FINISH_TRANSACTION, "FINISH_TRANSACTION");
+    if (ret <= 0)
+      return false;
+    RespondCode code;
+    std::string message;
+    readRespond(code, message);
+    if (ret <= 0)
+      return false;
+
+    // we've sent a FINISH_TRANSACTION. Now we'll wait for the peer to send a 'Confirm Transaction' response
+    if (code == CONFIRM_TRANSACTION) {
+      logger_->log_info(
+          "Site2Site transaction %s peer confirm transaction with CRC %s",
+          transactionID.c_str(), message.c_str());
+      if (this->_currentVersion > 3) {
+        long crcValue = transaction->getCRC();
+        std::string crc = std::to_string(crcValue);
+        if (message == crc) {
+          logger_->log_info("Site2Site transaction %s CRC matched",
+                            transactionID.c_str());
+          ret = writeRespond(CONFIRM_TRANSACTION, "CONFIRM_TRANSACTION");
+          if (ret <= 0)
+            return false;
+          transaction->_state = TRANSACTION_CONFIRMED;
+          return true;
+        } else {
+          logger_->log_info("Site2Site transaction %s CRC not matched %s",
+                            transactionID.c_str(), crc.c_str());
+          ret = writeRespond(BAD_CHECKSUM, "BAD_CHECKSUM");
+          /*
+           ret = writeRespond(CONFIRM_TRANSACTION, "CONFIRM_TRANSACTION");
+           if (ret <= 0)
+           return false;
+           transaction->_state = TRANSACTION_CONFIRMED;
+           return true; */
+          return false;
+        }
+      }
+      ret = writeRespond(CONFIRM_TRANSACTION, "CONFIRM_TRANSACTION");
+      if (ret <= 0)
+        return false;
+      transaction->_state = TRANSACTION_CONFIRMED;
+      return true;
+    } else {
+      logger_->log_info("Site2Site transaction %s peer unknown respond code %d",
+                        transactionID.c_str(), code);
+      return false;
+    }
+    return false;
+  }
 }
 
-void Site2SiteClientProtocol::cancel(std::string transactionID)
-{
-	Transaction *transaction = NULL;
-
-	if (_peerState != READY)
-	{
-		return;
-	}
-
-	std::map<std::string, Transaction *>::iterator it = this->_transactionMap.find(transactionID);
-
-	if (it == _transactionMap.end())
-	{
-		return;
-	}
-	else
-	{
-		transaction = it->second;
-	}
-
-	if (transaction->getState() == TRANSACTION_CANCELED || transaction->getState() == TRANSACTION_COMPLETED
-			|| transaction->getState() == TRANSACTION_ERROR)
-	{
-		return;
-	}
-
-	this->writeRespond(CANCEL_TRANSACTION, "Cancel");
-	transaction->_state = TRANSACTION_CANCELED;
-
-	tearDown();
-	return;
+void Site2SiteClientProtocol::cancel(std::string transactionID) {
+  Transaction *transaction = NULL;
+
+  if (_peerState != READY) {
+    return;
+  }
+
+  std::map<std::string, Transaction *>::iterator it =
+      this->_transactionMap.find(transactionID);
+
+  if (it == _transactionMap.end()) {
+    return;
+  } else {
+    transaction = it->second;
+  }
+
+  if (transaction->getState() == TRANSACTION_CANCELED
+      || transaction->getState() == TRANSACTION_COMPLETED
+      || transaction->getState() == TRANSACTION_ERROR) {
+    return;
+  }
+
+  this->writeRespond(CANCEL_TRANSACTION, "Cancel");
+  transaction->_state = TRANSACTION_CANCELED;
+
+  tearDown();
+  return;
 }
 
-void Site2SiteClientProtocol::deleteTransaction(std::string transactionID)
-{
-	Transaction *transaction = NULL;
+void Site2SiteClientProtocol::deleteTransaction(std::string transactionID) {
+  Transaction *transaction = NULL;
 
-	std::map<std::string, Transaction *>::iterator it = this->_transactionMap.find(transactionID);
+  std::map<std::string, Transaction *>::iterator it =
+      this->_transactionMap.find(transactionID);
 
-	if (it == _transactionMap.end())
-	{
-		return;
-	}
-	else
-	{
-		transaction = it->second;
-	}
+  if (it == _transactionMap.end()) {
+    return;
+  } else {
+    transaction = it->second;
+  }
 
-	logger_->log_info("Site2Site delete transaction %s", transaction->getUUIDStr().c_str());
-	delete transaction;
-	_transactionMap.erase(transactionID);
+  logger_->log_info("Site2Site delete transaction %s",
+                    transaction->getUUIDStr().c_str());
+  delete transaction;
+  _transactionMap.erase(transactionID);
 }
 
-void Site2SiteClientProtocol::error(std::string transactionID)
-{
-	Transaction *transaction = NULL;
+void Site2SiteClientProtocol::error(std::string transactionID) {
+  Transaction *transaction = NULL;
 
-	std::map<std::string, Transaction *>::iterator it = this->_transactionMap.find(transactionID);
+  std::map<std::string, Transaction *>::iterator it =
+      this->_transactionMap.find(transactionID);
 
-	if (it == _transactionMap.end())
-	{
-		return;
-	}
-	else
-	{
-		transaction = it->second;
-	}
+  if (it == _transactionMap.end()) {
+    return;
+  } else {
+    transaction = it->second;
+  }
 
-	transaction->_state = TRANSACTION_ERROR;
-	tearDown();
-	return;
+  transaction->_state = TRANSACTION_ERROR;
+  tearDown();
+  return;
 }
 
-//! Complete the transaction
-bool Site2SiteClientProtocol::complete(std::string transactionID)
-{
-	int ret;
-	Transaction *transaction = NULL;
-
-	if (_peerState != READY)
-	{
-		bootstrap();
-	}
-
-	if (_peerState != READY)
-	{
-		return false;
-	}
-
-	std::map<std::string, Transaction *>::iterator it = this->_transactionMap.find(transactionID);
-
-	if (it == _transactionMap.end())
-	{
-		return false;
-	}
-	else
-	{
-		transaction = it->second;
-	}
-
-	if (transaction->getState() != TRANSACTION_CONFIRMED)
-	{
-		return false;
-	}
-
-	if (transaction->getDirection() == RECEIVE)
-	{
-		if (transaction->_transfers == 0)
-		{
-			transaction->_state = TRANSACTION_COMPLETED;
-			return true;
-		}
-		else
-		{
-			logger_->log_info("Site2Site transaction %s send finished", transactionID.c_str());
-			ret = this->writeRespond(TRANSACTION_FINISHED, "Finished");
-			if (ret <= 0)
-				return false;
-			else
-			{
-				transaction->_state = TRANSACTION_COMPLETED;
-				return true;
-			}
-		}
-	}
-	else
-	{
-		RespondCode code;
-		std::string message;
-		int ret;
-
-		ret = readRespond(code, message);
-
-		if (ret <= 0)
-			return false;
-
-		if (code == TRANSACTION_FINISHED)
-		{
-			logger_->log_info("Site2Site transaction %s peer finished transaction", transactionID.c_str());
-			transaction->_state = TRANSACTION_COMPLETED;
-			return true;
-		}
-		else
-		{
-			logger_->log_info("Site2Site transaction %s peer unknown respond code %d",
-					transactionID.c_str(), code);
-			return false;
-		}
-	}
+// Complete the transaction
+bool Site2SiteClientProtocol::complete(std::string transactionID) {
+  int ret;
+  Transaction *transaction = NULL;
+
+  if (_peerState != READY) {
+    bootstrap();
+  }
+
+  if (_peerState != READY) {
+    return false;
+  }
+
+  std::map<std::string, Transaction *>::iterator it =
+      this->_transactionMap.find(transactionID);
+
+  if (it == _transactionMap.end()) {
+    return false;
+  } else {
+    transaction = it->second;
+  }
+
+  if (transaction->getState() != TRANSACTION_CONFIRMED) {
+    return false;
+  }
+
+  if (transaction->getDirection() == RECEIVE) {
+    if (transaction->_transfers == 0) {
+      transaction->_state = TRANSACTION_COMPLETED;
+      return true;
+    } else {
+      logger_->log_info("Site2Site transaction %s send finished",
+                        transactionID.c_str());
+      ret = this->writeRespond(TRANSACTION_FINISHED, "Finished");
+      if (ret <= 0)
+        return false;
+      else {
+        transaction->_state = TRANSACTION_COMPLETED;
+        return true;
+      }
+    }
+  } else {
+    RespondCode code;
+    std::string message;
+    int ret;
+
+    ret = readRespond(code, message);
+
+    if (ret <= 0)
+      return false;
+
+    if (code == TRANSACTION_FINISHED) {
+      logger_->log_info("Site2Site transaction %s peer finished transaction",
+                        transactionID.c_str());
+      transaction->_state = TRANSACTION_COMPLETED;
+      return true;
+    } else {
+      logger_->log_info("Site2Site transaction %s peer unknown respond code %d",
+                        transactionID.c_str(), code);
+      return false;
+    }
+  }
 }
 
-void Site2SiteClientProtocol::transferFlowFiles(ProcessContext *context, ProcessSession *session)
-{
-	FlowFileRecord *flow = session->get();
-	Transaction *transaction = NULL;
-
-	if (!flow)
-		return;
-
-	if (_peerState != READY)
-	{
-		bootstrap();
-	}
-
-	if (_peerState != READY)
-	{
-		context->yield();
-		tearDown();
-		throw Exception(SITE2SITE_EXCEPTION, "Can not establish handshake with peer");
-		return;
-	}
-
-	// Create the transaction
-	std::string transactionID;
-	transaction = createTransaction(transactionID, SEND);
-
-	if (transaction == NULL)
-	{
-		context->yield();
-		tearDown();
-		throw Exception(SITE2SITE_EXCEPTION, "Can not create transaction");
-		return;
-	}
-
-	bool continueTransaction = true;
-	uint64_t startSendingNanos = getTimeNano();
-
-	try
-	{
-		while (continueTransaction)
-		{
-			uint64_t startTime = getTimeMillis();
-			DataPacket packet(this, transaction, flow->getAttributes());
-
-			if (!send(transactionID, &packet, flow, session))
-			{
-				throw Exception(SITE2SITE_EXCEPTION, "Send Failed");
-				return;
-			}
-			logger_->log_info("Site2Site transaction %s send flow record %s",
-							transactionID.c_str(), flow->getUUIDStr().c_str());
-			uint64_t endTime = getTimeMillis();
-			std::string transitUri = peer_->getURL() + "/" + flow->getUUIDStr();
-			std::string details = "urn:nifi:" + flow->getUUIDStr() + "Remote Host=" + peer_->getHostName();
-			session->getProvenanceReporter()->send(flow, transitUri, details, endTime - startTime, false);
-			session->remove(flow);
-
-			uint64_t transferNanos = getTimeNano() - startSendingNanos;
-			if (transferNanos > _batchSendNanos)
-				break;
-
-			flow = session->get();
-			if (!flow)
-			{
-				continueTransaction = false;
-			}
-		} // while true
-
-		if (!confirm(transactionID))
-		{
-			throw Exception(SITE2SITE_EXCEPTION, "Confirm Failed");
-			return;
-		}
-		if (!complete(transactionID))
-		{
-			throw Exception(SITE2SITE_EXCEPTION, "Complete Failed");
-			return;
-		}
-		logger_->log_info("Site2Site transaction %s successfully send flow record %d, content bytes %d",
-				transactionID.c_str(), transaction->_transfers, transaction->_bytes);
-	}
-	catch (std::exception &exception)
-	{
-		if (transaction)
-			deleteTransaction(transactionID);
-		context->yield();
-		tearDown();
-		logger_->log_debug("Caught Exception %s", exception.what());
-		throw;
-	}
-	catch (...)
-	{
-		if (transaction)
-			deleteTransaction(transactionID);
-		context->yield();
-		tearDown();
-		logger_->log_debug("Caught Exception during Site2SiteClientProtocol::transferFlowFiles");
-		throw;
-	}
-
-	deleteTransaction(transactionID);
-
-	return;
+void Site2SiteClientProtocol::transferFlowFiles(
+    core::ProcessContext *context,
+    core::ProcessSession *session) {
+  std::shared_ptr<FlowFileRecord> flow = std::static_pointer_cast<FlowFileRecord>(session->get());;
+  Transaction *transaction = NULL;
+
+  if (!flow)
+    return;
+
+  if (_peerState != READY) {
+    bootstrap();
+  }
+
+  if (_peerState != READY) {
+    context->yield();
+    tearDown();
+    throw Exception(SITE2SITE_EXCEPTION,
+                    "Can not establish handshake with peer");
+    return;
+  }
+
+  // Create the transaction
+  std::string transactionID;
+  transaction = createTransaction(transactionID, SEND);
+
+  if (transaction == NULL) {
+    context->yield();
+    tearDown();
+    throw Exception(SITE2SITE_EXCEPTION, "Can not create transaction");
+    return;
+  }
+
+  bool continueTransaction = true;
+  uint64_t startSendingNanos = getTimeNano();
+
+  try {
+    while (continueTransaction) {
+      uint64_t startTime = getTimeMillis();
+      DataPacket packet(this, transaction, flow->getAttributes());
+
+      if (!send(transactionID, &packet, flow, session)) {
+        throw Exception(SITE2SITE_EXCEPTION, "Send Failed");
+        return;
+      }
+      logger_->log_info("Site2Site transaction %s send flow record %s",
+                        transactionID.c_str(), flow->getUUIDStr().c_str());
+      uint64_t endTime = getTimeMillis();
+      std::string transitUri = peer_->getURL() + "/" + flow->getUUIDStr();
+      std::string details = "urn:nifi:" + flow->getUUIDStr() + "Remote Host="
+          + peer_->getHostName();
+      session->getProvenanceReporter()->send(flow, transitUri, details,
+                                             endTime - startTime, false);
+      session->remove(flow);
+
+      uint64_t transferNanos = getTimeNano() - startSendingNanos;
+      if (transferNanos > _batchSendNanos)
+        break;
+
+      flow = std::static_pointer_cast<FlowFileRecord>(session->get());;
+      if (!flow) {
+        continueTransaction = false;
+      }
+    }  // while true
+
+    if (!confirm(transactionID)) {
+      throw Exception(SITE2SITE_EXCEPTION, "Confirm Failed");
+      return;
+    }
+    if (!complete(transactionID)) {
+      throw Exception(SITE2SITE_EXCEPTION, "Complete Failed");
+      return;
+    }
+    logger_->log_info(
+        "Site2Site transaction %s successfully send flow record %d, content bytes %d",
+        transactionID.c_str(), transaction->_transfers, transaction->_bytes);
+  } catch (std::exception &exception) {
+    if (transaction)
+      deleteTransaction(transactionID);
+    context->yield();
+    tearDown();
+    logger_->log_debug("Caught Exception %s", exception.what());
+    throw;
+  } catch (...) {
+    if (transaction)
+      deleteTransaction(transactionID);
+    context->yield();
+    tearDown();
+    logger_->log_debug(
+        "Caught Exception during Site2SiteClientProtocol::transferFlowFiles");
+    throw;
+  }
+
+  deleteTransaction(transactionID);
+
+  return;
 }
+
+
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/Site2SitePeer.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/Site2SitePeer.cpp b/libminifi/src/Site2SitePeer.cpp
index ae3cc2d..64732ac 100644
--- a/libminifi/src/Site2SitePeer.cpp
+++ b/libminifi/src/Site2SitePeer.cpp
@@ -31,6 +31,12 @@
 #include "Site2SitePeer.h"
 #include "FlowController.h"
 
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
 bool Site2SitePeer::Open() {
 
 	if (IsNullOrEmpty (host_))
@@ -53,3 +59,9 @@ void Site2SitePeer::Close() {
 	stream_->closeStream();
 }
 
+
+
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/TailFile.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/TailFile.cpp b/libminifi/src/TailFile.cpp
deleted file mode 100644
index 36b5e53..0000000
--- a/libminifi/src/TailFile.cpp
+++ /dev/null
@@ -1,269 +0,0 @@
-/**
- * @file TailFile.cpp
- * TailFile class implementation
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#include <vector>
-#include <queue>
-#include <map>
-#include <set>
-#include <sys/time.h>
-#include <sys/types.h>
-#include <sys/stat.h>
-#include <time.h>
-#include <sstream>
-#include <stdio.h>
-#include <string>
-#include <iostream>
-#include <dirent.h>
-#include <limits.h>
-#include <unistd.h>
-
-#include "utils/TimeUtil.h"
-#include "utils/StringUtils.h"
-#include "TailFile.h"
-#include "ProcessContext.h"
-#include "ProcessSession.h"
-
-const std::string TailFile::ProcessorName("TailFile");
-Property TailFile::FileName("File to Tail", "Fully-qualified filename of the file that should be tailed", "");
-Property TailFile::StateFile("State File",
-		"Specifies the file that should be used for storing state about what data has been ingested so that upon restart NiFi can resume from where it left off", "TailFileState");
-Relationship TailFile::Success("success", "All files are routed to success");
-
-void TailFile::initialize()
-{
-	//! Set the supported properties
-	std::set<Property> properties;
-	properties.insert(FileName);
-	properties.insert(StateFile);
-	setSupportedProperties(properties);
-	//! Set the supported relationships
-	std::set<Relationship> relationships;
-	relationships.insert(Success);
-	setSupportedRelationships(relationships);
-}
-
-std::string TailFile::trimLeft(const std::string& s)
-{
-	return StringUtils::trimLeft(s);
-}
-
-std::string TailFile::trimRight(const std::string& s)
-{
-	return StringUtils::trimRight(s);
-}
-
-void TailFile::parseStateFileLine(char *buf)
-{
-	char *line = buf;
-
-    while ((line[0] == ' ') || (line[0] =='\t'))
-    	++line;
-
-    char first = line[0];
-    if ((first == '\0') || (first == '#')  || (first == '\r') || (first == '\n') || (first == '='))
-    {
-    	return;
-    }
-
-    char *equal = strchr(line, '=');
-    if (equal == NULL)
-    {
-    	return;
-    }
-
-    equal[0] = '\0';
-    std::string key = line;
-
-    equal++;
-    while ((equal[0] == ' ') || (equal[0] == '\t'))
-    	++equal;
-
-    first = equal[0];
-    if ((first == '\0') || (first == '\r') || (first== '\n'))
-    {
-    	return;
-    }
-
-    std::string value = equal;
-    key = trimRight(key);
-    value = trimRight(value);
-
-    if (key == "FILENAME")
-    	this->_currentTailFileName = value;
-    if (key == "POSITION")
-    	this->_currentTailFilePosition = std::stoi(value);
-
-    return;
-}
-
-void TailFile::recoverState()
-{
-	std::ifstream file(_stateFile.c_str(), std::ifstream::in);
-	if (!file.good())
-	{
-		logger_->log_error("load state file failed %s", _stateFile.c_str());
-		return;
-	}
-	const unsigned int bufSize = 512;
-	char buf[bufSize];
-	for (file.getline(buf,bufSize); file.good(); file.getline(buf,bufSize))
-	{
-		parseStateFileLine(buf);
-	}
-}
-
-void TailFile::storeState()
-{
-	std::ofstream file(_stateFile.c_str());
-	if (!file.is_open())
-	{
-		logger_->log_error("store state file failed %s", _stateFile.c_str());
-		return;
-	}
-	file << "FILENAME=" << this->_currentTailFileName << "\n";
-	file << "POSITION=" << this->_currentTailFilePosition << "\n";
-	file.close();
-}
-
-static bool sortTailMatchedFileItem(TailMatchedFileItem i, TailMatchedFileItem j)
-{
-	return (i.modifiedTime < j.modifiedTime);
-}
-void TailFile::checkRollOver()
-{
-	struct stat statbuf;
-	std::vector<TailMatchedFileItem> matchedFiles;
-	std::string fullPath = this->_fileLocation + "/" + _currentTailFileName;
-
-	if (stat(fullPath.c_str(), &statbuf) == 0)
-	{
-		if (statbuf.st_size > this->_currentTailFilePosition)
-			// there are new input for the current tail file
-			return;
-
-		uint64_t modifiedTimeCurrentTailFile = ((uint64_t) (statbuf.st_mtime) * 1000);
-		std::string pattern = _fileName;
-		std::size_t found = _fileName.find_last_of(".");
-		if (found != std::string::npos)
-			pattern = _fileName.substr(0,found);
-		DIR *d;
-		d = opendir(this->_fileLocation.c_str());
-		if (!d)
-			return;
-		while (1)
-		{
-			struct dirent *entry;
-			entry = readdir(d);
-			if (!entry)
-				break;
-			std::string d_name = entry->d_name;
-			if (!(entry->d_type & DT_DIR))
-			{
-				std::string fileName = d_name;
-				std::string fileFullName = this->_fileLocation + "/" + d_name;
-				if (fileFullName.find(pattern) != std::string::npos && stat(fileFullName.c_str(), &statbuf) == 0)
-				{
-					if (((uint64_t) (statbuf.st_mtime) * 1000) >= modifiedTimeCurrentTailFile)
-					{
-						TailMatchedFileItem item;
-						item.fileName = fileName;
-						item.modifiedTime = ((uint64_t) (statbuf.st_mtime) * 1000);
-						matchedFiles.push_back(item);
-					}
-				}
-			}
-		}
-		closedir(d);
-
-		// Sort the list based on modified time
-		std::sort(matchedFiles.begin(), matchedFiles.end(), sortTailMatchedFileItem);
-		for (std::vector<TailMatchedFileItem>::iterator it = matchedFiles.begin(); it!=matchedFiles.end(); ++it)
-		{
-			TailMatchedFileItem item = *it;
-			if (item.fileName == _currentTailFileName)
-			{
-				++it;
-				if (it!=matchedFiles.end())
-				{
-					TailMatchedFileItem nextItem = *it;
-					logger_->log_info("TailFile File Roll Over from %s to %s", _currentTailFileName.c_str(), nextItem.fileName.c_str());
-					_currentTailFileName = nextItem.fileName;
-					_currentTailFilePosition = 0;
-					storeState();
-				}
-				break;
-			}
-		}
-	}
-	else
-		return;
-}
-
-
-void TailFile::onTrigger(ProcessContext *context, ProcessSession *session)
-{
-	std::string value;
-	if (context->getProperty(FileName.getName(), value))
-	{
-		std::size_t found = value.find_last_of("/\\");
-		this->_fileLocation = value.substr(0,found);
-		this->_fileName = value.substr(found+1);
-	}
-	if (context->getProperty(StateFile.getName(), value))
-	{
-		_stateFile = value + "." + getUUIDStr();
-	}
-	if (!this->_stateRecovered)
-	{
-		_stateRecovered = true;
-		this->_currentTailFileName = _fileName;
-		this->_currentTailFilePosition = 0;
-		// recover the state if we have not done so
-		this->recoverState();
-	}
-	checkRollOver();
-	std::string fullPath = this->_fileLocation + "/" + _currentTailFileName;
-	struct stat statbuf;
-	if (stat(fullPath.c_str(), &statbuf) == 0)
-	{
-		if (statbuf.st_size <= this->_currentTailFilePosition)
-			// there are no new input for the current tail file
-		{
-			context->yield();
-			return;
-		}
-		FlowFileRecord *flowFile = session->create();
-		if (!flowFile)
-			return;
-		std::size_t found = _currentTailFileName.find_last_of(".");
-		std::string baseName = _currentTailFileName.substr(0,found);
-		std::string extension = _currentTailFileName.substr(found+1);
-		flowFile->updateAttribute(PATH, _fileLocation);
-		flowFile->addAttribute(ABSOLUTE_PATH, fullPath);
-		session->import(fullPath, flowFile, true, this->_currentTailFilePosition);
-		session->transfer(flowFile, Success);
-		logger_->log_info("TailFile %s for %d bytes", _currentTailFileName.c_str(), flowFile->getSize());
-		std::string logName = baseName + "." + std::to_string(_currentTailFilePosition) + "-" +
-				std::to_string(_currentTailFilePosition + flowFile->getSize()) + "." + extension;
-		flowFile->updateAttribute(FILENAME, logName);
-		this->_currentTailFilePosition += flowFile->getSize();
-		storeState();
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/ThreadedSchedulingAgent.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/ThreadedSchedulingAgent.cpp b/libminifi/src/ThreadedSchedulingAgent.cpp
index 2008fec..6c04281 100644
--- a/libminifi/src/ThreadedSchedulingAgent.cpp
+++ b/libminifi/src/ThreadedSchedulingAgent.cpp
@@ -22,103 +22,125 @@
 
 #include "ThreadedSchedulingAgent.h"
 
-#include "ProcessContext.h"
-#include "ProcessSession.h"
-#include "ProcessSessionFactory.h"
-
-void ThreadedSchedulingAgent::schedule(Processor *processor)
-{
-	std::lock_guard<std::mutex> lock(_mtx);
-
-	_administrativeYieldDuration = 0;
-	std::string yieldValue;
-
-	if (configure_->get(Configure::nifi_administrative_yield_duration, yieldValue))
-	{
-		TimeUnit unit;
-		if (Property::StringToTime(yieldValue, _administrativeYieldDuration, unit) &&
-					Property::ConvertTimeUnitToMS(_administrativeYieldDuration, unit, _administrativeYieldDuration))
-		{
-			logger_->log_debug("nifi_administrative_yield_duration: [%d] ms", _administrativeYieldDuration);
-		}
-	}
-
-	_boredYieldDuration = 0;
-	if (configure_->get(Configure::nifi_bored_yield_duration, yieldValue))
-	{
-		TimeUnit unit;
-		if (Property::StringToTime(yieldValue, _boredYieldDuration, unit) &&
-					Property::ConvertTimeUnitToMS(_boredYieldDuration, unit, _boredYieldDuration))
-		{
-			logger_->log_debug("nifi_bored_yield_duration: [%d] ms", _boredYieldDuration);
-		}
-	}
-
-	if (processor->getScheduledState() != RUNNING)
-	{
-		logger_->log_info("Can not schedule threads for processor %s because it is not running", processor->getName().c_str());
-		return;
-	}
-
-	std::map<std::string, std::vector<std::thread *>>::iterator it =
-			_threads.find(processor->getUUIDStr());
-	if (it != _threads.end())
-	{
-		logger_->log_info("Can not schedule threads for processor %s because there are existing threads running");
-		return;
-	}
-
-	auto processContext = std::make_shared<ProcessContext>(processor);
-	auto sessionFactory = std::make_shared<ProcessSessionFactory>(processContext.get());
-
-	processor->onSchedule(processContext.get(), sessionFactory.get());
-
-	std::vector<std::thread *> threads;
-	for (int i = 0; i < processor->getMaxConcurrentTasks(); i++)
-	{
-	    ThreadedSchedulingAgent *agent = this;
-		std::thread *thread = new std::thread([agent, processor, processContext, sessionFactory] () {
-			agent->run(processor, processContext.get(), sessionFactory.get());
-		});
-		thread->detach();
-		threads.push_back(thread);
-		logger_->log_info("Scheduled thread %d running for process %s", thread->get_id(),
-				processor->getName().c_str());
-	}
-	_threads[processor->getUUIDStr().c_str()] = threads;
-
-	return;
+#include "core/Connectable.h"
+#include "core/ProcessorNode.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/ProcessSessionFactory.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+void ThreadedSchedulingAgent::schedule(
+    std::shared_ptr<core::Processor> processor) {
+  std::lock_guard < std::mutex > lock(mutex_);
+
+  _administrativeYieldDuration = 0;
+  std::string yieldValue;
+
+  if (configure_->get(Configure::nifi_administrative_yield_duration,
+                      yieldValue)) {
+    core::TimeUnit unit;
+    if (core::Property::StringToTime(
+        yieldValue, _administrativeYieldDuration, unit)
+        && core::Property::ConvertTimeUnitToMS(
+            _administrativeYieldDuration, unit, _administrativeYieldDuration)) {
+      logger_->log_debug("nifi_administrative_yield_duration: [%d] ms",
+                         _administrativeYieldDuration);
+    }
+  }
+
+  _boredYieldDuration = 0;
+  if (configure_->get(Configure::nifi_bored_yield_duration, yieldValue)) {
+    core::TimeUnit unit;
+    if (core::Property::StringToTime(
+        yieldValue, _boredYieldDuration, unit)
+        && core::Property::ConvertTimeUnitToMS(
+            _boredYieldDuration, unit, _boredYieldDuration)) {
+      logger_->log_debug("nifi_bored_yield_duration: [%d] ms",
+                         _boredYieldDuration);
+    }
+  }
+
+  if (processor->getScheduledState() != core::RUNNING) {
+    logger_->log_info(
+        "Can not schedule threads for processor %s because it is not running",
+        processor->getName().c_str());
+    return;
+  }
+
+  std::map<std::string, std::vector<std::thread *>>::iterator it =
+      _threads.find(processor->getUUIDStr());
+  if (it != _threads.end()) {
+    logger_->log_info(
+        "Can not schedule threads for processor %s because there are existing threads running");
+    return;
+  }
+
+  core::ProcessorNode processor_node(processor);
+  auto processContext = std::make_shared
+      < core::ProcessContext > (processor_node,repo_);
+  auto sessionFactory = std::make_shared
+      < core::ProcessSessionFactory
+      > (processContext.get());
+
+  processor->onSchedule(processContext.get(), sessionFactory.get());
+
+  std::vector<std::thread *> threads;
+  for (int i = 0; i < processor->getMaxConcurrentTasks(); i++) {
+    ThreadedSchedulingAgent *agent = this;
+    std::thread *thread = new std::thread(
+        [agent, processor, processContext, sessionFactory] () {
+          agent->run(processor, processContext.get(), sessionFactory.get());
+        });
+    thread->detach();
+    threads.push_back(thread);
+    logger_->log_info("Scheduled thread %d running for process %s",
+                      thread->get_id(), processor->getName().c_str());
+  }
+  _threads[processor->getUUIDStr().c_str()] = threads;
+
+  return;
 }
 
-void ThreadedSchedulingAgent::unschedule(Processor *processor)
-{
-	std::lock_guard<std::mutex> lock(_mtx);
-	
-	logger_->log_info("Shutting down threads for processor %s/%s",
-			processor->getName().c_str(),
-			processor->getUUIDStr().c_str());
-
-	if (processor->getScheduledState() != RUNNING)
-	{
-		logger_->log_info("Cannot unschedule threads for processor %s because it is not running", processor->getName().c_str());
-		return;
-	}
-
-	std::map<std::string, std::vector<std::thread *>>::iterator it =
-			_threads.find(processor->getUUIDStr());
-
-	if (it == _threads.end())
-	{
-		logger_->log_info("Cannot unschedule threads for processor %s because there are no existing threads running", processor->getName().c_str());
-		return;
-	}
-	for (std::vector<std::thread *>::iterator itThread = it->second.begin(); itThread != it->second.end(); ++itThread)
-	{
-		std::thread *thread = *itThread;
-		logger_->log_info("Scheduled thread %d deleted for process %s", thread->get_id(),
-				processor->getName().c_str());
-		delete thread;
-	}
-	_threads.erase(processor->getUUIDStr());
-	processor->clearActiveTask();
+void ThreadedSchedulingAgent::unschedule(std::shared_ptr<core::Processor> processor) {
+  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) {
+    logger_->log_info(
+        "Cannot unschedule threads for processor %s because it is not running",
+        processor->getName().c_str());
+    return;
+  }
+
+  std::map<std::string, std::vector<std::thread *>>::iterator it =
+      _threads.find(processor->getUUIDStr());
+
+  if (it == _threads.end()) {
+    logger_->log_info(
+        "Cannot unschedule threads for processor %s because there are no existing threads running",
+        processor->getName().c_str());
+    return;
+  }
+  for (std::vector<std::thread *>::iterator itThread = it->second.begin();
+      itThread != it->second.end(); ++itThread) {
+    std::thread *thread = *itThread;
+    logger_->log_info("Scheduled thread %d deleted for process %s",
+                      thread->get_id(), processor->getName().c_str());
+    delete thread;
+  }
+  _threads.erase(processor->getUUIDStr());
+  processor->clearActiveTask();
 }
+
+
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/TimerDrivenSchedulingAgent.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/TimerDrivenSchedulingAgent.cpp b/libminifi/src/TimerDrivenSchedulingAgent.cpp
index 30dc96c..3895e81 100644
--- a/libminifi/src/TimerDrivenSchedulingAgent.cpp
+++ b/libminifi/src/TimerDrivenSchedulingAgent.cpp
@@ -20,26 +20,38 @@
 #include <chrono>
 #include <thread>
 #include <iostream>
-#include "Property.h"
 #include "TimerDrivenSchedulingAgent.h"
+#include "core/Property.h"
 
-void TimerDrivenSchedulingAgent::run(Processor *processor, ProcessContext *processContext, ProcessSessionFactory *sessionFactory)
-{
-	while (this->_running)
-	{
-		bool shouldYield = this->onTrigger(processor, processContext, sessionFactory);
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
 
-		if (processor->isYield())
-		{
-			// Honor the yield
-			std::this_thread::sleep_for(std::chrono::milliseconds(processor->getYieldTime()));
-		}
-		else if (shouldYield && this->_boredYieldDuration > 0)
-		{
-			// No work to do or need to apply back pressure
-			std::this_thread::sleep_for(std::chrono::milliseconds(this->_boredYieldDuration));
-		}
-		std::this_thread::sleep_for(std::chrono::nanoseconds(processor->getSchedulingPeriodNano()));
-	}
-	return;
+void TimerDrivenSchedulingAgent::run(
+    std::shared_ptr<core::Processor> processor,
+    core::ProcessContext *processContext,
+    core::ProcessSessionFactory *sessionFactory) {
+  while (this->running_) {
+    bool shouldYield = this->onTrigger(processor, processContext,
+                                       sessionFactory);
+
+    if (processor->isYield()) {
+      // Honor the yield
+      std::this_thread::sleep_for(
+          std::chrono::milliseconds(processor->getYieldTime()));
+    } else if (shouldYield && this->_boredYieldDuration > 0) {
+      // No work to do or need to apply back pressure
+      std::this_thread::sleep_for(
+          std::chrono::milliseconds(this->_boredYieldDuration));
+    }
+    std::this_thread::sleep_for(
+        std::chrono::nanoseconds(processor->getSchedulingPeriodNano()));
+  }
+  return;
 }
+
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */


[13/16] nifi-minifi-cpp git commit: MINIFI-217: Updates namespaces and removes use of raw pointers for user facing API.

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/Site2SitePeer.h
----------------------------------------------------------------------
diff --git a/libminifi/include/Site2SitePeer.h b/libminifi/include/Site2SitePeer.h
index e89bb74..de3a42f 100644
--- a/libminifi/include/Site2SitePeer.h
+++ b/libminifi/include/Site2SitePeer.h
@@ -29,242 +29,261 @@
 #include <mutex>
 #include <atomic>
 #include <memory>
-#include "Logger.h"
-#include "Configure.h"
-#include "Property.h"
+
+#include "core/Property.h"
+#include "core/logging/Logger.h"
+#include "properties/Configure.h"
 #include "io/ClientSocket.h"
 #include "io/BaseStream.h"
 #include "utils/TimeUtil.h"
 
 
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
 
 static const char MAGIC_BYTES[] = { 'N', 'i', 'F', 'i' };
 
-//! Site2SitePeer Class
-class Site2SitePeer : public BaseStream{
-public:
-	
+// Site2SitePeer Class
+class Site2SitePeer : public org::apache::nifi::minifi::io::BaseStream {
+ public:
+
+  Site2SitePeer()
+      : stream_(nullptr),
+        host_(""),
+        port_(-1) {
+
+  }
+  /*
+   * Create a new site2site peer
+   */
+  explicit Site2SitePeer(
+      std::unique_ptr<org::apache::nifi::minifi::io::DataStream> injected_socket,
+      const std::string host_, uint16_t port_)
+      : host_(host_),
+        port_(port_),
+        stream_(injected_socket.release()) {
+    logger_ = logging::Logger::getLogger();
+    configure_ = Configure::getConfigure();
+    _yieldExpiration = 0;
+    _timeOut = 30000;  // 30 seconds
+    _url = "nifi://" + host_ + ":" + std::to_string(port_);
+  }
 
-	Site2SitePeer() : stream_(nullptr),host_(""),port_(-1){
-	  
-	}
-	/*
-	 * Create a new site2site peer
-	 */
-	explicit Site2SitePeer(std::unique_ptr<DataStream> injected_socket, const std::string host_, uint16_t port_ ) :
-		host_(host_), port_(port_), stream_(injected_socket.release()){
-		logger_ = Logger::getLogger();
-		configure_ = Configure::getConfigure();
-		_yieldExpiration = 0;
-		_timeOut = 30000; // 30 seconds
-		_url = "nifi://" + host_ + ":" + std::to_string(port_);
-	}
-	
-	explicit Site2SitePeer(Site2SitePeer &&ss) : stream_( ss.stream_.release()), host_( std::move(ss.host_) ), port_ (std::move(ss.port_) )
-	{
-	  logger_ = Logger::getLogger();
-	  configure_ = Configure::getConfigure();
-	  _yieldExpiration.store(ss._yieldExpiration);
-	  _timeOut.store(ss._timeOut);
-	  _url = std::move(ss._url);
-	}
-	//! Destructor
-	virtual ~Site2SitePeer() {
-		Close();
-	}
-	//! Set Processor yield period in MilliSecond
-	void setYieldPeriodMsec(uint64_t period) {
-		_yieldPeriodMsec = period;
-	}
-	//! get URL
-	std::string getURL() {
-		return _url;
-	}
-	//! Get Processor yield period in MilliSecond
-	uint64_t getYieldPeriodMsec(void) {
-		return (_yieldPeriodMsec);
-	}
-	//! Yield based on the yield period
-	void yield() {
-		_yieldExpiration = (getTimeMillis() + _yieldPeriodMsec);
-	}
-	//! setHostName
-	void setHostName(std::string host_) {
-		this->host_ = host_;
-		_url = "nifi://" + host_ + ":" + std::to_string(port_);
-	}
-	//! setPort
-	void setPort(uint16_t port_) {
-		this->port_ = port_;
-		_url = "nifi://" + host_ + ":" + std::to_string(port_);
-	}
-	//! getHostName
-	std::string getHostName() {
-		return host_;
-	}
-	//! getPort
-	uint16_t getPort() {
-		return port_;
-	}
-	//! Yield based on the input time
-	void yield(uint64_t time) {
-		_yieldExpiration = (getTimeMillis() + time);
-	}
-	//! whether need be to yield
-	bool isYield() {
-		if (_yieldExpiration > 0)
-			return (_yieldExpiration >= getTimeMillis());
-		else
-			return false;
-	}
-	// clear yield expiration
-	void clearYield() {
-		_yieldExpiration = 0;
-	}
-	//! Yield based on the yield period
-	void yield(std::string portId) {
-		std::lock_guard<std::mutex> lock(_mtx);
-		uint64_t yieldExpiration = (getTimeMillis() + _yieldPeriodMsec);
-		_yieldExpirationPortIdMap[portId] = yieldExpiration;
-	}
-	//! Yield based on the input time
-	void yield(std::string portId, uint64_t time) {
-		std::lock_guard<std::mutex> lock(_mtx);
-		uint64_t yieldExpiration = (getTimeMillis() + time);
-		_yieldExpirationPortIdMap[portId] = yieldExpiration;
-	}
-	//! whether need be to yield
-	bool isYield(std::string portId) {
-		std::lock_guard<std::mutex> lock(_mtx);
-		std::map<std::string, uint64_t>::iterator it =
-				this->_yieldExpirationPortIdMap.find(portId);
-		if (it != _yieldExpirationPortIdMap.end()) {
-			uint64_t yieldExpiration = it->second;
-			return (yieldExpiration >= getTimeMillis());
-		} else {
-			return false;
-		}
-	}
-	//! clear yield expiration
-	void clearYield(std::string portId) {
-		std::lock_guard<std::mutex> lock(_mtx);
-		std::map<std::string, uint64_t>::iterator it =
-				this->_yieldExpirationPortIdMap.find(portId);
-		if (it != _yieldExpirationPortIdMap.end()) {
-			_yieldExpirationPortIdMap.erase(portId);
-		}
-	}
-	//! setTimeOut
-	void setTimeOut(uint64_t time) {
-		_timeOut = time;
-	}
-	//! getTimeOut
-	uint64_t getTimeOut() {
-		return _timeOut;
-	}
-	int write(uint8_t value) {
-		return Serializable::write(value,stream_.get());
-	}
-	int write(char value) {
-		return Serializable::write(value,stream_.get());
-	}
-	int write(uint32_t value) {
+  explicit Site2SitePeer(Site2SitePeer &&ss)
+      : stream_(ss.stream_.release()),
+        host_(std::move(ss.host_)),
+        port_(std::move(ss.port_)) {
+    logger_ = logging::Logger::getLogger();
+    configure_ = Configure::getConfigure();
+    _yieldExpiration.store(ss._yieldExpiration);
+    _timeOut.store(ss._timeOut);
+    _url = std::move(ss._url);
+  }
+  // Destructor
+  virtual ~Site2SitePeer() {
+    Close();
+  }
+  // Set Processor yield period in MilliSecond
+  void setYieldPeriodMsec(uint64_t period) {
+    _yieldPeriodMsec = period;
+  }
+  // get URL
+  std::string getURL() {
+    return _url;
+  }
+  // Get Processor yield period in MilliSecond
+  uint64_t getYieldPeriodMsec(void) {
+    return (_yieldPeriodMsec);
+  }
+  // Yield based on the yield period
+  void yield() {
+    _yieldExpiration = (getTimeMillis() + _yieldPeriodMsec);
+  }
+  // setHostName
+  void setHostName(std::string host_) {
+    this->host_ = host_;
+    _url = "nifi://" + host_ + ":" + std::to_string(port_);
+  }
+  // setPort
+  void setPort(uint16_t port_) {
+    this->port_ = port_;
+    _url = "nifi://" + host_ + ":" + std::to_string(port_);
+  }
+  // getHostName
+  std::string getHostName() {
+    return host_;
+  }
+  // getPort
+  uint16_t getPort() {
+    return port_;
+  }
+  // Yield based on the input time
+  void yield(uint64_t time) {
+    _yieldExpiration = (getTimeMillis() + time);
+  }
+  // whether need be to yield
+  bool isYield() {
+    if (_yieldExpiration > 0)
+      return (_yieldExpiration >= getTimeMillis());
+    else
+      return false;
+  }
+  // clear yield expiration
+  void clearYield() {
+    _yieldExpiration = 0;
+  }
+  // Yield based on the yield period
+  void yield(std::string portId) {
+    std::lock_guard<std::mutex> lock(mutex_);
+    uint64_t yieldExpiration = (getTimeMillis() + _yieldPeriodMsec);
+    _yieldExpirationPortIdMap[portId] = yieldExpiration;
+  }
+  // Yield based on the input time
+  void yield(std::string portId, uint64_t time) {
+    std::lock_guard<std::mutex> lock(mutex_);
+    uint64_t yieldExpiration = (getTimeMillis() + time);
+    _yieldExpirationPortIdMap[portId] = yieldExpiration;
+  }
+  // whether need be to yield
+  bool isYield(std::string portId) {
+    std::lock_guard<std::mutex> lock(mutex_);
+    std::map<std::string, uint64_t>::iterator it = this
+        ->_yieldExpirationPortIdMap.find(portId);
+    if (it != _yieldExpirationPortIdMap.end()) {
+      uint64_t yieldExpiration = it->second;
+      return (yieldExpiration >= getTimeMillis());
+    } else {
+      return false;
+    }
+  }
+  // clear yield expiration
+  void clearYield(std::string portId) {
+    std::lock_guard<std::mutex> lock(mutex_);
+    std::map<std::string, uint64_t>::iterator it = this
+        ->_yieldExpirationPortIdMap.find(portId);
+    if (it != _yieldExpirationPortIdMap.end()) {
+      _yieldExpirationPortIdMap.erase(portId);
+    }
+  }
+  // setTimeOut
+  void setTimeOut(uint64_t time) {
+    _timeOut = time;
+  }
+  // getTimeOut
+  uint64_t getTimeOut() {
+    return _timeOut;
+  }
+  int write(uint8_t value) {
+    return Serializable::write(value, stream_.get());
+  }
+  int write(char value) {
+    return Serializable::write(value, stream_.get());
+  }
+  int write(uint32_t value) {
 
-		return Serializable::write(value,stream_.get());
+    return Serializable::write(value, stream_.get());
 
-	}
-	int write(uint16_t value) {
-		return Serializable::write(value,stream_.get());
-	}
-	int write(uint8_t *value, int len) {
-		return Serializable::write(value,len,stream_.get());
-	}
-	int write(uint64_t value) {
-		return Serializable::write(value,stream_.get());
-	}
-	int write(bool value) {
-		uint8_t temp = value;
-		return Serializable::write(temp,stream_.get());
-	}
-	int writeUTF(std::string str, bool widen = false){
-		return Serializable::writeUTF(str,stream_.get(),widen);
-	}
-	int read(uint8_t &value) {
-		return Serializable::read(value,stream_.get());
-	}
-	int read(uint16_t &value) {
-		return Serializable::read(value,stream_.get());
-	}
-	int read(char &value) {
-		return Serializable::read(value,stream_.get());
-	}
-	int read(uint8_t *value, int len) {
-		return Serializable::read(value,len,stream_.get());
-	}
-	int read(uint32_t &value) {
-		return Serializable::read(value,stream_.get());
-	}
-	int read(uint64_t &value) {
-		return Serializable::read(value,stream_.get());
-	}
-	int readUTF(std::string &str, bool widen = false)
-	{
-		return Serializable::readUTF(str,stream_.get(),widen);
-	}
-	//! open connection to the peer
-	bool Open();
-	//! close connection to the peer
-	void Close();
-	
-	/**
-	 * Move assignment operator.
-	 */
-	Site2SitePeer& operator=(Site2SitePeer&& other)
-	{
-	  stream_ = std::unique_ptr<DataStream>( other.stream_.release());
-	  host_ = std::move(other.host_);
-	  port_ = std::move(other.port_);
-	  logger_ = Logger::getLogger();
-	  configure_ = Configure::getConfigure();
-	  _yieldExpiration = 0;
-	  _timeOut = 30000; // 30 seconds
-	  _url = "nifi://" + host_ + ":" + std::to_string(port_);
-	  
-	  return *this;
-	}
+  }
+  int write(uint16_t value) {
+    return Serializable::write(value, stream_.get());
+  }
+  int write(uint8_t *value, int len) {
+    return Serializable::write(value, len, stream_.get());
+  }
+  int write(uint64_t value) {
+    return Serializable::write(value, stream_.get());
+  }
+  int write(bool value) {
+    uint8_t temp = value;
+    return Serializable::write(temp, stream_.get());
+  }
+  int writeUTF(std::string str, bool widen = false) {
+    return Serializable::writeUTF(str, stream_.get(), widen);
+  }
+  int read(uint8_t &value) {
+    return Serializable::read(value, stream_.get());
+  }
+  int read(uint16_t &value) {
+    return Serializable::read(value, stream_.get());
+  }
+  int read(char &value) {
+    return Serializable::read(value, stream_.get());
+  }
+  int read(uint8_t *value, int len) {
+    return Serializable::read(value, len, stream_.get());
+  }
+  int read(uint32_t &value) {
+    return Serializable::read(value, stream_.get());
+  }
+  int read(uint64_t &value) {
+    return Serializable::read(value, stream_.get());
+  }
+  int readUTF(std::string &str, bool widen = false) {
+    return org::apache::nifi::minifi::io::Serializable::readUTF(str,
+                                                                stream_.get(),
+                                                                widen);
+  }
+  // open connection to the peer
+  bool Open();
+  // close connection to the peer
+  void Close();
 
-	Site2SitePeer(const Site2SitePeer &parent) = delete;
-	Site2SitePeer &operator=(const Site2SitePeer &parent) = delete;
+  /**
+   * Move assignment operator.
+   */
+  Site2SitePeer& operator=(Site2SitePeer&& other) {
+    stream_ = std::unique_ptr<org::apache::nifi::minifi::io::DataStream>(
+        other.stream_.release());
+    host_ = std::move(other.host_);
+    port_ = std::move(other.port_);
+    logger_ = logging::Logger::getLogger();
+    configure_ = Configure::getConfigure();
+    _yieldExpiration = 0;
+    _timeOut = 30000;  // 30 seconds
+    _url = "nifi://" + host_ + ":" + std::to_string(port_);
 
-protected:
+    return *this;
+  }
 
-private:
+  Site2SitePeer(const Site2SitePeer &parent) = delete;
+  Site2SitePeer &operator=(const Site2SitePeer &parent) = delete;
 
-	std::unique_ptr<DataStream> stream_;
+ protected:
 
-	std::string host_;
-	uint16_t port_;
+ private:
 
-	//! Mutex for protection
-	std::mutex _mtx;
-	//! URL
-	std::string _url;
-	//! socket timeout;
-	std::atomic<uint64_t> _timeOut;
-	//! Logger
-	std::shared_ptr<Logger> logger_;
-	//! Configure
-	Configure *configure_;
-	//! Yield Period in Milliseconds
-	std::atomic<uint64_t> _yieldPeriodMsec;
-	//! Yield Expiration
-	std::atomic<uint64_t> _yieldExpiration;
-	//! Yield Expiration per destination PortID
-	std::map<std::string, uint64_t> _yieldExpirationPortIdMap;
-	//! OpenSSL connection state
-	// Prevent default copy constructor and assignment operation
-	// Only support pass by reference or pointer
+  std::unique_ptr<org::apache::nifi::minifi::io::DataStream> stream_;
+
+  std::string host_;
+  uint16_t port_;
+
+  // Mutex for protection
+  std::mutex mutex_;
+  // URL
+  std::string _url;
+  // socket timeout;
+  std::atomic<uint64_t> _timeOut;
+  // Logger
+  std::shared_ptr<logging::Logger> logger_;
+  // Configure
+  Configure *configure_;
+  // Yield Period in Milliseconds
+  std::atomic<uint64_t> _yieldPeriodMsec;
+  // Yield Expiration
+  std::atomic<uint64_t> _yieldExpiration;
+  // Yield Expiration per destination PortID
+  std::map<std::string, uint64_t> _yieldExpirationPortIdMap;
+  // OpenSSL connection state
+  // Prevent default copy constructor and assignment operation
+  // Only support pass by reference or pointer
 
 };
 
+
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
 #endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/TailFile.h
----------------------------------------------------------------------
diff --git a/libminifi/include/TailFile.h b/libminifi/include/TailFile.h
deleted file mode 100644
index d68748e..0000000
--- a/libminifi/include/TailFile.h
+++ /dev/null
@@ -1,93 +0,0 @@
-/**
- * @file TailFile.h
- * TailFile class declaration
- *
- * 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 __TAIL_FILE_H__
-#define __TAIL_FILE_H__
-
-#include "FlowFileRecord.h"
-#include "Processor.h"
-#include "ProcessSession.h"
-
-//! TailFile Class
-class TailFile : public Processor
-{
-public:
-	//! Constructor
-	/*!
-	 * Create a new processor
-	 */
-	TailFile(std::string name, uuid_t uuid = NULL)
-	: Processor(name, uuid)
-	{
-		logger_ = Logger::getLogger();
-		_stateRecovered = false;
-	}
-	//! Destructor
-	virtual ~TailFile()
-	{
-		storeState();
-	}
-	//! Processor Name
-	static const std::string ProcessorName;
-	//! Supported Properties
-	static Property FileName;
-	static Property StateFile;
-	//! Supported Relationships
-	static Relationship Success;
-
-public:
-	//! OnTrigger method, implemented by NiFi TailFile
-	virtual void onTrigger(ProcessContext *context, ProcessSession *session);
-	//! Initialize, over write by NiFi TailFile
-	virtual void initialize(void);
-	//! recoverState
-	void recoverState();
-	//! storeState
-	void storeState();
-
-protected:
-
-private:
-	//! Logger
-	std::shared_ptr<Logger> logger_;
-	std::string _fileLocation;
-	//! Property Specified Tailed File Name
-	std::string _fileName;
-	//! File to save state
-	std::string _stateFile;
-	//! State related to the tailed file
-	std::string _currentTailFileName;
-	uint64_t _currentTailFilePosition;
-	bool _stateRecovered;
-	uint64_t _currentTailFileCreatedTime;
-	//! Utils functions for parse state file
-	std::string trimLeft(const std::string& s);
-	std::string trimRight(const std::string& s);
-	void parseStateFileLine(char *buf);
-	void checkRollOver();
-
-};
-
-//! Matched File Item for Roll over check
-typedef struct {
-	std::string fileName;
-	uint64_t modifiedTime;
-} TailMatchedFileItem;
-
-#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/ThreadedSchedulingAgent.h
----------------------------------------------------------------------
diff --git a/libminifi/include/ThreadedSchedulingAgent.h b/libminifi/include/ThreadedSchedulingAgent.h
index 5eb5d8a..4e39da3 100644
--- a/libminifi/include/ThreadedSchedulingAgent.h
+++ b/libminifi/include/ThreadedSchedulingAgent.h
@@ -20,51 +20,60 @@
 #ifndef __THREADED_SCHEDULING_AGENT_H__
 #define __THREADED_SCHEDULING_AGENT_H__
 
-#include "Configure.h"
-#include "Logger.h"
-#include "Processor.h"
-#include "ProcessContext.h"
+#include "properties/Configure.h"
+#include "core/logging/Logger.h"
+#include "core/Processor.h"
+#include "core/Repository.h"
+#include "core/ProcessContext.h"
 #include "SchedulingAgent.h"
 
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
 /**
  * An abstract scheduling agent which creates and manages a pool of threads for
  * each processor scheduled.
  */
-class ThreadedSchedulingAgent : public SchedulingAgent
-{
-public:
-	//! Constructor
-	/*!
-	 * Create a new processor
-	 */
-	ThreadedSchedulingAgent()
-	: SchedulingAgent()
-	{
-	}
-	//! Destructor
-	virtual ~ThreadedSchedulingAgent()
-	{
-	}
+class ThreadedSchedulingAgent : public SchedulingAgent {
+ public:
+  // Constructor
+  /*!
+   * Create a new processor
+   */
+  ThreadedSchedulingAgent(std::shared_ptr<core::Repository> repo)
+      : SchedulingAgent(repo) {
+  }
+  // Destructor
+  virtual ~ThreadedSchedulingAgent() {
+  }
 
-	//! Run function for the thread
-	virtual void run(Processor *processor, ProcessContext *processContext, ProcessSessionFactory *sessionFactory) = 0;
+  // Run function for the thread
+  virtual void run(std::shared_ptr<core::Processor> processor,
+                   core::ProcessContext *processContext,
+                   core::ProcessSessionFactory *sessionFactory) = 0;
 
-public:
-	//! schedule, overwritten by different DrivenTimerDrivenSchedulingAgent
-	virtual void schedule(Processor *processor);
-	//! unschedule, overwritten by different DrivenTimerDrivenSchedulingAgent
-	virtual void unschedule(Processor *processor);
+ public:
+  // schedule, overwritten by different DrivenTimerDrivenSchedulingAgent
+  virtual void schedule(std::shared_ptr<core::Processor> processor);
+  // unschedule, overwritten by different DrivenTimerDrivenSchedulingAgent
+  virtual void unschedule(std::shared_ptr<core::Processor> processor);
 
-protected:
-	//! Threads
-	std::map<std::string, std::vector<std::thread *>> _threads;
+ protected:
+  // Threads
+  std::map<std::string, std::vector<std::thread *>> _threads;
 
-private:
-	// Prevent default copy constructor and assignment operation
-	// Only support pass by reference or pointer
-	ThreadedSchedulingAgent(const ThreadedSchedulingAgent &parent);
-	ThreadedSchedulingAgent &operator=(const ThreadedSchedulingAgent &parent);
+ private:
+  // Prevent default copy constructor and assignment operation
+  // Only support pass by reference or pointer
+  ThreadedSchedulingAgent(const ThreadedSchedulingAgent &parent);
+  ThreadedSchedulingAgent &operator=(const ThreadedSchedulingAgent &parent);
 
 };
 
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
 #endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/TimerDrivenSchedulingAgent.h
----------------------------------------------------------------------
diff --git a/libminifi/include/TimerDrivenSchedulingAgent.h b/libminifi/include/TimerDrivenSchedulingAgent.h
index 389ccf6..7da2abd 100644
--- a/libminifi/include/TimerDrivenSchedulingAgent.h
+++ b/libminifi/include/TimerDrivenSchedulingAgent.h
@@ -20,36 +20,47 @@
 #ifndef __TIMER_DRIVEN_SCHEDULING_AGENT_H__
 #define __TIMER_DRIVEN_SCHEDULING_AGENT_H__
 
-#include "Logger.h"
-#include "Processor.h"
-#include "ProcessContext.h"
+#include "core/logging/Logger.h"
+#include "core/Processor.h"
+#include "core/ProcessContext.h"
+#include "core/Repository.h"
 #include "ThreadedSchedulingAgent.h"
 
-//! TimerDrivenSchedulingAgent Class
-class TimerDrivenSchedulingAgent : public ThreadedSchedulingAgent
-{
-public:
-	//! Constructor
-	/*!
-	 * Create a new processor
-	 */
-	TimerDrivenSchedulingAgent()
-	: ThreadedSchedulingAgent()
-	{
-	}
-	//! Destructor
-	virtual ~TimerDrivenSchedulingAgent()
-	{
-	}
-	//! Run function for the thread
-	void run(Processor *processor, ProcessContext *processContext, ProcessSessionFactory *sessionFactory);
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+//  TimerDrivenSchedulingAgent Class
+class TimerDrivenSchedulingAgent : public ThreadedSchedulingAgent {
+ public:
+  //  Constructor
+  /*!
+   * Create a new processor
+   */
+  TimerDrivenSchedulingAgent(std::shared_ptr<core::Repository> repo)
+      : ThreadedSchedulingAgent(repo) {
+  }
+  //  Destructor
+  virtual ~TimerDrivenSchedulingAgent() {
+  }
+  /**
+   * Run function that accepts the processor, context and session factory.
+   */
+  void run(std::shared_ptr<core::Processor> processor,
+           core::ProcessContext *processContext,
+           core::ProcessSessionFactory *sessionFactory);
 
-private:
-	// Prevent default copy constructor and assignment operation
-	// Only support pass by reference or pointer
-	TimerDrivenSchedulingAgent(const TimerDrivenSchedulingAgent &parent);
-	TimerDrivenSchedulingAgent &operator=(const TimerDrivenSchedulingAgent &parent);
+ private:
+  // Prevent default copy constructor and assignment operation
+  // Only support pass by reference or pointer
+  TimerDrivenSchedulingAgent(const TimerDrivenSchedulingAgent &parent);
+  TimerDrivenSchedulingAgent &operator=(
+      const TimerDrivenSchedulingAgent &parent);
 
 };
 
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
 #endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/core/ConfigurableComponent.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ConfigurableComponent.h b/libminifi/include/core/ConfigurableComponent.h
new file mode 100644
index 0000000..c0cc623
--- /dev/null
+++ b/libminifi/include/core/ConfigurableComponent.h
@@ -0,0 +1,104 @@
+/**
+ *
+ * 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_CONFIGURABLECOMPONENT_H_
+#define LIBMINIFI_INCLUDE_CORE_CONFIGURABLECOMPONENT_H_
+
+#include "Property.h"
+#include "core/logging/Logger.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+/**
+ * Represents a configurable component
+ * Purpose: Extracts configuration items for all components and localized them
+ */
+class ConfigurableComponent {
+ public:
+
+
+  ConfigurableComponent() = delete;
+
+
+  explicit ConfigurableComponent(std::shared_ptr<logging::Logger> logger);
+
+  explicit ConfigurableComponent(const ConfigurableComponent &&other);
+
+  /**
+   * Get property using the provided name.
+   * @param name property name.
+   * @param value value passed in by reference
+   * @return result of getting property.
+   */
+  bool getProperty(const std::string name, std::string &value);
+  /**
+   * Sets the property using the provided name
+   * @param property name
+   * @param value property value.
+   * @return result of setting property.
+   */
+  bool setProperty(const std::string name, std::string value);
+  /**
+   * Sets the property using the provided name
+   * @param property name
+   * @param value property value.
+   * @return whether property was set or not
+   */
+  bool setProperty(Property &prop, std::string value);
+
+  /**
+   * Sets supported properties for the ConfigurableComponent
+   * @param supported properties
+   * @return result of set operation.
+   */
+  bool setSupportedProperties(std::set<Property> properties);
+  /**
+   * Sets supported properties for the ConfigurableComponent
+   * @param supported properties
+   * @return result of set operation.
+   */
+
+  virtual ~ConfigurableComponent();
+
+ protected:
+
+
+  /**
+   * Returns true if the instance can be edited.
+   * @return true/false
+   */
+  virtual bool canEdit()= 0;
+
+  std::mutex configuration_mutex_;
+  std::shared_ptr<logging::Logger> logger_;
+  // Supported properties
+  std::map<std::string, Property> properties_;
+
+};
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_CONFIGURABLECOMPONENT_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/core/ConfigurationFactory.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ConfigurationFactory.h b/libminifi/include/core/ConfigurationFactory.h
new file mode 100644
index 0000000..19ed5f4
--- /dev/null
+++ b/libminifi/include/core/ConfigurationFactory.h
@@ -0,0 +1,65 @@
+/**
+ *
+ * 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_CONFIGURATIONFACTORY_H_
+#define LIBMINIFI_INCLUDE_CORE_CONFIGURATIONFACTORY_H_
+
+#include "FlowConfiguration.h"
+#include  <type_traits>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+
+
+
+template<typename T>
+typename std::enable_if<!class_operations<T>::value, T*>::type instantiate(
+      std::shared_ptr<core::Repository> repo,
+      std::shared_ptr<core::Repository> flow_file_repo,const std::string path ) {
+  throw std::runtime_error("Cannot instantiate class");
+}
+
+template<typename T>
+typename std::enable_if<class_operations<T>::value, T*>::type instantiate(
+      std::shared_ptr<core::Repository> repo,
+      std::shared_ptr<core::Repository> flow_file_repo,const std::string path ) {
+  return new T(repo,flow_file_repo,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,
+      const std::string configuration_class_name, const std::string path = "",
+      bool fail_safe = false);
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_CONFIGURATIONFACTORY_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/core/Connectable.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/Connectable.h b/libminifi/include/core/Connectable.h
new file mode 100644
index 0000000..15e618f
--- /dev/null
+++ b/libminifi/include/core/Connectable.h
@@ -0,0 +1,165 @@
+/**
+ *
+ * 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_CONNECTABLE_H_
+#define LIBMINIFI_INCLUDE_CORE_CONNECTABLE_H_
+
+#include <set>
+#include "core.h"
+#include <condition_variable>
+#include "core/logging/Logger.h"
+#include "Relationship.h"
+#include "Scheduling.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+/**
+ * Represents the base connectable component
+ * Purpose: As in NiFi, this represents a connection point and allows the derived
+ * object to be connected to other connectables.
+ */
+class Connectable : public CoreComponent {
+ public:
+
+  explicit Connectable(std::string name, uuid_t uuid);
+
+  explicit Connectable(const Connectable &&other);
+
+  bool setSupportedRelationships(std::set<Relationship> relationships);
+
+  // Whether the relationship is supported
+  bool isSupportedRelationship(Relationship relationship);
+
+  /**
+   * Sets auto terminated relationships
+   * @param relationships
+   * @return result of set operation.
+   */
+  bool setAutoTerminatedRelationships(std::set<Relationship> relationships);
+
+  // Check whether the relationship is auto terminated
+  bool isAutoTerminated(Relationship relationship);
+
+  // Get Processor penalization period in MilliSecond
+  uint64_t getPenalizationPeriodMsec(void) {
+    return (_penalizationPeriodMsec);
+  }
+
+  /**
+   * Get outgoing connection based on relationship
+   * @return set of outgoing connections.
+   */
+  std::set<std::shared_ptr<Connectable>> getOutGoingConnections(
+      std::string relationship);
+
+  /**
+   * Get next incoming connection
+   * @return next incoming connection
+   */
+  std::shared_ptr<Connectable> getNextIncomingConnection();
+
+  /**
+   * @return true if incoming connections > 0
+   */
+  bool hasIncomingConnections() {
+    return (_incomingConnections.size() > 0);
+  }
+
+  uint8_t getMaxConcurrentTasks() {
+    return max_concurrent_tasks_;
+  }
+
+  void setMaxConcurrentTasks(const uint8_t tasks) {
+    max_concurrent_tasks_ = tasks;
+  }
+  /**
+   * Yield
+   */
+  virtual void yield() = 0;
+
+  virtual ~Connectable();
+
+  /**
+   * Determines if we are connected and operating
+   */
+  virtual bool isRunning() = 0;
+
+  /**
+   * Block until work is available on any input connection, or the given duration elapses
+   * @param timeoutMs timeout in milliseconds
+   */
+  void waitForWork(uint64_t timeoutMs);
+  /**
+   * Notify this processor that work may be available
+   */
+
+  void notifyWork();
+
+  /**
+   * Determines if work is available by this connectable
+   * @return boolean if work is available.
+   */
+  virtual bool isWorkAvailable() = 0;
+
+ protected:
+
+  // Penalization Period in MilliSecond
+  std::atomic<uint64_t> _penalizationPeriodMsec;
+
+  uint8_t max_concurrent_tasks_;
+
+  // Supported relationships
+  std::map<std::string, core::Relationship> relationships_;
+  // Autoterminated relationships
+  std::map<std::string, core::Relationship> auto_terminated_relationships_;
+
+  // Incoming connection Iterator
+  std::set<std::shared_ptr<Connectable>>::iterator incoming_connections_Iter;
+  // Incoming connections
+  std::set<std::shared_ptr<Connectable>> _incomingConnections;
+  // Outgoing connections map based on Relationship name
+  std::map<std::string, std::set<std::shared_ptr<Connectable>>>_outGoingConnections;
+
+  // Mutex for protection
+  std::mutex relationship_mutex_;
+
+  ///// work conditionals and locking mechanisms
+
+  // Concurrent condition mutex for whether there is incoming work to do
+  std::mutex work_available_mutex_;
+  // Condition for whether there is incoming work to do
+  std::atomic<bool> has_work_;
+  // Scheduling Strategy
+  std::atomic<SchedulingStrategy> strategy_;
+  // Concurrent condition variable for whether there is incoming work to do
+  std::condition_variable work_condition_;
+
+};
+
+}
+/* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_CONNECTABLE_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/core/FlowConfiguration.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/FlowConfiguration.h b/libminifi/include/core/FlowConfiguration.h
new file mode 100644
index 0000000..c7eedd2
--- /dev/null
+++ b/libminifi/include/core/FlowConfiguration.h
@@ -0,0 +1,118 @@
+/**
+ *
+ * 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_FLOWCONFIGURATION_H_
+#define LIBMINIFI_INCLUDE_CORE_FLOWCONFIGURATION_H_
+
+#include "core/core.h"
+#include "Connection.h"
+#include "RemoteProcessorGroupPort.h"
+#include "provenance/Provenance.h"
+#include "processors/GetFile.h"
+#include "processors/PutFile.h"
+#include "processors/TailFile.h"
+#include "processors/ListenSyslog.h"
+#include "processors/GenerateFlowFile.h"
+#include "processors/RealTimeDataCollector.h"
+#include "processors/ListenHTTP.h"
+#include "processors/LogAttribute.h"
+#include "processors/ExecuteProcess.h"
+#include "processors/AppendHostInfo.h"
+#include "core/Processor.h"
+#include "core/logging/Logger.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/ProcessGroup.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+/**
+ * Purpose: Flow configuration defines the mechanism
+ * by which we will configure our flow controller
+ */
+class FlowConfiguration : public CoreComponent {
+ public:
+  /**
+   * Constructor that will be used for configuring
+   * the flow controller.
+   */
+  FlowConfiguration(std::shared_ptr<core::Repository> repo,
+                    std::shared_ptr<core::Repository> flow_file_repo,
+                    const std::string path)
+      : CoreComponent(core::getClassName<FlowConfiguration>()),
+        flow_file_repo_(flow_file_repo),
+        config_path_(path) {
+
+  }
+
+  virtual ~FlowConfiguration();
+
+  // 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);
+  // Create Remote Processor Group
+  std::unique_ptr<core::ProcessGroup> createRemoteProcessGroup(std::string name,
+                                                               uuid_t uuid);
+  // Create Connection
+  std::shared_ptr<minifi::Connection> createConnection(std::string name,
+                                                       uuid_t uuid);
+
+  /**
+   * Returns the configuration path string
+   * @return config_path_
+   */
+  const std::string &getConfigurationPath() {
+    return config_path_;
+  }
+
+  virtual std::unique_ptr<core::ProcessGroup> getRoot() {
+    return getRoot(config_path_);
+  }
+
+  /**
+   * Base implementation that returns a null root pointer.
+   * @return Extensions should return a non-null pointer in order to
+   * properly configure flow controller.
+   */
+  virtual std::unique_ptr<core::ProcessGroup> getRoot(
+      const std::string &from_config) {
+    return nullptr;
+  }
+
+ protected:
+  // configuration path
+  std::string config_path_;
+  // flow file repo
+  std::shared_ptr<core::Repository> flow_file_repo_;
+
+};
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_FLOWCONFIGURATION_H_ */
+

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/core/FlowFile.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/FlowFile.h b/libminifi/include/core/FlowFile.h
new file mode 100644
index 0000000..247ad26
--- /dev/null
+++ b/libminifi/include/core/FlowFile.h
@@ -0,0 +1,283 @@
+/**
+ *
+ * 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 RECORD_H
+#define RECORD_H
+
+#include "utils/TimeUtil.h"
+#include "ResourceClaim.h"
+#include "Connectable.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class FlowFile {
+ public:
+  FlowFile();
+  ~FlowFile();
+  FlowFile& operator=(const FlowFile& other);
+
+  /**
+   * Returns a pointer to this flow file record's
+   * claim
+   */
+  std::shared_ptr<ResourceClaim> getResourceClaim();
+  /**
+   * Sets _claim to the inbound claim argument
+   */
+  void setResourceClaim(std::shared_ptr<ResourceClaim> &claim);
+
+  /**
+   * clear the resource claim
+   */
+  void clearResourceClaim();
+
+  /**
+   * Get lineage identifiers
+   */
+  std::set<std::string> &getlineageIdentifiers();
+
+  /**
+   * Returns whether or not this flow file record
+   * is marked as deleted.
+   * @return marked deleted
+   */
+  bool isDeleted();
+
+  /**
+   * Sets whether to mark this flow file record
+   * as deleted
+   * @param deleted deleted flag
+   */
+  void setDeleted(const bool deleted);
+
+  /**
+   * Get entry date for this record
+   * @return entry date uint64_t
+   */
+  uint64_t getEntryDate();
+
+  /**
+   * Gets the event time.
+   * @return event time.
+   */
+  uint64_t getEventTime();
+  /**
+   * Get lineage start date
+   * @return lineage start date uint64_t
+   */
+  uint64_t getlineageStartDate();
+
+  /**
+   * Sets the lineage start date
+   * @param date new lineage start date
+   */
+  void setLineageStartDate(const uint64_t date);
+
+  void setLineageIdentifiers(std::set<std::string> lineage_Identifiers) {
+    lineage_Identifiers_ = lineage_Identifiers;
+  }
+  /**
+   * Obtains an attribute if it exists. If it does the value is
+   * copied into value
+   * @param key key to look for
+   * @param value value to set
+   * @return result of finding key
+   */
+  bool getAttribute(std::string key, std::string &value);
+
+  /**
+   * Updates the value in the attribute map that corresponds
+   * to key
+   * @param key attribute name
+   * @param value value to set to attribute name
+   * @return result of finding key
+   */
+  bool updateAttribute(const std::string key, const std::string value);
+
+  /**
+   * Removes the attribute
+   * @param key attribute name to remove
+   * @return result of finding key
+   */
+  bool removeAttribute(const std::string key);
+
+  /**
+   * setAttribute, if attribute already there, update it, else, add it
+   */
+  void setAttribute(const std::string &key, const std::string &value) {
+    attributes_[key] = value;
+  }
+
+  /**
+   * Returns the map of attributes
+   * @return attributes.
+   */
+  std::map<std::string, std::string> getAttributes() {
+    return attributes_;
+  }
+
+  /**
+   * adds an attribute if it does not exist
+   *
+   */
+  bool addAttribute(const std::string &key, const std::string &value);
+
+  /**
+   * Set the size of this record.
+   * @param size size of record to set.�
+   */
+  void setSize(const uint64_t size) {
+    size_ = size;
+  }
+  /**
+   * Returns the size of corresponding flow file
+   * @return size as a uint64_t
+   */
+  uint64_t getSize();
+
+  /**
+   * Sets the offset
+   * @param offset offset to apply to this record.
+   */
+  void setOffset(const uint64_t offset) {
+    offset_ = offset;
+  }
+
+  /**
+   * Sets the penalty expiration
+   * @param penaltyExp new penalty expiration
+   */
+  void setPenaltyExpiration(const uint64_t penaltyExp) {
+    penaltyExpiration_ms_ = penaltyExp;
+  }
+
+  uint64_t getPenaltyExpiration() {
+    return penaltyExpiration_ms_;
+  }
+
+  /**
+   * Gets the offset within the flow file
+   * @return size as a uint64_t
+   */
+  uint64_t getOffset();
+
+  // Get the UUID as string
+  std::string getUUIDStr() {
+    return uuid_str_;
+  }
+  
+  bool getUUID(uuid_t other)
+  {
+    uuid_copy(other,uuid_);
+    return true;
+  }
+
+  // Check whether it is still being penalized
+  bool isPenalized() {
+    return (
+        penaltyExpiration_ms_ > 0 ?
+            penaltyExpiration_ms_ > getTimeMillis() : false);
+  }
+
+  /**
+   * Sets the original connection with a shared pointer.
+   * @param connection shared connection.
+   */
+  void setConnection(std::shared_ptr<core::Connectable> &connection);
+
+  /**
+   * Sets the original connection with a shared pointer.
+   * @param connection shared connection.
+   */
+  void setConnection(std::shared_ptr<core::Connectable> &&connection);
+
+  /**
+   * Returns the connection referenced by this record.
+   * @return shared connection pointer.
+   */
+  std::shared_ptr<core::Connectable> getConnection();
+  /**
+   * Sets the original connection with a shared pointer.
+   * @param connection shared connection.
+   */
+  void setOriginalConnection(std::shared_ptr<core::Connectable> &connection);
+  /**
+   * Returns the original connection referenced by this record.
+   * @return shared original connection pointer.
+   */
+  std::shared_ptr<core::Connectable> getOriginalConnection();
+
+  void setStoredToRepository(bool storedInRepository) {
+    stored = storedInRepository;
+  }
+
+  bool isStored() {
+    return stored;
+  }
+
+ protected:
+  bool stored;
+  // Mark for deletion
+  bool marked_delete_;
+  // Date at which the flow file entered the flow
+  uint64_t entry_date_;
+  // event time
+  uint64_t event_time_;
+  // Date at which the origin of this flow file entered the flow
+  uint64_t lineage_start_date_;
+  // Date at which the flow file was queued
+  uint64_t last_queue_date_;
+  // Size in bytes of the data corresponding to this flow file
+  uint64_t size_;
+  // A global unique identifier
+  uuid_t uuid_;
+  // A local unique identifier
+  uint64_t id_;
+  // Offset to the content
+  uint64_t offset_;
+  // Penalty expiration
+  uint64_t penaltyExpiration_ms_;
+  // Attributes key/values pairs for the flow record
+  std::map<std::string, std::string> attributes_;
+  // Pointer to the associated content resource claim
+  std::shared_ptr<ResourceClaim> claim_;
+  // UUID string
+  std::string uuid_str_;
+  // UUID string for all parents
+  std::set<std::string> lineage_Identifiers_;
+
+  // Logger
+  std::shared_ptr<logging::Logger> logger_;
+  
+  // Connection queue that this flow file will be transfer or current in
+  std::shared_ptr<core::Connectable> connection_;
+  // Orginal connection queue that this flow file was dequeued from
+  std::shared_ptr<core::Connectable> original_connection_;
+
+};
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif // RECORD_H

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/core/ProcessContext.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ProcessContext.h b/libminifi/include/core/ProcessContext.h
new file mode 100644
index 0000000..1da85cd
--- /dev/null
+++ b/libminifi/include/core/ProcessContext.h
@@ -0,0 +1,114 @@
+/**
+ *
+ * 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 __PROCESS_CONTEXT_H__
+#define __PROCESS_CONTEXT_H__
+
+#include <uuid/uuid.h>
+#include <vector>
+#include <queue>
+#include <map>
+#include <mutex>
+#include <atomic>
+#include <algorithm>
+
+#include "Property.h"
+#include "core/logging/Logger.h"
+#include "ProcessorNode.h"
+#include "core/Repository.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+// ProcessContext Class
+class ProcessContext {
+ public:
+  // Constructor
+  /*!
+   * Create a new process context associated with the processor/controller service/state manager
+   */
+  ProcessContext(ProcessorNode &processor,
+                 std::shared_ptr<core::Repository> repo)
+      : processor_node_(processor) {
+    logger_ = logging::Logger::getLogger();
+    repo_ = repo;
+  }
+  // Destructor
+  virtual ~ProcessContext() {
+  }
+  // Get Processor associated with the Process Context
+  ProcessorNode &getProcessorNode() {
+    return processor_node_;
+  }
+  bool getProperty(std::string name, std::string &value) {
+    return processor_node_.getProperty(name, value);
+  }
+  // Sets the property value using the property's string name
+  bool setProperty(std::string name, std::string value) {
+    return processor_node_.setProperty(name, value);
+  }
+  // Sets the property value using the Property object
+  bool setProperty(Property prop, std::string value) {
+    return processor_node_.setProperty(prop, value);
+  }
+  // Whether the relationship is supported
+  bool isSupportedRelationship(Relationship relationship) {
+    return processor_node_.isSupportedRelationship(relationship);
+  }
+
+  // Check whether the relationship is auto terminated
+  bool isAutoTerminated(Relationship relationship) {
+    return processor_node_.isAutoTerminated(relationship);
+  }
+  // Get ProcessContext Maximum Concurrent Tasks
+  uint8_t getMaxConcurrentTasks(void) {
+    return processor_node_.getMaxConcurrentTasks();
+  }
+  // Yield based on the yield period
+  void yield() {
+    processor_node_.yield();
+  }
+
+  std::shared_ptr<core::Repository> getProvenanceRepository() {
+    return repo_;
+  }
+
+  // Prevent default copy constructor and assignment operation
+  // Only support pass by reference or pointer
+  ProcessContext(const ProcessContext &parent) = delete;
+  ProcessContext &operator=(const ProcessContext &parent) = delete;
+
+ private:
+
+  // repository shared pointer.
+  std::shared_ptr<core::Repository> repo_;
+  // Processor
+  ProcessorNode processor_node_;
+  // Logger
+  std::shared_ptr<logging::Logger> logger_;
+
+};
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/core/ProcessGroup.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ProcessGroup.h b/libminifi/include/core/ProcessGroup.h
new file mode 100644
index 0000000..75bb0ba
--- /dev/null
+++ b/libminifi/include/core/ProcessGroup.h
@@ -0,0 +1,190 @@
+/**
+ *
+ * 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 __PROCESS_GROUP_H__
+#define __PROCESS_GROUP_H__
+
+#include <uuid/uuid.h>
+#include <vector>
+#include <queue>
+#include <map>
+#include <mutex>
+#include <atomic>
+#include <algorithm>
+#include <set>
+
+#include "Processor.h"
+#include "Exception.h"
+#include "TimerDrivenSchedulingAgent.h"
+#include "EventDrivenSchedulingAgent.h"
+#include "core/logging/Logger.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+// Process Group Type
+enum ProcessGroupType {
+  ROOT_PROCESS_GROUP = 0,
+  REMOTE_PROCESS_GROUP,
+  MAX_PROCESS_GROUP_TYPE
+};
+
+// ProcessGroup Class
+class ProcessGroup {
+ public:
+  // Constructor
+  /*!
+   * Create a new process group
+   */
+  ProcessGroup(ProcessGroupType type, std::string name, uuid_t uuid = NULL,
+               ProcessGroup *parent = NULL);
+  // Destructor
+  virtual ~ProcessGroup();
+  // Set Processor Name
+  void setName(std::string name) {
+    name_ = name;
+  }
+  // Get Process Name
+  std::string getName(void) {
+    return (name_);
+  }
+  // Set URL
+  void setURL(std::string url) {
+    url_ = url;
+  }
+  // Get URL
+  std::string getURL(void) {
+    return (url_);
+  }
+  // SetTransmitting
+  void setTransmitting(bool val) {
+    transmitting_ = val;
+  }
+  // Get Transmitting
+  bool getTransmitting() {
+    return transmitting_;
+  }
+  // setTimeOut
+  void setTimeOut(uint64_t time) {
+    timeOut_ = time;
+  }
+  uint64_t getTimeOut() {
+    return timeOut_;
+  }
+  // Set Processor yield period in MilliSecond
+  void setYieldPeriodMsec(uint64_t period) {
+    yield_period_msec_ = period;
+  }
+  // Get Processor yield period in MilliSecond
+  uint64_t getYieldPeriodMsec(void) {
+    return (yield_period_msec_);
+  }
+  // Set UUID
+  void setUUID(uuid_t uuid) {
+    uuid_copy(uuid_, uuid);
+  }
+  // Get UUID
+  bool getUUID(uuid_t uuid) {
+    if (uuid) {
+      uuid_copy(uuid, uuid_);
+      return true;
+    } else
+      return false;
+  }
+  // Start Processing
+  void startProcessing(TimerDrivenSchedulingAgent *timeScheduler,
+                       EventDrivenSchedulingAgent *eventScheduler);
+  // Stop Processing
+  void stopProcessing(TimerDrivenSchedulingAgent *timeScheduler,
+                      EventDrivenSchedulingAgent *eventScheduler);
+  // Whether it is root process group
+  bool isRootProcessGroup();
+  // set parent process group
+  void setParent(ProcessGroup *parent) {
+    std::lock_guard<std::mutex> lock(mutex_);
+    parent_process_group_ = parent;
+  }
+  // get parent process group
+  ProcessGroup *getParent(void) {
+    std::lock_guard<std::mutex> lock(mutex_);
+    return parent_process_group_;
+  }
+  // Add processor
+  void addProcessor(std::shared_ptr<Processor> processor);
+  // Remove processor
+  void removeProcessor(std::shared_ptr<Processor> processor);
+  // Add child processor group
+  void addProcessGroup(ProcessGroup *child);
+  // Remove child processor group
+  void removeProcessGroup(ProcessGroup *child);
+  // ! Add connections
+  void addConnection(std::shared_ptr<Connection> connection);
+  // findProcessor based on UUID
+  std::shared_ptr<Processor> findProcessor(uuid_t uuid);
+  // findProcessor based on name
+  std::shared_ptr<Processor> findProcessor(const std::string &processorName);
+  // removeConnection
+  void removeConnection(std::shared_ptr<Connection> connection);
+  // update property value
+  void updatePropertyValue(std::string processorName, std::string propertyName,
+                           std::string propertyValue);
+
+  void getConnections(
+      std::map<std::string, std::shared_ptr<Connection>> &connectionMap);
+
+ protected:
+  // A global unique identifier
+  uuid_t uuid_;
+  // Processor Group Name
+  std::string name_;
+  // Process Group Type
+  ProcessGroupType type_;
+  // Processors (ProcessNode) inside this process group which include Input/Output Port, Remote Process Group input/Output port
+  std::set<std::shared_ptr<Processor> > processors_;
+  std::set<ProcessGroup *> child_process_groups_;
+  // Connections between the processor inside the group;
+  std::set<std::shared_ptr<Connection> > connections_;
+  // Parent Process Group
+  ProcessGroup* parent_process_group_;
+  // Yield Period in Milliseconds
+  std::atomic<uint64_t> yield_period_msec_;
+  std::atomic<uint64_t> timeOut_;
+  // URL
+  std::string url_;
+  // Transmitting
+  std::atomic<bool> transmitting_;
+
+ private:
+
+  // Mutex for protection
+  std::mutex mutex_;
+  // Logger
+  std::shared_ptr<logging::Logger> logger_;
+  // Prevent default copy constructor and assignment operation
+  // Only support pass by reference or pointer
+  ProcessGroup(const ProcessGroup &parent);
+  ProcessGroup &operator=(const ProcessGroup &parent);
+};
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/core/ProcessSession.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ProcessSession.h b/libminifi/include/core/ProcessSession.h
new file mode 100644
index 0000000..b516817
--- /dev/null
+++ b/libminifi/include/core/ProcessSession.h
@@ -0,0 +1,167 @@
+/**
+ *
+ * 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 __PROCESS_SESSION_H__
+#define __PROCESS_SESSION_H__
+
+#include <uuid/uuid.h>
+#include <vector>
+#include <queue>
+#include <map>
+#include <mutex>
+#include <atomic>
+#include <algorithm>
+#include <set>
+
+#include "ProcessContext.h"
+#include "FlowFileRecord.h"
+#include "Exception.h"
+#include "core/logging/Logger.h"
+#include "FlowFile.h"
+#include "provenance/Provenance.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+// ProcessSession Class
+class ProcessSession {
+ public:
+  // Constructor
+  /*!
+   * Create a new process session
+   */
+  ProcessSession(ProcessContext *processContext = NULL)
+      : process_context_(processContext) {
+    logger_ = logging::Logger::getLogger();
+    logger_->log_trace("ProcessSession created for %s",
+                       process_context_->getProcessorNode().getName().c_str());
+    auto repo = processContext->getProvenanceRepository();
+    provenance_report_ = new provenance::ProvenanceReporter(
+        repo, process_context_->getProcessorNode().getUUIDStr(),
+        process_context_->getProcessorNode().getName());
+  }
+
+// Destructor
+  virtual ~ProcessSession() {
+    if (provenance_report_)
+      delete provenance_report_;
+  }
+// Commit the session
+  void commit();
+// Roll Back the session
+  void rollback();
+// Get Provenance Report
+  provenance::ProvenanceReporter *getProvenanceReporter() {
+    return provenance_report_;
+  }
+//
+// Get the FlowFile from the highest priority queue
+  std::shared_ptr<core::FlowFile> get();
+// Create a new UUID FlowFile with no content resource claim and without parent
+  std::shared_ptr<core::FlowFile> create();
+// Create a new UUID FlowFile with no content resource claim and inherit all attributes from parent
+  std::shared_ptr<core::FlowFile> create(std::shared_ptr<core::FlowFile> &parent);
+// Clone a new UUID FlowFile from parent both for content resource claim and attributes
+  std::shared_ptr<core::FlowFile> clone(
+      std::shared_ptr<core::FlowFile> &parent);
+// Clone a new UUID FlowFile from parent for attributes and sub set of parent content resource claim
+  std::shared_ptr<core::FlowFile> clone(std::shared_ptr<core::FlowFile> &parent,
+                                        long offset, long size);
+// Duplicate a FlowFile with the same UUID and all attributes and content resource claim for the roll back of the session
+  std::shared_ptr<core::FlowFile> duplicate(
+      std::shared_ptr<core::FlowFile> &original);
+// Transfer the FlowFile to the relationship
+  void transfer(std::shared_ptr<core::FlowFile> &flow,
+                Relationship relationship);
+  void transfer(std::shared_ptr<core::FlowFile> &&flow,
+                Relationship relationship);
+// Put Attribute
+  void putAttribute(std::shared_ptr<core::FlowFile> &flow, std::string key,
+                    std::string value);
+  void putAttribute(std::shared_ptr<core::FlowFile> &&flow, std::string key,
+                    std::string value);
+// Remove Attribute
+  void removeAttribute(std::shared_ptr<core::FlowFile> &flow, std::string key);
+  void removeAttribute(std::shared_ptr<core::FlowFile> &&flow, std::string key);
+// Remove Flow File
+  void remove(std::shared_ptr<core::FlowFile> &flow);
+  void remove(std::shared_ptr<core::FlowFile> &&flow);
+// Execute the given read callback against the content
+  void read(std::shared_ptr<core::FlowFile> &flow,
+            InputStreamCallback *callback);
+  void read(std::shared_ptr<core::FlowFile> &&flow,
+            InputStreamCallback *callback);
+// Execute the given write callback against the content
+  void write(std::shared_ptr<core::FlowFile> &flow,
+             OutputStreamCallback *callback);
+  void write(std::shared_ptr<core::FlowFile> &&flow,
+             OutputStreamCallback *callback);
+// Execute the given write/append callback against the content
+  void append(std::shared_ptr<core::FlowFile> &flow,
+              OutputStreamCallback *callback);
+  void append(std::shared_ptr<core::FlowFile> &&flow,
+              OutputStreamCallback *callback);
+// Penalize the flow
+  void penalize(std::shared_ptr<core::FlowFile> &flow);
+  void penalize(std::shared_ptr<core::FlowFile> &&flow);
+// Import the existed file into the flow
+  void import(std::string source, std::shared_ptr<core::FlowFile> &flow,
+              bool keepSource = true, uint64_t offset = 0);
+  void import(std::string source, std::shared_ptr<core::FlowFile> &&flow,
+              bool keepSource = true, uint64_t offset = 0);
+
+// Prevent default copy constructor and assignment operation
+// Only support pass by reference or pointer
+  ProcessSession(const ProcessSession &parent) = delete;
+  ProcessSession &operator=(const ProcessSession &parent) = delete;
+
+ protected:
+// FlowFiles being modified by current process session
+  std::map<std::string, std::shared_ptr<core::FlowFile> > _updatedFlowFiles;
+// Copy of the original FlowFiles being modified by current process session as above
+  std::map<std::string, std::shared_ptr<core::FlowFile> > _originalFlowFiles;
+// FlowFiles being added by current process session
+  std::map<std::string, std::shared_ptr<core::FlowFile> > _addedFlowFiles;
+// FlowFiles being deleted by current process session
+  std::map<std::string, std::shared_ptr<core::FlowFile> > _deletedFlowFiles;
+// FlowFiles being transfered to the relationship
+  std::map<std::string, Relationship> _transferRelationship;
+// FlowFiles being cloned for multiple connections per relationship
+  std::map<std::string, std::shared_ptr<core::FlowFile> > _clonedFlowFiles;
+
+ private:
+// Clone the flow file during transfer to multiple connections for a relationship
+  std::shared_ptr<core::FlowFile> cloneDuringTransfer(
+      std::shared_ptr<core::FlowFile> &parent);
+// ProcessContext
+  ProcessContext *process_context_;
+// Logger
+  std::shared_ptr<logging::Logger> logger_;
+// Provenance Report
+  provenance::ProvenanceReporter *provenance_report_;
+
+}
+;
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/core/ProcessSessionFactory.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ProcessSessionFactory.h b/libminifi/include/core/ProcessSessionFactory.h
new file mode 100644
index 0000000..e0ebe18
--- /dev/null
+++ b/libminifi/include/core/ProcessSessionFactory.h
@@ -0,0 +1,64 @@
+/**
+ * @file ProcessSessionFactory.h
+ * ProcessSessionFactory class declaration
+ *
+ * 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 __PROCESS_SESSION_FACTORY_H__
+#define __PROCESS_SESSION_FACTORY_H__
+
+#include <memory>
+
+#include "ProcessContext.h"
+#include "ProcessSession.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+// ProcessSessionFactory Class
+class ProcessSessionFactory {
+ public:
+  // Constructor
+  /*!
+   * Create a new process session factory
+   */
+  explicit ProcessSessionFactory(ProcessContext *processContext)
+      : process_context_(processContext) {
+  }
+
+  // Create the session
+  std::unique_ptr<ProcessSession> createSession();
+
+  // Prevent default copy constructor and assignment operation
+  // Only support pass by reference or pointer
+  ProcessSessionFactory(const ProcessSessionFactory &parent) = delete;
+  ProcessSessionFactory &operator=(const ProcessSessionFactory &parent) = delete;
+
+ private:
+  // ProcessContext
+  ProcessContext *process_context_;
+
+};
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/core/Processor.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/Processor.h b/libminifi/include/core/Processor.h
new file mode 100644
index 0000000..fd0411f
--- /dev/null
+++ b/libminifi/include/core/Processor.h
@@ -0,0 +1,270 @@
+/**
+ *
+ * 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 __PROCESSOR_H__
+#define __PROCESSOR_H__
+
+#include <uuid/uuid.h>
+#include <vector>
+#include <queue>
+#include <map>
+#include <mutex>
+#include <memory>
+#include <condition_variable>
+#include <atomic>
+#include <algorithm>
+#include <set>
+#include <chrono>
+#include <functional>
+
+#include "Connectable.h"
+#include "ConfigurableComponent.h"
+#include "Property.h"
+#include "utils/TimeUtil.h"
+#include "Relationship.h"
+#include "Connection.h"
+#include "ProcessContext.h"
+#include "ProcessSession.h"
+#include "ProcessSessionFactory.h"
+#include "Scheduling.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+// Minimum scheduling period in Nano Second
+#define MINIMUM_SCHEDULING_NANOS 30000
+
+// Default yield period in second
+#define DEFAULT_YIELD_PERIOD_SECONDS 1
+
+// Default penalization period in second
+#define DEFAULT_PENALIZATION_PERIOD_SECONDS 30
+
+// Processor Class
+class Processor : public Connectable, public ConfigurableComponent,
+    public std::enable_shared_from_this<Processor> {
+
+ public:
+  // Constructor
+  /*!
+   * Create a new processor
+   */
+  Processor(std::string name, uuid_t uuid = NULL);
+  // Destructor
+  virtual ~Processor() {
+  }
+
+  bool isRunning();
+  // Set Processor Scheduled State
+  void setScheduledState(ScheduledState state);
+  // Get Processor Scheduled State
+  ScheduledState getScheduledState(void) {
+    return state_;
+  }
+  // Set Processor Scheduling Strategy
+  void setSchedulingStrategy(SchedulingStrategy strategy) {
+    strategy_ = strategy;
+  }
+  // Get Processor Scheduling Strategy
+  SchedulingStrategy getSchedulingStrategy(void) {
+    return strategy_;
+  }
+  // Set Processor Loss Tolerant
+  void setlossTolerant(bool lossTolerant) {
+    loss_tolerant_ = lossTolerant;
+  }
+  // Get Processor Loss Tolerant
+  bool getlossTolerant(void) {
+    return loss_tolerant_;
+  }
+  // Set Processor Scheduling Period in Nano Second
+  void setSchedulingPeriodNano(uint64_t period) {
+    uint64_t minPeriod = MINIMUM_SCHEDULING_NANOS;
+    scheduling_period_nano_ = std::max(period, minPeriod);
+  }
+  // Get Processor Scheduling Period in Nano Second
+  uint64_t getSchedulingPeriodNano(void) {
+    return scheduling_period_nano_;
+  }
+  // Set Processor Run Duration in Nano Second
+  void setRunDurationNano(uint64_t period) {
+    run_durantion_nano_ = period;
+  }
+  // Get Processor Run Duration in Nano Second
+  uint64_t getRunDurationNano(void) {
+    return (run_durantion_nano_);
+  }
+  // Set Processor yield period in MilliSecond
+  void setYieldPeriodMsec(uint64_t period) {
+    yield_period_msec_ = period;
+  }
+  // Get Processor yield period in MilliSecond
+  uint64_t getYieldPeriodMsec(void) {
+    return (yield_period_msec_);
+  }
+  // Set Processor penalization period in MilliSecond
+  void setPenalizationPeriodMsec(uint64_t period) {
+    _penalizationPeriodMsec = period;
+  }
+  
+  
+  // Set Processor Maximum Concurrent Tasks
+  void setMaxConcurrentTasks(uint8_t tasks) {
+    max_concurrent_tasks_ = tasks;
+  }
+  // Get Processor Maximum Concurrent Tasks
+  uint8_t getMaxConcurrentTasks(void) {
+    return (max_concurrent_tasks_);
+  }
+  // Set Trigger when empty
+  void setTriggerWhenEmpty(bool value) {
+    _triggerWhenEmpty = value;
+  }
+  // Get Trigger when empty
+  bool getTriggerWhenEmpty(void) {
+    return (_triggerWhenEmpty);
+  }
+  // Get Active Task Counts
+  uint8_t getActiveTasks(void) {
+    return (active_tasks_);
+  }
+  // Increment Active Task Counts
+  void incrementActiveTasks(void) {
+    active_tasks_++;
+  }
+  // decrement Active Task Counts
+  void decrementActiveTask(void) {
+    active_tasks_--;
+  }
+  void clearActiveTask(void) {
+    active_tasks_ = 0;
+  }
+  // Yield based on the yield period
+  void yield() {
+    yield_expiration_ = (getTimeMillis() + yield_period_msec_);
+  }
+  // Yield based on the input time
+  void yield(uint64_t time) {
+    yield_expiration_ = (getTimeMillis() + time);
+  }
+  // whether need be to yield
+  bool isYield() {
+    if (yield_expiration_ > 0)
+      return (yield_expiration_ >= getTimeMillis());
+    else
+      return false;
+  }
+  // clear yield expiration
+  void clearYield() {
+    yield_expiration_ = 0;
+  }
+  // get yield time
+  uint64_t getYieldTime() {
+    uint64_t curTime = getTimeMillis();
+    if (yield_expiration_ > curTime)
+      return (yield_expiration_ - curTime);
+    else
+      return 0;;
+  }
+  // Whether flow file queued in incoming connection
+  bool flowFilesQueued();
+  // Whether flow file queue full in any of the outgoin connection
+  bool flowFilesOutGoingFull();
+
+  // Get outgoing connections based on relationship name
+  std::set<std::shared_ptr<Connection> > getOutGoingConnections(
+      std::string relationship);
+  // Add connection
+  bool addConnection(std::shared_ptr<Connectable> connection);
+  // Remove connection
+  void removeConnection(std::shared_ptr<Connectable> connection);
+  // Get the UUID as string
+  std::string getUUIDStr() {
+    return uuidStr_;
+  }
+  // Get the Next RoundRobin incoming connection
+  std::shared_ptr<Connection> getNextIncomingConnection();
+  // On Trigger
+  void onTrigger(ProcessContext *context,
+                 ProcessSessionFactory *sessionFactory);
+
+  virtual bool canEdit() {
+    return !isRunning();
+  }
+
+ public:
+
+
+  // OnTrigger method, implemented by NiFi Processor Designer
+  virtual void onTrigger(ProcessContext *context, ProcessSession *session) = 0;
+  // Initialize, overridden by NiFi Process Designer
+  virtual void initialize() {
+  }
+  // Scheduled event hook, overridden by NiFi Process Designer
+  virtual void onSchedule(ProcessContext *context,
+                          ProcessSessionFactory *sessionFactory) {
+  }
+
+ protected:
+
+  // Processor state
+  std::atomic<ScheduledState> state_;
+
+  // lossTolerant
+  std::atomic<bool> loss_tolerant_;
+  // SchedulePeriod in Nano Seconds
+  std::atomic<uint64_t> scheduling_period_nano_;
+  // Run Duration in Nano Seconds
+  std::atomic<uint64_t> run_durantion_nano_;
+  // Yield Period in Milliseconds
+  std::atomic<uint64_t> yield_period_msec_;
+  
+  // Active Tasks
+  std::atomic<uint8_t> active_tasks_;
+  // Trigger the Processor even if the incoming connection is empty
+  std::atomic<bool> _triggerWhenEmpty;
+
+private:
+
+  // Mutex for protection
+  std::mutex mutex_;
+  // Yield Expiration
+  std::atomic<uint64_t> yield_expiration_;
+  
+
+  // Check all incoming connections for work
+  bool isWorkAvailable();
+  // Logger
+  std::shared_ptr<logging::Logger> logger_;
+  // Prevent default copy constructor and assignment operation
+  // Only support pass by reference or pointer
+  Processor(const Processor &parent);
+  Processor &operator=(const Processor &parent);
+
+};
+
+}
+/* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/core/ProcessorConfig.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ProcessorConfig.h b/libminifi/include/core/ProcessorConfig.h
new file mode 100644
index 0000000..6b4a00a
--- /dev/null
+++ b/libminifi/include/core/ProcessorConfig.h
@@ -0,0 +1,53 @@
+/**
+ * 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_PROCESSORCONFIG_H_
+#define LIBMINIFI_INCLUDE_CORE_PROCESSORCONFIG_H_
+
+#include "core/core.h"
+#include "core/Property.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+
+struct ProcessorConfig {
+  std::string id;
+  std::string name;
+  std::string javaClass;
+  std::string maxConcurrentTasks;
+  std::string schedulingStrategy;
+  std::string schedulingPeriod;
+  std::string penalizationPeriod;
+  std::string yieldPeriod;
+  std::string runDurationNanos;
+  std::vector<std::string> autoTerminatedRelationships;
+  std::vector<core::Property> properties;
+};
+
+
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+
+#endif /* LIBMINIFI_INCLUDE_CORE_PROCESSORCONFIG_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/core/ProcessorNode.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ProcessorNode.h b/libminifi/include/core/ProcessorNode.h
new file mode 100644
index 0000000..8836f62
--- /dev/null
+++ b/libminifi/include/core/ProcessorNode.h
@@ -0,0 +1,246 @@
+/**
+ * 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_PROCESSOR_PROCESSORNODE_H_
+#define LIBMINIFI_INCLUDE_PROCESSOR_PROCESSORNODE_H_
+
+#include "ConfigurableComponent.h"
+#include "Connectable.h"
+#include "Property.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+/**
+ * Processor node functions as a pass through to the implementing Connectables
+ * ProcessorNode can be used by itself or with a pass through object, in which case
+ * we need to function as a passthrough or not.
+ */
+class ProcessorNode : public ConfigurableComponent, public Connectable {
+ public:
+  explicit ProcessorNode(const std::shared_ptr<Connectable> processor);
+
+  explicit ProcessorNode(const ProcessorNode &other);
+
+  /**
+   * Get property using the provided name.
+   * @param name property name.
+   * @param value value passed in by reference
+   * @return result of getting property.
+   */
+  std::shared_ptr<Connectable> getProcessor() const {
+    return processor_;
+  }
+
+  void yield() {
+    processor_->yield();
+  }
+
+  /**
+   * Get property using the provided name.
+   * @param name property name.
+   * @param value value passed in by reference
+   * @return result of getting property.
+   */
+  bool getProperty(const std::string name, std::string &value) {
+    const std::shared_ptr<ConfigurableComponent> processor_cast =
+        std::dynamic_pointer_cast<ConfigurableComponent>(processor_);
+    if (nullptr != processor_cast)
+      return processor_cast->getProperty(name, value);
+    else {
+      return ConfigurableComponent::getProperty(name, value);
+    }
+  }
+  /**
+   * Sets the property using the provided name
+   * @param property name
+   * @param value property value.
+   * @return result of setting property.
+   */
+  bool setProperty(const std::string name, std::string value) {
+    const std::shared_ptr<ConfigurableComponent> processor_cast =
+        std::dynamic_pointer_cast<ConfigurableComponent>(processor_);
+    bool ret = ConfigurableComponent::setProperty(name, value);
+    if (nullptr != processor_cast)
+      ret = processor_cast->setProperty(name, value);
+
+    return ret;
+
+  }
+
+  /**
+   * Sets the property using the provided name
+   * @param property name
+   * @param value property value.
+   * @return whether property was set or not
+   */
+  bool setProperty(Property &prop, std::string value) {
+    const std::shared_ptr<ConfigurableComponent> processor_cast =
+        std::dynamic_pointer_cast<ConfigurableComponent>(processor_);
+    bool ret = ConfigurableComponent::setProperty(prop, value);
+    if (nullptr != processor_cast)
+      ret = processor_cast->setProperty(prop, value);
+
+    return ret;
+  }
+
+  /**
+   * Sets supported properties for the ConfigurableComponent
+   * @param supported properties
+   * @return result of set operation.
+   */
+  bool setSupportedProperties(std::set<Property> properties) {
+    const std::shared_ptr<ConfigurableComponent> processor_cast =
+        std::dynamic_pointer_cast<ConfigurableComponent>(processor_);
+    bool ret = ConfigurableComponent::setSupportedProperties(properties);
+    if (nullptr != processor_cast)
+      ret = processor_cast->setSupportedProperties(properties);
+
+    return ret;
+  }
+  /**
+   * Sets supported properties for the ConfigurableComponent
+   * @param supported properties
+   * @return result of set operation.
+   */
+
+  bool setAutoTerminatedRelationships(std::set<Relationship> relationships) {
+    return processor_->setAutoTerminatedRelationships(relationships);
+  }
+
+  bool isAutoTerminated(Relationship relationship) {
+    return processor_->isAutoTerminated(relationship);
+  }
+
+  bool setSupportedRelationships(std::set<Relationship> relationships) {
+    return processor_->setSupportedRelationships(relationships);
+  }
+
+  bool isSupportedRelationship(Relationship relationship) {
+    return processor_->isSupportedRelationship(relationship);
+  }
+
+  /**
+   * Set name.
+   * @param name
+   */
+  void setName(const std::string name) {
+    Connectable::setName(name);
+    processor_->setName(name);
+  }
+
+  /**
+   * Set UUID in this instance
+   * @param uuid uuid to apply to the internal representation.
+   */
+  void setUUID(uuid_t uuid) {
+    Connectable::setUUID(uuid);
+    processor_->setUUID(uuid);
+  }
+
+// Get Processor penalization period in MilliSecond
+  uint64_t getPenalizationPeriodMsec(void) {
+    return processor_->getPenalizationPeriodMsec();
+  }
+
+  /**
+   * Get outgoing connection based on relationship
+   * @return set of outgoing connections.
+   */
+  std::set<std::shared_ptr<Connectable>> getOutGoingConnections(
+      std::string relationship) {
+    return processor_->getOutGoingConnections(relationship);
+  }
+
+  /**
+   * Get next incoming connection
+   * @return next incoming connection
+   */
+  std::shared_ptr<Connectable> getNextIncomingConnection() {
+    return processor_->getNextIncomingConnection();
+  }
+
+  /**
+   * @return true if incoming connections > 0
+   */
+  bool hasIncomingConnections() {
+    return processor_->hasIncomingConnections();
+  }
+
+  /**
+   * Returns the UUID through the provided object.
+   * @param uuid uuid struct to which we will copy the memory
+   * @return success of request
+   */
+  bool getUUID(uuid_t uuid) {
+    return processor_->getUUID(uuid);
+  }
+
+  unsigned const char *getUUID() {
+    return processor_->getUUID();
+  }
+  /**
+   * Return the UUID string
+   * @param constant reference to the UUID str
+   */
+  const std::string & getUUIDStr() const {
+    return processor_->getUUIDStr();
+  }
+
+// Get Process Name
+  std::string getName() const {
+    return processor_->getName();
+  }
+
+  uint8_t getMaxConcurrentTasks() {
+    return processor_->getMaxConcurrentTasks();
+  }
+
+  void setMaxConcurrentTasks(const uint8_t tasks) {
+    processor_->setMaxConcurrentTasks(tasks);
+  }
+
+  virtual bool isRunning();
+
+  virtual bool isWorkAvailable();
+
+  virtual ~ProcessorNode();
+
+ protected:
+
+  virtual bool canEdit() {
+    return !processor_->isRunning();
+  }
+
+  /**
+   * internal connectable.
+   */
+  std::shared_ptr<Connectable> processor_;
+
+}
+;
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_PROCESSOR_PROCESSORNODE_H_ */


[02/16] nifi-minifi-cpp git commit: MINIFI-217: Updates namespaces and removes use of raw pointers for user facing API.

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/processors/TailFile.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/TailFile.cpp b/libminifi/src/processors/TailFile.cpp
new file mode 100644
index 0000000..859daa6
--- /dev/null
+++ b/libminifi/src/processors/TailFile.cpp
@@ -0,0 +1,271 @@
+/**
+ * @file TailFile.cpp
+ * TailFile class implementation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <vector>
+#include <queue>
+#include <map>
+#include <set>
+#include <sys/time.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <time.h>
+#include <sstream>
+#include <stdio.h>
+#include <string>
+#include <iostream>
+#include <dirent.h>
+#include <limits.h>
+#include <unistd.h>
+
+#include "utils/TimeUtil.h"
+#include "utils/StringUtils.h"
+#include "processors/TailFile.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+const std::string TailFile::ProcessorName("TailFile");
+core::Property TailFile::FileName(
+    "File to Tail",
+    "Fully-qualified filename of the file that should be tailed", "");
+core::Property TailFile::StateFile(
+    "State File",
+    "Specifies the file that should be used for storing state about what data has been ingested so that upon restart NiFi can resume from where it left off",
+    "TailFileState");
+core::Relationship TailFile::Success(
+    "success", "All files are routed to success");
+
+void TailFile::initialize() {
+  // Set the supported properties
+  std::set<core::Property> properties;
+  properties.insert(FileName);
+  properties.insert(StateFile);
+  setSupportedProperties(properties);
+  // Set the supported relationships
+  std::set<core::Relationship> relationships;
+  relationships.insert(Success);
+  setSupportedRelationships(relationships);
+}
+
+std::string TailFile::trimLeft(const std::string& s) {
+  return org::apache::nifi::minifi::utils::StringUtils::trimLeft(s);
+}
+
+std::string TailFile::trimRight(const std::string& s) {
+  return org::apache::nifi::minifi::utils::StringUtils::trimRight(s);
+}
+
+void TailFile::parseStateFileLine(char *buf) {
+  char *line = buf;
+
+  while ((line[0] == ' ') || (line[0] == '\t'))
+    ++line;
+
+  char first = line[0];
+  if ((first == '\0') || (first == '#') || (first == '\r') || (first == '\n')
+      || (first == '=')) {
+    return;
+  }
+
+  char *equal = strchr(line, '=');
+  if (equal == NULL) {
+    return;
+  }
+
+  equal[0] = '\0';
+  std::string key = line;
+
+  equal++;
+  while ((equal[0] == ' ') || (equal[0] == '\t'))
+    ++equal;
+
+  first = equal[0];
+  if ((first == '\0') || (first == '\r') || (first == '\n')) {
+    return;
+  }
+
+  std::string value = equal;
+  key = trimRight(key);
+  value = trimRight(value);
+
+  if (key == "FILENAME")
+    this->_currentTailFileName = value;
+  if (key == "POSITION")
+    this->_currentTailFilePosition = std::stoi(value);
+
+  return;
+}
+
+void TailFile::recoverState() {
+  std::ifstream file(_stateFile.c_str(), std::ifstream::in);
+  if (!file.good()) {
+    logger_->log_error("load state file failed %s", _stateFile.c_str());
+    return;
+  }
+  const unsigned int bufSize = 512;
+  char buf[bufSize];
+  for (file.getline(buf, bufSize); file.good(); file.getline(buf, bufSize)) {
+    parseStateFileLine(buf);
+  }
+}
+
+void TailFile::storeState() {
+  std::ofstream file(_stateFile.c_str());
+  if (!file.is_open()) {
+    logger_->log_error("store state file failed %s", _stateFile.c_str());
+    return;
+  }
+  file << "FILENAME=" << this->_currentTailFileName << "\n";
+  file << "POSITION=" << this->_currentTailFilePosition << "\n";
+  file.close();
+}
+
+static bool sortTailMatchedFileItem(TailMatchedFileItem i,
+                                    TailMatchedFileItem j) {
+  return (i.modifiedTime < j.modifiedTime);
+}
+void TailFile::checkRollOver() {
+  struct stat statbuf;
+  std::vector<TailMatchedFileItem> matchedFiles;
+  std::string fullPath = this->_fileLocation + "/" + _currentTailFileName;
+
+  if (stat(fullPath.c_str(), &statbuf) == 0) {
+    if (statbuf.st_size > this->_currentTailFilePosition)
+      // there are new input for the current tail file
+      return;
+
+    uint64_t modifiedTimeCurrentTailFile =
+        ((uint64_t) (statbuf.st_mtime) * 1000);
+    std::string pattern = _fileName;
+    std::size_t found = _fileName.find_last_of(".");
+    if (found != std::string::npos)
+      pattern = _fileName.substr(0, found);
+    DIR *d;
+    d = opendir(this->_fileLocation.c_str());
+    if (!d)
+      return;
+    while (1) {
+      struct dirent *entry;
+      entry = readdir(d);
+      if (!entry)
+        break;
+      std::string d_name = entry->d_name;
+      if (!(entry->d_type & DT_DIR)) {
+        std::string fileName = d_name;
+        std::string fileFullName = this->_fileLocation + "/" + d_name;
+        if (fileFullName.find(pattern) != std::string::npos
+            && stat(fileFullName.c_str(), &statbuf) == 0) {
+          if (((uint64_t) (statbuf.st_mtime) * 1000)
+              >= modifiedTimeCurrentTailFile) {
+            TailMatchedFileItem item;
+            item.fileName = fileName;
+            item.modifiedTime = ((uint64_t) (statbuf.st_mtime) * 1000);
+            matchedFiles.push_back(item);
+          }
+        }
+      }
+    }
+    closedir(d);
+
+    // Sort the list based on modified time
+    std::sort(matchedFiles.begin(), matchedFiles.end(),
+              sortTailMatchedFileItem);
+    for (std::vector<TailMatchedFileItem>::iterator it = matchedFiles.begin();
+        it != matchedFiles.end(); ++it) {
+      TailMatchedFileItem item = *it;
+      if (item.fileName == _currentTailFileName) {
+        ++it;
+        if (it != matchedFiles.end()) {
+          TailMatchedFileItem nextItem = *it;
+          logger_->log_info("TailFile File Roll Over from %s to %s",
+                            _currentTailFileName.c_str(),
+                            nextItem.fileName.c_str());
+          _currentTailFileName = nextItem.fileName;
+          _currentTailFilePosition = 0;
+          storeState();
+        }
+        break;
+      }
+    }
+  } else
+    return;
+}
+
+void TailFile::onTrigger(
+    core::ProcessContext *context,
+    core::ProcessSession *session) {
+  std::string value;
+  if (context->getProperty(FileName.getName(), value)) {
+    std::size_t found = value.find_last_of("/\\");
+    this->_fileLocation = value.substr(0, found);
+    this->_fileName = value.substr(found + 1);
+  }
+  if (context->getProperty(StateFile.getName(), value)) {
+    _stateFile = value + "." + getUUIDStr();
+  }
+  if (!this->_stateRecovered) {
+    _stateRecovered = true;
+    this->_currentTailFileName = _fileName;
+    this->_currentTailFilePosition = 0;
+    // recover the state if we have not done so
+    this->recoverState();
+  }
+  checkRollOver();
+  std::string fullPath = this->_fileLocation + "/" + _currentTailFileName;
+  struct stat statbuf;
+  if (stat(fullPath.c_str(), &statbuf) == 0) {
+    if (statbuf.st_size <= this->_currentTailFilePosition)
+    // there are no new input for the current tail file
+        {
+      context->yield();
+      return;
+    }
+    std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast<FlowFileRecord>(session->create());;
+    if (!flowFile)
+      return;
+    std::size_t found = _currentTailFileName.find_last_of(".");
+    std::string baseName = _currentTailFileName.substr(0, found);
+    std::string extension = _currentTailFileName.substr(found + 1);
+    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.c_str(),
+                      flowFile->getSize());
+    std::string logName = baseName + "."
+        + std::to_string(_currentTailFilePosition) + "-"
+        + std::to_string(_currentTailFilePosition + flowFile->getSize()) + "."
+        + extension;
+    flowFile->updateKeyedAttribute(FILENAME, logName);
+    this->_currentTailFilePosition += flowFile->getSize();
+    storeState();
+  }
+}
+
+
+} /* namespace processors */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/provenance/Provenance.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/provenance/Provenance.cpp b/libminifi/src/provenance/Provenance.cpp
new file mode 100644
index 0000000..a90e182
--- /dev/null
+++ b/libminifi/src/provenance/Provenance.cpp
@@ -0,0 +1,578 @@
+/**
+ *
+ * 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 <cstdint>
+#include <vector>
+#include <arpa/inet.h>
+#include "io/DataStream.h"
+#include "io/Serializable.h"
+#include "provenance/Provenance.h"
+
+#include "core/logging/Logger.h"
+#include "core/Relationship.h"
+#include "FlowController.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace provenance {
+
+// DeSerialize
+bool ProvenanceEventRecord::DeSerialize(
+    const std::shared_ptr<core::Repository> &repo, std::string key) {
+  std::string value;
+  bool ret;
+
+  ret = repo->Get(key, value);
+
+  if (!ret) {
+    logger_->log_error("NiFi Provenance Store event %s can not found",
+                       key.c_str());
+    return false;
+  } else
+    logger_->log_debug("NiFi Provenance Read event %s length %d", key.c_str(),
+                       value.length());
+
+  org::apache::nifi::minifi::io::DataStream stream(
+      (const uint8_t*) value.data(), value.length());
+
+  ret = DeSerialize(stream);
+
+  if (ret) {
+    logger_->log_debug(
+        "NiFi Provenance retrieve event %s size %d eventType %d success",
+        _eventIdStr.c_str(), stream.getSize(), _eventType);
+  } else {
+    logger_->log_debug(
+        "NiFi Provenance retrieve event %s size %d eventType %d fail",
+        _eventIdStr.c_str(), stream.getSize(), _eventType);
+  }
+
+  return ret;
+}
+
+bool ProvenanceEventRecord::Serialize(
+    const std::shared_ptr<core::Repository> &repo) {
+
+  org::apache::nifi::minifi::io::DataStream outStream;
+
+  int ret;
+
+  ret = writeUTF(this->_eventIdStr, &outStream);
+  if (ret <= 0) {
+
+    return false;
+  }
+
+  uint32_t eventType = this->_eventType;
+  ret = write(eventType, &outStream);
+  if (ret != 4) {
+
+    return false;
+  }
+
+  ret = write(this->_eventTime, &outStream);
+  if (ret != 8) {
+
+    return false;
+  }
+
+  ret = write(this->_entryDate, &outStream);
+  if (ret != 8) {
+    return false;
+  }
+
+  ret = write(this->_eventDuration, &outStream);
+  if (ret != 8) {
+
+    return false;
+  }
+
+  ret = write(this->_lineageStartDate, &outStream);
+  if (ret != 8) {
+
+    return false;
+  }
+
+  ret = writeUTF(this->_componentId, &outStream);
+  if (ret <= 0) {
+
+    return false;
+  }
+
+  ret = writeUTF(this->_componentType, &outStream);
+  if (ret <= 0) {
+
+    return false;
+  }
+
+  ret = writeUTF(this->uuid_, &outStream);
+  if (ret <= 0) {
+
+    return false;
+  }
+
+  ret = writeUTF(this->_details, &outStream);
+  if (ret <= 0) {
+
+    return false;
+  }
+
+  // write flow attributes
+  uint32_t numAttributes = this->_attributes.size();
+  ret = write(numAttributes, &outStream);
+  if (ret != 4) {
+
+    return false;
+  }
+
+  for (auto itAttribute : _attributes) {
+    ret = writeUTF(itAttribute.first, &outStream, true);
+    if (ret <= 0) {
+
+      return false;
+    }
+    ret = writeUTF(itAttribute.second, &outStream, true);
+    if (ret <= 0) {
+
+      return false;
+    }
+  }
+
+  ret = writeUTF(this->_contentFullPath, &outStream);
+  if (ret <= 0) {
+
+    return false;
+  }
+
+  ret = write(this->_size, &outStream);
+  if (ret != 8) {
+
+    return false;
+  }
+
+  ret = write(this->_offset, &outStream);
+  if (ret != 8) {
+
+    return false;
+  }
+
+  ret = writeUTF(this->_sourceQueueIdentifier, &outStream);
+  if (ret <= 0) {
+
+    return false;
+  }
+
+  if (this->_eventType == ProvenanceEventRecord::FORK
+      || this->_eventType == ProvenanceEventRecord::CLONE
+      || this->_eventType == ProvenanceEventRecord::JOIN) {
+    // write UUIDs
+    uint32_t number = this->_parentUuids.size();
+    ret = write(number, &outStream);
+    if (ret != 4) {
+
+      return false;
+    }
+    for (auto parentUUID : _parentUuids) {
+      ret = writeUTF(parentUUID, &outStream);
+      if (ret <= 0) {
+
+        return false;
+      }
+    }
+    number = this->_childrenUuids.size();
+    ret = write(number, &outStream);
+    if (ret != 4) {
+      return false;
+    }
+    for (auto childUUID : _childrenUuids) {
+      ret = writeUTF(childUUID, &outStream);
+      if (ret <= 0) {
+
+        return false;
+      }
+    }
+  } else if (this->_eventType == ProvenanceEventRecord::SEND
+      || this->_eventType == ProvenanceEventRecord::FETCH) {
+    ret = writeUTF(this->_transitUri, &outStream);
+    if (ret <= 0) {
+
+      return false;
+    }
+  } else if (this->_eventType == ProvenanceEventRecord::RECEIVE) {
+    ret = writeUTF(this->_transitUri, &outStream);
+    if (ret <= 0) {
+
+      return false;
+    }
+    ret = writeUTF(this->_sourceSystemFlowFileIdentifier, &outStream);
+    if (ret <= 0) {
+
+      return false;
+    }
+  }
+
+  // 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());
+  } else {
+    logger_->log_error("NiFi Provenance Store event %s size %d fail",
+                       _eventIdStr.c_str(), outStream.getSize());
+  }
+
+  // cleanup
+
+  return true;
+}
+
+bool ProvenanceEventRecord::DeSerialize(const uint8_t *buffer,
+                                        const int bufferSize) {
+
+  int ret;
+
+  org::apache::nifi::minifi::io::DataStream outStream(buffer, bufferSize);
+
+  ret = readUTF(this->_eventIdStr, &outStream);
+
+  if (ret <= 0) {
+    return false;
+  }
+
+  uint32_t eventType;
+  ret = read(eventType, &outStream);
+  if (ret != 4) {
+    return false;
+  }
+  this->_eventType = (ProvenanceEventRecord::ProvenanceEventType) eventType;
+
+  ret = read(this->_eventTime, &outStream);
+  if (ret != 8) {
+    return false;
+  }
+
+  ret = read(this->_entryDate, &outStream);
+  if (ret != 8) {
+    return false;
+  }
+
+  ret = read(this->_eventDuration, &outStream);
+  if (ret != 8) {
+    return false;
+  }
+
+  ret = read(this->_lineageStartDate, &outStream);
+  if (ret != 8) {
+    return false;
+  }
+
+  ret = readUTF(this->_componentId, &outStream);
+  if (ret <= 0) {
+    return false;
+  }
+
+  ret = readUTF(this->_componentType, &outStream);
+  if (ret <= 0) {
+    return false;
+  }
+
+  ret = readUTF(this->uuid_, &outStream);
+  if (ret <= 0) {
+    return false;
+  }
+
+  ret = readUTF(this->_details, &outStream);
+
+  if (ret <= 0) {
+    return false;
+  }
+
+  // read flow attributes
+  uint32_t numAttributes = 0;
+  ret = read(numAttributes, &outStream);
+  if (ret != 4) {
+    return false;
+  }
+
+  for (uint32_t i = 0; i < numAttributes; i++) {
+    std::string key;
+    ret = readUTF(key, &outStream, true);
+    if (ret <= 0) {
+      return false;
+    }
+    std::string value;
+    ret = readUTF(value, &outStream, true);
+    if (ret <= 0) {
+      return false;
+    }
+    this->_attributes[key] = value;
+  }
+
+  ret = readUTF(this->_contentFullPath, &outStream);
+  if (ret <= 0) {
+    return false;
+  }
+
+  ret = read(this->_size, &outStream);
+  if (ret != 8) {
+    return false;
+  }
+
+  ret = read(this->_offset, &outStream);
+  if (ret != 8) {
+    return false;
+  }
+
+  ret = readUTF(this->_sourceQueueIdentifier, &outStream);
+  if (ret <= 0) {
+    return false;
+  }
+
+  if (this->_eventType == ProvenanceEventRecord::FORK
+      || this->_eventType == ProvenanceEventRecord::CLONE
+      || this->_eventType == ProvenanceEventRecord::JOIN) {
+    // read UUIDs
+    uint32_t number = 0;
+    ret = read(number, &outStream);
+    if (ret != 4) {
+      return false;
+    }
+
+    for (uint32_t i = 0; i < number; i++) {
+      std::string parentUUID;
+      ret = readUTF(parentUUID, &outStream);
+      if (ret <= 0) {
+        return false;
+      }
+      this->addParentUuid(parentUUID);
+    }
+    number = 0;
+    ret = read(number, &outStream);
+    if (ret != 4) {
+      return false;
+    }
+    for (uint32_t i = 0; i < number; i++) {
+      std::string childUUID;
+      ret = readUTF(childUUID, &outStream);
+      if (ret <= 0) {
+        return false;
+      }
+      this->addChildUuid(childUUID);
+    }
+  } else if (this->_eventType == ProvenanceEventRecord::SEND
+      || this->_eventType == ProvenanceEventRecord::FETCH) {
+    ret = readUTF(this->_transitUri, &outStream);
+    if (ret <= 0) {
+      return false;
+    }
+  } else if (this->_eventType == ProvenanceEventRecord::RECEIVE) {
+    ret = readUTF(this->_transitUri, &outStream);
+    if (ret <= 0) {
+      return false;
+    }
+    ret = readUTF(this->_sourceSystemFlowFileIdentifier, &outStream);
+    if (ret <= 0) {
+      return false;
+    }
+  }
+
+  return true;
+}
+
+void ProvenanceReporter::commit() {
+  for (auto event : _events) {
+    if (!repo_->isFull()) {
+      event->Serialize(repo_);
+    } else {
+      logger_->log_debug("Provenance Repository is full");
+    }
+  }
+}
+
+void ProvenanceReporter::create(std::shared_ptr<core::FlowFile> flow,
+                                std::string detail) {
+  ProvenanceEventRecord *event = allocate(ProvenanceEventRecord::CREATE, flow);
+
+  if (event) {
+    event->setDetails(detail);
+    add(event);
+  }
+}
+
+void ProvenanceReporter::route(std::shared_ptr<core::FlowFile> flow,
+                               core::Relationship relation, std::string detail,
+                               uint64_t processingDuration) {
+  ProvenanceEventRecord *event = allocate(ProvenanceEventRecord::ROUTE, flow);
+
+  if (event) {
+    event->setDetails(detail);
+    event->setRelationship(relation.getName());
+    event->setEventDuration(processingDuration);
+    add(event);
+  }
+}
+
+void ProvenanceReporter::modifyAttributes(std::shared_ptr<core::FlowFile> flow,
+                                          std::string detail) {
+  ProvenanceEventRecord *event = allocate(
+      ProvenanceEventRecord::ATTRIBUTES_MODIFIED, flow);
+
+  if (event) {
+    event->setDetails(detail);
+    add(event);
+  }
+}
+
+void ProvenanceReporter::modifyContent(std::shared_ptr<core::FlowFile> flow,
+                                       std::string detail,
+                                       uint64_t processingDuration) {
+  ProvenanceEventRecord *event = allocate(
+      ProvenanceEventRecord::CONTENT_MODIFIED, flow);
+
+  if (event) {
+    event->setDetails(detail);
+    event->setEventDuration(processingDuration);
+    add(event);
+  }
+}
+
+void ProvenanceReporter::clone(std::shared_ptr<core::FlowFile> parent,
+                               std::shared_ptr<core::FlowFile> child) {
+  ProvenanceEventRecord *event = allocate(ProvenanceEventRecord::CLONE, parent);
+
+  if (event) {
+    event->addChildFlowFile(child);
+    event->addParentFlowFile(parent);
+    add(event);
+  }
+}
+
+void ProvenanceReporter::join(
+    std::vector<std::shared_ptr<core::FlowFile> > parents,
+    std::shared_ptr<core::FlowFile> child, std::string detail,
+    uint64_t processingDuration) {
+  ProvenanceEventRecord *event = allocate(ProvenanceEventRecord::JOIN, child);
+
+  if (event) {
+    event->addChildFlowFile(child);
+    std::vector<std::shared_ptr<core::FlowFile> >::iterator it;
+    for (it = parents.begin(); it != parents.end(); it++) {
+      std::shared_ptr<core::FlowFile> record = *it;
+      event->addParentFlowFile(record);
+    }
+    event->setDetails(detail);
+    event->setEventDuration(processingDuration);
+    add(event);
+  }
+}
+
+void ProvenanceReporter::fork(
+    std::vector<std::shared_ptr<core::FlowFile> > child,
+    std::shared_ptr<core::FlowFile> parent, std::string detail,
+    uint64_t processingDuration) {
+  ProvenanceEventRecord *event = allocate(ProvenanceEventRecord::FORK, parent);
+
+  if (event) {
+    event->addParentFlowFile(parent);
+    std::vector<std::shared_ptr<core::FlowFile> >::iterator it;
+    for (it = child.begin(); it != child.end(); it++) {
+      std::shared_ptr<core::FlowFile> record = *it;
+      event->addChildFlowFile(record);
+    }
+    event->setDetails(detail);
+    event->setEventDuration(processingDuration);
+    add(event);
+  }
+}
+
+void ProvenanceReporter::expire(std::shared_ptr<core::FlowFile> flow,
+                                std::string detail) {
+  ProvenanceEventRecord *event = allocate(ProvenanceEventRecord::EXPIRE, flow);
+
+  if (event) {
+    event->setDetails(detail);
+    add(event);
+  }
+}
+
+void ProvenanceReporter::drop(std::shared_ptr<core::FlowFile> flow,
+                              std::string reason) {
+  ProvenanceEventRecord *event = allocate(ProvenanceEventRecord::DROP, flow);
+
+  if (event) {
+    std::string dropReason = "Discard reason: " + reason;
+    event->setDetails(dropReason);
+    add(event);
+  }
+}
+
+void ProvenanceReporter::send(std::shared_ptr<core::FlowFile> flow,
+                              std::string transitUri, std::string detail,
+                              uint64_t processingDuration, bool force) {
+  ProvenanceEventRecord *event = allocate(ProvenanceEventRecord::SEND, flow);
+
+  if (event) {
+    event->setTransitUri(transitUri);
+    event->setDetails(detail);
+    event->setEventDuration(processingDuration);
+    if (!force) {
+      add(event);
+    } else {
+      if (!repo_->isFull())
+        event->Serialize(repo_);
+      delete event;
+    }
+  }
+}
+
+void ProvenanceReporter::receive(std::shared_ptr<core::FlowFile> flow,
+                                 std::string transitUri,
+                                 std::string sourceSystemFlowFileIdentifier,
+                                 std::string detail,
+                                 uint64_t processingDuration) {
+  ProvenanceEventRecord *event = allocate(ProvenanceEventRecord::RECEIVE, flow);
+
+  if (event) {
+    event->setTransitUri(transitUri);
+    event->setDetails(detail);
+    event->setEventDuration(processingDuration);
+    event->setSourceSystemFlowFileIdentifier(sourceSystemFlowFileIdentifier);
+    add(event);
+  }
+}
+
+void ProvenanceReporter::fetch(std::shared_ptr<core::FlowFile> flow,
+                               std::string transitUri, std::string detail,
+                               uint64_t processingDuration) {
+  ProvenanceEventRecord *event = allocate(ProvenanceEventRecord::FETCH, flow);
+
+  if (event) {
+    event->setTransitUri(transitUri);
+    event->setDetails(detail);
+    event->setEventDuration(processingDuration);
+    add(event);
+  }
+}
+
+} /* namespace provenance */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/provenance/ProvenanceRepository.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/provenance/ProvenanceRepository.cpp b/libminifi/src/provenance/ProvenanceRepository.cpp
new file mode 100644
index 0000000..88455be
--- /dev/null
+++ b/libminifi/src/provenance/ProvenanceRepository.cpp
@@ -0,0 +1,75 @@
+/**
+ *
+ * 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 "provenance/Provenance.h"
+#include "provenance/ProvenanceRepository.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace provenance{
+
+
+
+void ProvenanceRepository::run() {
+  // threshold for purge
+  uint64_t purgeThreshold = max_partition_bytes_ * 3 / 4;
+  while (running_) {
+    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((uint8_t *) it->value().data(),
+                                  (int) it->value().size())) {
+          if ((curTime - eventRead.getEventTime())
+              > max_partition_millis_)
+            purgeList.push_back(key);
+        } else {
+          logger_->log_debug("NiFi Provenance retrieve event %s fail",
+                                   key.c_str());
+          purgeList.push_back(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;
+    else
+      repo_full_ = false;
+  }
+  return;
+}
+} /* namespace provenance */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/test/Server.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/Server.cpp b/libminifi/test/Server.cpp
index 65245f6..9428ee0 100644
--- a/libminifi/test/Server.cpp
+++ b/libminifi/test/Server.cpp
@@ -22,7 +22,7 @@
 #define DEFAULT_REPORT_INTERVAL 1000 // 1 sec
 #define MAX_READ_TIMEOUT 30000 // 30 seconds
 
-//! FlowControl Protocol Msg Type
+// FlowControl Protocol Msg Type
 typedef enum {
 	REGISTER_REQ, // Device Register Request from device to server which contain device serial number, current running flow YAML version
 	REGISTER_RESP, // Device Register Respond from server to device, may contain new flow.yml from server ask device to apply and also device report interval
@@ -31,7 +31,7 @@ typedef enum {
 	MAX_FLOW_CONTROL_MSG_TYPE
 } FlowControlMsgType;
 
-//! FlowControl Protocol Msg Type String
+// FlowControl Protocol Msg Type String
 static const char *FlowControlMsgTypeStr[MAX_FLOW_CONTROL_MSG_TYPE] =
 {
 		"REGISTER_REQ",
@@ -40,7 +40,7 @@ static const char *FlowControlMsgTypeStr[MAX_FLOW_CONTROL_MSG_TYPE] =
 		"REPORT_RESP"
 };
 
-//! Flow Control Msg Type to String
+// Flow Control Msg Type to String
 inline const char *FlowControlMsgTypeToStr(FlowControlMsgType type)
 {
 	if (type < MAX_FLOW_CONTROL_MSG_TYPE)
@@ -49,7 +49,7 @@ inline const char *FlowControlMsgTypeToStr(FlowControlMsgType type)
 		return NULL;
 }
 
-//! FlowControll Protocol Msg ID (Some Messages are fix length, Some are variable length (TLV)
+// FlowControll Protocol Msg ID (Some Messages are fix length, Some are variable length (TLV)
 typedef enum {
 	//Fix length 8 bytes: client to server in register request, required field
 	FLOW_SERIAL_NUMBER,
@@ -70,7 +70,7 @@ typedef enum {
 	MAX_FLOW_MSG_ID
 } FlowControlMsgID;
 
-//! FlowControl Protocol Msg ID String
+// FlowControl Protocol Msg ID String
 static const char *FlowControlMsgIDStr[MAX_FLOW_MSG_ID] =
 {
 		"FLOW_SERIAL_NUMBER",
@@ -86,7 +86,7 @@ static const char *FlowControlMsgIDStr[MAX_FLOW_MSG_ID] =
 #define TYPE_HDR_LEN 4 // Fix Hdr Type
 #define TLV_HDR_LEN 8 // Type 4 bytes and Len 4 bytes
 
-//! FlowControl Protocol Msg Len
+// FlowControl Protocol Msg Len
 inline int FlowControlMsgIDEncodingLen(FlowControlMsgID id, int payLoadLen)
 {
 	if (id == FLOW_SERIAL_NUMBER)
@@ -99,7 +99,7 @@ inline int FlowControlMsgIDEncodingLen(FlowControlMsgID id, int payLoadLen)
 		return -1;
 }
 
-//! Flow Control Msg Id to String
+// Flow Control Msg Id to String
 inline const char *FlowControlMsgIdToStr(FlowControlMsgID id)
 {
 	if (id < MAX_FLOW_MSG_ID)
@@ -108,7 +108,7 @@ inline const char *FlowControlMsgIdToStr(FlowControlMsgID id)
 		return NULL;
 }
 
-//! Flow Control Respond status code
+// Flow Control Respond status code
 typedef enum {
 	RESP_SUCCESS,
 	RESP_TRIGGER_REGISTER, // Server respond to client report to re trigger register
@@ -118,7 +118,7 @@ typedef enum {
 	MAX_RESP_CODE
 } FlowControlRespCode;
 
-//! FlowControl Resp Code str
+// FlowControl Resp Code str
 static const char *FlowControlRespCodeStr[MAX_RESP_CODE] =
 {
 		"RESP_SUCCESS",
@@ -128,7 +128,7 @@ static const char *FlowControlRespCodeStr[MAX_RESP_CODE] =
 		"RESP_FAILURE"
 };
 
-//! Flow Control Resp Code to String
+// Flow Control Resp Code to String
 inline const char *FlowControlRespCodeToStr(FlowControlRespCode code)
 {
 	if (code < MAX_RESP_CODE)
@@ -137,16 +137,16 @@ inline const char *FlowControlRespCodeToStr(FlowControlRespCode code)
 		return NULL;
 }
 
-//! Common FlowControlProtocol Header
+// Common FlowControlProtocol Header
 typedef struct {
-	uint32_t msgType; //! Msg Type
-	uint32_t seqNumber; //! Seq Number to match Req with Resp
-	uint32_t status; //! Resp Code, see FlowControlRespCode
-	uint32_t payloadLen; //! Msg Payload length
+	uint32_t msgType; // Msg Type
+	uint32_t seqNumber; // Seq Number to match Req with Resp
+	uint32_t status; // Resp Code, see FlowControlRespCode
+	uint32_t payloadLen; // Msg Payload length
 } FlowControlProtocolHeader;
 
 
-//! encode uint32_t
+// encode uint32_t
 uint8_t *encode(uint8_t *buf, uint32_t value)
 {
 		*buf++ = (value & 0xFF000000) >> 24;
@@ -156,14 +156,14 @@ uint8_t *encode(uint8_t *buf, uint32_t value)
 		return buf;
 }
 
-//! encode uint32_t
+// encode uint32_t
 uint8_t *decode(uint8_t *buf, uint32_t &value)
 {
 		value = ((buf[0]<<24)|(buf[1]<<16)|(buf[2]<<8)|(buf[3]));
 		return (buf + 4);
 }
 
-//! encode byte array
+// encode byte array
 uint8_t *encode(uint8_t *buf, uint8_t *bufArray, int size)
 {
 		memcpy(buf, bufArray, size);
@@ -171,7 +171,7 @@ uint8_t *encode(uint8_t *buf, uint8_t *bufArray, int size)
 		return buf;
 }
 
-//! encode std::string
+// encode std::string
 uint8_t *encode(uint8_t *buf, std::string value)
 {
 		// add the \0 for size

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/test/TestBase.h
----------------------------------------------------------------------
diff --git a/libminifi/test/TestBase.h b/libminifi/test/TestBase.h
index 97c32e6..f73174b 100644
--- a/libminifi/test/TestBase.h
+++ b/libminifi/test/TestBase.h
@@ -23,63 +23,52 @@
 #include "ResourceClaim.h"
 #include "catch.hpp"
 #include <vector>
-#include "Logger.h"
+#include "core/logging/Logger.h"
+#include "core/core.h"
 
 
 class LogTestController {
-public:
-	LogTestController(const std::string level = "debug") {
-		Logger::getLogger()->setLogLevel(level);
-	}
-
-
-	void enableDebug()
-	{
-		Logger::getLogger()->setLogLevel("debug");
-	}
-
-	~LogTestController() {
-		Logger::getLogger()->setLogLevel(LOG_LEVEL_E::info);
-	}
+ public:
+  LogTestController(const std::string level = "debug") {
+    logging::Logger::getLogger()->setLogLevel(level);
+  }
+
+  void enableDebug() {
+    logging::Logger::getLogger()->setLogLevel("debug");
+  }
+
+  ~LogTestController() {
+    logging::Logger::getLogger()->setLogLevel(logging::LOG_LEVEL_E::info);
+  }
 };
 
-class TestController{
-public:
-
-
+class TestController {
+ public:
 
-	TestController() : log("info")
-	{
-		ResourceClaim::default_directory_path = "./";
-	}
+  TestController()
+      : log("info") {
+    minifi::ResourceClaim::default_directory_path = "./";
+  }
 
-	~TestController()
-	{
-		for(auto dir : directories)
-		{
-			rmdir(dir);
-		}
-	}
+  ~TestController() {
+    for (auto dir : directories) {
+      rmdir(dir);
+    }
+  }
 
-	void enableDebug() {
-		log.enableDebug();
-	}
+  void enableDebug() {
+    log.enableDebug();
+  }
 
-	char *createTempDirectory(char *format)
-	{
-		char *dir = mkdtemp(format);
-		return dir;
-	}
-
-protected:
-	LogTestController log;
-	std::vector<char*> directories;
+  char *createTempDirectory(char *format) {
+    char *dir = mkdtemp(format);
+    return dir;
+  }
 
+ protected:
+  LogTestController log;
+  std::vector<char*> directories;
 
 };
 
-
-
-
-
 #endif /* LIBMINIFI_TEST_TESTBASE_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/test/nodefs/NoLevelDB.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/nodefs/NoLevelDB.cpp b/libminifi/test/nodefs/NoLevelDB.cpp
new file mode 100644
index 0000000..00c9212
--- /dev/null
+++ b/libminifi/test/nodefs/NoLevelDB.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 "../TestBase.h"
+
+#include "core/core.h"
+#include "core/RepositoryFactory.h"
+
+TEST_CASE("NoLevelDBTest1", "[NoLevelDBTest]") {
+  std::shared_ptr<core::Repository> prov_repo = core::createRepository(
+      "provenancerepository", true);
+  REQUIRE(nullptr != prov_repo);
+}
+
+TEST_CASE("NoLevelDBTest2", "[NoLevelDBTest]") {
+  std::shared_ptr<core::Repository> prov_repo = core::createRepository(
+      "flowfilerepository", true);
+  REQUIRE(nullptr != prov_repo);
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/test/nodefs/NoYamlConfiguration.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/nodefs/NoYamlConfiguration.cpp b/libminifi/test/nodefs/NoYamlConfiguration.cpp
new file mode 100644
index 0000000..9a9b10e
--- /dev/null
+++ b/libminifi/test/nodefs/NoYamlConfiguration.cpp
@@ -0,0 +1,38 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+#include "core/core.h"
+#include "core/RepositoryFactory.h"
+
+
+#include "core/ConfigurationFactory.h"
+
+TEST_CASE("NoYamlSupport1", "[NoYamlSupport1]") {
+  std::shared_ptr<core::Repository> prov_repo = core::createRepository(
+      "provenancerepository", true);
+REQUIRE(nullptr != prov_repo);
+std::unique_ptr<core::FlowConfiguration> flow_configuration = std::move(
+      core::createFlowConfiguration(prov_repo, prov_repo,
+                                   "yamlconfiguration"));
+
+
+  REQUIRE(nullptr != flow_configuration);
+  
+}
+

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/test/unit/CRCTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/CRCTests.cpp b/libminifi/test/unit/CRCTests.cpp
index 197b5be..54f27bc 100644
--- a/libminifi/test/unit/CRCTests.cpp
+++ b/libminifi/test/unit/CRCTests.cpp
@@ -22,60 +22,58 @@
 #include "io/DataStream.h"
 #include "../TestBase.h"
 
-
 TEST_CASE("Test CRC1", "[testcrc1]") {
 
-	BaseStream base;
-	CRCStream<BaseStream> test(&base);
-	test.writeData((uint8_t*)"cow",3);
-	REQUIRE(2580823964 == test.getCRC());
-	
-	
+  org::apache::nifi::minifi::io::BaseStream base;
+  org::apache::nifi::minifi::io::CRCStream<
+      org::apache::nifi::minifi::io::BaseStream> test(&base);
+  test.writeData((uint8_t*) "cow", 3);
+  REQUIRE(2580823964 == test.getCRC());
+
+
 }
 
 TEST_CASE("Test CRC2", "[testcrc2]") {
 
-	BaseStream base;
-	CRCStream<BaseStream> test(&base);
-	std::string fox = "the quick brown fox jumped over the brown fox";
-	std::vector<uint8_t> charvect(fox.begin(), fox.end()); 
-	test.writeData(charvect,charvect.size());
-	REQUIRE(1922388889 == test.getCRC());
-	
-	
-}
+  org::apache::nifi::minifi::io::BaseStream base;
+  org::apache::nifi::minifi::io::CRCStream<
+      org::apache::nifi::minifi::io::BaseStream> test(&base);
+  std::string fox = "the quick brown fox jumped over the brown fox";
+  std::vector<uint8_t> charvect(fox.begin(), fox.end());
+  test.writeData(charvect, charvect.size());
+  REQUIRE(1922388889 == test.getCRC());
 
+}
 
 TEST_CASE("Test CRC3", "[testcrc3]") {
 
-	BaseStream base;
-	CRCStream<BaseStream> test(&base);
-	uint64_t number=7;
-	test.write(number);
-	REQUIRE(4215687882 == test.getCRC());
-	
-	
-}
+  org::apache::nifi::minifi::io::BaseStream base;
+  org::apache::nifi::minifi::io::CRCStream<
+      org::apache::nifi::minifi::io::BaseStream> test(&base);
+  uint64_t number = 7;
+  test.write(number);
+  REQUIRE(4215687882 == test.getCRC());
 
+}
 
 TEST_CASE("Test CRC4", "[testcrc4]") {
 
-	BaseStream base;
-	CRCStream<BaseStream> test(&base);
-	uint32_t number=7;
-	test.write(number);
-	REQUIRE(3206564543 == test.getCRC());
-	
-	
+  org::apache::nifi::minifi::io::BaseStream base;
+  org::apache::nifi::minifi::io::CRCStream<
+      org::apache::nifi::minifi::io::BaseStream> test(&base);
+  uint32_t number = 7;
+  test.write(number);
+  REQUIRE(3206564543 == test.getCRC());
+
 }
 
 TEST_CASE("Test CRC5", "[testcrc5]") {
 
-	BaseStream base;
-	CRCStream<BaseStream> test(&base);
-	uint16_t number=7;
-	test.write(number);
-	REQUIRE(3753740124 == test.getCRC());
-	
-	
-}
\ No newline at end of file
+  org::apache::nifi::minifi::io::BaseStream base;
+  org::apache::nifi::minifi::io::CRCStream<
+      org::apache::nifi::minifi::io::BaseStream> test(&base);
+  uint16_t number = 7;
+  test.write(number);
+  REQUIRE(3753740124 == test.getCRC());
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/test/unit/LoggerTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/LoggerTests.cpp b/libminifi/test/unit/LoggerTests.cpp
index 8359037..efa30bf 100644
--- a/libminifi/test/unit/LoggerTests.cpp
+++ b/libminifi/test/unit/LoggerTests.cpp
@@ -19,125 +19,191 @@
 #include <memory>
 
 #include "../TestBase.h"
-#include "../../include/LogAppenders.h"
+#include "core/logging/LogAppenders.h"
+
+using namespace logging;
 
 bool contains(std::string stringA, std::string ending) {
-	return (ending.length() > 0 && stringA.find(ending) != std::string::npos);
+  return (ending.length() > 0 && stringA.find(ending) != std::string::npos);
 }
 
 TEST_CASE("Test log Levels", "[ttl1]") {
-	std::ostringstream oss;
-
-	std::unique_ptr<BaseLogger> outputLogger = std::unique_ptr<BaseLogger>(new OutputStreamAppender(oss,0));
-	std::shared_ptr<Logger> logger = Logger::getLogger();
-	logger->updateLogger(std::move(outputLogger));
-	logger->setLogLevel("trace");
-	logger->log_info("hello world");
-
-	REQUIRE( true == contains(oss.str(),"[minifi log -- OutputStreamAppender] [info] hello world"));
+  std::ostringstream oss;
+
+  std::unique_ptr<BaseLogger> outputLogger = std::unique_ptr<BaseLogger>(
+      new org::apache::nifi::minifi::core::logging::OutputStreamAppender(oss,
+                                                                         0));
+  std::shared_ptr<logging::Logger> logger = logging::Logger::getLogger();
+  logger->updateLogger(std::move(outputLogger));
+  logger->setLogLevel("trace");
+  logger->log_info("hello world");
+
+  REQUIRE(
+      true
+          == contains(
+              oss.str(),
+              "[minifi log -- org::apache::nifi::minifi::core::logging::OutputStreamAppender] [info] hello world"));
+
+  std::unique_ptr<BaseLogger> nullAppender = std::unique_ptr<BaseLogger>(
+      new org::apache::nifi::minifi::core::logging::NullAppender());
+
+  logger->updateLogger(std::move(nullAppender));
 }
 
 TEST_CASE("Test log Levels debug", "[ttl2]") {
-	std::ostringstream oss;
-
-	std::unique_ptr<BaseLogger> outputLogger = std::unique_ptr<BaseLogger>(new OutputStreamAppender(oss,0));
-	std::shared_ptr<Logger> logger = Logger::getLogger();
-	logger->updateLogger(std::move(outputLogger));
-	logger->setLogLevel("trace");
-	logger->log_debug("hello world");
-
-	REQUIRE( true == contains(oss.str(),"[minifi log -- OutputStreamAppender] [debug] hello world"));
+  std::ostringstream oss;
+
+  std::unique_ptr<BaseLogger> outputLogger = std::unique_ptr<BaseLogger>(
+      new org::apache::nifi::minifi::core::logging::OutputStreamAppender(oss,
+                                                                         0));
+  std::shared_ptr<logging::Logger> logger = logging::Logger::getLogger();
+  logger->updateLogger(std::move(outputLogger));
+  logger->setLogLevel("trace");
+  logger->log_debug("hello world");
+
+  REQUIRE(
+      true
+          == contains(
+              oss.str(),
+              "[minifi log -- org::apache::nifi::minifi::core::logging::OutputStreamAppender] [debug] hello world"));
+
+  std::unique_ptr<BaseLogger> nullAppender = std::unique_ptr<BaseLogger>(
+      new org::apache::nifi::minifi::core::logging::NullAppender());
+
+  logger->updateLogger(std::move(nullAppender));
 }
 
 TEST_CASE("Test log Levels trace", "[ttl3]") {
-	std::ostringstream oss;
+  std::ostringstream oss;
+
+  std::unique_ptr<BaseLogger> outputLogger = std::unique_ptr<BaseLogger>(
+      new org::apache::nifi::minifi::core::logging::OutputStreamAppender(oss,
+                                                                         0));
+  std::shared_ptr<logging::Logger> logger = logging::Logger::getLogger();
+  logger->updateLogger(std::move(outputLogger));
+  logger->setLogLevel("trace");
+
+  logger->log_trace("hello world");
 
-	std::unique_ptr<BaseLogger> outputLogger = std::unique_ptr<BaseLogger>(new OutputStreamAppender(oss,0));
-	std::shared_ptr<Logger> logger = Logger::getLogger();
-	logger->updateLogger(std::move(outputLogger));
-	logger->setLogLevel("trace");
+  REQUIRE(
+      true
+          == contains(
+              oss.str(),
+              "[minifi log -- org::apache::nifi::minifi::core::logging::OutputStreamAppender] [trace] hello world"));
 
-	logger->log_trace("hello world");
+  std::unique_ptr<BaseLogger> nullAppender = std::unique_ptr<BaseLogger>(
+      new org::apache::nifi::minifi::core::logging::NullAppender());
 
-	REQUIRE( true == contains(oss.str(),"[minifi log -- OutputStreamAppender] [trace] hello world"));
+  logger->updateLogger(std::move(nullAppender));
 }
 
 TEST_CASE("Test log Levels error", "[ttl4]") {
-	std::ostringstream oss;
+  std::ostringstream oss;
 
-	std::unique_ptr<BaseLogger> outputLogger = std::unique_ptr<BaseLogger>(new OutputStreamAppender(oss,0));
-	std::shared_ptr<Logger> logger = Logger::getLogger();
-	logger->updateLogger(std::move(outputLogger));
-	logger->setLogLevel("trace");
+  std::unique_ptr<BaseLogger> outputLogger = std::unique_ptr<BaseLogger>(
+      new org::apache::nifi::minifi::core::logging::OutputStreamAppender(oss,
+                                                                         0));
+  std::shared_ptr<logging::Logger> logger = logging::Logger::getLogger();
+  logger->updateLogger(std::move(outputLogger));
+  logger->setLogLevel("trace");
 
-	logger->log_error("hello world");
+  logger->log_error("hello world");
 
-	REQUIRE( true == contains(oss.str(),"[minifi log -- OutputStreamAppender] [error] hello world"));
+  REQUIRE(
+      true
+          == contains(
+              oss.str(),
+              "[minifi log -- org::apache::nifi::minifi::core::logging::OutputStreamAppender] [error] hello world"));
+
+  std::unique_ptr<BaseLogger> nullAppender = std::unique_ptr<BaseLogger>(
+      new org::apache::nifi::minifi::core::logging::NullAppender());
+
+  logger->updateLogger(std::move(nullAppender));
 }
 
 TEST_CASE("Test log Levels change", "[ttl5]") {
-	std::ostringstream oss;
+  std::ostringstream oss;
+
+  std::unique_ptr<BaseLogger> outputLogger = std::unique_ptr<BaseLogger>(
+      new org::apache::nifi::minifi::core::logging::OutputStreamAppender(oss,
+                                                                         0));
+  std::shared_ptr<logging::Logger> logger = logging::Logger::getLogger();
+  logger->updateLogger(std::move(outputLogger));
+  logger->setLogLevel("trace");
 
-	std::unique_ptr<BaseLogger> outputLogger = std::unique_ptr<BaseLogger>(new OutputStreamAppender(oss,0));
-	std::shared_ptr<Logger> logger = Logger::getLogger();
-	logger->updateLogger(std::move(outputLogger));
-	logger->setLogLevel("trace");
+  logger->log_error("hello world");
 
-	logger->log_error("hello world");
+  REQUIRE(
+      true
+          == contains(
+              oss.str(),
+              "[minifi log -- org::apache::nifi::minifi::core::logging::OutputStreamAppender] [error] hello world"));
+  oss.str("");
+  oss.clear();
+  REQUIRE(0 == oss.str().length());
+  logger->setLogLevel("off");
 
-	REQUIRE( true == contains(oss.str(),"[minifi log -- OutputStreamAppender] [error] hello world"));
-	oss.str("");
-	oss.clear();
-	REQUIRE( 0 == oss.str().length() );
-	logger->setLogLevel("off");
+  logger->log_error("hello world");
 
-	logger->log_error("hello world");
+  REQUIRE(0 == oss.str().length());
+
+  std::unique_ptr<BaseLogger> nullAppender = std::unique_ptr<BaseLogger>(
+      new org::apache::nifi::minifi::core::logging::NullAppender());
+
+  logger->updateLogger(std::move(nullAppender));
 
-	REQUIRE( 0 == oss.str().length() );
 }
 
 TEST_CASE("Test log LevelsConfigured", "[ttl6]") {
-	std::ostringstream oss;
+  std::ostringstream oss;
+
+  minifi::Configure *config = minifi::Configure::getConfigure();
 
-	Configure *config = Configure::getConfigure();
+  config->set(BaseLogger::nifi_log_appender,
+              "OutputStreamAppender");
+  config->set(
+      org::apache::nifi::minifi::core::logging::OutputStreamAppender::nifi_log_output_stream_error_stderr,
+      "true");
 
-	config->set(BaseLogger::nifi_log_appender,"outputstreamappender");
-	config->set(OutputStreamAppender::nifi_log_output_stream_error_stderr,"true");
+  std::shared_ptr<logging::Logger> logger = logging::Logger::getLogger();
 
-	std::shared_ptr<Logger> logger = Logger::getLogger();
+  auto oldrdbuf = std::cerr.rdbuf();
+  std::cerr.rdbuf(oss.rdbuf());
 
-	auto oldrdbuf = std::cerr.rdbuf();
-	std::cerr.rdbuf(oss.rdbuf());
+  std::unique_ptr<BaseLogger> newLogger = LogInstance::getConfiguredLogger(
+      config);
 
-	std::unique_ptr<BaseLogger> newLogger =LogInstance::getConfiguredLogger(config);
+  logger->updateLogger(std::move(newLogger));
 
-	logger->updateLogger(std::move(newLogger));
+  logger->setLogLevel("trace");
 
-	logger->setLogLevel("trace");
+  // capture stderr
+  logger->log_error("hello world");
 
-	// capture stderr
-	logger->log_error("hello world");
+  REQUIRE(
+      true
+          == contains(
+              oss.str(),
+              "[minifi log -- org::apache::nifi::minifi::core::logging::OutputStreamAppender] [error] hello world"));
 
-	REQUIRE( true == contains(oss.str(),"[minifi log -- OutputStreamAppender] [error] hello world"));
+  std::cerr.rdbuf(oldrdbuf);
 
-	std::cerr.rdbuf(oldrdbuf);
+  config->set(BaseLogger::nifi_log_appender, "nullappender");
 
-	config->set(BaseLogger::nifi_log_appender,"nullappender");
+  newLogger = LogInstance::getConfiguredLogger(config);
 
-	newLogger =LogInstance::getConfiguredLogger(config);
+  logger->updateLogger(std::move(newLogger));
 
-	logger->updateLogger(std::move(newLogger));
+  oss.str("");
+  oss.clear();
+  REQUIRE(0 == oss.str().length());
 
-	oss.str("");
-	oss.clear();
-	REQUIRE( 0 == oss.str().length() );
+  // should have nothing from the null appender
+  logger->log_info("hello world");
+  logger->log_debug("hello world");
+  logger->log_trace("hello world");
 
-	// should have nothing from the null appender
-	logger->log_info("hello world");
-	logger->log_debug("hello world");
-	logger->log_trace("hello world");
-//	logger->log_error("hello world");
+  REQUIRE(0 == oss.str().length());
 
-	REQUIRE( 0 == oss.str().length() );
 
 }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/test/unit/ProcessorTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/ProcessorTests.cpp b/libminifi/test/unit/ProcessorTests.cpp
index ae5f7e5..4f08d5d 100644
--- a/libminifi/test/unit/ProcessorTests.cpp
+++ b/libminifi/test/unit/ProcessorTests.cpp
@@ -20,157 +20,255 @@
 #include "FlowController.h"
 #include "ProvenanceTestHelper.h"
 #include "../TestBase.h"
-#include <memory>
-#include "../../include/LogAppenders.h"
-#include "GetFile.h"
+#include "core/logging/LogAppenders.h"
+#include "core/logging/BaseLogger.h"
+#include "processors/GetFile.h"
+#include "core/core.h"
+#include "../../include/core/FlowFile.h"
+#include "core/Processor.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSession.h"
+#include "core/ProcessorNode.h"
+
+TEST_CASE("Test Creation of GetFile", "[getfileCreate]") {
+  org::apache::nifi::minifi::processors::GetFile processor("processorname");
+  REQUIRE(processor.getName() == "processorname");
+}
 
+TEST_CASE("Test Find file", "[getfileCreate2]") {
 
-TEST_CASE("Test Creation of GetFile", "[getfileCreate]"){
-	GetFile processor("processorname");
-	REQUIRE( processor.getName() == "processorname");
-}
+  TestController testController;
+
+  testController.enableDebug();
 
+  
 
-TEST_CASE("Test Find file", "[getfileCreate2]"){
+  std::shared_ptr<core::Processor> processor = std::make_shared<
+      org::apache::nifi::minifi::processors::GetFile>("getfileCreate2");
 
-	TestController testController;
+  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);
 
-	Configure *config = Configure::getConfigure();
+      
 
-	config->set(BaseLogger::nifi_log_appender,"rollingappender");
-	config->set(OutputStreamAppender::nifi_log_output_stream_error_stderr,"true");
-	std::shared_ptr<Logger> logger = Logger::getLogger();
-	std::unique_ptr<BaseLogger> newLogger =LogInstance::getConfiguredLogger(config);
-	logger->updateLogger(std::move(newLogger));
-	logger->setLogLevel("debug");
+  char format[] = "/tmp/gt.XXXXXX";
+  char *dir = testController.createTempDirectory(format);
 
-	ProvenanceTestRepository provenanceRepo;
-	FlowTestRepository flowRepo;
-	TestFlowController controller(provenanceRepo, flowRepo);
-	FlowControllerFactory::getFlowController( dynamic_cast<FlowController*>(&controller));
+  uuid_t processoruuid;
+  REQUIRE(true == processor->getUUID(processoruuid));
 
-	GetFile processor("getfileCreate2");
 
-	char format[] ="/tmp/gt.XXXXXX";
-	char *dir = testController.createTempDirectory(format);
+  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);
 
-	uuid_t processoruuid;
-	REQUIRE( true == processor.getUUID(processoruuid) );
+  connection->setSourceUUID(processoruuid);
+  connection->setDestinationUUID(processoruuid);
 
-	Connection connection("getfileCreate2Connection");
-	connection.setRelationship(Relationship("success","description"));
+  processor->addConnection(connection);
+  REQUIRE(dir != NULL);
 
-	// link the connections so that we can test results at the end for this
+  core::ProcessorNode node(processor);
 
-	connection.setSourceProcessor(&processor);
-	connection.setDestinationProcessor(&processor);
+  core::ProcessContext context(node, test_repo);
+  context.setProperty(org::apache::nifi::minifi::processors::GetFile::Directory,
+                      dir);
+  core::ProcessSession session(&context);
 
-	connection.setSourceProcessorUUID(processoruuid);
-	connection.setDestinationProcessorUUID(processoruuid);
 
-	processor.addConnection(&connection);
-	REQUIRE( dir != NULL );
+  REQUIRE(processor->getName() == "getfileCreate2");
 
-	ProcessContext context(&processor);
-	context.setProperty(GetFile::Directory,dir);
-	ProcessSession session(&context);
+  std::shared_ptr<core::FlowFile> record;
+  processor->setScheduledState(core::ScheduledState::RUNNING);
+  processor->onTrigger(&context, &session);
 
-	REQUIRE( processor.getName() == "getfileCreate2");
+  provenance::ProvenanceReporter *reporter = session.getProvenanceReporter();
+  std::set<provenance::ProvenanceEventRecord*> records = reporter->getEvents();
+  record = session.get();
+  REQUIRE(record == nullptr);
+  REQUIRE(records.size() == 0);
 
-	FlowFileRecord *record;
-	processor.setScheduledState(ScheduledState::RUNNING);
-	processor.onTrigger(&context,&session);
+  std::fstream file;
+  std::stringstream ss;
+  ss << dir << "/" << "tstFile.ext";
+  file.open(ss.str(), std::ios::out);
+  file << "tempFile";
+  file.close();
 
-	ProvenanceReporter *reporter = session.getProvenanceReporter();
-	std::set<ProvenanceEventRecord*> records = reporter->getEvents();
+  processor->incrementActiveTasks();
+  processor->setScheduledState(core::ScheduledState::RUNNING);
+  processor->onTrigger(&context, &session);
+  unlink(ss.str().c_str());
+  rmdir(dir);
+  reporter = session.getProvenanceReporter();
 
-	record = session.get();
-	REQUIRE( record== 0 );
-	REQUIRE( records.size() == 0 );
+  REQUIRE( processor->getName() == "getfileCreate2");
 
-	std::fstream file;
-	std::stringstream ss;
-	std::string fileName("tstFile.ext");
-	ss << dir << "/" << fileName;
-	file.open(ss.str(),std::ios::out);
-	file << "tempFile";
-	int64_t fileSize = file.tellp();
-	file.close();
+  records = reporter->getEvents();
 
-	processor.incrementActiveTasks();
-	processor.setScheduledState(ScheduledState::RUNNING);
+  for (provenance::ProvenanceEventRecord *provEventRecord : records) {
+    REQUIRE(provEventRecord->getComponentType() == processor->getName());
+  }
+  session.commit();
+  std::shared_ptr<core::FlowFile> ffr = session.get();
 
-	processor.onTrigger(&context,&session);
-	unlink(ss.str().c_str());
-	rmdir(dir);
+  ffr->getResourceClaim()->decreaseFlowFileRecordOwnedCount();
+  REQUIRE(2 == repo->getRepoMap().size());
 
-	reporter = session.getProvenanceReporter();
+  for (auto entry : repo->getRepoMap()) {
+    provenance::ProvenanceEventRecord newRecord;
+    newRecord.DeSerialize((uint8_t*) entry.second.data(),
+                          entry.second.length());
 
-	records = reporter->getEvents();
+    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");
 
-	for(ProvenanceEventRecord *provEventRecord : records)
-	{
-		REQUIRE (provEventRecord->getComponentType() == processor.getName());
-	}
-	session.commit();
 
-        // verify flow file repo
-	REQUIRE( 1 == flowRepo.getRepoMap().size() );
+  }
 
-	for(auto  entry: flowRepo.getRepoMap())
-	{
-		FlowFileEventRecord newRecord;
-		newRecord.DeSerialize((uint8_t*)entry.second.data(),entry.second.length());
-		REQUIRE (fileSize == newRecord.getFileSize());
-		REQUIRE (0 == newRecord.getFileOffset());
-		std::map<std::string, std::string> attrs = newRecord.getAttributes();
-		std::string key = FlowAttributeKey(FILENAME);
-		REQUIRE (attrs[key] == fileName);
-	}
+}
 
-	FlowFileRecord *ffr = session.get();
+TEST_CASE("LogAttributeTest", "[getfileCreate3]") {
+  std::ostringstream oss;
+  std::unique_ptr<logging::BaseLogger> outputLogger = std::unique_ptr<
+      logging::BaseLogger>(
+      new org::apache::nifi::minifi::core::logging::OutputStreamAppender(oss,
+                                                                         0));
+  std::shared_ptr<logging::Logger> logger = logging::Logger::getLogger();
+  logger->updateLogger(std::move(outputLogger));
 
-	ffr->getResourceClaim()->decreaseFlowFileRecordOwnedCount();
+  TestController testController;
 
-	delete ffr;
+  testController.enableDebug();
 
-	std::set<FlowFileRecord*> expiredFlows;
 
-	REQUIRE( 2 == provenanceRepo.getRepoMap().size() );
+  std::shared_ptr<core::Repository> repo = std::make_shared<
+      TestRepository>();
 
-	for(auto  entry: provenanceRepo.getRepoMap())
-	{
-		ProvenanceEventRecord newRecord;
-		newRecord.DeSerialize((uint8_t*)entry.second.data(),entry.second.length());
+  std::shared_ptr<core::Processor> processor = std::make_shared<
+      org::apache::nifi::minifi::processors::GetFile>("getfileCreate2");
 
-		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<core::Processor> logAttribute = std::make_shared<
+      org::apache::nifi::minifi::processors::LogAttribute>("logattribute");
 
+  char format[] = "/tmp/gt.XXXXXX";
+  char *dir = testController.createTempDirectory(format);
 
+  uuid_t processoruuid;
+  REQUIRE(true == processor->getUUID(processoruuid));
 
+  uuid_t logattribute_uuid;
+  REQUIRE(true == logAttribute->getUUID(logattribute_uuid));
 
 
+  std::shared_ptr<minifi::Connection> connection = std::make_shared<
+      minifi::Connection>(repo,"getfileCreate2Connection");
+  connection->setRelationship(core::Relationship("success", "description"));
 
+  std::shared_ptr<minifi::Connection> connection2 = std::make_shared<
+      minifi::Connection>(repo,"logattribute");
+  connection2->setRelationship(core::Relationship("success", "description"));
 
+  // link the connections so that we can test results at the end for this
+  connection->setSource(processor);
 
-}
 
+  // link the connections so that we can test results at the end for this
+  connection->setDestination(logAttribute);
+
+  connection2->setSource(logAttribute);
+
+
+  connection2->setSourceUUID(logattribute_uuid);
+  connection->setSourceUUID(processoruuid);
+  connection->setDestinationUUID(logattribute_uuid);
+
+  processor->addConnection(connection);
+  logAttribute->addConnection(connection);
+  logAttribute->addConnection(connection2);
+  REQUIRE(dir != NULL);
+
+  core::ProcessorNode node(processor);
+  core::ProcessorNode node2(logAttribute);
+
+  core::ProcessContext context(node, repo);
+  core::ProcessContext context2(node2, repo);
+  context.setProperty(org::apache::nifi::minifi::processors::GetFile::Directory,
+                      dir);
+  core::ProcessSession session(&context);
+  core::ProcessSession session2(&context2);
 
+  REQUIRE(processor->getName() == "getfileCreate2");
 
+  std::shared_ptr<core::FlowFile> record;
+  processor->setScheduledState(core::ScheduledState::RUNNING);
+  processor->onTrigger(&context, &session);
 
+  logAttribute->incrementActiveTasks();
+  logAttribute->setScheduledState(core::ScheduledState::RUNNING);
+  logAttribute->onTrigger(&context2, &session2);
+
+  provenance::ProvenanceReporter *reporter = session.getProvenanceReporter();
+  std::set<provenance::ProvenanceEventRecord*> records = reporter->getEvents();
+  record = session.get();
+  REQUIRE(record == nullptr);
+  REQUIRE(records.size() == 0);
+
+  std::fstream file;
+  std::stringstream ss;
+  ss << dir << "/" << "tstFile.ext";
+  file.open(ss.str(), std::ios::out);
+  file << "tempFile";
+  file.close();
+
+  processor->incrementActiveTasks();
+  processor->setScheduledState(core::ScheduledState::RUNNING);
+  processor->onTrigger(&context, &session);
+  unlink(ss.str().c_str());
+  rmdir(dir);
+  reporter = session.getProvenanceReporter();
+
+  records = reporter->getEvents();
+  session.commit();
+  oss.str("");
+  oss.clear();
+
+  logAttribute->incrementActiveTasks();
+  logAttribute->setScheduledState(core::ScheduledState::RUNNING);
+  logAttribute->onTrigger(&context2, &session2);
+
+  //session2.commit();
+  records = reporter->getEvents();
+
+  std::string log_attribute_output = oss.str();
+  REQUIRE(
+      log_attribute_output.find("key:absolute.path value:" + ss.str())
+          != std::string::npos);
+  REQUIRE(log_attribute_output.find("Size:8 Offset:0") != std::string::npos);
+  REQUIRE(
+      log_attribute_output.find("key:path value:" + std::string(dir))
+          != std::string::npos);
+
+  outputLogger = std::unique_ptr<logging::BaseLogger>(
+      new org::apache::nifi::minifi::core::logging::NullAppender());
+  logger->updateLogger(std::move(outputLogger));
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/test/unit/PropertyTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/PropertyTests.cpp b/libminifi/test/unit/PropertyTests.cpp
index e9f1c19..dee809f 100644
--- a/libminifi/test/unit/PropertyTests.cpp
+++ b/libminifi/test/unit/PropertyTests.cpp
@@ -16,22 +16,23 @@
  * limitations under the License.
  */
 
+#include "../../include/core/Property.h"
 #include "utils/StringUtils.h"
 #include "../TestBase.h"
-#include "Property.h"
+
 
 TEST_CASE("Test Boolean Conversion", "[testboolConversion]") {
 
 	bool b;
-	REQUIRE(true == StringUtils::StringToBool("true",b));
-	REQUIRE(true == StringUtils::StringToBool("True",b));
-	REQUIRE(true == StringUtils::StringToBool("TRue",b));
-	REQUIRE(true == StringUtils::StringToBool("tRUE",b));
+	REQUIRE(true == org::apache::nifi::minifi::utils::StringUtils::StringToBool("true",b));
+	REQUIRE(true == org::apache::nifi::minifi::utils::StringUtils::StringToBool("True",b));
+	REQUIRE(true == org::apache::nifi::minifi::utils::StringUtils::StringToBool("TRue",b));
+	REQUIRE(true == org::apache::nifi::minifi::utils::StringUtils::StringToBool("tRUE",b));
 
-	REQUIRE(false == StringUtils::StringToBool("FALSE",b));
-	REQUIRE(false == StringUtils::StringToBool("FALLSEY",b));
-	REQUIRE(false == StringUtils::StringToBool("FaLSE",b));
-	REQUIRE(false == StringUtils::StringToBool("false",b));
+	REQUIRE(false == org::apache::nifi::minifi::utils::StringUtils::StringToBool("FALSE",b));
+	REQUIRE(false == org::apache::nifi::minifi::utils::StringUtils::StringToBool("FALLSEY",b));
+	REQUIRE(false == org::apache::nifi::minifi::utils::StringUtils::StringToBool("FaLSE",b));
+	REQUIRE(false == org::apache::nifi::minifi::utils::StringUtils::StringToBool("false",b));
 
 }
 
@@ -41,7 +42,7 @@ TEST_CASE("Test Trimmer Right", "[testTrims]") {
 
 	REQUIRE(test.c_str()[test.length() - 1] == '\n');
 	REQUIRE(test.c_str()[test.length() - 2] == '\t');
-	test = StringUtils::trimRight(test);
+	test = org::apache::nifi::minifi::utils::StringUtils::trimRight(test);
 
 	REQUIRE(test.c_str()[test.length() - 1] == 'd');
 	REQUIRE(test.c_str()[test.length() - 2] == 'a');
@@ -51,7 +52,7 @@ TEST_CASE("Test Trimmer Right", "[testTrims]") {
 	REQUIRE(test.c_str()[test.length() - 1] == '\t');
 	REQUIRE(test.c_str()[test.length() - 2] == '\v');
 
-	test = StringUtils::trimRight(test);
+	test = org::apache::nifi::minifi::utils::StringUtils::trimRight(test);
 
 	REQUIRE(test.c_str()[test.length() - 1] == 'd');
 	REQUIRE(test.c_str()[test.length() - 2] == 'a');
@@ -61,7 +62,7 @@ TEST_CASE("Test Trimmer Right", "[testTrims]") {
 	REQUIRE(test.c_str()[test.length() - 1] == '\f');
 	REQUIRE(test.c_str()[test.length() - 2] == ' ');
 
-	test = StringUtils::trimRight(test);
+	test = org::apache::nifi::minifi::utils::StringUtils::trimRight(test);
 
 	REQUIRE(test.c_str()[test.length() - 1] == 'd');
 
@@ -74,7 +75,7 @@ TEST_CASE("Test Trimmer Left", "[testTrims]") {
 	REQUIRE(test.c_str()[0] == '\t');
 	REQUIRE(test.c_str()[1] == '\n');
 
-	test = StringUtils::trimLeft(test);
+	test = org::apache::nifi::minifi::utils::StringUtils::trimLeft(test);
 
 	REQUIRE(test.c_str()[0] == 'a');
 	REQUIRE(test.c_str()[1] == ' ');
@@ -84,7 +85,7 @@ TEST_CASE("Test Trimmer Left", "[testTrims]") {
 	REQUIRE(test.c_str()[0] == '\v');
 	REQUIRE(test.c_str()[1] == '\t');
 
-	test = StringUtils::trimLeft(test);
+	test = org::apache::nifi::minifi::utils::StringUtils::trimLeft(test);
 
 	REQUIRE(test.c_str()[0] == 'a');
 	REQUIRE(test.c_str()[1] == ' ');
@@ -94,7 +95,7 @@ TEST_CASE("Test Trimmer Left", "[testTrims]") {
 	REQUIRE(test.c_str()[0] == ' ');
 	REQUIRE(test.c_str()[1] == '\f');
 
-	test = StringUtils::trimLeft(test);
+	test = org::apache::nifi::minifi::utils::StringUtils::trimLeft(test);
 
 	REQUIRE(test.c_str()[0] == 'a');
 	REQUIRE(test.c_str()[1] == ' ');

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/test/unit/ProvenanceTestHelper.h
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/ProvenanceTestHelper.h b/libminifi/test/unit/ProvenanceTestHelper.h
index 1ee6a4c..cb8f520 100644
--- a/libminifi/test/unit/ProvenanceTestHelper.h
+++ b/libminifi/test/unit/ProvenanceTestHelper.h
@@ -18,177 +18,120 @@
 #ifndef LIBMINIFI_TEST_UNIT_PROVENANCETESTHELPER_H_
 #define LIBMINIFI_TEST_UNIT_PROVENANCETESTHELPER_H_
 
-#include "Provenance.h"
+#include "provenance/Provenance.h"
 #include "FlowController.h"
-#include "FlowFileRepository.h"
-
-/**
- * Test repository
- */
-class FlowTestRepository : public FlowFileRepository
-{
-public:
-	FlowTestRepository()
-{
-}
-		//! initialize
-		bool initialize()
-		{
-			return true;
-		}
-
-		//! Destructor
-		virtual ~FlowTestRepository() {
-
-		}
-
-		bool Put(std::string key, uint8_t *buf, int bufLen)
-		{
-			repositoryResults.insert(std::pair<std::string,std::string>(key,std::string((const char*)buf,bufLen)));
-			return true;
-		}
-		//! Delete
-		bool Delete(std::string key)
-		{
-			repositoryResults.erase(key);
-			return true;
-		}
-		//! Get
-		bool Get(std::string key, std::string &value)
-		{
-			auto result = repositoryResults.find(key);
-			if (result != repositoryResults.end())
-			{
-				value = result->second;
-				return true;
-			}
-			else
-			{
-				return false;
-			}
-		}
-
-		const std::map<std::string,std::string> &getRepoMap() const
-		{
-			return repositoryResults;
-		}
-
-protected:
-		std::map<std::string,std::string> repositoryResults;
-};
-
+#include "core/Repository.h"
+#include "core/core.h"
 /**
  * Test repository
  */
-class ProvenanceTestRepository : public ProvenanceRepository
-{
-public:
-	ProvenanceTestRepository()
-{
-}
-		//! initialize
-		bool initialize()
-		{
-			return true;
-		}
-
-		//! Destructor
-		virtual ~ProvenanceTestRepository() {
-
-		}
-
-		bool Put(std::string key, uint8_t *buf, int bufLen)
-		{
-			repositoryResults.insert(std::pair<std::string,std::string>(key,std::string((const char*)buf,bufLen)));
-			return true;
-		}
-		//! Delete
-		bool Delete(std::string key)
-		{
-			repositoryResults.erase(key);
-			return true;
-		}
-		//! Get
-		bool Get(std::string key, std::string &value)
-		{
-			auto result = repositoryResults.find(key);
-			if (result != repositoryResults.end())
-			{
-				value = result->second;
-				return true;
-			}
-			else
-			{
-				return false;
-			}
-		}
-
-		const std::map<std::string,std::string> &getRepoMap() const
-		{
-			return repositoryResults;
-		}
-
-protected:
-		std::map<std::string,std::string> repositoryResults;
+class TestRepository : public core::Repository {
+ public:
+  TestRepository()
+      : Repository("repo_name", "./dir", 1000, 100, 0) {
+  }
+  // initialize
+  bool initialize() {
+    return true;
+  }
+
+  // Destructor
+  virtual ~TestRepository() {
+
+  }
+
+  bool Put(std::string key, uint8_t *buf, int bufLen) {
+    repositoryResults.insert(
+        std::pair<std::string, std::string>(
+            key, std::string((const char*) buf, bufLen)));
+    return true;
+  }
+  // Delete
+  bool Delete(std::string key) {
+    repositoryResults.erase(key);
+    return true;
+  }
+  // Get
+  bool Get(std::string key, std::string &value) {
+    auto result = repositoryResults.find(key);
+    if (result != repositoryResults.end()) {
+      value = result->second;
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  const std::map<std::string, std::string> &getRepoMap() const {
+    return repositoryResults;
+  }
+
+  void run() {
+    // do nothing
+  }
+ protected:
+  std::map<std::string, std::string> repositoryResults;
 };
 
-
-class TestFlowController : public FlowController
-{
-
-public:
-	TestFlowController(ProvenanceTestRepository &provenanceRepo, FlowTestRepository &flowRepo) : ::FlowController()
-	{
-		_provenanceRepo = dynamic_cast<ProvenanceRepository*>(&provenanceRepo);
-		_flowfileRepo = dynamic_cast<FlowFileRepository*>(&flowRepo);
-	}
-	~TestFlowController()
-	{
-
-	}
-	void load(){
-
-	}
-
-	bool start()
-	{
-		_running.store(true);
-		return true;
-	}
-
-	void stop(bool force)
-	{
-		_running.store(false);
-	}
-	void waitUnload(const uint64_t timeToWaitMs)
-	{
-		stop(true);
-	}
-
-	void unload()
-	{
-		stop(true);
-	}
-
-	void reload(std::string file)
-	{
-
-	}
-
-	bool isRunning()
-	{
-		return true;
-	}
-
-
-	Processor *createProcessor(std::string name, uuid_t uuid){ return 0;}
-
-	ProcessGroup *createRootProcessGroup(std::string name, uuid_t uuid){ return 0;}
-
-	ProcessGroup *createRemoteProcessGroup(std::string name, uuid_t uuid){ return 0; }
-
-	Connection *createConnection(std::string name, uuid_t uuid){ return 0; }
+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, nullptr, "",true) {
+  }
+  ~TestFlowController() {
+
+  }
+  void load() {
+
+  }
+
+  bool start() {
+    running_.store(true);
+    return true;
+  }
+
+  void stop(bool force) {
+    running_.store(false);
+  }
+  void waitUnload(const uint64_t timeToWaitMs) {
+    stop(true);
+  }
+
+  void unload() {
+    stop(true);
+  }
+
+  void reload(std::string file) {
+
+  }
+
+  bool isRunning() {
+    return true;
+  }
+
+  std::shared_ptr<core::Processor> createProcessor(std::string name,
+                                                   uuid_t uuid) {
+    return 0;
+  }
+
+  core::ProcessGroup *createRootProcessGroup(std::string name, uuid_t uuid) {
+    return 0;
+  }
+
+  core::ProcessGroup *createRemoteProcessGroup(std::string name, uuid_t uuid) {
+    return 0;
+  }
+
+  std::shared_ptr<minifi::Connection> createConnection(std::string name,
+                                                       uuid_t uuid) {
+    return 0;
+  }
+ protected:
+  void initializePaths(const std::string &adjustedFilename) {
+    std::cout << "what" << std::endl;
+  }
 };
 
-
 #endif /* LIBMINIFI_TEST_UNIT_PROVENANCETESTHELPER_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/test/unit/ProvenanceTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/ProvenanceTests.cpp b/libminifi/test/unit/ProvenanceTests.cpp
index d78de47..624601c 100644
--- a/libminifi/test/unit/ProvenanceTests.cpp
+++ b/libminifi/test/unit/ProvenanceTests.cpp
@@ -16,81 +16,81 @@
  * limitations under the License.
  */
 
-
 #ifndef PROVENANCE_TESTS
 #define PROVENANCE_TESTS
 #include "../TestBase.h"
 
 #include "ProvenanceTestHelper.h"
-#include "Provenance.h"
+#include "provenance/Provenance.h"
 #include "FlowFileRecord.h"
+#include "core/core.h"
+#include "core/repository/FlowFileRepository.h"
 
+TEST_CASE("Test Provenance record create", "[Testprovenance::ProvenanceEventRecord]") {
 
-
-TEST_CASE("Test Provenance record create", "[TestProvenanceEventRecord]"){
-
-	ProvenanceEventRecord record1(ProvenanceEventRecord::ProvenanceEventType::CREATE,"blah","blahblah");
-	REQUIRE( record1.getAttributes().size() == 0);
-	REQUIRE( record1.getAlternateIdentifierUri().length() == 0);
+  provenance::ProvenanceEventRecord record1(
+      provenance::ProvenanceEventRecord::ProvenanceEventType::CREATE, "blah",
+      "blahblah");
+  REQUIRE(record1.getAttributes().size() == 0);
+  REQUIRE(record1.getAlternateIdentifierUri().length() == 0);
 
 }
 
+TEST_CASE("Test Provenance record serialization", "[Testprovenance::ProvenanceEventRecordSerializeDeser]") {
 
-TEST_CASE("Test Provenance record serialization", "[TestProvenanceEventRecordSerializeDeser]"){
+  provenance::ProvenanceEventRecord record1(
+      provenance::ProvenanceEventRecord::ProvenanceEventType::CREATE, "componentid",
+      "componenttype");
 
-	ProvenanceEventRecord record1(ProvenanceEventRecord::ProvenanceEventType::CREATE,"componentid","componenttype");
+  std::string eventId = record1.getEventId();
 
-	std::string eventId = record1.getEventId();
-	
-	std::string smileyface = ":)" ;
-	record1.setDetails(smileyface);
+  std::string smileyface = ":)";
+  record1.setDetails(smileyface);
 
-	ProvenanceTestRepository repo;
-	uint64_t sample = 65555;
-	ProvenanceRepository *testRepository = dynamic_cast<ProvenanceRepository*>(&repo);
-	record1.setEventDuration(sample);
+  uint64_t sample = 65555;
+  std::shared_ptr<core::Repository> testRepository =std::make_shared<TestRepository>();
+  record1.setEventDuration(sample);
 
-	record1.Serialize(testRepository);
-	ProvenanceEventRecord record2;
-	REQUIRE( record2.DeSerialize(testRepository,eventId) == true);
-	REQUIRE( record2.getEventId() == record1.getEventId());
-	REQUIRE( record2.getComponentId() == record1.getComponentId());
-	REQUIRE( record2.getComponentType() == record1.getComponentType());
-	REQUIRE( record2.getDetails() == record1.getDetails());
-	REQUIRE( record2.getDetails() == smileyface);
-	REQUIRE( record2.getEventDuration() == sample);
+  record1.Serialize(testRepository);
+  provenance::ProvenanceEventRecord record2;
+  REQUIRE(record2.DeSerialize(testRepository,eventId) == true);
+  REQUIRE(record2.getEventId() == record1.getEventId());
+  REQUIRE(record2.getComponentId() == record1.getComponentId());
+  REQUIRE(record2.getComponentType() == record1.getComponentType());
+  REQUIRE(record2.getDetails() == record1.getDetails());
+  REQUIRE(record2.getDetails() == smileyface);
+  REQUIRE(record2.getEventDuration() == sample);
 }
 
-
-TEST_CASE("Test Flowfile record added to provenance", "[TestFlowAndProv1]"){
-
-	ProvenanceEventRecord record1(ProvenanceEventRecord::ProvenanceEventType::CLONE,"componentid","componenttype");
-	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"));
-	FlowFileRecord ffr1(attributes);
-
-	record1.addChildFlowFile(&ffr1);
-
-	ProvenanceTestRepository repo;
-	uint64_t sample = 65555;
-	ProvenanceRepository *testRepository = dynamic_cast<ProvenanceRepository*>(&repo);
-	record1.setEventDuration(sample);
-
-	record1.Serialize(testRepository);
-	ProvenanceEventRecord record2;
-	REQUIRE( record2.DeSerialize(testRepository,eventId) == true);
-	REQUIRE( record1.getChildrenUuids().size() == 1);
-	REQUIRE( record2.getChildrenUuids().size() == 1);
-	std::string childId = record2.getChildrenUuids().at(0);
-	REQUIRE( childId == ffr1.getUUIDStr());
-	record2.removeChildUuid(childId);
-	REQUIRE( record2.getChildrenUuids().size() == 0);
-
+TEST_CASE("Test Flowfile record added to provenance", "[TestFlowAndProv1]") {
+
+  provenance::ProvenanceEventRecord record1(
+      provenance::ProvenanceEventRecord::ProvenanceEventType::CLONE, "componentid",
+      "componenttype");
+  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>("./content_repository",0,0,0);
+  std::shared_ptr<minifi::FlowFileRecord> ffr1 = std::make_shared<
+      minifi::FlowFileRecord>(frepo,attributes);
+
+  record1.addChildFlowFile(ffr1);
+
+   uint64_t sample = 65555;
+  std::shared_ptr<core::Repository> testRepository =std::make_shared<TestRepository>();
+  record1.setEventDuration(sample);
+
+  record1.Serialize(testRepository);
+  provenance::ProvenanceEventRecord record2;
+  REQUIRE(record2.DeSerialize(testRepository,eventId) == true);
+  REQUIRE(record1.getChildrenUuids().size() == 1);
+  REQUIRE(record2.getChildrenUuids().size() == 1);
+  std::string childId = record2.getChildrenUuids().at(0);
+  REQUIRE(childId == ffr1->getUUIDStr());
+  record2.removeChildUuid(childId);
+  REQUIRE(record2.getChildrenUuids().size() == 0);
 
 }
 
-
-
 #endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/test/unit/SerializationTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/SerializationTests.cpp b/libminifi/test/unit/SerializationTests.cpp
index 96540bd..c6ada33 100644
--- a/libminifi/test/unit/SerializationTests.cpp
+++ b/libminifi/test/unit/SerializationTests.cpp
@@ -28,13 +28,13 @@
 #define FMT_DEFAULT fmt_lower
 
 
-
+using namespace org::apache::nifi::minifi::io;
 TEST_CASE("TestSetPortId", "[S2S1]"){
 
 
-	Site2SitePeer peer(std::unique_ptr<DataStream>(new DataStream()),"fake_host",65433);
+	std::unique_ptr<minifi::Site2SitePeer> peer = std::unique_ptr<minifi::Site2SitePeer>( new minifi::Site2SitePeer(std::unique_ptr<DataStream>(new DataStream()),"fake_host",65433));
 
-	Site2SiteClientProtocol protocol(&peer);
+	minifi::Site2SiteClientProtocol protocol(std::move(peer));
 
 
 	std::string uuid_str = "c56a4180-65aa-42ec-a945-5fd21dec0538";
@@ -54,9 +54,9 @@ TEST_CASE("TestSetPortId", "[S2S1]"){
 TEST_CASE("TestSetPortIdUppercase", "[S2S2]"){
 
 
-	Site2SitePeer peer(std::unique_ptr<DataStream>(new DataStream()),"fake_host",65433);
+  std::unique_ptr<minifi::Site2SitePeer> peer = std::unique_ptr<minifi::Site2SitePeer>( new minifi::Site2SitePeer(std::unique_ptr<DataStream>(new DataStream()),"fake_host",65433));
 
-	Site2SiteClientProtocol protocol(&peer);
+  minifi::Site2SiteClientProtocol protocol(std::move(peer));
 
 
 	std::string uuid_str = "C56A4180-65AA-42EC-A945-5FD21DEC0538";

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/test/unit/SocketTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/SocketTests.cpp b/libminifi/test/unit/SocketTests.cpp
index 3fb7c9e..e735f15 100644
--- a/libminifi/test/unit/SocketTests.cpp
+++ b/libminifi/test/unit/SocketTests.cpp
@@ -17,170 +17,159 @@
  */
 
 #include "../TestBase.h"
-#include "../../include/io/ClientSocket.h"
+#include "io/ClientSocket.h"
+
+using namespace org::apache::nifi::minifi::io;
 TEST_CASE("TestSocket", "[TestSocket1]") {
 
-	Socket socket("localhost",8183);
-	REQUIRE(-1 == socket.initialize() );
-	REQUIRE("localhost" == socket.getHostname());
-	socket.closeStream();
+  Socket socket("localhost", 8183);
+  REQUIRE(-1 == socket.initialize());
+  REQUIRE("localhost" == socket.getHostname());
+  socket.closeStream();
 
 }
 
 TEST_CASE("TestSocketWriteTest1", "[TestSocket2]") {
 
-	Socket socket("localhost",8183);
-	REQUIRE(-1 == socket.initialize() );
+  Socket socket("localhost", 8183);
+  REQUIRE(-1 == socket.initialize());
 
-	socket.writeData(0,0);
+  socket.writeData(0, 0);
 
-	std::vector<uint8_t> buffer;
-	buffer.push_back('a');
+  std::vector<uint8_t> buffer;
+  buffer.push_back('a');
 
-	REQUIRE(-1 == socket.writeData(buffer,1));
+  REQUIRE(-1 == socket.writeData(buffer, 1));
 
-	socket.closeStream();	
+  socket.closeStream();
 
 }
 
 TEST_CASE("TestSocketWriteTest2", "[TestSocket3]") {
 
-		std::vector<uint8_t> buffer;
-	buffer.push_back('a');
+  std::vector<uint8_t> buffer;
+  buffer.push_back('a');
+
+  Socket server("localhost", 9183, 1);
 
-	Socket server("localhost",9183,1);
+  REQUIRE(-1 != server.initialize());
 
-	REQUIRE(-1 != server.initialize() );
+  Socket client("localhost", 9183);
 
-	Socket client("localhost",9183);
+  REQUIRE(-1 != client.initialize());
 
-	REQUIRE(-1 != client.initialize() );
+  REQUIRE(1 == client.writeData(buffer, 1));
 
-	REQUIRE( 1 == client.writeData(buffer,1) );
+  std::vector<uint8_t> readBuffer;
+  readBuffer.resize(1);
 
-	std::vector<uint8_t> readBuffer;
-	readBuffer.resize(1);
+  REQUIRE(1 == server.readData(readBuffer, 1));
 
-	REQUIRE( 1== server.readData(readBuffer,1) );
+  REQUIRE(readBuffer == buffer);
 
-	REQUIRE( readBuffer == buffer );
+  server.closeStream();
 
-	server.closeStream();
-	
-	
-	client.closeStream();	
+  client.closeStream();
 
 }
 
 TEST_CASE("TestGetHostName", "[TestSocket4]") {
 
-	REQUIRE( Socket::getMyHostName().length() > 0 );
-
-
+  REQUIRE(Socket::getMyHostName().length() > 0);
 
 }
 
 TEST_CASE("TestWriteEndian64", "[TestSocket4]") {
 
-		std::vector<uint8_t> buffer;
-	buffer.push_back('a');
+  std::vector<uint8_t> buffer;
+  buffer.push_back('a');
 
-	Socket server("localhost",9183,1);
+  Socket server("localhost", 9183, 1);
 
-	REQUIRE(-1 != server.initialize() );
+  REQUIRE(-1 != server.initialize());
 
-	Socket client("localhost",9183);
+  Socket client("localhost", 9183);
 
-	REQUIRE(-1 != client.initialize() );
+  REQUIRE(-1 != client.initialize());
 
-	uint64_t negative_one = -1;
-	REQUIRE( 8 == client.write(negative_one) );
+  uint64_t negative_one = -1;
+  REQUIRE(8 == client.write(negative_one));
 
+  uint64_t negative_two = 0;
+  REQUIRE(8 == server.read(negative_two));
 
-	uint64_t negative_two = 0;
-	REQUIRE( 8 == server.read(negative_two) );
+  REQUIRE(negative_two == negative_one);
 
-	REQUIRE( negative_two == negative_one );
+  server.closeStream();
 
-
-	server.closeStream();
-	
-	
-	client.closeStream();	
+  client.closeStream();
 
 }
 
 TEST_CASE("TestWriteEndian32", "[TestSocket5]") {
 
-		std::vector<uint8_t> buffer;
-	buffer.push_back('a');
+  std::vector<uint8_t> buffer;
+  buffer.push_back('a');
 
-	Socket server("localhost",9183,1);
+  Socket server("localhost", 9183, 1);
 
-	REQUIRE(-1 != server.initialize() );
+  REQUIRE(-1 != server.initialize());
 
-	Socket client("localhost",9183);
+  Socket client("localhost", 9183);
 
-	REQUIRE(-1 != client.initialize() );
+  REQUIRE(-1 != client.initialize());
 
-	{
-	uint32_t negative_one = -1;
-	REQUIRE( 4 == client.write(negative_one) );
+  {
+    uint32_t negative_one = -1;
+    REQUIRE(4 == client.write(negative_one));
 
+    uint32_t negative_two = 0;
+    REQUIRE(4 == server.read(negative_two));
 
-	uint32_t negative_two = 0;
-	REQUIRE( 4 == server.read(negative_two) );
+    REQUIRE(negative_two == negative_one);
+  }
 
-	REQUIRE( negative_two == negative_one );
-	}
-	
-	{
-	uint16_t negative_one = -1;
-	REQUIRE( 2 == client.write(negative_one) );
+  {
+    uint16_t negative_one = -1;
+    REQUIRE(2 == client.write(negative_one));
 
+    uint16_t negative_two = 0;
+    REQUIRE(2 == server.read(negative_two));
 
-	uint16_t negative_two = 0;
-	REQUIRE( 2 == server.read(negative_two) );
+    REQUIRE(negative_two == negative_one);
+  }
+  server.closeStream();
 
-	REQUIRE( negative_two == negative_one );
-	}
-	server.closeStream();
-	
-	
-	client.closeStream();	
+  client.closeStream();
 
 }
 
-
 TEST_CASE("TestSocketWriteTestAfterClose", "[TestSocket6]") {
 
-		std::vector<uint8_t> buffer;
-	buffer.push_back('a');
+  std::vector<uint8_t> buffer;
+  buffer.push_back('a');
 
-	Socket server("localhost",9183,1);
+  Socket server("localhost", 9183, 1);
 
-	REQUIRE(-1 != server.initialize() );
+  REQUIRE(-1 != server.initialize());
 
-	Socket client("localhost",9183);
+  Socket client("localhost", 9183);
 
-	REQUIRE(-1 != client.initialize() );
+  REQUIRE(-1 != client.initialize());
 
-	REQUIRE( 1 == client.writeData(buffer,1) );
+  REQUIRE(1 == client.writeData(buffer, 1));
 
-	std::vector<uint8_t> readBuffer;
-	readBuffer.resize(1);
+  std::vector<uint8_t> readBuffer;
+  readBuffer.resize(1);
 
-	REQUIRE( 1== server.readData(readBuffer,1) );
+  REQUIRE(1 == server.readData(readBuffer, 1));
 
-	REQUIRE( readBuffer == buffer );
-	
-	client.closeStream();
-	
-	REQUIRE( -1 == client.writeData(buffer,1) );	
-	
-	server.closeStream();
+  REQUIRE(readBuffer == buffer);
 
+  client.closeStream();
 
+  REQUIRE(-1 == client.writeData(buffer, 1));
 
-}
+  server.closeStream();
 
+}


[12/16] nifi-minifi-cpp git commit: MINIFI-217: Updates namespaces and removes use of raw pointers for user facing API.

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/core/Property.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/Property.h b/libminifi/include/core/Property.h
new file mode 100644
index 0000000..c681449
--- /dev/null
+++ b/libminifi/include/core/Property.h
@@ -0,0 +1,264 @@
+/**
+ *
+ * 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 __PROPERTY_H__
+#define __PROPERTY_H__
+
+#include <algorithm>
+#include <sstream>
+#include <string>
+#include <vector>
+#include <queue>
+#include <map>
+#include <mutex>
+#include <atomic>
+#include <functional>
+#include <set>
+#include <stdlib.h>
+#include <math.h>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+// Time Unit
+enum TimeUnit {
+  DAY,
+  HOUR,
+  MINUTE,
+  SECOND,
+  MILLISECOND,
+  NANOSECOND
+};
+
+// Property Class
+class Property {
+
+ public:
+  // Constructor
+  /*!
+   * Create a new property
+   */
+  Property(const std::string name, const std::string description,
+           const std::string value)
+      : name_(name),
+        description_(description),
+        value_(value) {
+  }
+  Property() {
+  }
+  // Destructor
+  virtual ~Property() {
+  }
+  // Get Name for the property
+  std::string getName() const;
+  // Get Description for the property
+  std::string getDescription();
+  // Get value for the property
+  std::string getValue() const;
+  // Set value for the property
+  void setValue(std::string value);
+  const Property &operator=(const Property &other);
+  // Compare
+  bool operator <(const Property & right) const;
+
+  // Convert TimeUnit to MilliSecond
+  static bool ConvertTimeUnitToMS(int64_t input, TimeUnit unit, int64_t &out) {
+    if (unit == MILLISECOND) {
+      out = input;
+      return true;
+    } else if (unit == SECOND) {
+      out = input * 1000;
+      return true;
+    } else if (unit == MINUTE) {
+      out = input * 60 * 1000;
+      return true;
+    } else if (unit == HOUR) {
+      out = input * 60 * 60 * 1000;
+      return true;
+    } else if (unit == DAY) {
+      out = 24 * 60 * 60 * 1000;
+      return true;
+    } else if (unit == NANOSECOND) {
+      out = input / 1000 / 1000;
+      return true;
+    } else {
+      return false;
+    }
+  }
+  // Convert TimeUnit to NanoSecond
+  static bool ConvertTimeUnitToNS(int64_t input, TimeUnit unit, int64_t &out) {
+    if (unit == MILLISECOND) {
+      out = input * 1000 * 1000;
+      return true;
+    } else if (unit == SECOND) {
+      out = input * 1000 * 1000 * 1000;
+      return true;
+    } else if (unit == MINUTE) {
+      out = input * 60 * 1000 * 1000 * 1000;
+      return true;
+    } else if (unit == HOUR) {
+      out = input * 60 * 60 * 1000 * 1000 * 1000;
+      return true;
+    } else if (unit == NANOSECOND) {
+      out = input;
+      return true;
+    } else {
+      return false;
+    }
+  }
+  // Convert String
+  static bool StringToTime(std::string input, int64_t &output,
+                           TimeUnit &timeunit) {
+    if (input.size() == 0) {
+      return false;
+    }
+
+    const char *cvalue = input.c_str();
+    char *pEnd;
+    long int ival = strtol(cvalue, &pEnd, 0);
+
+    if (pEnd[0] == '\0') {
+      return false;
+    }
+
+    while (*pEnd == ' ') {
+      // Skip the space
+      pEnd++;
+    }
+
+    std::string unit(pEnd);
+
+    if (unit == "sec" || unit == "s" || unit == "second" || unit == "seconds"
+        || unit == "secs") {
+      timeunit = SECOND;
+      output = ival;
+      return true;
+    } else if (unit == "min" || unit == "m" || unit == "mins"
+        || unit == "minute" || unit == "minutes") {
+      timeunit = MINUTE;
+      output = ival;
+      return true;
+    } else if (unit == "ns" || unit == "nano" || unit == "nanos"
+        || unit == "nanoseconds") {
+      timeunit = NANOSECOND;
+      output = ival;
+      return true;
+    } else if (unit == "ms" || unit == "milli" || unit == "millis"
+        || unit == "milliseconds") {
+      timeunit = MILLISECOND;
+      output = ival;
+      return true;
+    } else if (unit == "h" || unit == "hr" || unit == "hour" || unit == "hrs"
+        || unit == "hours") {
+      timeunit = HOUR;
+      output = ival;
+      return true;
+    } else if (unit == "d" || unit == "day" || unit == "days") {
+      timeunit = DAY;
+      output = ival;
+      return true;
+    } else
+      return false;
+  }
+
+  // Convert String to Integer
+  static bool StringToInt(std::string input, int64_t &output) {
+    if (input.size() == 0) {
+      return false;
+    }
+
+    const char *cvalue = input.c_str();
+    char *pEnd;
+    long int ival = strtol(cvalue, &pEnd, 0);
+
+    if (pEnd[0] == '\0') {
+      output = ival;
+      return true;
+    }
+
+    while (*pEnd == ' ') {
+      // Skip the space
+      pEnd++;
+    }
+
+    char end0 = toupper(pEnd[0]);
+    if ((end0 == 'K') || (end0 == 'M') || (end0 == 'G') || (end0 == 'T')
+        || (end0 == 'P')) {
+      if (pEnd[1] == '\0') {
+        unsigned long int multiplier = 1000;
+
+        if ((end0 != 'K')) {
+          multiplier *= 1000;
+          if (end0 != 'M') {
+            multiplier *= 1000;
+            if (end0 != 'G') {
+              multiplier *= 1000;
+              if (end0 != 'T') {
+                multiplier *= 1000;
+              }
+            }
+          }
+        }
+        output = ival * multiplier;
+        return true;
+
+      } else if ((pEnd[1] == 'b' || pEnd[1] == 'B') && (pEnd[2] == '\0')) {
+
+        unsigned long int multiplier = 1024;
+
+        if ((end0 != 'K')) {
+          multiplier *= 1024;
+          if (end0 != 'M') {
+            multiplier *= 1024;
+            if (end0 != 'G') {
+              multiplier *= 1024;
+              if (end0 != 'T') {
+                multiplier *= 1024;
+              }
+            }
+          }
+        }
+        output = ival * multiplier;
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+ protected:
+  // Name
+  std::string name_;
+  // Description
+  std::string description_;
+  // Value
+  std::string value_;
+
+ private:
+
+};
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/core/Relationship.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/Relationship.h b/libminifi/include/core/Relationship.h
new file mode 100644
index 0000000..416ede6
--- /dev/null
+++ b/libminifi/include/core/Relationship.h
@@ -0,0 +1,96 @@
+/**
+ * @file Relationship.h
+ * Relationship class declaration
+ *
+ * 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 __RELATIONSHIP_H__
+#define __RELATIONSHIP_H__
+
+#include <string>
+#include <uuid/uuid.h>
+#include <vector>
+#include <queue>
+#include <map>
+#include <mutex>
+#include <atomic>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+// undefined relationship for remote process group outgoing port and root process group incoming port
+#define UNDEFINED_RELATIONSHIP "undefined"
+
+inline bool isRelationshipNameUndefined(std::string name) {
+  if (name == UNDEFINED_RELATIONSHIP)
+    return true;
+  else
+    return false;
+}
+
+// Relationship Class
+class Relationship {
+
+ public:
+  /*
+   * Create a new relationship 
+   */
+  Relationship(const std::string name, const std::string description)
+      : name_(name),
+        description_(description) {
+  }
+  Relationship()
+      : name_(UNDEFINED_RELATIONSHIP) {
+  }
+  // Destructor
+  virtual ~Relationship() {
+  }
+  // Get Name for the relationship
+  std::string getName() const {
+    return name_;
+  }
+  // Get Description for the relationship
+  std::string getDescription() const {
+    return description_;
+  }
+  // Compare
+  bool operator <(const Relationship & right) const {
+    return name_ < right.name_;
+  }
+  // Whether it is a undefined relationship
+  bool isRelationshipUndefined() {
+    return isRelationshipNameUndefined(name_);
+  }
+
+ protected:
+
+  // Name
+  std::string name_;
+  // Description
+  std::string description_;
+
+ private:
+};
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/core/Repository.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/Repository.h b/libminifi/include/core/Repository.h
new file mode 100644
index 0000000..a668df5
--- /dev/null
+++ b/libminifi/include/core/Repository.h
@@ -0,0 +1,153 @@
+/**
+ * @file Repository 
+ * Repository class declaration
+ *
+ * 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 __REPOSITORY_H__
+#define __REPOSITORY_H__
+
+#include <ftw.h>
+#include <uuid/uuid.h>
+#include <atomic>
+#include <cstdint>
+#include <cstring>
+#include <iostream>
+#include <map>
+#include <set>
+#include <string>
+#include <thread>
+#include <vector>
+
+#include "properties/Configure.h"
+#include "core/logging/Logger.h"
+#include "core/Property.h"
+#include "ResourceClaim.h"
+#include "io/Serializable.h"
+#include "utils/TimeUtil.h"
+#include "utils/StringUtils.h"
+#include "core.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class Repository : public CoreComponent {
+ public:
+  /*
+   * Constructor for the repository
+   */
+  Repository(std::string repo_name, std::string directory,
+             int64_t maxPartitionMillis, int64_t maxPartitionBytes,
+             uint64_t purgePeriod)
+      : CoreComponent(repo_name),
+        thread_() {
+    directory_ = directory;
+    max_partition_millis_ = maxPartitionMillis;
+    max_partition_bytes_ = maxPartitionBytes;
+    purge_period_ = purgePeriod;
+    configure_ = Configure::getConfigure();
+    running_ = false;
+    repo_full_ = false;
+  }
+
+  // Destructor
+  virtual ~Repository() {
+    stop();
+  }
+
+  // initialize
+  virtual bool initialize(){
+    return true;
+  }
+  // Put
+  virtual bool Put(std::string key, uint8_t *buf, int bufLen){
+    return true;
+  }
+  // Delete
+  virtual bool Delete(std::string key){
+    return true;
+  }
+
+  virtual bool Get(std::string key, std::string &value) {
+    return true;
+  }
+
+  // Run function for the thread
+  virtual  void run(){
+    // no op
+  }
+  // Start the repository monitor thread
+  virtual void start();
+  // Stop the repository monitor thread
+  virtual void stop();
+  // whether the repo is full
+  virtual bool isFull() {
+    return repo_full_;
+  }
+  // whether the repo is enable
+  virtual bool isRunning() {
+    return running_;
+  }
+  uint64_t incrementSize(const char *fpath, const struct stat *sb,
+                         int typeflag) {
+    return (repo_size_ += sb->st_size);
+  }
+
+  // Prevent default copy constructor and assignment operation
+  // Only support pass by reference or pointer
+  Repository(const Repository &parent) = delete;
+  Repository &operator=(const Repository &parent) = delete;
+
+ protected:
+  // Mutex for protection
+  std::mutex mutex_;
+  // repository directory
+  std::string directory_;
+  // Configure
+  Configure *configure_;
+  // max db entry life time
+  int64_t max_partition_millis_;
+  // max db size
+  int64_t max_partition_bytes_;
+  // purge period
+  uint64_t purge_period_;
+  // thread
+  std::thread thread_;
+  // whether the monitoring thread is running for the repo while it was enabled
+  bool running_;
+  // whether stop accepting provenace event
+  std::atomic<bool> repo_full_;
+  // repoSize
+  uint64_t repoSize();
+  // size of the directory
+  std::atomic<uint64_t> repo_size_;
+
+ private:
+  // Run function for the thread
+    void threadExecutor(){
+      run();
+    }
+};
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/core/RepositoryFactory.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/RepositoryFactory.h b/libminifi/include/core/RepositoryFactory.h
new file mode 100644
index 0000000..03ed524
--- /dev/null
+++ b/libminifi/include/core/RepositoryFactory.h
@@ -0,0 +1,44 @@
+/**
+ *
+ * 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_REPOSITORYFACTORY_H_
+#define LIBMINIFI_INCLUDE_CORE_REPOSITORYFACTORY_H_
+
+
+#include "core/Repository.h"
+#include "core.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+namespace core {
+
+  std::shared_ptr<core::Repository> createRepository(
+      const std::string configuration_class_name, bool fail_safe = false);
+
+
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_REPOSITORYFACTORY_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/core/Scheduling.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/Scheduling.h b/libminifi/include/core/Scheduling.h
new file mode 100644
index 0000000..0c983df
--- /dev/null
+++ b/libminifi/include/core/Scheduling.h
@@ -0,0 +1,64 @@
+/**
+ *
+ * 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_SCHEDULING_H_
+#define LIBMINIFI_INCLUDE_CORE_SCHEDULING_H_
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+/*
+ * Indicates the valid values for the state of a entity
+ * with respect to scheduling the entity to run.
+ */
+enum ScheduledState {
+
+  /**
+   * Entity cannot be scheduled to run
+   */
+  DISABLED,
+  /**
+   * Entity can be scheduled to run but currently is not
+   */
+  STOPPED,
+  /**
+   * Entity is currently scheduled to run
+   */
+  RUNNING
+};
+
+/*
+ * Scheduling Strategy
+ */
+enum SchedulingStrategy {
+  // Event driven
+  EVENT_DRIVEN,
+  // Timer driven
+  TIMER_DRIVEN,
+  // Cron Driven
+  CRON_DRIVEN
+};
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+#endif /* LIBMINIFI_INCLUDE_CORE_SCHEDULING_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/core/core.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/core.h b/libminifi/include/core/core.h
new file mode 100644
index 0000000..9f86100
--- /dev/null
+++ b/libminifi/include/core/core.h
@@ -0,0 +1,177 @@
+/**
+ *
+ * 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_CORE_H_
+#define LIBMINIFI_INCLUDE_CORE_CORE_H_
+
+#include <uuid/uuid.h>
+#include <cxxabi.h>
+#include "core/logging/Logger.h"
+/**
+ * namespace aliasing
+ */
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+}
+namespace processors {
+}
+namespace provenance {
+
+}
+namespace core {
+
+template<typename T>
+static inline std::string getClassName() {
+  char *b =   abi::__cxa_demangle(typeid(T).name(), 0, 0, 0);
+  std::string name = b;
+  delete [] b;
+  return name;
+}
+
+template<typename T>
+struct class_operations {
+  
+  template<typename Q=T>
+  static std::true_type canDestruct(decltype(std::declval<Q>().~Q()) *) {
+    return std::true_type();
+  }
+
+  
+  template<typename Q=T>
+  static std::false_type canDestruct(...) {
+    return std::false_type();
+  }
+
+  typedef decltype(canDestruct<T>(0)) type;
+
+  static const bool value = type::value; /* Which is it? */
+};
+
+
+template<typename T>
+typename std::enable_if<!class_operations<T>::value, T*>::type instantiate() {
+  throw std::runtime_error("Cannot instantiate class");
+}
+
+template<typename T>
+typename std::enable_if<class_operations<T>::value, T*>::type instantiate() {
+  return new T();
+}
+
+/**
+ * Base component within MiNiFi
+ * Purpose: Many objects store a name and UUID, therefore
+ * the functionality is localized here to avoid duplication
+ */
+class CoreComponent {
+
+ public:
+
+  /**
+   * Constructor that sets the name and uuid.
+   */
+  explicit CoreComponent(const std::string name, uuid_t uuid = 0)
+      : logger_(logging::Logger::getLogger()),
+        name_(name) {
+    if (!uuid)
+      // Generate the global UUID for the flow record
+      uuid_generate(uuid_);
+    else
+      uuid_copy(uuid_, uuid);
+
+    char uuidStr[37];
+    uuid_unparse_lower(uuid_, uuidStr);
+    uuidStr_ = uuidStr;
+  }
+
+  /**
+   * Move Constructor.
+   */
+  explicit CoreComponent(const CoreComponent &&other)
+      : name_(std::move(other.name_)),
+        logger_(logging::Logger::getLogger()) {
+    uuid_copy(uuid_, other.uuid_);
+  }
+
+  // Get component name Name
+  std::string getName();
+
+  /**
+   * Set name.
+   * @param name
+   */
+  void setName(const std::string name);
+
+  /**
+   * Set UUID in this instance
+   * @param uuid uuid to apply to the internal representation.
+   */
+  void setUUID(uuid_t uuid);
+
+  /**
+   * Returns the UUID through the provided object.
+   * @param uuid uuid struct to which we will copy the memory
+   * @return success of request
+   */
+  bool getUUID(uuid_t uuid);
+
+  unsigned const char *getUUID();
+  /**
+   * Return the UUID string
+   * @param constant reference to the UUID str
+   */
+  const std::string & getUUIDStr()  {
+    return uuidStr_;
+  }
+
+ protected:
+  // A global unique identifier
+  uuid_t uuid_;
+  // UUID string
+  std::string uuidStr_;
+
+  // logger shared ptr
+  std::shared_ptr<org::apache::nifi::minifi::core::logging::Logger> logger_;
+
+  // Connectable's name
+  std::string name_;
+};
+
+namespace logging {
+}
+}
+}
+}
+}
+}
+
+namespace minifi = org::apache::nifi::minifi;
+
+namespace core = org::apache::nifi::minifi::core;
+
+namespace processors = org::apache::nifi::minifi::processors;
+
+namespace logging = org::apache::nifi::minifi::core::logging;
+
+namespace utils = org::apache::nifi::minifi::utils;
+
+namespace provenance = org::apache::nifi::minifi::provenance;
+
+#endif /* LIBMINIFI_INCLUDE_CORE_CORE_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/core/logging/BaseLogger.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/logging/BaseLogger.h b/libminifi/include/core/logging/BaseLogger.h
new file mode 100644
index 0000000..bfdf26f
--- /dev/null
+++ b/libminifi/include/core/logging/BaseLogger.h
@@ -0,0 +1,224 @@
+/**
+ *
+ * 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_BASELOGGER_H_
+#define LIBMINIFI_INCLUDE_BASELOGGER_H_
+
+#include <string>
+#include <memory>
+#include "spdlog/spdlog.h"
+#include <iostream>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+namespace logging {
+
+// 5M default log file size
+#define DEFAULT_LOG_FILE_SIZE (5*1024*1024)
+// 3 log files rotation
+#define DEFAULT_LOG_FILE_NUMBER 3
+#define LOG_NAME "minifi log"
+#define LOG_FILE_NAME "minifi-app.log"
+
+/**
+ * Log level enumeration.
+ */
+typedef enum {
+  trace = 0,
+  debug = 1,
+  info = 2,
+  warn = 3,
+  err = 4,
+  critical = 5,
+  off = 6
+} LOG_LEVEL_E;
+
+#define LOG_BUFFER_SIZE 1024
+#define FILL_BUFFER  char buffer[LOG_BUFFER_SIZE]; \
+    va_list args; \
+    va_start(args, format); \
+    std::vsnprintf(buffer, LOG_BUFFER_SIZE,format, args); \
+    va_end(args);
+
+/**
+ * Base class that represents a logger configuration.
+ */
+class BaseLogger {
+
+ public:
+  static const char *nifi_log_level;
+  static const char *nifi_log_appender;
+
+  /**
+   * Base Constructor
+   */
+  BaseLogger() {
+    setLogLevel("info");
+    logger_ = nullptr;
+    stderr_ = nullptr;
+  }
+
+  /**
+   * Logger configuration constructorthat will set the base log level.
+   * @param config incoming configuration.
+   */
+  BaseLogger(std::string log_level, std::shared_ptr<spdlog::logger> logger)
+      : logger_(logger) {
+    setLogLevel(log_level);
+
+  }
+
+  virtual ~BaseLogger() {
+
+  }
+
+  /**
+   * Move constructor that will atomically swap configuration
+   * shared pointers.
+   */
+  BaseLogger(const BaseLogger &&other)
+      : configured_level_(other.configured_level_.load()) {
+    // must atomically exchange the pointers
+    logger_ = std::move(other.logger_);
+    set_error_logger(other.stderr_);
+
+  }
+
+  /**
+   * Returns the log level for this instance.
+   */
+  virtual LOG_LEVEL_E getLogLevel() const {
+    return configured_level_;
+  }
+
+  /**
+   * @brief Log error message
+   * @param format format string ('man printf' for syntax)
+   * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
+   */
+  virtual void log_error(const char * const format, ...);
+  /**
+   * @brief Log warn message
+   * @param format format string ('man printf' for syntax)
+   * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
+   */
+  virtual void log_warn(const char * const format, ...);
+  /**
+   * @brief Log info message
+   * @param format format string ('man printf' for syntax)
+   * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
+   */
+  virtual void log_info(const char * const format, ...);
+  /**
+   * @brief Log debug message
+   * @param format format string ('man printf' for syntax)
+   * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
+   */
+  virtual void log_debug(const char * const format, ...);
+  /**
+   * @brief Log trace message
+   * @param format format string ('man printf' for syntax)
+   * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
+   */
+  virtual void log_trace(const char * const format, ...);
+
+  /**
+   * @brief Log error message
+   * @param format format string ('man printf' for syntax)
+   * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
+   */
+  virtual void log_str(LOG_LEVEL_E level, const std::string &buffer);
+
+  /**
+   * Sets the log level for this instance based on the string
+   * @param level desired log leve.
+   * @param defaultLevel default level if we cannot match level.
+   */
+  virtual void setLogLevel(const std::string &level, LOG_LEVEL_E defaultLevel =
+                               info);
+
+  /**
+   * Sets the log level atomic and sets it
+   * within logger if it can
+   * @param level desired log level.
+   */
+  virtual void setLogLevel(LOG_LEVEL_E level) {
+    configured_level_ = level;
+    setLogLevel();
+  }
+
+  bool shouldLog(LOG_LEVEL_E level) {
+    return level >= configured_level_.load(std::memory_order_relaxed);
+  }
+
+  /**
+   * Move operator overload
+   */
+  BaseLogger &operator=(const BaseLogger &&other) {
+    configured_level_ = (other.configured_level_.load());
+    // must atomically exchange the pointers
+    logger_ = std::move(other.logger_);
+    set_error_logger(other.stderr_);
+    return *this;
+  }
+
+ protected:
+
+  /**
+   * Logger configuration constructorthat will set the base log level.
+   * @param config incoming configuration.
+   */
+  BaseLogger(std::string log_level)
+      : logger_(nullptr) {
+    setLogLevel(log_level);
+  }
+
+  void setLogger(std::shared_ptr<spdlog::logger> logger) {
+    logger_ = logger;
+  }
+
+  /**
+   * Since a thread may be using stderr and it can be null,
+   * we must atomically exchange the shared pointers.
+   * @param other other shared pointer. can be null ptr
+   */
+  void set_error_logger(std::shared_ptr<spdlog::logger> other);
+
+  /**
+   * Sets the log level on the spdlogger if it is not null.
+   */
+  void setLogLevel() {
+    if (logger_ != nullptr)
+      logger_->set_level((spdlog::level::level_enum) configured_level_.load());
+
+  }
+
+  std::atomic<LOG_LEVEL_E> configured_level_;
+  std::shared_ptr<spdlog::logger> logger_;
+  std::shared_ptr<spdlog::logger> stderr_;
+};
+
+} /* namespace logging */
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+#endif /* LIBMINIFI_INCLUDE_BASELOGGER_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/core/logging/LogAppenders.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/logging/LogAppenders.h b/libminifi/include/core/logging/LogAppenders.h
new file mode 100644
index 0000000..7bdc3be
--- /dev/null
+++ b/libminifi/include/core/logging/LogAppenders.h
@@ -0,0 +1,301 @@
+/**
+ *
+ * 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_LOGAPPENDERS_H_
+#define LIBMINIFI_INCLUDE_LOGAPPENDERS_H_
+
+#include "BaseLogger.h"
+#include "spdlog/sinks/null_sink.h"
+#include "spdlog/sinks/ostream_sink.h"
+#include <cxxabi.h>
+#include "properties/Configure.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+namespace logging {
+
+template<typename T>
+static std::string getUniqueName() {
+  std::string name = LOG_NAME;
+  name += " -- ";
+  name += abi::__cxa_demangle(typeid(T).name(), 0, 0, 0);
+  spdlog::drop(name);
+  return name;
+}
+
+/**
+ * Null appender sets a null sink, thereby performing no logging.
+ */
+class NullAppender : public BaseLogger {
+ public:
+  /**
+   * Base constructor that creates the null sink.
+   */
+  explicit NullAppender()
+      : BaseLogger("off") {
+    auto null_sink = std::make_shared<spdlog::sinks::null_sink_st>();
+    std::string unique_name = getUniqueName<NullAppender>();
+    logger_ = std::make_shared<spdlog::logger>(unique_name, null_sink);
+    configured_level_ = off;
+    setLogLevel();
+  }
+
+  /**
+   * Move constructor for the null appender.
+   */
+  explicit NullAppender(const NullAppender &&other)
+      : BaseLogger(std::move(other)) {
+
+  }
+
+};
+
+/**
+ * Basic output stream configuration that uses a supplied ostream
+ *
+ * Design : extends LoggerConfiguration using the logger and log level
+ * encapsulated within the base configuration class.
+ */
+class OutputStreamAppender : public BaseLogger {
+
+ public:
+
+  static const char *nifi_log_output_stream_error_stderr;
+
+  /**
+   * Output stream move constructor.
+   */
+  explicit OutputStreamAppender(const OutputStreamAppender &&other)
+      : BaseLogger(std::move(other)) {
+
+  }
+
+  /**
+   * Base constructor. Creates a ostream sink.
+   * @param stream incoming stream reference.
+   * @param config configuration.
+   */
+  explicit OutputStreamAppender(Configure *config)
+      : BaseLogger("info") {
+    auto ostream_sink = std::make_shared<spdlog::sinks::ostream_sink_mt>(
+        std::cout);
+
+    std::string unique_name = getUniqueName<OutputStreamAppender>();
+    logger_ = std::make_shared<spdlog::logger>(unique_name, ostream_sink);
+
+    std::string use_std_err;
+
+    if (NULL != config
+        && config->get(nifi_log_output_stream_error_stderr, use_std_err)) {
+
+      std::transform(use_std_err.begin(), use_std_err.end(),
+                     use_std_err.begin(), ::tolower);
+
+      if (use_std_err == "true") {
+        std::string err_unique_name = getUniqueName<OutputStreamAppender>();
+        auto error_ostream_sink = std::make_shared<
+            spdlog::sinks::ostream_sink_mt>(std::cerr);
+        stderr_ = std::make_shared<spdlog::logger>(err_unique_name,
+                                                   error_ostream_sink);
+      }
+    } else {
+      stderr_ = nullptr;
+    }
+
+    std::string log_level;
+    if (NULL != config && config->get(BaseLogger::nifi_log_level, log_level)) {
+      setLogLevel(log_level);
+    } else {
+      setLogLevel("info");
+    }
+
+  }
+
+  /**
+   * Base constructor. Creates a ostream sink.
+   * @param stream incoming stream reference.
+   * @param config configuration.
+   */
+  OutputStreamAppender(std::ostream &stream, Configure *config)
+      : BaseLogger("info") {
+    auto ostream_sink = std::make_shared<spdlog::sinks::ostream_sink_mt>(
+        stream);
+    std::string unique_name = getUniqueName<OutputStreamAppender>();
+    logger_ = std::make_shared<spdlog::logger>(unique_name, ostream_sink);
+
+    stderr_ = nullptr;
+
+    std::string log_level;
+    if (NULL != config && config->get(BaseLogger::nifi_log_level, log_level)) {
+      setLogLevel(log_level);
+    } else {
+      setLogLevel("info");
+    }
+
+  }
+
+ protected:
+
+};
+
+/**
+ * Rolling configuration
+ * Design : extends LoggerConfiguration using the logger and log level
+ * encapsulated within the base configuration class.
+ */
+class RollingAppender : public BaseLogger {
+ public:
+  static const char *nifi_log_rolling_apender_file;
+  static const char *nifi_log_rolling_appender_max_files;
+  static const char *nifi_log_rolling_appender_max_file_size;
+
+  /**
+   * RollingAppenderConfiguration move constructor.
+   */
+  explicit RollingAppender(const RollingAppender&& other)
+      : BaseLogger(std::move(other)),
+        max_files_(std::move(other.max_files_)),
+        file_name_(std::move(other.file_name_)),
+        max_file_size_(std::move(other.max_file_size_)) {
+  }
+  /**
+   * Base Constructor.
+   * @param config pointer to the configuration for this instance.
+   */
+  explicit RollingAppender(Configure * config = 0)
+      : BaseLogger("info") {
+    std::string file_name = "";
+    if (NULL != config
+        && config->get(nifi_log_rolling_apender_file, file_name)) {
+      file_name_ = file_name;
+    } else {
+      file_name_ = LOG_FILE_NAME;
+    }
+
+    std::string max_files = "";
+    if (NULL != config
+        && config->get(nifi_log_rolling_appender_max_files, max_files)) {
+      try {
+        max_files_ = std::stoi(max_files);
+      } catch (const std::invalid_argument &ia) {
+        max_files_ = DEFAULT_LOG_FILE_NUMBER;
+      } catch (const std::out_of_range &oor) {
+        max_files_ = DEFAULT_LOG_FILE_NUMBER;
+      }
+    } else {
+      max_files_ = DEFAULT_LOG_FILE_NUMBER;
+    }
+
+    std::string max_file_size = "";
+    if (NULL != config
+        && config->get(nifi_log_rolling_appender_max_file_size,
+                       max_file_size)) {
+      try {
+        max_file_size_ = std::stoi(max_file_size);
+      } catch (const std::invalid_argument &ia) {
+        max_file_size_ = DEFAULT_LOG_FILE_SIZE;
+      } catch (const std::out_of_range &oor) {
+        max_file_size_ = DEFAULT_LOG_FILE_SIZE;
+      }
+    } else {
+      max_file_size_ = DEFAULT_LOG_FILE_SIZE;
+    }
+
+    std::string unique_name = getUniqueName<OutputStreamAppender>();
+    logger_ = spdlog::rotating_logger_mt(unique_name, file_name_,
+                                         max_file_size_, max_files_);
+
+    std::string log_level;
+    if (NULL != config && config->get(BaseLogger::nifi_log_level, log_level)) {
+      setLogLevel(log_level);
+    }
+  }
+
+  /**
+   * To maintain current functionality we will flush on write.
+   */
+  void log_str(LOG_LEVEL_E level, const std::string &buffer) {
+    BaseLogger::log_str(level, buffer);
+    logger_->flush();
+  }
+
+ protected:
+
+  /**
+   * file name.
+   */
+  std::string file_name_;
+  /**
+   * maximum number of files to keep in the rotation.
+   */
+  size_t max_files_;
+  /**
+   * Maximum file size per rotated file.
+   */
+  size_t max_file_size_;
+
+};
+
+class LogInstance {
+ public:
+  /**
+   * Returns a logger configuration based on
+   * the configuration within this instance.
+   * @param config configuration for this instance.
+   */
+  static std::unique_ptr<BaseLogger> getConfiguredLogger(Configure *config) {
+    std::string appender = "";
+
+    if (config->get(BaseLogger::nifi_log_appender, appender)) {
+      std::transform(appender.begin(), appender.end(), appender.begin(),
+                     ::tolower);
+
+      if ("nullappender" == appender || "null appender" == appender
+          || "null" == appender) {
+
+        return std::move(std::unique_ptr<BaseLogger>(new NullAppender()));
+
+      } else if ("rollingappender" == appender || "rolling appender" == appender
+          || "rolling" == appender) {
+
+        return std::move(
+            std::unique_ptr<BaseLogger>(new RollingAppender(config)));
+
+      } else if ("outputstream" == appender
+          || "outputstreamappender" == appender
+          || "outputstream appender" == appender) {
+
+        return std::move(
+            std::unique_ptr<BaseLogger>(new OutputStreamAppender(config)));
+
+      }
+    }
+    return nullptr;
+
+  }
+};
+
+} /* namespace logging */
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+#endif /* LIBMINIFI_INCLUDE_LOGAPPENDERS_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/core/logging/Logger.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/logging/Logger.h b/libminifi/include/core/logging/Logger.h
new file mode 100644
index 0000000..08ef702
--- /dev/null
+++ b/libminifi/include/core/logging/Logger.h
@@ -0,0 +1,214 @@
+/**
+ * @file Logger.h
+ * Logger class declaration
+ * This is a C++ wrapper for spdlog, a lightweight C++ logging library
+ *
+ * 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 __LOGGER_H__
+#define __LOGGER_H__
+
+#include <string>
+#include <atomic>
+#include <memory>
+#include <utility>
+#include <algorithm>
+#include <cstdio>
+#include <iostream>
+
+#include "BaseLogger.h"
+#include "spdlog/spdlog.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+namespace logging {
+
+/**
+ * Logger class
+ * Design: Extends BaseLogger, leaving this class to be the facade to the underlying
+ * logging mechanism. Is a facade to BaseLogger's underlying log stream. This allows
+ * the underlying implementation to be replaced real time.
+ */
+class Logger : public BaseLogger {
+ protected:
+  struct singleton;
+ public:
+
+  /**
+   * Returns a shared pointer to the logger instance.
+   * Note that while there is no synchronization this is expected
+   * to be called and initialized first
+   * @returns shared pointer to the base logger.
+   */
+  static std::shared_ptr<Logger> getLogger() {
+
+    if (singleton_logger_ == nullptr)
+      singleton_logger_ = std::make_shared<Logger>(singleton { 0 });
+    return singleton_logger_;
+  }
+
+  /**
+   * Returns the log level for this instance.
+   */
+  LOG_LEVEL_E getLogLevel() const {
+    return current_logger_.load()->getLogLevel();
+  }
+
+  /**
+   * Sets the log level atomic and sets it
+   * within logger if it can
+   * @param level desired log level.
+   */
+  void setLogLevel(LOG_LEVEL_E level) {
+    current_logger_.load()->setLogLevel(level);
+  }
+
+  /**
+   * Sets the log level for this instance based on the string
+   * @param level desired log leve.
+   * @param defaultLevel default level if we cannot match level.
+   */
+  void setLogLevel(const std::string &level, LOG_LEVEL_E defaultLevel = info) {
+    current_logger_.load()->setLogLevel(level, info);
+  }
+
+  void updateLogger(std::unique_ptr<BaseLogger> logger) {
+
+    if (logger == nullptr)
+      return;
+    current_logger_.store(logger.release());
+  }
+
+  /**
+   * @brief Log error message
+   * @param format format string ('man printf' for syntax)
+   * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
+   */
+  void log_error(const char * const format, ...) {
+    if (!current_logger_.load()->shouldLog(err))
+      return;
+    FILL_BUFFER
+    current_logger_.load()->log_str(err, buffer);
+  }
+  /**
+   * @brief Log warn message
+   * @param format format string ('man printf' for syntax)
+   * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
+   */
+  void log_warn(const char * const format, ...) {
+    if (!current_logger_.load()->shouldLog(warn))
+      return;
+    FILL_BUFFER
+    current_logger_.load()->log_str(warn, buffer);
+  }
+  /**
+   * @brief Log info message
+   * @param format format string ('man printf' for syntax)
+   * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
+   */
+  void log_info(const char * const format, ...) {
+    if (!current_logger_.load()->shouldLog(info))
+      return;
+    FILL_BUFFER
+    current_logger_.load()->log_str(info, buffer);
+  }
+  /**
+   * @brief Log debug message
+   * @param format format string ('man printf' for syntax)
+   * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
+   */
+  void log_debug(const char * const format, ...) {
+
+    if (!current_logger_.load()->shouldLog(debug))
+      return;
+    FILL_BUFFER
+    current_logger_.load()->log_str(debug, buffer);
+  }
+  /**
+   * @brief Log trace message
+   * @param format format string ('man printf' for syntax)
+   * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
+   */
+  void log_trace(const char * const format, ...) {
+
+    if (!current_logger_.load()->shouldLog(trace))
+      return;
+    FILL_BUFFER
+    current_logger_.load()->log_str(trace, buffer);
+  }
+
+  /**
+   * @brief Log message
+   * @param format format string ('man printf' for syntax)
+   * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
+   */
+  virtual void log_str(LOG_LEVEL_E level, const std::string &buffer) {
+    current_logger_.load()->log_str(level, buffer);
+  }
+
+  // Destructor
+  ~Logger() {
+  }
+
+  explicit Logger(const singleton &a) {
+
+    /**
+     * flush on info to maintain current functionality
+     */
+    std::shared_ptr<spdlog::logger> defaultsink = spdlog::rotating_logger_mt(
+        LOG_NAME,
+        LOG_FILE_NAME,
+        DEFAULT_LOG_FILE_SIZE, DEFAULT_LOG_FILE_NUMBER);
+    defaultsink->flush_on(spdlog::level::level_enum::info);
+
+    std::unique_ptr<BaseLogger> new_logger_ = std::unique_ptr<BaseLogger>(
+        new BaseLogger("info", defaultsink));
+
+    new_logger_->setLogLevel(info);
+    current_logger_.store(new_logger_.release());
+  }
+
+  Logger(const Logger &parent) = delete;
+  Logger &operator=(const Logger &parent) = delete;
+
+ protected:
+
+  /**
+   * Allows for a null constructor above so that we can have a public constructor that
+   * effectively limits us to being a singleton by having a protected argument in the constructor
+   */
+  struct singleton {
+    explicit singleton(int) {
+    }
+  };
+
+  std::atomic<BaseLogger*> current_logger_;
+
+// Singleton logger instance
+  static std::shared_ptr<Logger> singleton_logger_;
+};
+
+} /* namespace logging */
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/core/repository/FlowFileRepository.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/repository/FlowFileRepository.h b/libminifi/include/core/repository/FlowFileRepository.h
new file mode 100644
index 0000000..31e655a
--- /dev/null
+++ b/libminifi/include/core/repository/FlowFileRepository.h
@@ -0,0 +1,169 @@
+/**
+ *
+ * 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_FLOWFILEREPOSITORY_H_
+#define LIBMINIFI_INCLUDE_CORE_REPOSITORY_FLOWFILEREPOSITORY_H_
+
+#include "leveldb/db.h"
+#include "leveldb/options.h"
+#include "leveldb/slice.h"
+#include "leveldb/status.h"
+#include "core/Repository.h"
+#include "core/core.h"
+#include "Connection.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+namespace repository {
+
+
+
+#define FLOWFILE_REPOSITORY_DIRECTORY "./flowfile_repository"
+#define MAX_FLOWFILE_REPOSITORY_STORAGE_SIZE (10*1024*1024) // 10M
+#define MAX_FLOWFILE_REPOSITORY_ENTRY_LIFE_TIME (600000) // 10 minute
+#define FLOWFILE_REPOSITORY_PURGE_PERIOD (2500) // 2500 msec
+
+/**
+ * Flow File repository
+ * Design: Extends Repository and implements the run function, using LevelDB as the primary substrate.
+ */
+class FlowFileRepository : public core::Repository, public std::enable_shared_from_this<FlowFileRepository> {
+ public:
+  // Constructor
+
+
+   
+  FlowFileRepository(std::string directory, int64_t maxPartitionMillis,
+                     int64_t maxPartitionBytes, uint64_t purgePeriod)
+      : Repository(core::getClassName<FlowFileRepository>(), directory,
+                   maxPartitionMillis, maxPartitionBytes, purgePeriod)
+
+  {
+    db_ = NULL;
+  }
+  
+  FlowFileRepository() : FlowFileRepository(FLOWFILE_REPOSITORY_DIRECTORY,
+			MAX_FLOWFILE_REPOSITORY_ENTRY_LIFE_TIME, MAX_FLOWFILE_REPOSITORY_STORAGE_SIZE, FLOWFILE_REPOSITORY_PURGE_PERIOD)
+  {
+  }
+
+  // Destructor
+  ~FlowFileRepository() {
+    if (db_)
+      delete db_;
+  }
+
+  // initialize
+  virtual bool initialize() {
+    std::string value;
+
+    if (configure_->get(Configure::nifi_flowfile_repository_directory_default,
+                        value)) {
+      directory_ = value;
+    }
+    logger_->log_info("NiFi FlowFile Repository Directory %s",
+                      directory_.c_str());
+    if (configure_->get(Configure::nifi_flowfile_repository_max_storage_size,
+                        value)) {
+      Property::StringToInt(value, max_partition_bytes_);
+    }
+    logger_->log_info("NiFi FlowFile Max Partition Bytes %d",
+                      max_partition_bytes_);
+    if (configure_->get(Configure::nifi_flowfile_repository_max_storage_time,
+                        value)) {
+      TimeUnit unit;
+      if (Property::StringToTime(value, max_partition_millis_, unit)
+          && Property::ConvertTimeUnitToMS(max_partition_millis_, unit,
+                                           max_partition_millis_)) {
+      }
+    }
+    logger_->log_info("NiFi FlowFile Max Storage Time: [%d] ms",
+                      max_partition_millis_);
+    leveldb::Options options;
+    options.create_if_missing = true;
+    leveldb::Status status = leveldb::DB::Open(options, directory_.c_str(),
+                                               &db_);
+    if (status.ok()) {
+      logger_->log_info("NiFi FlowFile Repository database open %s success",
+                        directory_.c_str());
+    } else {
+      logger_->log_error("NiFi FlowFile Repository database open %s fail",
+                         directory_.c_str());
+      return false;
+    }
+    return true;
+  }
+
+  virtual void run();
+  
+  virtual bool Put(std::string key, uint8_t *buf, int bufLen)
+  {
+		  
+	  // persistent to the DB
+	  leveldb::Slice value((const char *) buf, bufLen);
+	  leveldb::Status status;
+	  status = db_->Put(leveldb::WriteOptions(), key, value);
+	  if (status.ok())
+		  return true;
+	  else
+		  return false;
+  }
+  /**
+  * 
+  * Deletes the key
+  * @return status of the delete operation
+  */
+  virtual bool Delete(std::string key)
+  {
+	  leveldb::Status status;
+	  status = db_->Delete(leveldb::WriteOptions(), key);
+	  if (status.ok())
+		  return true;
+	  else
+		  return false;
+  }
+  /**
+    * Sets the value from the provided key
+    * @return status of the get operation.
+    */
+  virtual bool Get(std::string key, std::string &value)
+  {
+	  leveldb::Status status;
+	  status = db_->Get(leveldb::ReadOptions(), key, &value);
+	  if (status.ok())
+		  return true;
+	  else
+		  return false;
+  }
+  
+  void loadFlowFileToConnections(std::map<std::string, std::shared_ptr<minifi::Connection>> &connectionMap);
+  
+ private:
+  leveldb::DB* db_;
+};
+
+} /* namespace repository */
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_REPOSITORY_FLOWFILEREPOSITORY_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/core/yaml/YamlConfiguration.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/yaml/YamlConfiguration.h b/libminifi/include/core/yaml/YamlConfiguration.h
new file mode 100644
index 0000000..0ca9190
--- /dev/null
+++ b/libminifi/include/core/yaml/YamlConfiguration.h
@@ -0,0 +1,99 @@
+/**
+ *
+ * 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_YAMLCONFIGURATION_H_
+#define LIBMINIFI_INCLUDE_CORE_YAMLCONFIGURATION_H_
+
+#include "core/ProcessorConfig.h"
+#include "yaml-cpp/yaml.h"
+#include "../FlowConfiguration.h"
+#include "Site2SiteClientProtocol.h"
+#include <string>
+#include "io/validation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+#define DEFAULT_FLOW_YAML_FILE_NAME "conf/flow.yml"
+#define CONFIG_YAML_PROCESSORS_KEY "Processors"
+
+class YamlConfiguration : public FlowConfiguration {
+
+ public:
+  YamlConfiguration(std::shared_ptr<core::Repository> repo,
+                    std::shared_ptr<core::Repository> flow_file_repo,
+                    const std::string path = DEFAULT_FLOW_YAML_FILE_NAME)
+      : FlowConfiguration(repo, flow_file_repo, path) {
+    if (IsNullOrEmpty(config_path_)) {
+      config_path_ = DEFAULT_FLOW_YAML_FILE_NAME;
+    }
+  }
+
+  virtual ~YamlConfiguration() {
+
+  }
+
+  std::unique_ptr<core::ProcessGroup> getRoot(const std::string &from_config) {
+
+    YAML::Node flow = YAML::LoadFile(from_config);
+
+    YAML::Node flowControllerNode = flow["Flow Controller"];
+    YAML::Node processorsNode = flow[CONFIG_YAML_PROCESSORS_KEY];
+    YAML::Node connectionsNode = flow["Connections"];
+    YAML::Node remoteProcessingGroupNode = flow["Remote Processing Groups"];
+
+    // Create the root process group
+    core::ProcessGroup * root = parseRootProcessGroupYaml(flowControllerNode);
+    parseProcessorNodeYaml(processorsNode, root);
+    parseRemoteProcessGroupYaml(&remoteProcessingGroupNode, root);
+    parseConnectionYaml(&connectionsNode, root);
+
+    return std::unique_ptr<core::ProcessGroup>(root);
+
+  }
+ protected:
+  // Process Processor Node YAML
+  void parseProcessorNodeYaml(YAML::Node processorNode,
+                              core::ProcessGroup * parent);
+  // Process Port YAML
+  void parsePortYaml(YAML::Node *portNode, core::ProcessGroup *parent,
+                     TransferDirection direction);
+  // Process Root Processor Group YAML
+  core::ProcessGroup *parseRootProcessGroupYaml(YAML::Node rootNode);
+  // Process Property YAML
+  void parseProcessorPropertyYaml(YAML::Node *doc, YAML::Node *node,
+                                  std::shared_ptr<core::Processor> processor);
+  // Process connection YAML
+  void parseConnectionYaml(YAML::Node *node, core::ProcessGroup * parent);
+  // Process Remote Process Group YAML
+  void parseRemoteProcessGroupYaml(YAML::Node *node,
+                                   core::ProcessGroup * parent);
+  // Parse Properties Node YAML for a processor
+  void parsePropertiesNodeYaml(YAML::Node *propertiesNode,
+                               std::shared_ptr<core::Processor> processor);
+};
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif /* LIBMINIFI_INCLUDE_CORE_YAMLCONFIGURATION_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/io/BaseStream.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/BaseStream.h b/libminifi/include/io/BaseStream.h
index c3ebe42..b0b3589 100644
--- a/libminifi/include/io/BaseStream.h
+++ b/libminifi/include/io/BaseStream.h
@@ -19,132 +19,144 @@
 #ifndef LIBMINIFI_INCLUDE_IO_BASESTREAM_H_
 #define LIBMINIFI_INCLUDE_IO_BASESTREAM_H_
 
-
 #include <cstdint>
 #include "EndianCheck.h"
 #include "DataStream.h"
 #include "Serializable.h"
-class BaseStream: public DataStream, public Serializable {
-
-public:
-	BaseStream() {
-
-	}
-	virtual ~BaseStream() {
-
-	}
-	/**
-	 * write 4 bytes to stream
-	 * @param base_value non encoded value
-	 * @param stream output stream
-	 * @param is_little_endian endianness determination
-	 * @return resulting write size
-	 **/
-	virtual int write(uint32_t base_value, bool is_little_endian =
-			EndiannessCheck::IS_LITTLE);
-
-	/**
-	 * write 2 bytes to stream
-	 * @param base_value non encoded value
-	 * @param stream output stream
-	 * @param is_little_endian endianness determination
-	 * @return resulting write size
-	 **/
-	virtual int write(uint16_t base_value, bool is_little_endian =
-			EndiannessCheck::IS_LITTLE);
-
-	/**
-	 * write valueto stream
-	 * @param value non encoded value
-	 * @param len length of value
-	 * @param strema output stream
-	 * @return resulting write size
-	 **/
-	virtual int write(uint8_t *value, int len);
-
-	/**
-	 * write 8 bytes to stream
-	 * @param base_value non encoded value
-	 * @param stream output stream
-	 * @param is_little_endian endianness determination
-	 * @return resulting write size
-	 **/
-	virtual int write(uint64_t base_value, bool is_little_endian =
-			EndiannessCheck::IS_LITTLE);
-
-	/**
-	 * write bool to stream
-	 * @param value non encoded value
-	 * @return resulting write size
-	 **/
-	virtual int write(bool value);
-
-	/**
-	 * write UTF string to stream
-	 * @param str string to write
-	 * @return resulting write size
-	 **/
-	virtual int writeUTF(std::string str, bool widen = false);
-
-	/**
-	 * reads a byte from the stream
-	 * @param value reference in which will set the result
-	 * @param stream stream from which we will read
-	 * @return resulting read size
-	 **/
-	virtual int read(uint8_t &value);
-
-	/**
-	 * reads two bytes from the stream
-	 * @param value reference in which will set the result
-	 * @param stream stream from which we will read
-	 * @return resulting read size
-	 **/
-	virtual int read(uint16_t &base_value, bool is_little_endian =
-			EndiannessCheck::IS_LITTLE);
-
-	/**
-	 * reads a byte from the stream
-	 * @param value reference in which will set the result
-	 * @param stream stream from which we will read
-	 * @return resulting read size
-	 **/
-	virtual int read(char &value);
-
-	/**
-	 * reads a byte array from the stream
-	 * @param value reference in which will set the result
-	 * @param len length to read
-	 * @param stream stream from which we will read
-	 * @return resulting read size
-	 **/
-	virtual int read(uint8_t *value, int len);
-
-	/**
-	 * reads four bytes from the stream
-	 * @param value reference in which will set the result
-	 * @param stream stream from which we will read
-	 * @return resulting read size
-	 **/
-	virtual int read(uint32_t &value,
-			bool is_little_endian = EndiannessCheck::IS_LITTLE);
-
-	/**
-	 * reads eight byte from the stream
-	 * @param value reference in which will set the result
-	 * @param stream stream from which we will read
-	 * @return resulting read size
-	 **/
-	virtual int read(uint64_t &value,
-			bool is_little_endian = EndiannessCheck::IS_LITTLE);
-
-	/**
-	 * read UTF from stream
-	 * @param str reference string
-	 * @param stream stream from which we will read
-	 * @return resulting read size
-	 **/
-	virtual int readUTF(std::string &str, bool widen = false);
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace io {
+
+
+class BaseStream : public DataStream, public Serializable {
+
+ public:
+  BaseStream() {
+
+  }
+  virtual ~BaseStream() {
+
+  }
+  /**
+   * write 4 bytes to stream
+   * @param base_value non encoded value
+   * @param stream output stream
+   * @param is_little_endian endianness determination
+   * @return resulting write size
+   **/
+  virtual int write(uint32_t base_value, bool is_little_endian =
+                        EndiannessCheck::IS_LITTLE);
+
+  /**
+   * write 2 bytes to stream
+   * @param base_value non encoded value
+   * @param stream output stream
+   * @param is_little_endian endianness determination
+   * @return resulting write size
+   **/
+  virtual int write(uint16_t base_value, bool is_little_endian =
+                        EndiannessCheck::IS_LITTLE);
+
+  /**
+   * write valueto stream
+   * @param value non encoded value
+   * @param len length of value
+   * @param strema output stream
+   * @return resulting write size
+   **/
+  virtual int write(uint8_t *value, int len);
+
+  /**
+   * write 8 bytes to stream
+   * @param base_value non encoded value
+   * @param stream output stream
+   * @param is_little_endian endianness determination
+   * @return resulting write size
+   **/
+  virtual int write(uint64_t base_value, bool is_little_endian =
+                        EndiannessCheck::IS_LITTLE);
+
+  /**
+   * write bool to stream
+   * @param value non encoded value
+   * @return resulting write size
+   **/
+  virtual int write(bool value);
+
+  /**
+   * write UTF string to stream
+   * @param str string to write
+   * @return resulting write size
+   **/
+  virtual int writeUTF(std::string str, bool widen = false);
+
+  /**
+   * reads a byte from the stream
+   * @param value reference in which will set the result
+   * @param stream stream from which we will read
+   * @return resulting read size
+   **/
+  virtual int read(uint8_t &value);
+
+  /**
+   * reads two bytes from the stream
+   * @param value reference in which will set the result
+   * @param stream stream from which we will read
+   * @return resulting read size
+   **/
+  virtual int read(uint16_t &base_value, bool is_little_endian =
+                       EndiannessCheck::IS_LITTLE);
+
+  /**
+   * reads a byte from the stream
+   * @param value reference in which will set the result
+   * @param stream stream from which we will read
+   * @return resulting read size
+   **/
+  virtual int read(char &value);
+
+  /**
+   * reads a byte array from the stream
+   * @param value reference in which will set the result
+   * @param len length to read
+   * @param stream stream from which we will read
+   * @return resulting read size
+   **/
+  virtual int read(uint8_t *value, int len);
+
+  /**
+   * reads four bytes from the stream
+   * @param value reference in which will set the result
+   * @param stream stream from which we will read
+   * @return resulting read size
+   **/
+  virtual int read(uint32_t &value, bool is_little_endian =
+                       EndiannessCheck::IS_LITTLE);
+
+  /**
+   * reads eight byte from the stream
+   * @param value reference in which will set the result
+   * @param stream stream from which we will read
+   * @return resulting read size
+   **/
+  virtual int read(uint64_t &value, bool is_little_endian =
+                       EndiannessCheck::IS_LITTLE);
+
+  /**
+   * read UTF from stream
+   * @param str reference string
+   * @param stream stream from which we will read
+   * @return resulting read size
+   **/
+  virtual int readUTF(std::string &str, bool widen = false);
 };
 
+} /* namespace io */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
 #endif /* LIBMINIFI_INCLUDE_IO_BASESTREAM_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/io/CRCStream.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/CRCStream.h b/libminifi/include/io/CRCStream.h
index 01b6199..99fdfc3 100644
--- a/libminifi/include/io/CRCStream.h
+++ b/libminifi/include/io/CRCStream.h
@@ -25,282 +25,279 @@
 #include "BaseStream.h"
 #include "Serializable.h"
 
-#define htonll_r(x) ((((uint64_t)htonl(x)) << 32) + htonl((x) >> 32))
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace io {
 
+#define htonll_r(x) ((((uint64_t)htonl(x)) << 32) + htonl((x) >> 32))
 
 template<typename T>
-class CRCStream: public BaseStream {
-public:
-	/**
-	 * Raw pointer because the caller guarantees that
-	 * it will exceed our lifetime.
-	 */
-	explicit CRCStream(T *stream);
-
-	explicit CRCStream( CRCStream<T> &&move );
-
-	virtual ~CRCStream() {
-
-	}
-
-	/**
-	 * 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);
-	
-	/**
-	 * write 4 bytes to stream
-	 * @param base_value non encoded value
-	 * @param stream output stream
-	 * @param is_little_endian endianness determination
-	 * @return resulting write size
-	 **/
-	virtual int write(uint32_t base_value, bool is_little_endian =
-			EndiannessCheck::IS_LITTLE);
-	/**
-	 * write 2 bytes to stream
-	 * @param base_value non encoded value
-	 * @param stream output stream
-	 * @param is_little_endian endianness determination
-	 * @return resulting write size
-	 **/
-	virtual int write(uint16_t base_value, bool is_little_endian =
-			EndiannessCheck::IS_LITTLE);
-
-
-	/**
-	 * write 8 bytes to stream
-	 * @param base_value non encoded value
-	 * @param stream output stream
-	 * @param is_little_endian endianness determination
-	 * @return resulting write size
-	 **/
-	virtual int write(uint64_t base_value, bool is_little_endian =
-			EndiannessCheck::IS_LITTLE);
-	
-	
-	
-	/**
-	 * Reads a system word
-	 * @param value value to write
-	 */
-	virtual int read(uint64_t &value, bool is_little_endian =
-			EndiannessCheck::IS_LITTLE);
-
-	/**
-	 * Reads a uint32_t
-	 * @param value value to write
-	 */
-	virtual int read(uint32_t &value, bool is_little_endian =
-			EndiannessCheck::IS_LITTLE);
-
-	/**
-	 * Reads a system short
-	 * @param value value to write
-	 */
-	virtual int read(uint16_t &value, bool is_little_endian =
-			EndiannessCheck::IS_LITTLE);
-	
-
-	virtual short initialize() {
-		child_stream_->initialize();
-		reset();
-		return 0;
-	}
-
-
-	void updateCRC(uint8_t *buffer, uint32_t length);
-
-	uint64_t getCRC() {
-		return crc_;
-	}
-
-	void reset();
-protected:
-  
+class CRCStream : public BaseStream {
+ public:
   /**
-	 * Creates a vector and returns the vector using the provided
-	 * type name.
-	 * @param t incoming object
-	 * @returns vector.
-	 */
-	template<typename K>
-	std::vector<uint8_t> readBuffer(const K& t){
-	    std::vector<uint8_t> buf;
-	    buf.resize(sizeof t);
-	    readData((uint8_t*) &buf[0], sizeof(t));
-	    return buf;
-	}
-	
-  
-	uint64_t crc_;
-	T *child_stream_;
-};
+   * Raw pointer because the caller guarantees that
+   * it will exceed our lifetime.
+   */
+  explicit CRCStream(T *stream);
+
+  explicit CRCStream(CRCStream<T> &&move);
+
+  virtual ~CRCStream() {
+
+  }
+
+  /**
+   * 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);
+
+  /**
+   * write 4 bytes to stream
+   * @param base_value non encoded value
+   * @param stream output stream
+   * @param is_little_endian endianness determination
+   * @return resulting write size
+   **/
+  virtual int write(uint32_t base_value, bool is_little_endian =
+                        EndiannessCheck::IS_LITTLE);
+  /**
+   * write 2 bytes to stream
+   * @param base_value non encoded value
+   * @param stream output stream
+   * @param is_little_endian endianness determination
+   * @return resulting write size
+   **/
+  virtual int write(uint16_t base_value, bool is_little_endian =
+                        EndiannessCheck::IS_LITTLE);
+
+  /**
+   * write 8 bytes to stream
+   * @param base_value non encoded value
+   * @param stream output stream
+   * @param is_little_endian endianness determination
+   * @return resulting write size
+   **/
+  virtual int write(uint64_t base_value, bool is_little_endian =
+                        EndiannessCheck::IS_LITTLE);
 
+  /**
+   * Reads a system word
+   * @param value value to write
+   */
+  virtual int read(uint64_t &value, bool is_little_endian =
+                       EndiannessCheck::IS_LITTLE);
+
+  /**
+   * Reads a uint32_t
+   * @param value value to write
+   */
+  virtual int read(uint32_t &value, bool is_little_endian =
+                       EndiannessCheck::IS_LITTLE);
+
+  /**
+   * Reads a system short
+   * @param value value to write
+   */
+  virtual int read(uint16_t &value, bool is_little_endian =
+                       EndiannessCheck::IS_LITTLE);
+
+  virtual short initialize() {
+    child_stream_->initialize();
+    reset();
+    return 0;
+  }
+
+  void updateCRC(uint8_t *buffer, uint32_t length);
+
+  uint64_t getCRC() {
+    return crc_;
+  }
+
+  void reset();
+ protected:
+
+  /**
+   * Creates a vector and returns the vector using the provided
+   * type name.
+   * @param t incoming object
+   * @returns vector.
+   */
+  template<typename K>
+  std::vector<uint8_t> readBuffer(const K& t) {
+    std::vector<uint8_t> buf;
+    buf.resize(sizeof t);
+    readData((uint8_t*) &buf[0], sizeof(t));
+    return buf;
+  }
+
+  uint64_t crc_;
+  T *child_stream_;
+};
 
 template<typename T>
-CRCStream<T>::CRCStream(T *other) :
-		child_stream_(other) {
-	crc_ = crc32(0L, Z_NULL, 0);
+CRCStream<T>::CRCStream(T *other)
+    : child_stream_(other) {
+  crc_ = crc32(0L, Z_NULL, 0);
 }
 
 template<typename T>
-CRCStream<T>::CRCStream(CRCStream<T> &&move) :
-		crc_(std::move(move.crc_)), child_stream_(std::move(move.child_stream_)) {
+CRCStream<T>::CRCStream(CRCStream<T> &&move)
+    : crc_(std::move(move.crc_)),
+      child_stream_(std::move(move.child_stream_)) {
 
 }
 
 template<typename T>
 int CRCStream<T>::readData(std::vector<uint8_t> &buf, int buflen) {
 
-	if (buf.capacity() < buflen)
-		buf.resize(buflen);
-	return readData((uint8_t*) &buf[0], buflen);
+  if (buf.capacity() < buflen)
+    buf.resize(buflen);
+  return readData((uint8_t*) &buf[0], buflen);
 }
 
 template<typename T>
 int CRCStream<T>::readData(uint8_t *buf, int buflen) {
-	int ret = child_stream_->read(buf, buflen);
-	crc_ = crc32(crc_, buf, buflen);
-	return ret;
+  int ret = child_stream_->read(buf, buflen);
+  crc_ = crc32(crc_, buf, buflen);
+  return ret;
 }
 
 template<typename T>
 int CRCStream<T>::writeData(std::vector<uint8_t> &buf, int buflen) {
 
-	if (buf.capacity() < buflen)
-		buf.resize(buflen);
-	return writeData((uint8_t*) &buf[0], buflen);
+  if (buf.capacity() < buflen)
+    buf.resize(buflen);
+  return writeData((uint8_t*) &buf[0], buflen);
 }
 
 template<typename T>
 int CRCStream<T>::writeData(uint8_t *value, int size) {
 
-	int ret = child_stream_->write(value, size);
-	crc_ = crc32(crc_, value, size);
-	return ret;
+  int ret = child_stream_->write(value, size);
+  crc_ = crc32(crc_, value, size);
+  return ret;
 
 }
 template<typename T>
 void CRCStream<T>::reset() {
-	crc_ = crc32(0L, Z_NULL, 0);
+  crc_ = crc32(0L, Z_NULL, 0);
 }
 template<typename T>
 void CRCStream<T>::updateCRC(uint8_t *buffer, uint32_t length) {
-	crc_ = crc32(crc_, buffer, length);
+  crc_ = crc32(crc_, buffer, length);
 }
 
 template<typename T>
-int CRCStream<T>::write(uint64_t base_value, bool is_little_endian){
-  
-   const uint64_t value =
-        is_little_endian == 1 ? htonll_r(base_value) : base_value;
-    uint8_t bytes[sizeof value];
-    std::copy(static_cast<const char*>(static_cast<const void*>(&value)),
-              static_cast<const char*>(static_cast<const void*>(&value)) + sizeof value,
-              bytes);
-    return writeData(bytes,sizeof value);
-}
+int CRCStream<T>::write(uint64_t base_value, bool is_little_endian) {
 
+  const uint64_t value =
+      is_little_endian == 1 ? htonll_r(base_value) : base_value;
+  uint8_t bytes[sizeof value];
+  std::copy(
+      static_cast<const char*>(static_cast<const void*>(&value)),
+      static_cast<const char*>(static_cast<const void*>(&value)) + sizeof value,
+      bytes);
+  return writeData(bytes, sizeof value);
+}
 
 template<typename T>
-int CRCStream<T>::write(uint32_t base_value, bool is_little_endian){
-   const uint32_t value = is_little_endian ? htonl(base_value) : base_value;
-    uint8_t bytes[sizeof value];
-    std::copy(static_cast<const char*>(static_cast<const void*>(&value)),
-              static_cast<const char*>(static_cast<const void*>(&value)) + sizeof value,
-              bytes);
-    return writeData(bytes,sizeof value);
+int CRCStream<T>::write(uint32_t base_value, bool is_little_endian) {
+  const uint32_t value = is_little_endian ? htonl(base_value) : base_value;
+  uint8_t bytes[sizeof value];
+  std::copy(
+      static_cast<const char*>(static_cast<const void*>(&value)),
+      static_cast<const char*>(static_cast<const void*>(&value)) + sizeof value,
+      bytes);
+  return writeData(bytes, sizeof value);
 }
 
 template<typename T>
-int CRCStream<T>::write(uint16_t base_value, bool is_little_endian){
-  const uint16_t value =
-        is_little_endian == 1 ? htons(base_value) : base_value;
+int CRCStream<T>::write(uint16_t base_value, bool is_little_endian) {
+  const uint16_t value = is_little_endian == 1 ? htons(base_value) : base_value;
   uint8_t bytes[sizeof value];
-    std::copy(static_cast<const char*>(static_cast<const void*>(&value)),
-              static_cast<const char*>(static_cast<const void*>(&value)) + sizeof value,
-              bytes);
-    return writeData(bytes,sizeof value);
+  std::copy(
+      static_cast<const char*>(static_cast<const void*>(&value)),
+      static_cast<const char*>(static_cast<const void*>(&value)) + sizeof value,
+      bytes);
+  return writeData(bytes, sizeof value);
 }
 
-
 template<typename T>
 int CRCStream<T>::read(uint64_t &value, bool is_little_endian) {
 
-	auto buf = readBuffer(value);
-
-	if (is_little_endian) {
-		value = ((uint64_t) buf[0] << 56) | ((uint64_t) (buf[1] & 255) << 48)
-				| ((uint64_t) (buf[2] & 255) << 40)
-				| ((uint64_t) (buf[3] & 255) << 32)
-				| ((uint64_t) (buf[4] & 255) << 24)
-				| ((uint64_t) (buf[5] & 255) << 16)
-				| ((uint64_t) (buf[6] & 255) << 8)
-				| ((uint64_t) (buf[7] & 255) << 0);
-	} else {
-		value = ((uint64_t) buf[0] << 0) | ((uint64_t) (buf[1] & 255) << 8)
-				| ((uint64_t) (buf[2] & 255) << 16)
-				| ((uint64_t) (buf[3] & 255) << 24)
-				| ((uint64_t) (buf[4] & 255) << 32)
-				| ((uint64_t) (buf[5] & 255) << 40)
-				| ((uint64_t) (buf[6] & 255) << 48)
-				| ((uint64_t) (buf[7] & 255) << 56);
-	}
-	return sizeof(value);
+  auto buf = readBuffer(value);
+
+  if (is_little_endian) {
+    value = ((uint64_t) buf[0] << 56) | ((uint64_t) (buf[1] & 255) << 48)
+        | ((uint64_t) (buf[2] & 255) << 40) | ((uint64_t) (buf[3] & 255) << 32)
+        | ((uint64_t) (buf[4] & 255) << 24) | ((uint64_t) (buf[5] & 255) << 16)
+        | ((uint64_t) (buf[6] & 255) << 8) | ((uint64_t) (buf[7] & 255) << 0);
+  } else {
+    value = ((uint64_t) buf[0] << 0) | ((uint64_t) (buf[1] & 255) << 8)
+        | ((uint64_t) (buf[2] & 255) << 16) | ((uint64_t) (buf[3] & 255) << 24)
+        | ((uint64_t) (buf[4] & 255) << 32) | ((uint64_t) (buf[5] & 255) << 40)
+        | ((uint64_t) (buf[6] & 255) << 48) | ((uint64_t) (buf[7] & 255) << 56);
+  }
+  return sizeof(value);
 }
 
 template<typename T>
 int CRCStream<T>::read(uint32_t &value, bool is_little_endian) {
 
-	auto buf = readBuffer(value);
+  auto buf = readBuffer(value);
 
-	if (is_little_endian) {
-		value = (buf[0] << 24) | (buf[1] << 16) | (buf[2] << 8) | buf[3];
-	} else {
-		value = buf[0] | buf[1] << 8 | buf[2] << 16 | buf[3] << 24;
+  if (is_little_endian) {
+    value = (buf[0] << 24) | (buf[1] << 16) | (buf[2] << 8) | buf[3];
+  } else {
+    value = buf[0] | buf[1] << 8 | buf[2] << 16 | buf[3] << 24;
 
-	}
+  }
 
-	return sizeof(value);
+  return sizeof(value);
 }
 
 template<typename T>
 int CRCStream<T>::read(uint16_t &value, bool is_little_endian) {
 
-	auto buf = readBuffer(value);
+  auto buf = readBuffer(value);
 
-	if (is_little_endian) {
-		value = (buf[0] << 8) | buf[1];
-	} else {
-		value = buf[0] | buf[1] << 8;
+  if (is_little_endian) {
+    value = (buf[0] << 8) | buf[1];
+  } else {
+    value = buf[0] | buf[1] << 8;
 
-	}
-	return sizeof(value);
+  }
+  return sizeof(value);
 }
 
-
+} /* namespace io */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
 #endif /* LIBMINIFI_INCLUDE_IO_CRCSTREAM_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/io/ClientSocket.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/ClientSocket.h b/libminifi/include/io/ClientSocket.h
index 3f8aae1..97cace2 100644
--- a/libminifi/include/io/ClientSocket.h
+++ b/libminifi/include/io/ClientSocket.h
@@ -26,10 +26,17 @@
 #include <mutex>
 #include <atomic>
 #include "io/BaseStream.h"
-#include "Logger.h"
+#include "core/core.h"
+#include "core/logging/Logger.h"
 
 #include "io/validation.h"
 
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace io {
+
 /**
  * Socket class.
  * Purpose: Provides a general purpose socket interface that abstracts
@@ -39,207 +46,209 @@
  *
  *
  */
-class Socket: public BaseStream {
-public:
-	/**
-	 * Constructor that accepts host name, port and listeners. With this
-	 * contructor we will be creating a server socket
-	 * @param hostname our host name
-	 * @param port connecting port
-	 * @param listeners number of listeners in the queue
-	 */
-	explicit Socket(const std::string &hostname, const uint16_t port,
-			const uint16_t listeners);
-
-	/**
-	 * Constructor that creates a client socket.
-	 * @param hostname hostname we are connecting to.
-	 * @param port port we are connecting to.
-	 */
-	explicit Socket(const std::string &hostname, const uint16_t port);
-
-	/**
-	 * Move constructor.
-	 */
-	explicit Socket(const Socket &&);
-
-	static std::string HOSTNAME;
-
-	/**
-	 * Static function to return the current machine's host name
-	 */
-	static std::string getMyHostName(std::string *str = &HOSTNAME) {
-		if (__builtin_expect(!IsNullOrEmpty(str), 0))
-			return *str;
-		else {
-			char hostname[1024];
-			gethostname(hostname, 1024);
-			Socket mySock(hostname, 0);
-			mySock.initialize();
-			return mySock.getHostname();
-		}
-	}
-
-	/**
-	 * Destructor
-	 */
-
-	virtual ~Socket();
-
-	virtual void closeStream();
-	/**
-	 * Initializes the socket
-	 * @return result of the creation operation.
-	 */
-	virtual short initialize();
-
-	std::string getHostname() const;
-
-	/**
-	 * Return the port for this socket
-	 * @returns port
-	 */
-	uint16_t getPort();
-
-	// 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);
-
-	
-	
-	/**
-	 * Writes a system word
-	 * @param value value to write
-	 */
-	virtual int write(uint64_t value, bool is_little_endian =
-			EndiannessCheck::IS_LITTLE);
-
-	/**
-	 * Writes a uint32_t
-	 * @param value value to write
-	 */
-	virtual int write(uint32_t value, bool is_little_endian =
-			EndiannessCheck::IS_LITTLE);
-
-	/**
-	 * Writes a system short
-	 * @param value value to write
-	 */
-	virtual int write(uint16_t value, bool is_little_endian =
-			EndiannessCheck::IS_LITTLE);
-
-	
-	/**
-	 * Reads a system word
-	 * @param value value to write
-	 */
-	virtual int read(uint64_t &value, bool is_little_endian =
-			EndiannessCheck::IS_LITTLE);
-
-	/**
-	 * Reads a uint32_t
-	 * @param value value to write
-	 */
-	virtual int read(uint32_t &value, bool is_little_endian =
-			EndiannessCheck::IS_LITTLE);
-
-	/**
-	 * Reads a system short
-	 * @param value value to write
-	 */
-	virtual int read(uint16_t &value, bool is_little_endian =
-			EndiannessCheck::IS_LITTLE);
-
-	/**
-	 * Returns the underlying buffer
-	 * @return vector's array
-	 **/
-	const uint8_t *getBuffer() const {
-		return ::DataStream::getBuffer();
-	}
-
-	/**
-	 * Retrieve size of data stream
-	 * @return size of data stream
-	 **/
-	const uint32_t getSize() const {
-		return ::DataStream::getSize();
-	}
-
-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&);
-
-	/**
-	 * Creates a connection using the address info object.
-	 * @param p addrinfo structure.
-	 * @returns fd.
-	 */
-	virtual int8_t createConnection(const addrinfo *p,in_addr_t &addr);
-
-	/**
-	 * Sets socket options depending on the instance.
-	 * @param sock socket file descriptor.
-	 */
-	virtual short setSocketOptions(const int sock);
-
-	/**
-	 * Attempt to select the socket file descriptor
-	 * @param msec timeout interval to wait
-	 * @returns file descriptor
-	 */
-	virtual short select_descriptor(const uint16_t msec);
-
-	std::shared_ptr<Logger> logger_;
-
-	addrinfo *addr_info_;
-
-	std::recursive_mutex selection_mutex_;
-
-	std::string requested_hostname_;
-	std::string canonical_hostname_;
-	uint16_t port_;
-
-	// connection information
-	int32_t socket_file_descriptor_;
-
-	fd_set total_list_;
-	fd_set read_fds_;
-	std::atomic<uint16_t> socket_max_;
-	uint16_t listeners_;
+class Socket : public BaseStream {
+ public:
+  /**
+   * Constructor that accepts host name, port and listeners. With this
+   * contructor we will be creating a server socket
+   * @param hostname our host name
+   * @param port connecting port
+   * @param listeners number of listeners in the queue
+   */
+  explicit Socket(const std::string &hostname, const uint16_t port,
+                  const uint16_t listeners);
+
+  /**
+   * Constructor that creates a client socket.
+   * @param hostname hostname we are connecting to.
+   * @param port port we are connecting to.
+   */
+  explicit Socket(const std::string &hostname, const uint16_t port);
+
+  /**
+   * Move constructor.
+   */
+  explicit Socket(const Socket &&);
+
+  static std::string HOSTNAME;
+
+  /**
+   * Static function to return the current machine's host name
+   */
+  static std::string getMyHostName(std::string *str = &HOSTNAME) {
+    if (__builtin_expect(!IsNullOrEmpty(str), 0))
+      return *str;
+    else {
+      char hostname[1024];
+      gethostname(hostname, 1024);
+      Socket mySock(hostname, 0);
+      mySock.initialize();
+      return mySock.getHostname();
+    }
+  }
+
+  /**
+   * Destructor
+   */
+
+  virtual ~Socket();
+
+  virtual void closeStream();
+  /**
+   * Initializes the socket
+   * @return result of the creation operation.
+   */
+  virtual short initialize();
+
+  std::string getHostname() const;
+
+  /**
+   * Return the port for this socket
+   * @returns port
+   */
+  uint16_t getPort();
+
+  // 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);
+
+  /**
+   * Writes a system word
+   * @param value value to write
+   */
+  virtual int write(uint64_t value, bool is_little_endian =
+                        EndiannessCheck::IS_LITTLE);
+
+  /**
+   * Writes a uint32_t
+   * @param value value to write
+   */
+  virtual int write(uint32_t value, bool is_little_endian =
+                        EndiannessCheck::IS_LITTLE);
+
+  /**
+   * Writes a system short
+   * @param value value to write
+   */
+  virtual int write(uint16_t value, bool is_little_endian =
+                        EndiannessCheck::IS_LITTLE);
+
+  /**
+   * Reads a system word
+   * @param value value to write
+   */
+  virtual int read(uint64_t &value, bool is_little_endian =
+                       EndiannessCheck::IS_LITTLE);
+
+  /**
+   * Reads a uint32_t
+   * @param value value to write
+   */
+  virtual int read(uint32_t &value, bool is_little_endian =
+                       EndiannessCheck::IS_LITTLE);
+
+  /**
+   * Reads a system short
+   * @param value value to write
+   */
+  virtual int read(uint16_t &value, bool is_little_endian =
+                       EndiannessCheck::IS_LITTLE);
+
+  /**
+   * Returns the underlying buffer
+   * @return vector's array
+   **/
+  const uint8_t *getBuffer() const {
+    return DataStream::getBuffer();
+  }
+
+  /**
+   * Retrieve size of data stream
+   * @return size of data stream
+   **/
+  const uint32_t getSize() const {
+    return DataStream::getSize();
+  }
+
+ 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&);
+
+  /**
+   * Creates a connection using the address info object.
+   * @param p addrinfo structure.
+   * @returns fd.
+   */
+  virtual int8_t createConnection(const addrinfo *p, in_addr_t &addr);
+
+  /**
+   * Sets socket options depending on the instance.
+   * @param sock socket file descriptor.
+   */
+  virtual short setSocketOptions(const int sock);
+
+  /**
+   * Attempt to select the socket file descriptor
+   * @param msec timeout interval to wait
+   * @returns file descriptor
+   */
+  virtual short select_descriptor(const uint16_t msec);
+
+  std::shared_ptr<logging::Logger> logger_;
+
+  addrinfo *addr_info_;
+
+  std::recursive_mutex selection_mutex_;
+
+  std::string requested_hostname_;
+  std::string canonical_hostname_;
+  uint16_t port_;
+
+  // connection information
+  int32_t socket_file_descriptor_;
+
+  fd_set total_list_;
+  fd_set read_fds_;
+  std::atomic<uint16_t> socket_max_;
+  uint16_t listeners_;
 
 };
 
+} /* namespace io */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
 #endif /* LIBMINIFI_INCLUDE_IO_CLIENTSOCKET_H_ */


[09/16] nifi-minifi-cpp git commit: MINIFI-217: Updates namespaces and removes use of raw pointers for user facing API.

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/FlowController.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/FlowController.cpp b/libminifi/src/FlowController.cpp
index eca84be..e472a9a 100644
--- a/libminifi/src/FlowController.cpp
+++ b/libminifi/src/FlowController.cpp
@@ -30,765 +30,271 @@
 #include <unistd.h>
 #include <future>
 #include "FlowController.h"
-#include "ProcessContext.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessGroup.h"
 #include "utils/StringUtils.h"
+#include "core/core.h"
+#include "core/repository/FlowFileRepository.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+#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::unique_ptr<core::FlowConfiguration> flow_configuration,
+    const std::string name, bool headless_mode)
+    : CoreComponent(core::getClassName<FlowController>()),
+      root_(nullptr),
+      max_timer_driven_threads_(0),
+      max_event_driven_threads_(0),
+      running_(false),
+      initialized_(false),
+      provenance_repo_(provenance_repo),
+      flow_file_repo_(flow_file_repo),
+      protocol_(0),
+      _timerScheduler(provenance_repo_),
+      _eventScheduler(provenance_repo_),
+      flow_configuration_(std::move(flow_configuration)) {
+  if (provenance_repo == nullptr)
+    throw std::runtime_error("Provenance Repo should not be null");
+  if (flow_file_repo == nullptr)
+    throw std::runtime_error("Flow Repo should not be null");
+
+  uuid_generate(uuid_);
+  setUUID(uuid_);
+
+  // Setup the default values
+  if (flow_configuration_ != nullptr) {
+    configuration_filename_ = flow_configuration_->getConfigurationPath();
+  }
+  max_event_driven_threads_ = DEFAULT_MAX_EVENT_DRIVEN_THREAD;
+  max_timer_driven_threads_ = DEFAULT_MAX_TIMER_DRIVEN_THREAD;
+  running_ = false;
+  initialized_ = false;
+  root_ = NULL;
+
+  protocol_ = new FlowControlProtocol(this);
+
+  // NiFi config properties
+  configure_ = Configure::getConfigure();
+
+  if (!headless_mode) {
+    std::string rawConfigFileString;
+    configure_->get(Configure::nifi_flow_configuration_file,
+                    rawConfigFileString);
+
+    if (!rawConfigFileString.empty()) {
+      configuration_filename_ = rawConfigFileString;
+    }
+
+    std::string adjustedFilename;
+    if (!configuration_filename_.empty()) {
+      // perform a naive determination if this is a relative path
+      if (configuration_filename_.c_str()[0] != '/') {
+        adjustedFilename = adjustedFilename + configure_->getHome() + "/"
+            + configuration_filename_;
+      } else {
+        adjustedFilename = configuration_filename_;
+      }
+    }
+
+    initializePaths(adjustedFilename);
+  }
 
-FlowController *FlowControllerFactory::_flowController(NULL);
-
-FlowControllerImpl::FlowControllerImpl(std::string name)  {
-	uuid_generate(_uuid);
-
-	_name = name;
-	// Setup the default values
-	_configurationFileName = DEFAULT_FLOW_YAML_FILE_NAME;
-	_maxEventDrivenThreads = DEFAULT_MAX_EVENT_DRIVEN_THREAD;
-	_maxTimerDrivenThreads = DEFAULT_MAX_TIMER_DRIVEN_THREAD;
-	_running = false;
-	_initialized = false;
-	_root = NULL;
-	logger_ = Logger::getLogger();
-	_protocol = new FlowControlProtocol(this);
-
-	// NiFi config properties
-	configure_ = Configure::getConfigure();
-
-	std::string rawConfigFileString;
-	configure_->get(Configure::nifi_flow_configuration_file,
-			rawConfigFileString);
-
-	if (!rawConfigFileString.empty()) {
-		_configurationFileName = rawConfigFileString;
-	}
-
-	char *path = NULL;
-	char full_path[PATH_MAX];
-
-	std::string adjustedFilename;
-	if (!_configurationFileName.empty()) {
-		// perform a naive determination if this is a relative path
-		if (_configurationFileName.c_str()[0] != '/') {
-			adjustedFilename = adjustedFilename + configure_->getHome() + "/"
-					+ _configurationFileName;
-		} else {
-			adjustedFilename = _configurationFileName;
-		}
-	}
-
-	path = realpath(adjustedFilename.c_str(), full_path);
-
-	std::string pathString(path);
-	_configurationFileName = pathString;
-	logger_->log_info("FlowController NiFi Configuration file %s", pathString.c_str());
-
-	// Create the content repo directory if needed
-	struct stat contentDirStat;
-
-	if (stat(ResourceClaim::default_directory_path.c_str(), &contentDirStat) != -1 && S_ISDIR(contentDirStat.st_mode))
-	{
-		path = realpath(ResourceClaim::default_directory_path.c_str(), full_path);
-		logger_->log_info("FlowController content directory %s", full_path);
-	}
-	else
-	{
-	   if (mkdir(ResourceClaim::default_directory_path.c_str(), 0777) == -1)
-	   {
-		   logger_->log_error("FlowController content directory creation failed");
-		   exit(1);
-	   }
-	}
-
-	
-	std::string clientAuthStr;
-
-	if (!path) {
-		logger_->log_error(
-				"Could not locate path from provided configuration file name (%s).  Exiting.",
-				full_path);
-		exit(1);
-	}
-
-
-	// Create repos for flow record and provenance
-	_flowfileRepo = new FlowFileRepository();
-	_flowfileRepo->initialize();
-	_provenanceRepo = new ProvenanceRepository();
-	_provenanceRepo->initialize();
 }
 
-FlowControllerImpl::~FlowControllerImpl() {
+void FlowController::initializePaths(const std::string &adjustedFilename) {
+  char *path = NULL;
+  char full_path[PATH_MAX];
+  path = realpath(adjustedFilename.c_str(), full_path);
+
+  if (path == NULL) {
+    throw std::runtime_error(
+        "Path is not specified. Either manually set MINIFI_HOME or ensure ../conf exists");
+  }
+  std::string pathString(path);
+  configuration_filename_ = pathString;
+  logger_->log_info("FlowController NiFi Configuration file %s",
+                    pathString.c_str());
+
+  // Create the content repo directory if needed
+  struct stat contentDirStat;
+
+  if (stat(ResourceClaim::default_directory_path.c_str(), &contentDirStat)
+      != -1&& S_ISDIR(contentDirStat.st_mode)) {
+    path = realpath(ResourceClaim::default_directory_path.c_str(), full_path);
+    logger_->log_info("FlowController content directory %s", full_path);
+  } else {
+    if (mkdir(ResourceClaim::default_directory_path.c_str(), 0777) == -1) {
+      logger_->log_error("FlowController content directory creation failed");
+      exit(1);
+    }
+  }
 
-	stop(true);
-	unload();
-	if (NULL != _protocol)
-		delete _protocol;
-	if (NULL != _provenanceRepo)
-		delete _provenanceRepo;
-	if (NULL != _flowfileRepo)
-		delete _flowfileRepo;
+  std::string clientAuthStr;
 
-}
+  if (!path) {
+    logger_->log_error(
+        "Could not locate path from provided configuration file name (%s).  Exiting.",
+        full_path);
+    exit(1);
+  }
 
-void FlowControllerImpl::stop(bool force) {
+}
 
-	if (_running) {
-		// immediately indicate that we are not running
-		_running = false;
+FlowController::~FlowController() {
+  stop(true);
+  unload();
+  if (NULL != protocol_)
+    delete protocol_;
 
-		logger_->log_info("Stop Flow Controller");
-		this->_timerScheduler.stop();
-		this->_eventScheduler.stop();
-		this->_flowfileRepo->stop();
-		this->_provenanceRepo->stop();
-		// Wait for sometime for thread stop
-		std::this_thread::sleep_for(std::chrono::milliseconds(1000));
-		if (this->_root)
-			this->_root->stopProcessing(&this->_timerScheduler,
-					&this->_eventScheduler);
+}
 
-	}
+void FlowController::stop(bool force) {
+  std::lock_guard<std::recursive_mutex> flow_lock(mutex_);
+  if (running_) {
+    // immediately indicate that we are not running
+    running_ = false;
+
+    logger_->log_info("Stop Flow Controller");
+    this->_timerScheduler.stop();
+    this->_eventScheduler.stop();
+    this->flow_file_repo_->stop();
+    this->provenance_repo_->stop();
+    // Wait for sometime for thread stop
+    std::this_thread::sleep_for(std::chrono::milliseconds(1000));
+    if (this->root_)
+      this->root_->stopProcessing(&this->_timerScheduler,
+                                  &this->_eventScheduler);
+
+  }
 }
 
 /**
  * This function will attempt to unload yaml and stop running Processors.
  *
  * If the latter attempt fails or does not complete within the prescribed
- * period, _running will be set to false and we will return.
+ * period, running_ will be set to false and we will return.
  *
  * @param timeToWaitMs Maximum time to wait before manually
  * marking running as false.
  */
-void FlowControllerImpl::waitUnload(const uint64_t timeToWaitMs) {
-	if (_running) {
-		// use the current time and increment with the provided argument.
-		std::chrono::system_clock::time_point wait_time =
-				std::chrono::system_clock::now()
-						+ std::chrono::milliseconds(timeToWaitMs);
-
-		// create an asynchronous future.
-		std::future<void> unload_task = std::async(std::launch::async,
-				[this]() {unload();});
-
-		if (std::future_status::ready == unload_task.wait_until(wait_time)) {
-			_running = false;
-		}
-
-	}
-}
-
-
-void FlowControllerImpl::unload() {
-	if (_running) {
-		stop(true);
-	}
-	if (_initialized) {
-		logger_->log_info("Unload Flow Controller");
-		if (_root)
-			delete _root;
-		_root = NULL;
-		_initialized = false;
-		_name = "";
-	}
-
-	return;
-}
-
-Processor *FlowControllerImpl::createProcessor(std::string name, uuid_t uuid) {
-	Processor *processor = NULL;
-	if (name == GenerateFlowFile::ProcessorName) {
-		processor = new GenerateFlowFile(name, uuid);
-	} else if (name == LogAttribute::ProcessorName) {
-		processor = new LogAttribute(name, uuid);
-	} else if (name == RealTimeDataCollector::ProcessorName) {
-		processor = new RealTimeDataCollector(name, uuid);
-	} else if (name == GetFile::ProcessorName) {
-		processor = new GetFile(name, uuid);
-	} else if (name == PutFile::ProcessorName) {
-		processor = new PutFile(name, uuid);
-	} else if (name == TailFile::ProcessorName) {
-		processor = new TailFile(name, uuid);
-	} else if (name == ListenSyslog::ProcessorName) {
-		processor = new ListenSyslog(name, uuid);
-	} else if (name == ListenHTTP::ProcessorName) {
-        processor = new ListenHTTP(name, uuid);
-	} else if (name == ExecuteProcess::ProcessorName) {
-		processor = new ExecuteProcess(name, uuid);
-	} else if (name == AppendHostInfo::ProcessorName) {
-		processor = new AppendHostInfo(name, uuid);
-	} else {
-		logger_->log_error("No Processor defined for %s", name.c_str());
-		return NULL;
-	}
-
-	//! initialize the processor
-	processor->initialize();
-
-	return processor;
-}
-
-ProcessGroup *FlowControllerImpl::createRootProcessGroup(std::string name,
-		uuid_t uuid) {
-	return new ProcessGroup(ROOT_PROCESS_GROUP, name, uuid);
-}
-
-ProcessGroup *FlowControllerImpl::createRemoteProcessGroup(std::string name,
-		uuid_t uuid) {
-	return new ProcessGroup(REMOTE_PROCESS_GROUP, name, uuid);
-}
-
-Connection *FlowControllerImpl::createConnection(std::string name,
-		uuid_t uuid) {
-	return new Connection(name, uuid);
-}
-
-#ifdef YAML_SUPPORT
-void FlowControllerImpl::parseRootProcessGroupYaml(YAML::Node rootFlowNode) {
-	uuid_t uuid;
-	ProcessGroup *group = NULL;
-
-	std::string flowName = rootFlowNode["name"].as<std::string>();
-	std::string id = rootFlowNode["id"].as<std::string>();
-
-	uuid_parse(id.c_str(), uuid);
-
-	logger_->log_debug("parseRootProcessGroup: id => [%s]", id.c_str());
-	logger_->log_debug("parseRootProcessGroup: name => [%s]", flowName.c_str());
-	group = this->createRootProcessGroup(flowName, uuid);
-	this->_root = group;
-	this->_name = flowName;
-}
-
-void FlowControllerImpl::parseProcessorNodeYaml(YAML::Node processorsNode,
-		ProcessGroup *parentGroup) {
-	int64_t schedulingPeriod = -1;
-	int64_t penalizationPeriod = -1;
-	int64_t yieldPeriod = -1;
-	int64_t runDurationNanos = -1;
-	uuid_t uuid;
-	Processor *processor = NULL;
-
-	if (!parentGroup) {
-		logger_->log_error("parseProcessNodeYaml: no parent group exists");
-		return;
-	}
-
-	if (processorsNode) {
-
-		if (processorsNode.IsSequence()) {
-			// Evaluate sequence of processors
-			int numProcessors = processorsNode.size();
-
-			for (YAML::const_iterator iter = processorsNode.begin();
-					iter != processorsNode.end(); ++iter) {
-				ProcessorConfig procCfg;
-				YAML::Node procNode = iter->as<YAML::Node>();
-
-				procCfg.name = procNode["name"].as<std::string>();
-				procCfg.id = procNode["id"].as<std::string>();
-				logger_->log_debug("parseProcessorNode: name => [%s] id => [%s]",
-						procCfg.name.c_str(), procCfg.id.c_str());
-				procCfg.javaClass = procNode["class"].as<std::string>();
-				logger_->log_debug("parseProcessorNode: class => [%s]",
-						procCfg.javaClass.c_str());
-
-				uuid_parse(procCfg.id.c_str(), uuid);
-
-				// Determine the processor name only from the Java class
-				int lastOfIdx = procCfg.javaClass.find_last_of(".");
-				if (lastOfIdx != std::string::npos) {
-					lastOfIdx++; // if a value is found, increment to move beyond the .
-					int nameLength = procCfg.javaClass.length() - lastOfIdx;
-					std::string processorName = procCfg.javaClass.substr(
-							lastOfIdx, nameLength);
-					processor = this->createProcessor(processorName, uuid);
-				}
-
-				if (!processor) {
-					logger_->log_error(
-							"Could not create a processor %s with name %s",
-							procCfg.name.c_str(), procCfg.id.c_str());
-					throw std::invalid_argument(
-							"Could not create processor " + procCfg.name);
-				}
-				processor->setName(procCfg.name);
-
-				procCfg.maxConcurrentTasks =
-						procNode["max concurrent tasks"].as<std::string>();
-				logger_->log_debug(
-						"parseProcessorNode: max concurrent tasks => [%s]",
-						procCfg.maxConcurrentTasks.c_str());
-				procCfg.schedulingStrategy = procNode["scheduling strategy"].as<
-						std::string>();
-				logger_->log_debug(
-						"parseProcessorNode: scheduling strategy => [%s]",
-						procCfg.schedulingStrategy.c_str());
-				procCfg.schedulingPeriod = procNode["scheduling period"].as<
-						std::string>();
-				logger_->log_debug(
-						"parseProcessorNode: scheduling period => [%s]",
-						procCfg.schedulingPeriod.c_str());
-				procCfg.penalizationPeriod = procNode["penalization period"].as<
-						std::string>();
-				logger_->log_debug(
-						"parseProcessorNode: penalization period => [%s]",
-						procCfg.penalizationPeriod.c_str());
-				procCfg.yieldPeriod =
-						procNode["yield period"].as<std::string>();
-				logger_->log_debug("parseProcessorNode: yield period => [%s]",
-						procCfg.yieldPeriod.c_str());
-				procCfg.yieldPeriod = procNode["run duration nanos"].as<
-						std::string>();
-				logger_->log_debug(
-						"parseProcessorNode: run duration nanos => [%s]",
-						procCfg.runDurationNanos.c_str());
-
-				// handle auto-terminated relationships
-				YAML::Node autoTerminatedSequence =
-						procNode["auto-terminated relationships list"];
-				std::vector<std::string> rawAutoTerminatedRelationshipValues;
-				if (autoTerminatedSequence.IsSequence()
-						&& !autoTerminatedSequence.IsNull()
-						&& autoTerminatedSequence.size() > 0) {
-					for (YAML::const_iterator relIter =
-							autoTerminatedSequence.begin();
-							relIter != autoTerminatedSequence.end();
-							++relIter) {
-						std::string autoTerminatedRel =
-								relIter->as<std::string>();
-						rawAutoTerminatedRelationshipValues.push_back(
-								autoTerminatedRel);
-					}
-				}
-				procCfg.autoTerminatedRelationships =
-						rawAutoTerminatedRelationshipValues;
-
-				// handle processor properties
-				YAML::Node propertiesNode = procNode["Properties"];
-				parsePropertiesNodeYaml(&propertiesNode, processor);
-
-				// Take care of scheduling
-				TimeUnit unit;
-				if (Property::StringToTime(procCfg.schedulingPeriod,
-						schedulingPeriod, unit)
-						&& Property::ConvertTimeUnitToNS(schedulingPeriod, unit,
-								schedulingPeriod)) {
-					logger_->log_debug(
-							"convert: parseProcessorNode: schedulingPeriod => [%d] ns",
-							schedulingPeriod);
-					processor->setSchedulingPeriodNano(schedulingPeriod);
-				}
-
-				if (Property::StringToTime(procCfg.penalizationPeriod,
-						penalizationPeriod, unit)
-						&& Property::ConvertTimeUnitToMS(penalizationPeriod,
-								unit, penalizationPeriod)) {
-					logger_->log_debug(
-							"convert: parseProcessorNode: penalizationPeriod => [%d] ms",
-							penalizationPeriod);
-					processor->setPenalizationPeriodMsec(penalizationPeriod);
-				}
-
-				if (Property::StringToTime(procCfg.yieldPeriod, yieldPeriod,
-						unit)
-						&& Property::ConvertTimeUnitToMS(yieldPeriod, unit,
-								yieldPeriod)) {
-					logger_->log_debug(
-							"convert: parseProcessorNode: yieldPeriod => [%d] ms",
-							yieldPeriod);
-					processor->setYieldPeriodMsec(yieldPeriod);
-				}
-
-				// Default to running
-				processor->setScheduledState(RUNNING);
-
-				if (procCfg.schedulingStrategy == "TIMER_DRIVEN") {
-					processor->setSchedulingStrategy(TIMER_DRIVEN);
-					logger_->log_debug("setting scheduling strategy as %s",
-							procCfg.schedulingStrategy.c_str());
-				} else if (procCfg.schedulingStrategy == "EVENT_DRIVEN") {
-					processor->setSchedulingStrategy(EVENT_DRIVEN);
-					logger_->log_debug("setting scheduling strategy as %s",
-							procCfg.schedulingStrategy.c_str());
-				} else {
-					processor->setSchedulingStrategy(CRON_DRIVEN);
-					logger_->log_debug("setting scheduling strategy as %s",
-							procCfg.schedulingStrategy.c_str());
-
-				}
-
-				int64_t maxConcurrentTasks;
-				if (Property::StringToInt(procCfg.maxConcurrentTasks,
-						maxConcurrentTasks)) {
-					logger_->log_debug(
-							"parseProcessorNode: maxConcurrentTasks => [%d]",
-							maxConcurrentTasks);
-					processor->setMaxConcurrentTasks(maxConcurrentTasks);
-				}
-
-				if (Property::StringToInt(procCfg.runDurationNanos,
-						runDurationNanos)) {
-					logger_->log_debug(
-							"parseProcessorNode: runDurationNanos => [%d]",
-							runDurationNanos);
-					processor->setRunDurationNano(runDurationNanos);
-				}
-
-				std::set<Relationship> autoTerminatedRelationships;
-				for (auto &&relString : procCfg.autoTerminatedRelationships) {
-					Relationship relationship(relString, "");
-					logger_->log_debug(
-							"parseProcessorNode: autoTerminatedRelationship  => [%s]",
-							relString.c_str());
-					autoTerminatedRelationships.insert(relationship);
-				}
-
-				processor->setAutoTerminatedRelationships(
-						autoTerminatedRelationships);
-
-				parentGroup->addProcessor(processor);
-			}
-		}
-	} else {
-		throw new std::invalid_argument(
-				"Cannot instantiate a MiNiFi instance without a defined Processors configuration node.");
-	}
-}
-
-void FlowControllerImpl::parseRemoteProcessGroupYaml(YAML::Node *rpgNode,
-		ProcessGroup *parentGroup) {
-	uuid_t uuid;
-
-	if (!parentGroup) {
-		logger_->log_error(
-				"parseRemoteProcessGroupYaml: no parent group exists");
-		return;
-	}
-
-	if (rpgNode) {
-		if (rpgNode->IsSequence()) {
-			for (YAML::const_iterator iter = rpgNode->begin();
-					iter != rpgNode->end(); ++iter) {
-				YAML::Node rpgNode = iter->as<YAML::Node>();
-
-				auto name = rpgNode["name"].as<std::string>();
-				auto id = rpgNode["id"].as<std::string>();
-
-				logger_->log_debug("parseRemoteProcessGroupYaml: name => [%s], id => [%s]",
-						name.c_str(), id.c_str());
-
-				std::string url = rpgNode["url"].as<std::string>();
-				logger_->log_debug("parseRemoteProcessGroupYaml: url => [%s]",
-						url.c_str());
-
-				std::string timeout = rpgNode["timeout"].as<std::string>();
-				logger_->log_debug(
-						"parseRemoteProcessGroupYaml: timeout => [%s]",
-						timeout.c_str());
-
-				std::string yieldPeriod =
-						rpgNode["yield period"].as<std::string>();
-				logger_->log_debug(
-						"parseRemoteProcessGroupYaml: yield period => [%s]",
-						yieldPeriod.c_str());
-
-				YAML::Node inputPorts = rpgNode["Input Ports"].as<YAML::Node>();
-				YAML::Node outputPorts =
-						rpgNode["Output Ports"].as<YAML::Node>();
-				ProcessGroup *group = NULL;
-
-				uuid_parse(id.c_str(), uuid);
-
-				int64_t timeoutValue = -1;
-				int64_t yieldPeriodValue = -1;
-
-				group = this->createRemoteProcessGroup(name.c_str(), uuid);
-				group->setParent(parentGroup);
-				parentGroup->addProcessGroup(group);
-
-				TimeUnit unit;
-
-				if (Property::StringToTime(yieldPeriod, yieldPeriodValue, unit)
-						&& Property::ConvertTimeUnitToMS(yieldPeriodValue, unit,
-								yieldPeriodValue) && group) {
-					logger_->log_debug(
-							"parseRemoteProcessGroupYaml: yieldPeriod => [%d] ms",
-							yieldPeriodValue);
-					group->setYieldPeriodMsec(yieldPeriodValue);
-				}
-
-				if (Property::StringToTime(timeout, timeoutValue, unit)
-						&& Property::ConvertTimeUnitToMS(timeoutValue, unit,
-								timeoutValue) && group) {
-					logger_->log_debug(
-							"parseRemoteProcessGroupYaml: timeoutValue => [%d] ms",
-							timeoutValue);
-					group->setTimeOut(timeoutValue);
-				}
-
-				group->setTransmitting(true);
-				group->setURL(url);
-
-				if (inputPorts && inputPorts.IsSequence()) {
-					for (YAML::const_iterator portIter = inputPorts.begin();
-							portIter != inputPorts.end(); ++portIter) {
-						logger_->log_debug("Got a current port, iterating...");
-
-						YAML::Node currPort = portIter->as<YAML::Node>();
-
-						this->parsePortYaml(&currPort, group, SEND);
-					} // for node
-				}
-				if (outputPorts && outputPorts.IsSequence()) {
-					for (YAML::const_iterator portIter = outputPorts.begin();
-							portIter != outputPorts.end(); ++portIter) {
-						logger_->log_debug("Got a current port, iterating...");
-
-						YAML::Node currPort = portIter->as<YAML::Node>();
-
-						this->parsePortYaml(&currPort, group, RECEIVE);
-					} // for node
-				}
-
-			}
-		}
-	}
-}
-
-void FlowControllerImpl::parseConnectionYaml(YAML::Node *connectionsNode,
-		ProcessGroup *parent) {
-	uuid_t uuid;
-	Connection *connection = NULL;
-
-	if (!parent) {
-		logger_->log_error("parseProcessNode: no parent group was provided");
-		return;
-	}
-
-	if (connectionsNode) {
-
-		if (connectionsNode->IsSequence()) {
-			for (YAML::const_iterator iter = connectionsNode->begin();
-					iter != connectionsNode->end(); ++iter) {
-
-				YAML::Node connectionNode = iter->as<YAML::Node>();
-
-				std::string name = connectionNode["name"].as<std::string>();
-				std::string id = connectionNode["id"].as<std::string>();
-				std::string destId = connectionNode["destination id"].as<
-						std::string>();
-
-				uuid_parse(id.c_str(), uuid);
-
-				logger_->log_debug(
-						"Created connection with UUID %s and name %s", id.c_str(),
-						name.c_str());
-				connection = this->createConnection(name, uuid);
-				auto rawRelationship =
-						connectionNode["source relationship name"].as<
-								std::string>();
-				Relationship relationship(rawRelationship, "");
-				logger_->log_debug("parseConnection: relationship => [%s]",
-						rawRelationship.c_str());
-				if (connection)
-					connection->setRelationship(relationship);
-				std::string connectionSrcProcId =
-						connectionNode["source id"].as<std::string>();
-				uuid_t srcUUID;
-				uuid_parse(connectionSrcProcId.c_str(), srcUUID);
-
-				Processor *srcProcessor = this->_root->findProcessor(
-						srcUUID);
-
-				if (!srcProcessor) {
-					logger_->log_error(
-							"Could not locate a source with id %s to create a connection",
-							connectionSrcProcId.c_str());
-					throw std::invalid_argument(
-							"Could not locate a source with id %s to create a connection "
-									+ connectionSrcProcId);
-				}
-
-				uuid_t destUUID;
-				uuid_parse(destId.c_str(), destUUID);
-				Processor *destProcessor = this->_root->findProcessor(destUUID);
-				// If we could not find name, try by UUID
-				if (!destProcessor) {
-					uuid_t destUuid;
-					uuid_parse(destId.c_str(), destUuid);
-					destProcessor = this->_root->findProcessor(destUuid);
-				}
-				if (destProcessor) {
-					std::string destUuid = destProcessor->getUUIDStr();
-				}
-
-				uuid_t srcUuid;
-				uuid_t destUuid;
-				srcProcessor->getUUID(srcUuid);
-				connection->setSourceProcessorUUID(srcUuid);
-				destProcessor->getUUID(destUuid);
-				connection->setDestinationProcessorUUID(destUuid);
-
-				if (connection) {
-					parent->addConnection(connection);
-				}
-			}
-		}
-
-		if (connection)
-			parent->addConnection(connection);
-
-		return;
-	}
-}
-
-void FlowControllerImpl::parsePortYaml(YAML::Node *portNode,
-		ProcessGroup *parent, TransferDirection direction) {
-	uuid_t uuid;
-	Processor *processor = NULL;
-	RemoteProcessorGroupPort *port = NULL;
-
-	if (!parent) {
-		logger_->log_error("parseProcessNode: no parent group existed");
-		return;
-	}
-
-	YAML::Node inputPortsObj = portNode->as<YAML::Node>();
-
-	// generate the random UIID
-	uuid_generate(uuid);
-
-	auto portId = inputPortsObj["id"].as<std::string>();
-	auto nameStr = inputPortsObj["name"].as<std::string>();
-	uuid_parse(portId.c_str(), uuid);
-
-	port = new RemoteProcessorGroupPort(nameStr.c_str(), uuid);
-
-	processor = (Processor *) port;
-	port->setDirection(direction);
-	port->setTimeOut(parent->getTimeOut());
-	port->setTransmitting(true);
-	processor->setYieldPeriodMsec(parent->getYieldPeriodMsec());
-	processor->initialize();
-
-	// handle port properties
-	YAML::Node nodeVal = portNode->as<YAML::Node>();
-	YAML::Node propertiesNode = nodeVal["Properties"];
-
-	parsePropertiesNodeYaml(&propertiesNode, processor);
-
-	// add processor to parent
-	parent->addProcessor(processor);
-	processor->setScheduledState(RUNNING);
-	auto rawMaxConcurrentTasks = inputPortsObj["max concurrent tasks"].as<
-			std::string>();
-	int64_t maxConcurrentTasks;
-	if (Property::StringToInt(rawMaxConcurrentTasks, maxConcurrentTasks)) {
-		processor->setMaxConcurrentTasks(maxConcurrentTasks);
-	}
-	logger_->log_debug("parseProcessorNode: maxConcurrentTasks => [%d]",
-			maxConcurrentTasks);
-	processor->setMaxConcurrentTasks(maxConcurrentTasks);
+void FlowController::waitUnload(const uint64_t timeToWaitMs) {
+  if (running_) {
+    // use the current time and increment with the provided argument.
+    std::chrono::system_clock::time_point wait_time =
+        std::chrono::system_clock::now()
+            + std::chrono::milliseconds(timeToWaitMs);
+
+    // create an asynchronous future.
+    std::future<void> unload_task = std::async(std::launch::async,
+                                               [this]() {unload();});
+
+    if (std::future_status::ready == unload_task.wait_until(wait_time)) {
+      running_ = false;
+    }
 
+  }
 }
 
-void FlowControllerImpl::parsePropertiesNodeYaml(YAML::Node *propertiesNode,
-		Processor *processor) {
-	// Treat generically as a YAML node so we can perform inspection on entries to ensure they are populated
-	for (YAML::const_iterator propsIter = propertiesNode->begin();
-			propsIter != propertiesNode->end(); ++propsIter) {
-		std::string propertyName = propsIter->first.as<std::string>();
-		YAML::Node propertyValueNode = propsIter->second;
-		if (!propertyValueNode.IsNull() && propertyValueNode.IsDefined()) {
-			std::string rawValueString = propertyValueNode.as<std::string>();
-			if (!processor->setProperty(propertyName, rawValueString)) {
-				logger_->log_warn(
-						"Received property %s with value %s but is not one of the properties for %s",
-						propertyName.c_str(), rawValueString.c_str(),
-						processor->getName().c_str());
-			}
-		}
-	}
+void FlowController::unload() {
+  std::lock_guard<std::recursive_mutex> flow_lock(mutex_);
+  if (running_) {
+    stop(true);
+  }
+  if (initialized_) {
+    logger_->log_info("Unload Flow Controller");
+    root_ = nullptr;
+    initialized_ = false;
+    name_ = "";
+  }
+
+  return;
 }
-#endif /* ifdef YAML_SUPPORT */
 
-void FlowControllerImpl::load() {
-    if (_running) {
-        stop(true);
-    }
-    if (!_initialized) {
-        logger_->log_info("Load Flow Controller from file %s", _configurationFileName.c_str());
-
-#ifdef YAML_SUPPORT
-		YAML::Node flow = YAML::LoadFile(_configurationFileName);
-
-		YAML::Node flowControllerNode = flow["Flow Controller"];
-		YAML::Node processorsNode = flow[CONFIG_YAML_PROCESSORS_KEY];
-		YAML::Node connectionsNode = flow["Connections"];
-		YAML::Node remoteProcessingGroupNode = flow["Remote Processing Groups"];
+void FlowController::load() {
+  std::lock_guard<std::recursive_mutex> flow_lock(mutex_);
+  if (running_) {
+    stop(true);
+  }
+  if (!initialized_) {
+    logger_->log_info("Load Flow Controller from file %s",
+                      configuration_filename_.c_str());
 
-		// Create the root process group
-		parseRootProcessGroupYaml(flowControllerNode);
-		parseProcessorNodeYaml(processorsNode, this->_root);
-		parseRemoteProcessGroupYaml(&remoteProcessingGroupNode, this->_root);
-		parseConnectionYaml(&connectionsNode, this->_root);
+    this->root_ = flow_configuration_->getRoot(configuration_filename_);
 
-		// Load Flow File from Repo
-		loadFlowRepo();
-#endif
+    // Load Flow File from Repo
+    loadFlowRepo();
 
-		_initialized = true;
-    }
+    initialized_ = true;
+  }
 }
 
-void FlowControllerImpl::loadFlowRepo()
-{
-	if (this->_flowfileRepo && this->_flowfileRepo->isEnable())
-	{
-		std::map<std::string, Connection *> connectionMap;
-		this->_root->getConnections(&connectionMap);
-		this->_flowfileRepo->loadFlowFileToConnections(&connectionMap);
-	}
-}
-
-void FlowControllerImpl::reload(std::string yamlFile)
-{
-    logger_->log_info("Starting to reload Flow Controller with yaml %s", yamlFile.c_str());
+void FlowController::reload(std::string yamlFile) {
+  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();
+  std::string oldYamlFile = this->configuration_filename_;
+  this->configuration_filename_ = yamlFile;
+  load();
+  start();
+  if (this->root_ != nullptr) {
+    this->configuration_filename_ = oldYamlFile;
+    logger_->log_info("Rollback Flow Controller to YAML %s",
+                      oldYamlFile.c_str());
     stop(true);
     unload();
-    std::string oldYamlFile = this->_configurationFileName;
-    this->_configurationFileName = yamlFile;
     load();
     start();
-       if (!this->_root)
-       {
-        this->_configurationFileName = oldYamlFile;
-        logger_->log_info("Rollback Flow Controller to YAML %s", oldYamlFile.c_str());
-        stop(true);
-        unload();
-        load();
-        start();
+  }
+}
+
+void FlowController::loadFlowRepo() {
+  if (this->flow_file_repo_) {
+    std::map<std::string, std::shared_ptr<Connection>> connectionMap;
+    if (this->root_ != nullptr) {
+      this->root_->getConnections(connectionMap);
     }
+    auto rep = std::static_pointer_cast<core::repository::FlowFileRepository>(
+        flow_file_repo_);
+    rep->loadFlowFileToConnections(connectionMap);
+  }
 }
 
-bool FlowControllerImpl::start() {
-	if (!_initialized) {
-		logger_->log_error(
-				"Can not start Flow Controller because it has not been initialized");
-		return false;
-	} else {
-
-		if (!_running) {
-			logger_->log_info("Starting Flow Controller");
-			this->_timerScheduler.start();
-			this->_eventScheduler.start();
-			if (this->_root)
-				this->_root->startProcessing(&this->_timerScheduler,
-						&this->_eventScheduler);
-			_running = true;
-			this->_protocol->start();
-			this->_provenanceRepo->start();
-			this->_flowfileRepo->start();
-			logger_->log_info("Started Flow Controller");
-		}
-		return true;
-	}
+bool FlowController::start() {
+  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;
+  } else {
+
+    if (!running_) {
+      logger_->log_info("Starting Flow Controller");
+      this->_timerScheduler.start();
+      this->_eventScheduler.start();
+      if (this->root_ != nullptr) {
+        this->root_->startProcessing(&this->_timerScheduler,
+                                     &this->_eventScheduler);
+      }
+      running_ = true;
+      this->protocol_->start();
+      this->provenance_repo_->start();
+      this->flow_file_repo_->start();
+      logger_->log_info("Started Flow Controller");
+    }
+    return true;
+  }
 }
+
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/FlowFileRecord.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/FlowFileRecord.cpp b/libminifi/src/FlowFileRecord.cpp
index a2f2323..7383574 100644
--- a/libminifi/src/FlowFileRecord.cpp
+++ b/libminifi/src/FlowFileRecord.cpp
@@ -27,253 +27,339 @@
 #include <cstdio>
 
 #include "FlowFileRecord.h"
-#include "Relationship.h"
-#include "Logger.h"
-#include "FlowController.h"
-#include "FlowFileRepository.h"
-
-std::atomic<uint64_t> FlowFileRecord::_localFlowSeqNumber(0);
-
-FlowFileRecord::FlowFileRecord(std::map<std::string, std::string> attributes, ResourceClaim *claim)
-: _size(0),
-  _id(_localFlowSeqNumber.load()),
-  _offset(0),
-  _penaltyExpirationMs(0),
-  _claim(claim),
-  _isStoredToRepo(false),
-  _markedDelete(false),
-  _connection(NULL),
-  _orginalConnection(NULL)
-{
-	_entryDate = getTimeMillis();
-	_lineageStartDate = _entryDate;
-
-	char uuidStr[37];
-
-	// Generate the global UUID for the flow record
-	uuid_generate(_uuid);
-	// Increase the local ID for the flow record
-	++_localFlowSeqNumber;
-	uuid_unparse_lower(_uuid, uuidStr);
-	_uuidStr = uuidStr;
-
-	// Populate the default attributes
-    addAttribute(FILENAME, std::to_string(getTimeNano()));
-    addAttribute(PATH, DEFAULT_FLOWFILE_PATH);
-    addAttribute(UUID, uuidStr);
-	// Populate the attributes from the input
-    std::map<std::string, std::string>::iterator it;
-    for (it = attributes.begin(); it!= attributes.end(); it++)
-    {
-    	addAttribute(it->first, it->second);
-    }
+#include "core/logging/Logger.h"
+#include "core/Relationship.h"
+#include "core/Repository.h"
 
-    _snapshot = false;
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
 
-	if (_claim)
-		// Increase the flow file record owned count for the resource claim
-		_claim->increaseFlowFileRecordOwnedCount();
-	logger_ = Logger::getLogger();
-}
+std::atomic<uint64_t> FlowFileRecord::local_flow_seq_number_(0);
 
-FlowFileRecord::FlowFileRecord(FlowFileEventRecord *event)
-: _size(0),
-  _id(_localFlowSeqNumber.load()),
-  _offset(0),
-  _penaltyExpirationMs(0),
-  _claim(NULL),
-  _isStoredToRepo(false),
-  _markedDelete(false),
-  _connection(NULL),
-  _orginalConnection(NULL)
-{
-	_entryDate = event->getFlowFileEntryDate();
-	_lineageStartDate = event->getlineageStartDate();
-	_size = event->getFileSize();
-	_offset = event->getFileOffset();
-	_lineageIdentifiers = event->getLineageIdentifiers();
-	_attributes = event->getAttributes();
-    _snapshot = false;
-    _uuidStr = event->getFlowFileUuid();
-    uuid_parse(_uuidStr.c_str(), _uuid);
-
-    if (_size > 0)
-    {
-    	_claim = new ResourceClaim();
-    }
+FlowFileRecord::FlowFileRecord(
+    std::shared_ptr<core::Repository> flow_repository,
+    std::map<std::string, std::string> attributes,
+    std::shared_ptr<ResourceClaim> claim)
+    : FlowFile(),
+      flow_repository_(flow_repository) {
+
+  id_ = local_flow_seq_number_.load();
+  claim_ = claim;
+  // Increase the local ID for the flow record
+  ++local_flow_seq_number_;
+  // Populate the default attributes
+  addKeyedAttribute(FILENAME, std::to_string(getTimeNano()));
+  addKeyedAttribute(PATH, DEFAULT_FLOWFILE_PATH);
+  addKeyedAttribute(UUID, getUUIDStr());
+  // Populate the attributes from the input
+  std::map<std::string, std::string>::iterator it;
+  for (it = attributes.begin(); it != attributes.end(); it++) {
+    FlowFile::addAttribute(it->first, it->second);
+  }
+
+  snapshot_ = false;
 
-	if (_claim)
-	{
-		_claim->setContentFullPath(event->getContentFullPath());
-		// Increase the flow file record owned count for the resource claim
-		_claim->increaseFlowFileRecordOwnedCount();
+  if (claim_ != nullptr)
+    // Increase the flow file record owned count for the resource claim
+    claim_->increaseFlowFileRecordOwnedCount();
+  logger_ = logging::Logger::getLogger();
+}
+
+FlowFileRecord::FlowFileRecord(
+    std::shared_ptr<core::Repository> flow_repository,
+    std::shared_ptr<core::FlowFile> &event, const std::string &uuidConnection)
+    : FlowFile(),
+      snapshot_(""),
+      flow_repository_(flow_repository) {
+	entry_date_ = event->getEntryDate();
+	lineage_start_date_ = event->getlineageStartDate();
+	lineage_Identifiers_ = event->getlineageIdentifiers();
+	uuid_str_ = event->getUUIDStr();
+	attributes_ = event->getAttributes();
+	size_ = event->getSize();
+	offset_ = event->getOffset();
+	event->getUUID(uuid_);
+	uuid_connection_ = uuidConnection;
+	if (event->getResourceClaim()) {
+	  content_full_fath_ = event->getResourceClaim()->getContentFullPath();
 	}
-	logger_ = Logger::getLogger();
-	++_localFlowSeqNumber;
 }
 
+FlowFileRecord::FlowFileRecord(
+    std::shared_ptr<core::Repository> flow_repository,
+    std::shared_ptr<core::FlowFile> &event)
+    : FlowFile(),
+      uuid_connection_(""),
+      snapshot_(""),
+      flow_repository_(flow_repository) {
 
-FlowFileRecord::~FlowFileRecord()
-{
-	if (!_snapshot)
-		logger_->log_debug("Delete FlowFile UUID %s", _uuidStr.c_str());
-	else
-		logger_->log_debug("Delete SnapShot FlowFile UUID %s", _uuidStr.c_str());
-	if (_claim)
-	{
-		// Decrease the flow file record owned count for the resource claim
-		_claim->decreaseFlowFileRecordOwnedCount();
-		if (_claim->getFlowFileRecordOwnedCount() <= 0)
-		{
-			logger_->log_debug("Delete Resource Claim %s", _claim->getContentFullPath().c_str());
-			std::string value;
-			if (!FlowControllerFactory::getFlowController()->getFlowFileRepository() ||
-					!FlowControllerFactory::getFlowController()->getFlowFileRepository()->isEnable() ||
-					!this->_isStoredToRepo ||
-					!FlowControllerFactory::getFlowController()->getFlowFileRepository()->Get(_uuidStr, value))
-			{
-				// if it is persistent to DB already while it is in the queue, we keep the content
-				std::remove(_claim->getContentFullPath().c_str());
-			}
-			delete _claim;
-		}
-	}
 }
 
-bool FlowFileRecord::addAttribute(FlowAttribute key, std::string value)
-{
-	const char *keyStr = FlowAttributeKey(key);
-	if (keyStr)
-	{
-		std::string keyString = keyStr;
-		return addAttribute(keyString, value);
-	}
-	else
-	{
-		return false;
-	}
+FlowFileRecord::~FlowFileRecord() {
+  if (!snapshot_)
+    logger_->log_debug("Delete FlowFile UUID %s", uuid_str_.c_str());
+  else
+    logger_->log_debug("Delete SnapShot FlowFile UUID %s", uuid_str_.c_str());
+  if (claim_) {
+    // Decrease the flow file record owned count for the resource claim
+    claim_->decreaseFlowFileRecordOwnedCount();
+    std::string value;
+    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());
+      }
+    }
+  }
 }
 
-bool FlowFileRecord::addAttribute(std::string key, std::string value)
-{
-	std::map<std::string, std::string>::iterator it = _attributes.find(key);
-	if (it != _attributes.end())
-	{
-		// attribute already there in the map
-		return false;
-	}
-	else
-	{
-		_attributes[key] = value;
-		return true;
-	}
+bool FlowFileRecord::addKeyedAttribute(FlowAttribute key, std::string value) {
+  const char *keyStr = FlowAttributeKey(key);
+  if (keyStr) {
+    const std::string keyString = keyStr;
+    return FlowFile::addAttribute(keyString, value);
+  } else {
+    return false;
+  }
 }
 
-bool FlowFileRecord::removeAttribute(FlowAttribute key)
-{
-	const char *keyStr = FlowAttributeKey(key);
-	if (keyStr)
-	{
-		std::string keyString = keyStr;
-		return removeAttribute(keyString);
-	}
-	else
-	{
-		return false;
-	}
+bool FlowFileRecord::removeKeyedAttribute(FlowAttribute key) {
+  const char *keyStr = FlowAttributeKey(key);
+  if (keyStr) {
+    std::string keyString = keyStr;
+    return FlowFile::removeAttribute(keyString);
+  } else {
+    return false;
+  }
 }
 
-bool FlowFileRecord::removeAttribute(std::string key)
-{
-	std::map<std::string, std::string>::iterator it = _attributes.find(key);
-	if (it != _attributes.end())
-	{
-		_attributes.erase(key);
-		return true;
-	}
-	else
-	{
-		return false;
-	}
+bool FlowFileRecord::updateKeyedAttribute(FlowAttribute key,
+                                          std::string value) {
+  const char *keyStr = FlowAttributeKey(key);
+  if (keyStr) {
+    std::string keyString = keyStr;
+    return FlowFile::updateAttribute(keyString, value);
+  } else {
+    return false;
+  }
 }
 
-bool FlowFileRecord::updateAttribute(FlowAttribute key, std::string value)
-{
-	const char *keyStr = FlowAttributeKey(key);
-	if (keyStr)
-	{
-		std::string keyString = keyStr;
-		return updateAttribute(keyString, value);
-	}
-	else
-	{
-		return false;
-	}
+bool FlowFileRecord::getKeyedAttribute(FlowAttribute key, std::string &value) {
+  const char *keyStr = FlowAttributeKey(key);
+  if (keyStr) {
+    std::string keyString = keyStr;
+    return FlowFile::getAttribute(keyString, value);
+  } else {
+    return false;
+  }
 }
 
-bool FlowFileRecord::updateAttribute(std::string key, std::string value)
-{
-	std::map<std::string, std::string>::iterator it = _attributes.find(key);
-	if (it != _attributes.end())
-	{
-		_attributes[key] = value;
-		return true;
-	}
-	else
-	{
-		return false;
-	}
+FlowFileRecord &FlowFileRecord::operator=(const FlowFileRecord &other) {
+  core::FlowFile::operator=(other);
+  uuid_connection_ = other.uuid_connection_;
+  content_full_fath_ = other.content_full_fath_;
+  snapshot_ = other.snapshot_;
+  return *this;
 }
 
-bool FlowFileRecord::getAttribute(FlowAttribute key, std::string &value)
-{
-	const char *keyStr = FlowAttributeKey(key);
-	if (keyStr)
-	{
-		std::string keyString = keyStr;
-		return getAttribute(keyString, value);
-	}
-	else
-	{
-		return false;
-	}
+bool FlowFileRecord::DeSerialize(std::string key) {
+  std::string value;
+  bool ret;
+
+  ret = flow_repository_->Get(key, value);
+
+  if (!ret) {
+    logger_->log_error("NiFi FlowFile Store event %s can not found",
+                       key.c_str());
+    return false;
+  } else
+    logger_->log_debug("NiFi FlowFile Read event %s length %d", key.c_str(),
+                       value.length());
+
+  io::DataStream stream((const uint8_t*) value.data(), value.length());
+
+  ret = DeSerialize(stream);
+
+  if (ret) {
+    logger_->log_debug(
+        "NiFi FlowFile retrieve uuid %s size %d connection %s success",
+        uuid_str_.c_str(), stream.getSize(), uuid_connection_.c_str());
+  } else {
+    logger_->log_debug(
+        "NiFi FlowFile retrieve uuid %s size %d connection %d fail",
+        uuid_str_.c_str(), stream.getSize(), uuid_connection_.c_str());
+  }
+
+  return ret;
 }
 
-bool FlowFileRecord::getAttribute(std::string key, std::string &value)
-{
-	std::map<std::string, std::string>::iterator it = _attributes.find(key);
-	if (it != _attributes.end())
-	{
-		value = it->second;
-		return true;
-	}
-	else
-	{
-		return false;
-	}
+bool FlowFileRecord::Serialize() {
+
+  io::DataStream outStream;
+
+  int ret;
+
+  ret = write(this->event_time_, &outStream);
+  if (ret != 8) {
+
+    return false;
+  }
+
+  ret = write(this->entry_date_, &outStream);
+  if (ret != 8) {
+    return false;
+  }
+
+  ret = write(this->lineage_start_date_, &outStream);
+  if (ret != 8) {
+
+    return false;
+  }
+
+  ret = writeUTF(this->uuid_str_, &outStream);
+  if (ret <= 0) {
+
+    return false;
+  }
+
+  ret = writeUTF(this->uuid_connection_, &outStream);
+  if (ret <= 0) {
+
+    return false;
+  }
+  // write flow attributes
+  uint32_t numAttributes = this->attributes_.size();
+  ret = write(numAttributes, &outStream);
+  if (ret != 4) {
+
+    return false;
+  }
+
+  for (auto itAttribute : attributes_) {
+    ret = writeUTF(itAttribute.first, &outStream, true);
+    if (ret <= 0) {
+
+      return false;
+    }
+    ret = writeUTF(itAttribute.second, &outStream, true);
+    if (ret <= 0) {
+
+      return false;
+    }
+  }
+
+  ret = writeUTF(this->content_full_fath_, &outStream);
+  if (ret <= 0) {
+
+    return false;
+  }
+
+  ret = write(this->size_, &outStream);
+  if (ret != 8) {
+
+    return false;
+  }
+
+  ret = write(this->offset_, &outStream);
+  if (ret != 8) {
+
+    return false;
+  }
+
+  // Persistent to the DB
+  
+
+  if (flow_repository_->Put(uuid_str_,
+                            const_cast<uint8_t*>(outStream.getBuffer()),
+                            outStream.getSize())) {
+    logger_->log_debug("NiFi FlowFile Store event %s size %d success",
+                       uuid_str_.c_str(), outStream.getSize());
+    return true;
+  } else {
+    logger_->log_error("NiFi FlowFile Store event %s size %d fail",
+                       uuid_str_.c_str(), outStream.getSize());
+    return false;
+  }
+
+  // cleanup
+
+  return true;
 }
 
-void FlowFileRecord::duplicate(FlowFileRecord *original)
-{
-	uuid_copy(this->_uuid, original->_uuid);
-	this->_attributes = original->_attributes;
-	this->_entryDate = original->_entryDate;
-	this->_id = original->_id;
-	this->_lastQueueDate = original->_lastQueueDate;
-	this->_lineageStartDate = original->_lineageStartDate;
-	this->_offset = original->_offset;
-	this->_penaltyExpirationMs = original->_penaltyExpirationMs;
-	this->_size = original->_size;
-	this->_lineageIdentifiers = original->_lineageIdentifiers;
-	this->_orginalConnection = original->_orginalConnection;
-	this->_uuidStr = original->_uuidStr;
-	this->_connection = original->_connection;
-	this->_markedDelete = original->_markedDelete;
-
-	this->_claim = original->_claim;
-	if (this->_claim)
-		this->_claim->increaseFlowFileRecordOwnedCount();
-
-	this->_snapshot = true;
+bool FlowFileRecord::DeSerialize(const uint8_t *buffer, const int bufferSize) {
+
+  int ret;
+
+  io::DataStream outStream(buffer, bufferSize);
+
+  ret = read(this->event_time_, &outStream);
+  if (ret != 8) {
+    return false;
+  }
+
+  ret = read(this->entry_date_, &outStream);
+  if (ret != 8) {
+    return false;
+  }
+
+  ret = read(this->lineage_start_date_, &outStream);
+  if (ret != 8) {
+    return false;
+  }
+
+  ret = readUTF(this->uuid_str_, &outStream);
+  if (ret <= 0) {
+    return false;
+  }
+
+  ret = readUTF(this->uuid_connection_, &outStream);
+  if (ret <= 0) {
+    return false;
+  }
+
+  // read flow attributes
+  uint32_t numAttributes = 0;
+  ret = read(numAttributes, &outStream);
+  if (ret != 4) {
+    return false;
+  }
+
+  for (uint32_t i = 0; i < numAttributes; i++) {
+    std::string key;
+    ret = readUTF(key, &outStream, true);
+    if (ret <= 0) {
+      return false;
+    }
+    std::string value;
+    ret = readUTF(value, &outStream, true);
+    if (ret <= 0) {
+      return false;
+    }
+    this->attributes_[key] = value;
+  }
+
+  ret = readUTF(this->content_full_fath_, &outStream);
+  if (ret <= 0) {
+    return false;
+  }
+
+  ret = read(this->size_, &outStream);
+  if (ret != 8) {
+    return false;
+  }
+
+  ret = read(this->offset_, &outStream);
+  if (ret != 8) {
+    return false;
+  }
+
+  return true;
 }
 
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/FlowFileRepository.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/FlowFileRepository.cpp b/libminifi/src/FlowFileRepository.cpp
deleted file mode 100644
index 3388738..0000000
--- a/libminifi/src/FlowFileRepository.cpp
+++ /dev/null
@@ -1,282 +0,0 @@
-/**
- * @file FlowFileRepository.cpp
- * FlowFile implemenatation 
- *
- * 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 <cstdint>
-#include <vector>
-#include <arpa/inet.h>
-#include "io/DataStream.h"
-#include "io/Serializable.h"
-#include "FlowFileRecord.h"
-#include "Relationship.h"
-#include "Logger.h"
-#include "FlowController.h"
-#include "FlowFileRepository.h"
-
-//! DeSerialize
-bool FlowFileEventRecord::DeSerialize(FlowFileRepository *repo,
-		std::string key) {
-	std::string value;
-	bool ret;
-
-	ret = repo->Get(key, value);
-
-	if (!ret) {
-		logger_->log_error("NiFi FlowFile Store event %s can not found",
-				key.c_str());
-		return false;
-	} else
-		logger_->log_debug("NiFi FlowFile Read event %s length %d",
-				key.c_str(), value.length());
-
-
-	DataStream stream((const uint8_t*)value.data(),value.length());
-
-	ret = DeSerialize(stream);
-
-	if (ret) {
-		logger_->log_debug(
-				"NiFi FlowFile retrieve uuid %s size %d connection %s success",
-				_uuid.c_str(), stream.getSize(), _uuidConnection.c_str());
-	} else {
-		logger_->log_debug(
-				"NiFi FlowFile retrieve uuid %s size %d connection %d fail",
-				_uuid.c_str(), stream.getSize(), _uuidConnection.c_str());
-	}
-
-	return ret;
-}
-
-bool FlowFileEventRecord::Serialize(FlowFileRepository *repo) {
-
-	DataStream outStream;
-
-	int ret;
-
-	ret = write(this->_eventTime,&outStream);
-	if (ret != 8) {
-
-		return false;
-	}
-
-	ret = write(this->_entryDate,&outStream);
-	if (ret != 8) {
-		return false;
-	}
-
-	ret = write(this->_lineageStartDate,&outStream);
-	if (ret != 8) {
-
-		return false;
-	}
-
-	ret = writeUTF(this->_uuid,&outStream);
-	if (ret <= 0) {
-
-		return false;
-	}
-
-	ret = writeUTF(this->_uuidConnection,&outStream);
-	if (ret <= 0) {
-
-		return false;
-	}
-
-	// write flow attributes
-	uint32_t numAttributes = this->_attributes.size();
-	ret = write(numAttributes,&outStream);
-	if (ret != 4) {
-
-		return false;
-	}
-
-	for (auto itAttribute : _attributes) {
-		ret = writeUTF(itAttribute.first,&outStream, true);
-		if (ret <= 0) {
-
-			return false;
-		}
-		ret = writeUTF(itAttribute.second,&outStream, true);
-		if (ret <= 0) {
-
-			return false;
-		}
-	}
-
-	ret = writeUTF(this->_contentFullPath,&outStream);
-	if (ret <= 0) {
-
-		return false;
-	}
-
-	ret = write(this->_size,&outStream);
-	if (ret != 8) {
-
-		return false;
-	}
-
-	ret = write(this->_offset,&outStream);
-	if (ret != 8) {
-
-		return false;
-	}
-
-	// Persistent to the DB
-
-	if (repo->Put(_uuid, const_cast<uint8_t*>(outStream.getBuffer()), outStream.getSize())) {
-		logger_->log_debug("NiFi FlowFile Store event %s size %d success",
-				_uuid.c_str(), outStream.getSize());
-		return true;
-	} else {
-		logger_->log_error("NiFi FlowFile Store event %s size %d fail",
-				_uuid.c_str(), outStream.getSize());
-		return false;
-	}
-
-	// cleanup
-
-	return true;
-}
-
-bool FlowFileEventRecord::DeSerialize(const uint8_t *buffer, const int bufferSize) {
-
-	int ret;
-
-	DataStream outStream(buffer,bufferSize);
-
-	ret = read(this->_eventTime,&outStream);
-	if (ret != 8) {
-		return false;
-	}
-
-	ret = read(this->_entryDate,&outStream);
-	if (ret != 8) {
-		return false;
-	}
-
-	ret = read(this->_lineageStartDate,&outStream);
-	if (ret != 8) {
-		return false;
-	}
-
-	ret = readUTF(this->_uuid,&outStream);
-	if (ret <= 0) {
-		return false;
-	}
-
-	ret = readUTF(this->_uuidConnection,&outStream);
-	if (ret <= 0) {
-		return false;
-	}
-
-	// read flow attributes
-	uint32_t numAttributes = 0;
-	ret = read(numAttributes,&outStream);
-	if (ret != 4) {
-		return false;
-	}
-
-	for (uint32_t i = 0; i < numAttributes; i++) {
-		std::string key;
-		ret = readUTF(key,&outStream, true);
-		if (ret <= 0) {
-			return false;
-		}
-		std::string value;
-		ret = readUTF(value,&outStream, true);
-		if (ret <= 0) {
-			return false;
-		}
-		this->_attributes[key] = value;
-	}
-
-	ret = readUTF(this->_contentFullPath,&outStream);
-	if (ret <= 0) {
-		return false;
-	}
-
-	ret = read(this->_size,&outStream);
-	if (ret != 8) {
-		return false;
-	}
-
-	ret = read(this->_offset,&outStream);
-	if (ret != 8) {
-		return false;
-	}
-
-	return true;
-}
-
-void FlowFileRepository::loadFlowFileToConnections(std::map<std::string, Connection *> *connectionMap)
-{
-#ifdef LEVELDB_SUPPORT
-	if (!_enable)
-		return;
-
-	std::vector<std::string> purgeList;
-	leveldb::Iterator* it = _db->NewIterator(
-						leveldb::ReadOptions());
-
-	for (it->SeekToFirst(); it->Valid(); it->Next())
-	{
-		FlowFileEventRecord eventRead;
-		std::string key = it->key().ToString();
-		if (eventRead.DeSerialize((uint8_t *) it->value().data(),
-				(int) 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
-				FlowFileRecord *record = new FlowFileRecord(&eventRead);
-				// set store to repo to true so that we do need to persistent again in enqueue
-				record->setStoredToRepository(true);
-				search->second->put(record);
-			}
-			else
-			{
-				if (eventRead.getContentFullPath().length() > 0)
-				{
-					std::remove(eventRead.getContentFullPath().c_str());
-				}
-				purgeList.push_back(key);
-			}
-		}
-		else
-		{
-			purgeList.push_back(key);
-		}
-	}
-
-	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",
-										RepositoryTypeStr[_type],
-										eventId.c_str());
-		Delete(eventId);
-	}
-#endif
-
-	return;
-}
-

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/GenerateFlowFile.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/GenerateFlowFile.cpp b/libminifi/src/GenerateFlowFile.cpp
deleted file mode 100644
index 12d7f70..0000000
--- a/libminifi/src/GenerateFlowFile.cpp
+++ /dev/null
@@ -1,135 +0,0 @@
-/**
- * @file GenerateFlowFile.cpp
- * GenerateFlowFile class implementation
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#include <vector>
-#include <queue>
-#include <map>
-#include <set>
-#include <sys/time.h>
-#include <time.h>
-#include <chrono>
-#include <thread>
-#include <random>
-#include "utils/StringUtils.h"
-
-#include "GenerateFlowFile.h"
-#include "ProcessContext.h"
-#include "ProcessSession.h"
-
-const char *GenerateFlowFile::DATA_FORMAT_BINARY = "Binary";
-const char *GenerateFlowFile::DATA_FORMAT_TEXT = "Text";
-const std::string GenerateFlowFile::ProcessorName("GenerateFlowFile");
-Property GenerateFlowFile::FileSize("File Size", "The size of the file that will be used", "1 kB");
-Property GenerateFlowFile::BatchSize("Batch Size", "The number of FlowFiles to be transferred in each invocation", "1");
-Property GenerateFlowFile::DataFormat("Data Format", "Specifies whether the data should be Text or Binary", GenerateFlowFile::DATA_FORMAT_BINARY);
-Property GenerateFlowFile::UniqueFlowFiles("Unique FlowFiles",
-		"If true, each FlowFile that is generated will be unique. If false, a random value will be generated and all FlowFiles", "true");
-Relationship GenerateFlowFile::Success("success", "success operational on the flow record");
-
-void GenerateFlowFile::initialize()
-{
-	//! Set the supported properties
-	std::set<Property> properties;
-	properties.insert(FileSize);
-	properties.insert(BatchSize);
-	properties.insert(DataFormat);
-	properties.insert(UniqueFlowFiles);
-	setSupportedProperties(properties);
-	//! Set the supported relationships
-	std::set<Relationship> relationships;
-	relationships.insert(Success);
-	setSupportedRelationships(relationships);
-}
-
-void GenerateFlowFile::onTrigger(ProcessContext *context, ProcessSession *session)
-{
-	int64_t batchSize = 1;
-	bool uniqueFlowFile = true;
-	int64_t fileSize = 1024;
-
-	std::string value;
-	if (context->getProperty(FileSize.getName(), value))
-	{
-		Property::StringToInt(value, fileSize);
-	}
-	if (context->getProperty(BatchSize.getName(), value))
-	{
-		Property::StringToInt(value, batchSize);
-	}
-	if (context->getProperty(UniqueFlowFiles.getName(), value))
-	{
-		StringUtils::StringToBool(value, uniqueFlowFile);
-	}
-
-	if (!uniqueFlowFile)
-	{
-		char *data;
-		data = new char[fileSize];
-		if (!data)
-			return;
-		uint64_t dataSize = fileSize;
-		GenerateFlowFile::WriteCallback callback(data, dataSize);
-		char *current = data;
-		for (int i = 0; i < fileSize; i+= sizeof(int))
-		{
-			int randValue = random();
-			*((int *) current) = randValue;
-			current += sizeof(int);
-		}
-		for (int i = 0; i < batchSize; i++)
-		{
-			// For each batch
-			FlowFileRecord *flowFile = session->create();
-			if (!flowFile)
-				return;
-			if (fileSize > 0)
-				session->write(flowFile, &callback);
-			session->transfer(flowFile, Success);
-		}
-		delete[] data;
-	}
-	else
-	{
-		if (!_data)
-		{
-			// We have not create the unique data yet
-			_data = new char[fileSize];
-			_dataSize = fileSize;
-			char *current = _data;
-			for (int i = 0; i < fileSize; i+= sizeof(int))
-			{
-				int randValue = random();
-				*((int *) current) = randValue;
-				// *((int *) current) = (0xFFFFFFFF & i);
-				current += sizeof(int);
-			}
-		}
-		GenerateFlowFile::WriteCallback callback(_data, _dataSize);
-		for (int i = 0; i < batchSize; i++)
-		{
-			// For each batch
-			FlowFileRecord *flowFile = session->create();
-			if (!flowFile)
-				return;
-			if (fileSize > 0)
-				session->write(flowFile, &callback);
-			session->transfer(flowFile, Success);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/GetFile.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/GetFile.cpp b/libminifi/src/GetFile.cpp
deleted file mode 100644
index 40dd387..0000000
--- a/libminifi/src/GetFile.cpp
+++ /dev/null
@@ -1,329 +0,0 @@
-/**
- * @file GetFile.cpp
- * GetFile class implementation
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#include <vector>
-#include <queue>
-#include <map>
-#include <set>
-#include <sys/time.h>
-#include <sys/types.h>
-#include <sys/stat.h>
-#include <time.h>
-#include <sstream>
-#include <stdio.h>
-#include <string>
-#include <iostream>
-#include <dirent.h>
-#include <limits.h>
-#include <unistd.h>
-#if  (__GNUC__ >= 4) 
-	#if (__GNUC_MINOR__ < 9)
-		#include <regex.h>
-	#endif
-#endif
-#include "utils/StringUtils.h"
-#include <regex>
-#include "utils/TimeUtil.h"
-#include "GetFile.h"
-#include "ProcessContext.h"
-#include "ProcessSession.h"
-
-const std::string GetFile::ProcessorName("GetFile");
-Property GetFile::BatchSize("Batch Size", "The maximum number of files to pull in each iteration", "10");
-Property GetFile::Directory("Input Directory", "The input directory from which to pull files", ".");
-Property GetFile::IgnoreHiddenFile("Ignore Hidden Files", "Indicates whether or not hidden files should be ignored", "true");
-Property GetFile::KeepSourceFile("Keep Source File",
-		"If true, the file is not deleted after it has been copied to the Content Repository", "false");
-Property GetFile::MaxAge("Maximum File Age",
-		"The minimum age that a file must be in order to be pulled; any file younger than this amount of time (according to last modification date) will be ignored", "0 sec");
-Property GetFile::MinAge("Minimum File Age",
-		"The maximum age that a file must be in order to be pulled; any file older than this amount of time (according to last modification date) will be ignored", "0 sec");
-Property GetFile::MaxSize("Maximum File Size", "The maximum size that a file can be in order to be pulled", "0 B");
-Property GetFile::MinSize("Minimum File Size", "The minimum size that a file must be in order to be pulled", "0 B");
-Property GetFile::PollInterval("Polling Interval", "Indicates how long to wait before performing a directory listing", "0 sec");
-Property GetFile::Recurse("Recurse Subdirectories", "Indicates whether or not to pull files from subdirectories", "true");
-Property GetFile::FileFilter("File Filter", "Only files whose names match the given regular expression will be picked up", "[^\\.].*");
-Relationship GetFile::Success("success", "All files are routed to success");
-
-void GetFile::initialize()
-{
-	//! Set the supported properties
-	std::set<Property> properties;
-	properties.insert(BatchSize);
-	properties.insert(Directory);
-	properties.insert(IgnoreHiddenFile);
-	properties.insert(KeepSourceFile);
-	properties.insert(MaxAge);
-	properties.insert(MinAge);
-	properties.insert(MaxSize);
-	properties.insert(MinSize);
-	properties.insert(PollInterval);
-	properties.insert(Recurse);
-	properties.insert(FileFilter);
-	setSupportedProperties(properties);
-	//! Set the supported relationships
-	std::set<Relationship> relationships;
-	relationships.insert(Success);
-	setSupportedRelationships(relationships);
-}
-
-void GetFile::onTrigger(ProcessContext *context, ProcessSession *session)
-{
-	std::string value;
-
-	logger_->log_info("onTrigger GetFile");
-	if (context->getProperty(Directory.getName(), value))
-	{
-		_directory = value;
-	}
-	if (context->getProperty(BatchSize.getName(), value))
-	{
-		Property::StringToInt(value, _batchSize);
-	}
-	if (context->getProperty(IgnoreHiddenFile.getName(), value))
-	{
-		StringUtils::StringToBool(value, _ignoreHiddenFile);
-	}
-	if (context->getProperty(KeepSourceFile.getName(), value))
-	{
-		StringUtils::StringToBool(value, _keepSourceFile);
-	}
-
-	logger_->log_info("onTrigger GetFile");
-	if (context->getProperty(MaxAge.getName(), value))
-	{
-		TimeUnit unit;
-		if (Property::StringToTime(value, _maxAge, unit) &&
-			Property::ConvertTimeUnitToMS(_maxAge, unit, _maxAge))
-		{
-
-		}
-	}
-	if (context->getProperty(MinAge.getName(), value))
-	{
-		TimeUnit unit;
-		if (Property::StringToTime(value, _minAge, unit) &&
-			Property::ConvertTimeUnitToMS(_minAge, unit, _minAge))
-		{
-
-		}
-	}
-	if (context->getProperty(MaxSize.getName(), value))
-	{
-		Property::StringToInt(value, _maxSize);
-	}
-	if (context->getProperty(MinSize.getName(), value))
-	{
-		Property::StringToInt(value, _minSize);
-	}
-	if (context->getProperty(PollInterval.getName(), value))
-	{
-		TimeUnit unit;
-		if (Property::StringToTime(value, _pollInterval, unit) &&
-			Property::ConvertTimeUnitToMS(_pollInterval, unit, _pollInterval))
-		{
-
-		}
-	}
-	if (context->getProperty(Recurse.getName(), value))
-	{
-		StringUtils::StringToBool(value, _recursive);
-	}
-
-	if (context->getProperty(FileFilter.getName(), value))
-	{
-		_fileFilter = value;
-	}
-
-	// Perform directory list
-	logger_->log_info("Is listing empty %i",isListingEmpty());
-	if (isListingEmpty())
-	{
-		if (_pollInterval == 0 || (getTimeMillis() - _lastDirectoryListingTime) > _pollInterval)
-		{
-			performListing(_directory);
-		}
-	}
-	logger_->log_info("Is listing empty %i",isListingEmpty());
-
-	if (!isListingEmpty())
-	{
-		try
-		{
-			std::queue<std::string> list;
-			pollListing(list, _batchSize);
-			while (!list.empty())
-			{
-
-				std::string fileName = list.front();
-				list.pop();
-				logger_->log_info("GetFile process %s", fileName.c_str());
-				FlowFileRecord *flowFile = session->create();
-				if (!flowFile)
-					return;
-				std::size_t found = fileName.find_last_of("/\\");
-				std::string path = fileName.substr(0,found);
-				std::string name = fileName.substr(found+1);
-				flowFile->updateAttribute(FILENAME, name);
-				flowFile->updateAttribute(PATH, path);
-				flowFile->addAttribute(ABSOLUTE_PATH, fileName);
-				session->import(fileName, flowFile, _keepSourceFile);
-				session->transfer(flowFile, Success);
-			}
-		}
-		catch (std::exception &exception)
-		{
-			logger_->log_debug("GetFile Caught Exception %s", exception.what());
-			throw;
-		}
-		catch (...)
-		{
-			throw;
-		}
-	}
-
-}
-
-bool GetFile::isListingEmpty()
-{
-	std::lock_guard<std::mutex> lock(_mtx);
-
-	return _dirList.empty();
-}
-
-void GetFile::putListing(std::string fileName)
-{
-	std::lock_guard<std::mutex> lock(_mtx);
-
-	_dirList.push(fileName);
-}
-
-void GetFile::pollListing(std::queue<std::string> &list, int maxSize)
-{
-	std::lock_guard<std::mutex> lock(_mtx);
-
-	while (!_dirList.empty() && (maxSize == 0 || list.size() < maxSize))
-	{
-		std::string fileName = _dirList.front();
-		_dirList.pop();
-		list.push(fileName);
-	}
-
-	return;
-}
-
-bool GetFile::acceptFile(std::string fullName, std::string name)
-{
-	struct stat statbuf;
-
-	if (stat(fullName.c_str(), &statbuf) == 0)
-	{
-		if (_minSize > 0 && statbuf.st_size <_minSize)
-			return false;
-
-		if (_maxSize > 0 && statbuf.st_size > _maxSize)
-			return false;
-
-		uint64_t modifiedTime = ((uint64_t) (statbuf.st_mtime) * 1000);
-		uint64_t fileAge = getTimeMillis() - modifiedTime;
-		if (_minAge > 0 && fileAge < _minAge)
-			return false;
-		if (_maxAge > 0 && fileAge > _maxAge)
-			return false;
-
-		if (_ignoreHiddenFile && fullName.c_str()[0] == '.')
-			return false;
-
-		if (access(fullName.c_str(), R_OK) != 0)
-			return false;
-
-		if (_keepSourceFile == false && access(fullName.c_str(), W_OK) != 0)
-			return false;
-
-
-	#ifdef __GNUC__ 
-		#if (__GNUC__ >= 4)
-			#if (__GNUC_MINOR__ < 9)
-				regex_t regex;
-				int ret = regcomp(&regex, _fileFilter.c_str(),0);
-				if (ret)
-					return false;
-				ret = regexec(&regex,name.c_str(),(size_t)0,NULL,0);
-				regfree(&regex);
-				if (ret)
-					return false;	
-			#else
-				try{
-					std::regex re(_fileFilter);
-	
-					if (!std::regex_match(name, re)) {
-						return false;
-   					}
-				} catch (std::regex_error e) {
-					logger_->log_error("Invalid File Filter regex: %s.", e.what());
-					return false;
-				}
-			#endif
-		#endif
-		#else
-			logger_->log_info("Cannot support regex filtering");
-		#endif
-		return true;
-	}
-
-	return false;
-}
-
-void GetFile::performListing(std::string dir)
-{
-	logger_->log_info("Performing file listing against %s",dir.c_str());
-	DIR *d;
-	d = opendir(dir.c_str());
-	if (!d)
-		return;
-	// only perform a listing while we are not empty
-	logger_->log_info("Performing file listing against %s",dir.c_str());
-	while (isRunning())
-	{
-		struct dirent *entry;
-		entry = readdir(d);
-		if (!entry)
-			break;
-		std::string d_name = entry->d_name;
-		if ((entry->d_type & DT_DIR))
-		{
-			// if this is a directory
-			if (_recursive && strcmp(d_name.c_str(), "..") != 0 && strcmp(d_name.c_str(), ".") != 0)
-			{
-				std::string path = dir + "/" + d_name;
-				performListing(path);
-			}
-		}
-		else
-		{
-			std::string fileName = dir + "/" + d_name;
-			if (acceptFile(fileName, d_name))
-			{
-				// check whether we can take this file
-				putListing(fileName);
-			}
-		}
-	}
-	closedir(d);
-}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/ListenHTTP.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/ListenHTTP.cpp b/libminifi/src/ListenHTTP.cpp
deleted file mode 100644
index 89ce1d2..0000000
--- a/libminifi/src/ListenHTTP.cpp
+++ /dev/null
@@ -1,395 +0,0 @@
-/**
- * @file ListenHTTP.cpp
- * ListenHTTP class implementation
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#include <sstream>
-#include <stdio.h>
-#include <string>
-#include <iostream>
-#include <fstream>
-#include <uuid/uuid.h>
-
-#include <CivetServer.h>
-
-#include "ListenHTTP.h"
-
-#include "utils/TimeUtil.h"
-#include "ProcessContext.h"
-#include "ProcessSession.h"
-#include "ProcessSessionFactory.h"
-
-const std::string ListenHTTP::ProcessorName("ListenHTTP");
-
-Property ListenHTTP::BasePath("Base Path", "Base path for incoming connections", "contentListener");
-Property ListenHTTP::Port("Listening Port", "The Port to listen on for incoming connections", "");
-Property ListenHTTP::AuthorizedDNPattern("Authorized DN Pattern", "A Regular Expression to apply against the Distinguished Name of incoming connections. If the Pattern does not match the DN, the connection will be refused.", ".*");
-Property ListenHTTP::SSLCertificate("SSL Certificate", "File containing PEM-formatted file including TLS/SSL certificate and key", "");
-Property ListenHTTP::SSLCertificateAuthority("SSL Certificate Authority", "File containing trusted PEM-formatted certificates", "");
-Property ListenHTTP::SSLVerifyPeer("SSL Verify Peer", "Whether or not to verify the client's certificate (yes/no)", "no");
-Property ListenHTTP::SSLMinimumVersion("SSL Minimum Version", "Minimum TLS/SSL version allowed (SSL2, SSL3, TLS1.0, TLS1.1, TLS1.2)", "SSL2");
-Property ListenHTTP::HeadersAsAttributesRegex("HTTP Headers to receive as Attributes (Regex)", "Specifies the Regular Expression that determines the names of HTTP Headers that should be passed along as FlowFile attributes", "");
-
-Relationship ListenHTTP::Success("success", "All files are routed to success");
-
-void ListenHTTP::initialize()
-{
-	_logger->log_info("Initializing ListenHTTP");
-
-	//! Set the supported properties
-	std::set<Property> properties;
-	properties.insert(BasePath);
-	properties.insert(Port);
-	properties.insert(AuthorizedDNPattern);
-	properties.insert(SSLCertificate);
-	properties.insert(SSLCertificateAuthority);
-	properties.insert(SSLVerifyPeer);
-	properties.insert(SSLMinimumVersion);
-	properties.insert(HeadersAsAttributesRegex);
-	setSupportedProperties(properties);
-	//! Set the supported relationships
-	std::set<Relationship> relationships;
-	relationships.insert(Success);
-	setSupportedRelationships(relationships);
-}
-
-void ListenHTTP::onSchedule(ProcessContext *context, ProcessSessionFactory *sessionFactory)
-{
-
-	std::string basePath;
-
-	if (!context->getProperty(BasePath.getName(), basePath))
-	{
-		_logger->log_info("%s attribute is missing, so default value of %s will be used",
-				BasePath.getName().c_str(),
-				BasePath.getValue().c_str());
-		basePath = BasePath.getValue();
-	}
-
-	basePath.insert(0, "/");
-
-	std::string listeningPort;
-
-	if (!context->getProperty(Port.getName(), listeningPort))
-	{
-		_logger->log_error("%s attribute is missing or invalid",
-				Port.getName().c_str());
-		return;
-	}
-
-	std::string authDNPattern;
-
-	if (context->getProperty(AuthorizedDNPattern.getName(), authDNPattern) && !authDNPattern.empty())
-	{
-		_logger->log_info("ListenHTTP using %s: %s",
-				AuthorizedDNPattern.getName().c_str(),
-				authDNPattern.c_str());
-	}
-
-	std::string sslCertFile;
-
-	if (context->getProperty(SSLCertificate.getName(), sslCertFile) && !sslCertFile.empty())
-	{
-		_logger->log_info("ListenHTTP using %s: %s",
-				SSLCertificate.getName().c_str(),
-				sslCertFile.c_str());
-	}
-
-	// Read further TLS/SSL options only if TLS/SSL usage is implied by virtue of certificate value being set
-	std::string sslCertAuthorityFile;
-	std::string sslVerifyPeer;
-	std::string sslMinVer;
-
-	if (!sslCertFile.empty())
-	{
-		if (context->getProperty(SSLCertificateAuthority.getName(), sslCertAuthorityFile)
-				&& !sslCertAuthorityFile.empty())
-		{
-			_logger->log_info("ListenHTTP using %s: %s",
-					SSLCertificateAuthority.getName().c_str(),
-					sslCertAuthorityFile.c_str());
-		}
-
-		if (context->getProperty(SSLVerifyPeer.getName(), sslVerifyPeer))
-		{
-			if (sslVerifyPeer.empty() || sslVerifyPeer.compare("no") == 0)
-			{
-				_logger->log_info("ListenHTTP will not verify peers");
-			}
-			else
-			{
-				_logger->log_info("ListenHTTP will verify peers");
-			}
-		}
-		else
-		{
-			_logger->log_info("ListenHTTP will not verify peers");
-		}
-
-		if (context->getProperty(SSLMinimumVersion.getName(), sslMinVer))
-		{
-			_logger->log_info("ListenHTTP using %s: %s",
-					SSLMinimumVersion.getName().c_str(),
-					sslMinVer.c_str());
-		}
-	}
-
-	std::string headersAsAttributesPattern;
-
-	if (context->getProperty(HeadersAsAttributesRegex.getName(),headersAsAttributesPattern)
-			&& !headersAsAttributesPattern.empty())
-	{
-		_logger->log_info("ListenHTTP using %s: %s",
-				HeadersAsAttributesRegex.getName().c_str(),
-				headersAsAttributesPattern.c_str());
-	}
-
-	auto numThreads = getMaxConcurrentTasks();
-
-	_logger->log_info("ListenHTTP starting HTTP server on port %s and path %s with %d threads",
-			listeningPort.c_str(),
-			basePath.c_str(),
-			numThreads);
-
-	// Initialize web server
-	std::vector<std::string> options;
-	options.push_back("enable_keep_alive");
-	options.push_back("yes");
-	options.push_back("keep_alive_timeout_ms");
-	options.push_back("15000");
-	options.push_back("num_threads");
-	options.push_back(std::to_string(numThreads));
-
-	if (sslCertFile.empty())
-	{
-		options.push_back("listening_ports");
-		options.push_back(listeningPort);
-	}
-	else
-	{
-		listeningPort += "s";
-		options.push_back("listening_ports");
-		options.push_back(listeningPort);
-
-		options.push_back("ssl_certificate");
-		options.push_back(sslCertFile);
-
-		if (!sslCertAuthorityFile.empty())
-		{
-			options.push_back("ssl_ca_file");
-			options.push_back(sslCertAuthorityFile);
-		}
-
-		if (sslVerifyPeer.empty() || sslVerifyPeer.compare("no") == 0)
-		{
-			options.push_back("ssl_verify_peer");
-			options.push_back("no");
-		}
-		else
-		{
-			options.push_back("ssl_verify_peer");
-			options.push_back("yes");
-		}
-
-		if (sslMinVer.compare("SSL2") == 0)
-		{
-			options.push_back("ssl_protocol_version");
-			options.push_back(std::to_string(0));
-		}
-		else if (sslMinVer.compare("SSL3") == 0)
-		{
-			options.push_back("ssl_protocol_version");
-			options.push_back(std::to_string(1));
-		}
-		else if (sslMinVer.compare("TLS1.0") == 0)
-		{
-			options.push_back("ssl_protocol_version");
-			options.push_back(std::to_string(2));
-		}
-		else if (sslMinVer.compare("TLS1.1") == 0)
-		{
-			options.push_back("ssl_protocol_version");
-			options.push_back(std::to_string(3));
-		}
-		else
-		{
-			options.push_back("ssl_protocol_version");
-			options.push_back(std::to_string(4));
-		}
-	}
-
-	_server.reset(new CivetServer(options));
-	_handler.reset(new Handler(context,
-			sessionFactory,
-			std::move(authDNPattern),
-			std::move(headersAsAttributesPattern)));
-	_server->addHandler(basePath, _handler.get());
-}
-
-void ListenHTTP::onTrigger(ProcessContext *context, ProcessSession *session)
-{
-
-	FlowFileRecord *flowFile = session->get();
-
-	// Do nothing if there are no incoming files
-	if (!flowFile)
-	{
-		return;
-	}
-}
-
-ListenHTTP::Handler::Handler(ProcessContext *context,
-		ProcessSessionFactory *sessionFactory,
-		std::string &&authDNPattern,
-		std::string &&headersAsAttributesPattern)
-: _authDNRegex(std::move(authDNPattern))
-, _headersAsAttributesRegex(std::move(headersAsAttributesPattern))
-{
-	_processContext = context;
-	_processSessionFactory = sessionFactory;
-}
-
-void ListenHTTP::Handler::sendErrorResponse(struct mg_connection *conn)
-{
-	mg_printf(conn,
-			"HTTP/1.1 500 Internal Server Error\r\n"
-			"Content-Type: text/html\r\n"
-			"Content-Length: 0\r\n\r\n");
-}
-
-bool ListenHTTP::Handler::handlePost(CivetServer *server, struct mg_connection *conn)
-{
-	_logger = Logger::getLogger();
-
-	auto req_info = mg_get_request_info(conn);
-	_logger->log_info("ListenHTTP handling POST request of length %d", req_info->content_length);
-
-	// If this is a two-way TLS connection, authorize the peer against the configured pattern
-	if (req_info->is_ssl && req_info->client_cert != nullptr)
-	{
-		if (!std::regex_match(req_info->client_cert->subject, _authDNRegex))
-		{
-			mg_printf(conn,
-					"HTTP/1.1 403 Forbidden\r\n"
-					"Content-Type: text/html\r\n"
-					"Content-Length: 0\r\n\r\n");
-			_logger->log_warn("ListenHTTP client DN not authorized: %s", req_info->client_cert->subject);
-			return true;
-		}
-	}
-
-	// Always send 100 Continue, as allowed per standard to minimize client delay (https://www.w3.org/Protocols/rfc2616/rfc2616-sec8.html)
-	mg_printf(conn, "HTTP/1.1 100 Continue\r\n\r\n");
-
-	auto session = _processSessionFactory->createSession();
-	ListenHTTP::WriteCallback callback(conn, req_info);
-	auto flowFile = session->create();
-
-	if (!flowFile)
-	{
-		sendErrorResponse(conn);
-		return true;
-	}
-
-	try
-	{
-		session->write(flowFile, &callback);
-
-		// Add filename from "filename" header value (and pattern headers)
-		for (int i = 0; i < req_info->num_headers; i++)
-		{
-			auto header = &req_info->http_headers[i];
-
-			if (strcmp("filename", header->name) == 0)
-			{
-				if (!flowFile->updateAttribute("filename", header->value))
-				{
-					flowFile->addAttribute("filename", header->value);
-				}
-			}
-			else if (std::regex_match(header->name, _headersAsAttributesRegex))
-			{
-				if (!flowFile->updateAttribute(header->name, header->value))
-				{
-					flowFile->addAttribute(header->name, header->value);
-				}
-			}
-		}
-
-		session->transfer(flowFile, Success);
-		session->commit();
-	}
-	catch (std::exception &exception)
-	{
-		_logger->log_debug("ListenHTTP Caught Exception %s", exception.what());
-		sendErrorResponse(conn);
-		session->rollback();
-		throw;
-	}
-	catch (...)
-	{
-		_logger->log_debug("ListenHTTP Caught Exception Processor::onTrigger");
-		sendErrorResponse(conn);
-		session->rollback();
-		throw;
-	}
-
-	mg_printf(conn,
-			"HTTP/1.1 200 OK\r\n"
-			"Content-Type: text/html\r\n"
-			"Content-Length: 0\r\n\r\n");
-
-	return true;
-}
-
-ListenHTTP::WriteCallback::WriteCallback(struct mg_connection *conn, const struct mg_request_info *reqInfo)
-{
-	_logger = Logger::getLogger();
-	_conn = conn;
-	_reqInfo = reqInfo;
-}
-
-void ListenHTTP::WriteCallback::process(std::ofstream *stream)
-{
-	long long rlen;
-	long long nlen = 0;
-	long long tlen = _reqInfo->content_length;
-	char buf[16384];
-
-	while (nlen < tlen)
-	{
-		rlen = tlen - nlen;
-
-		if (rlen > sizeof(buf))
-		{
-			rlen = sizeof(buf);
-		}
-
-		// Read a buffer of data from client
-		rlen = mg_read(_conn, &buf[0], (size_t)rlen);
-
-		if (rlen <= 0)
-		{
-			break;
-		}
-
-		// Transfer buffer data to the output stream
-		stream->write(&buf[0], rlen);
-
-		nlen += rlen;
-	}
-}


[04/16] nifi-minifi-cpp git commit: MINIFI-217: Updates namespaces and removes use of raw pointers for user facing API.

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/core/Repository.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/Repository.cpp b/libminifi/src/core/Repository.cpp
new file mode 100644
index 0000000..9a27785
--- /dev/null
+++ b/libminifi/src/core/Repository.cpp
@@ -0,0 +1,65 @@
+/**
+ *
+ * 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 <cstdint>
+#include <vector>
+#include <arpa/inet.h>
+#include "io/DataStream.h"
+#include "io/Serializable.h"
+#include "core/Relationship.h"
+#include "core/logging/Logger.h"
+#include "FlowController.h"
+#include "core/Repository.h"
+#include "provenance/Provenance.h"
+#include "core/repository/FlowFileRepository.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+void Repository::start() {
+  if (this->purge_period_ <= 0)
+    return;
+  if (running_)
+    return;
+  thread_ = std::thread(&Repository::threadExecutor, this);
+  thread_.detach();
+  running_ = true;
+  logger_->log_info("%s Repository Monitor Thread Start", name_.c_str());
+}
+
+void Repository::stop() {
+  if (!running_)
+    return;
+  running_ = false;
+  if (thread_.joinable())
+    thread_.join();
+  logger_->log_info("%s Repository Monitor Thread Stop", name_.c_str());
+}
+
+// repoSize
+uint64_t Repository::repoSize() {
+  return repo_size_;
+}
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/core/RepositoryFactory.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/RepositoryFactory.cpp b/libminifi/src/core/RepositoryFactory.cpp
new file mode 100644
index 0000000..ef0b9ef
--- /dev/null
+++ b/libminifi/src/core/RepositoryFactory.cpp
@@ -0,0 +1,69 @@
+#include "core/Repository.h"
+#include "core/Repository.h"
+
+#ifdef LEVELDB_SUPPORT
+#include "core/repository/FlowFileRepository.h"
+#include "provenance/ProvenanceRepository.h"
+#endif
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+#ifndef LEVELDB_SUPPORT
+  namespace provenance{
+  class ProvenanceRepository;
+  }
+#endif
+namespace core {
+
+#ifndef LEVELDB_SUPPORT
+  class FlowFileRepository;
+#endif
+
+
+ std::shared_ptr<core::Repository> createRepository(
+      const std::string configuration_class_name, bool fail_safe = false) {
+
+    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::Repository> return_obj = nullptr;
+      if (class_name_lc == "flowfilerepository") {
+
+        return_obj = std::shared_ptr<core::Repository>((core::Repository*)instantiate<core::repository::FlowFileRepository>());
+      } else if (class_name_lc == "provenancerepository") {
+
+
+	return_obj = std::shared_ptr<core::Repository>((core::Repository*)instantiate<provenance::ProvenanceRepository>());
+
+      }
+      
+      if (return_obj){
+        return return_obj;
+      }
+      if (fail_safe) {
+        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);
+      }
+    }
+
+    throw std::runtime_error(
+        "Support for the provided configuration class could not be found");
+  }
+
+  
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/core/logging/BaseLogger.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/logging/BaseLogger.cpp b/libminifi/src/core/logging/BaseLogger.cpp
new file mode 100644
index 0000000..a6b43a8
--- /dev/null
+++ b/libminifi/src/core/logging/BaseLogger.cpp
@@ -0,0 +1,161 @@
+/**
+ *
+ * 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/logging/BaseLogger.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+namespace logging {
+
+// Logger related configuration items.
+const char *BaseLogger::nifi_log_level = "nifi.log.level";
+const char *BaseLogger::nifi_log_appender = "nifi.log.appender";
+
+/**
+ * @brief Log error message
+ * @param format format string ('man printf' for syntax)
+ * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
+ */
+void BaseLogger::log_error(const char * const format, ...) {
+  if (logger_ == NULL || !logger_->should_log(spdlog::level::level_enum::err))
+    return;
+  FILL_BUFFER
+  log_str(err, buffer);
+}
+/**
+ * @brief Log warn message
+ * @param format format string ('man printf' for syntax)
+ * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
+ */
+void BaseLogger::log_warn(const char * const format, ...) {
+  if (logger_ == NULL || !logger_->should_log(spdlog::level::level_enum::warn))
+    return;
+  FILL_BUFFER
+  log_str(warn, buffer);
+}
+/**
+ * @brief Log info message
+ * @param format format string ('man printf' for syntax)
+ * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
+ */
+void BaseLogger::log_info(const char * const format, ...) {
+  if (logger_ == NULL || !logger_->should_log(spdlog::level::level_enum::info))
+    return;
+  FILL_BUFFER
+  log_str(info, buffer);
+}
+/**
+ * @brief Log debug message
+ * @param format format string ('man printf' for syntax)
+ * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
+ */
+void BaseLogger::log_debug(const char * const format, ...) {
+
+  if (logger_ == NULL || !logger_->should_log(spdlog::level::level_enum::debug))
+    return;
+  FILL_BUFFER
+  log_str(debug, buffer);
+}
+/**
+ * @brief Log trace message
+ * @param format format string ('man printf' for syntax)
+ * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
+ */
+void BaseLogger::log_trace(const char * const format, ...) {
+
+  if (logger_ == NULL || !logger_->should_log(spdlog::level::level_enum::trace))
+    return;
+  FILL_BUFFER
+  log_str(debug, buffer);
+}
+
+// overridables
+
+/**
+ * @brief Log error message
+ * @param format format string ('man printf' for syntax)
+ * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
+ */
+void BaseLogger::log_str(LOG_LEVEL_E level, const std::string &buffer) {
+  switch (level) {
+    case err:
+    case critical:
+      if (stderr_ != nullptr) {
+        stderr_->error(buffer);
+      } else {
+        logger_->error(buffer);
+      }
+      break;
+    case warn:
+      logger_->warn(buffer);
+      break;
+    case info:
+      logger_->info(buffer);
+      break;
+    case debug:
+      logger_->debug(buffer);
+      break;
+    case trace:
+      logger_->trace(buffer);
+      break;
+    case off:
+      break;
+    default:
+      logger_->info(buffer);
+      break;
+  }
+
+}
+
+void BaseLogger::setLogLevel(const std::string &level,
+                             LOG_LEVEL_E defaultLevel) {
+  std::string logLevel = level;
+  std::transform(logLevel.begin(), logLevel.end(), logLevel.begin(), ::tolower);
+
+  if (logLevel == "trace") {
+    setLogLevel(trace);
+  } else if (logLevel == "debug") {
+    setLogLevel(debug);
+  } else if (logLevel == "info") {
+    setLogLevel(info);
+  } else if (logLevel == "warn") {
+    setLogLevel(warn);
+  } else if (logLevel == "error") {
+    setLogLevel(err);
+  } else if (logLevel == "critical") {
+    setLogLevel(critical);
+  } else if (logLevel == "off") {
+    setLogLevel(off);
+  } else {
+    setLogLevel(defaultLevel);
+  }
+}
+
+void BaseLogger::set_error_logger(std::shared_ptr<spdlog::logger> other) {
+  stderr_ = std::move(other);
+}
+
+} /* namespace logging */
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/core/logging/LogAppenders.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/logging/LogAppenders.cpp b/libminifi/src/core/logging/LogAppenders.cpp
new file mode 100644
index 0000000..5d92334
--- /dev/null
+++ b/libminifi/src/core/logging/LogAppenders.cpp
@@ -0,0 +1,39 @@
+/**
+ *
+ * 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/logging/LogAppenders.h"
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+namespace logging {
+
+const char *OutputStreamAppender::nifi_log_output_stream_error_stderr="nifi.log.outputstream.appender.error.stderr";
+
+const char *RollingAppender::nifi_log_rolling_apender_file = "nifi.log.rolling.appender.file";
+const char *RollingAppender::nifi_log_rolling_appender_max_files = "nifi.log.rolling.appender.max.files";
+const char *RollingAppender::nifi_log_rolling_appender_max_file_size = "nifi.log.rolling.appender.max.file_size";
+
+
+} /* namespace logging */
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/core/logging/Logger.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/logging/Logger.cpp b/libminifi/src/core/logging/Logger.cpp
new file mode 100644
index 0000000..d8cadf3
--- /dev/null
+++ b/libminifi/src/core/logging/Logger.cpp
@@ -0,0 +1,40 @@
+/**
+ * @file Logger.cpp
+ * Logger class implementation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include "core/logging/Logger.h"
+
+#include <vector>
+#include <queue>
+#include <map>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+namespace logging {
+
+std::shared_ptr<Logger> Logger::singleton_logger_(nullptr);
+
+} /* namespace logging */
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/core/repository/FlowFileRepository.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/repository/FlowFileRepository.cpp b/libminifi/src/core/repository/FlowFileRepository.cpp
new file mode 100644
index 0000000..c495a67
--- /dev/null
+++ b/libminifi/src/core/repository/FlowFileRepository.cpp
@@ -0,0 +1,109 @@
+#include "core/repository/FlowFileRepository.h"
+#include "FlowFileRecord.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+namespace repository {
+
+void FlowFileRepository::run() {
+  // threshold for purge
+  uint64_t purgeThreshold = max_partition_bytes_ * 3 / 4;
+  while (running_) {
+    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()) {
+        std::shared_ptr<FlowFileRecord> eventRead = std::make_shared<FlowFileRecord>(shared_from_this());
+        std::string key = it->key().ToString();
+        if (eventRead->DeSerialize((uint8_t *) it->value().data(),
+                                  (int) it->value().size())) {
+          if ((curTime - eventRead->getEventTime()) > max_partition_millis_)
+            purgeList.push_back(key);
+        } else {
+          logger_->log_debug("NiFi %s retrieve event %s fail", name_.c_str(),
+                             key.c_str());
+          purgeList.push_back(key);
+        }
+      }
+      delete it;
+      for (auto eventId : purgeList) {
+        logger_->log_info("Repository Repo %s Purge %s", name_.c_str(),
+                          eventId.c_str());
+        Delete(eventId);
+      }
+    }
+    if (size > max_partition_bytes_)
+      repo_full_ = true;
+    else
+      repo_full_ = false;
+  }
+  return;
+}
+
+void FlowFileRepository::loadFlowFileToConnections(std::map<std::string, std::shared_ptr<minifi::Connection>> &connectionMap)
+ {
+
+  std::vector<std::string> 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::string key = it->key().ToString();
+    if (eventRead->DeSerialize((uint8_t *) it->value().data(),
+        (int) 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);
+        // set store to repo to true so that we do need to persistent again in enqueue
+        record->setStoredToRepository(true);
+        search->second->put(record);
+      }
+      else
+      {
+        if (eventRead->getContentFullPath().length() > 0)
+        {
+          std::remove(eventRead->getContentFullPath().c_str());
+        }
+        purgeList.push_back(key);
+      }
+    }
+    else
+    {
+      purgeList.push_back(key);
+    }
+  }
+
+  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);
+  }
+
+  return;
+}
+
+} /* namespace repository */
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/core/yaml/YamlConfiguration.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/yaml/YamlConfiguration.cpp b/libminifi/src/core/yaml/YamlConfiguration.cpp
new file mode 100644
index 0000000..8e59363
--- /dev/null
+++ b/libminifi/src/core/yaml/YamlConfiguration.cpp
@@ -0,0 +1,490 @@
+/**
+ *
+ * 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/yaml/YamlConfiguration.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+core::ProcessGroup *YamlConfiguration::parseRootProcessGroupYaml(
+    YAML::Node rootFlowNode) {
+  uuid_t uuid;
+
+  std::string flowName = rootFlowNode["name"].as<std::string>();
+  std::string id ;
+  
+  try {
+    rootFlowNode["id"].as<std::string>();
+
+    uuid_parse(id.c_str(), uuid);
+  }catch(...)
+  {
+    logger_->log_warn("Generating random ID for root node");
+    uuid_generate(uuid);
+    char uuid_str[37];
+    uuid_unparse(uuid,uuid_str);
+    id = uuid_str;
+  }
+
+  logger_->log_debug("parseRootProcessGroup: id => [%s]", id.c_str());
+  logger_->log_debug("parseRootProcessGroup: name => [%s]", flowName.c_str());
+  std::unique_ptr<core::ProcessGroup> group =
+      FlowConfiguration::createRootProcessGroup(flowName, uuid);
+
+  this->name_ = flowName;
+
+  return group.release();
+}
+
+void YamlConfiguration::parseProcessorNodeYaml(
+    YAML::Node processorsNode, core::ProcessGroup * parentGroup) {
+  int64_t schedulingPeriod = -1;
+  int64_t penalizationPeriod = -1;
+  int64_t yieldPeriod = -1;
+  int64_t runDurationNanos = -1;
+  uuid_t uuid;
+  std::shared_ptr<core::Processor> processor = nullptr;
+
+  if (!parentGroup) {
+    logger_->log_error("parseProcessNodeYaml: no parent group exists");
+    return;
+  }
+
+  if (processorsNode) {
+
+    if (processorsNode.IsSequence()) {
+      // Evaluate sequence of processors
+      int numProcessors = processorsNode.size();
+
+      for (YAML::const_iterator iter = processorsNode.begin();
+          iter != processorsNode.end(); ++iter) {
+        core::ProcessorConfig procCfg;
+        YAML::Node procNode = iter->as<YAML::Node>();
+
+        procCfg.name = procNode["name"].as<std::string>();
+        procCfg.id = procNode["id"].as<std::string>();
+        logger_->log_debug("parseProcessorNode: name => [%s] id => [%s]",
+                           procCfg.name.c_str(), procCfg.id.c_str());
+        procCfg.javaClass = procNode["class"].as<std::string>();
+        logger_->log_debug("parseProcessorNode: class => [%s]",
+                           procCfg.javaClass.c_str());
+
+        uuid_parse(procCfg.id.c_str(), uuid);
+
+        // Determine the processor name only from the Java class
+        int lastOfIdx = procCfg.javaClass.find_last_of(".");
+        if (lastOfIdx != std::string::npos) {
+          lastOfIdx++;  // if a value is found, increment to move beyond the .
+          int nameLength = procCfg.javaClass.length() - lastOfIdx;
+          std::string processorName = procCfg.javaClass.substr(lastOfIdx,
+                                                               nameLength);
+          processor = this->createProcessor(processorName, uuid);
+        }
+
+        if (!processor) {
+          logger_->log_error("Could not create a processor %s with name %s",
+                             procCfg.name.c_str(), procCfg.id.c_str());
+          throw std::invalid_argument(
+              "Could not create processor " + procCfg.name);
+        }
+        processor->setName(procCfg.name);
+
+        procCfg.maxConcurrentTasks = procNode["max concurrent tasks"]
+            .as<std::string>();
+        logger_->log_debug("parseProcessorNode: max concurrent tasks => [%s]",
+                           procCfg.maxConcurrentTasks.c_str());
+        procCfg.schedulingStrategy = procNode["scheduling strategy"]
+            .as<std::string>();
+        logger_->log_debug("parseProcessorNode: scheduling strategy => [%s]",
+                           procCfg.schedulingStrategy.c_str());
+        procCfg.schedulingPeriod =
+            procNode["scheduling period"].as<std::string>();
+        logger_->log_debug("parseProcessorNode: scheduling period => [%s]",
+                           procCfg.schedulingPeriod.c_str());
+        procCfg.penalizationPeriod = procNode["penalization period"]
+            .as<std::string>();
+        logger_->log_debug("parseProcessorNode: penalization period => [%s]",
+                           procCfg.penalizationPeriod.c_str());
+        procCfg.yieldPeriod = procNode["yield period"].as<std::string>();
+        logger_->log_debug("parseProcessorNode: yield period => [%s]",
+                           procCfg.yieldPeriod.c_str());
+        procCfg.yieldPeriod = procNode["run duration nanos"].as<std::string>();
+        logger_->log_debug("parseProcessorNode: run duration nanos => [%s]",
+                           procCfg.runDurationNanos.c_str());
+
+        // handle auto-terminated relationships
+        YAML::Node autoTerminatedSequence =
+            procNode["auto-terminated relationships list"];
+        std::vector<std::string> rawAutoTerminatedRelationshipValues;
+        if (autoTerminatedSequence.IsSequence()
+            && !autoTerminatedSequence.IsNull()
+            && autoTerminatedSequence.size() > 0) {
+          for (YAML::const_iterator relIter = autoTerminatedSequence.begin();
+              relIter != autoTerminatedSequence.end(); ++relIter) {
+            std::string autoTerminatedRel = relIter->as<std::string>();
+            rawAutoTerminatedRelationshipValues.push_back(autoTerminatedRel);
+          }
+        }
+        procCfg.autoTerminatedRelationships =
+            rawAutoTerminatedRelationshipValues;
+
+        // handle processor properties
+        YAML::Node propertiesNode = procNode["Properties"];
+        parsePropertiesNodeYaml(&propertiesNode, processor);
+
+        // Take care of scheduling
+        core::TimeUnit unit;
+        if (core::Property::StringToTime(procCfg.schedulingPeriod,
+                                         schedulingPeriod, unit)
+            && core::Property::ConvertTimeUnitToNS(schedulingPeriod, unit,
+                                                   schedulingPeriod)) {
+          logger_->log_debug(
+              "convert: parseProcessorNode: schedulingPeriod => [%d] ns",
+              schedulingPeriod);
+          processor->setSchedulingPeriodNano(schedulingPeriod);
+        }
+
+        if (core::Property::StringToTime(procCfg.penalizationPeriod,
+                                         penalizationPeriod, unit)
+            && core::Property::ConvertTimeUnitToMS(penalizationPeriod, unit,
+                                                   penalizationPeriod)) {
+          logger_->log_debug(
+              "convert: parseProcessorNode: penalizationPeriod => [%d] ms",
+              penalizationPeriod);
+          processor->setPenalizationPeriodMsec(penalizationPeriod);
+        }
+
+        if (core::Property::StringToTime(procCfg.yieldPeriod, yieldPeriod, unit)
+            && core::Property::ConvertTimeUnitToMS(yieldPeriod, unit,
+                                                   yieldPeriod)) {
+          logger_->log_debug(
+              "convert: parseProcessorNode: yieldPeriod => [%d] ms",
+              yieldPeriod);
+          processor->setYieldPeriodMsec(yieldPeriod);
+        }
+
+        // Default to running
+        processor->setScheduledState(core::RUNNING);
+
+        if (procCfg.schedulingStrategy == "TIMER_DRIVEN") {
+          processor->setSchedulingStrategy(core::TIMER_DRIVEN);
+          logger_->log_debug("setting scheduling strategy as %s",
+                             procCfg.schedulingStrategy.c_str());
+        } else if (procCfg.schedulingStrategy == "EVENT_DRIVEN") {
+          processor->setSchedulingStrategy(core::EVENT_DRIVEN);
+          logger_->log_debug("setting scheduling strategy as %s",
+                             procCfg.schedulingStrategy.c_str());
+        } else {
+          processor->setSchedulingStrategy(core::CRON_DRIVEN);
+          logger_->log_debug("setting scheduling strategy as %s",
+                             procCfg.schedulingStrategy.c_str());
+
+        }
+
+        int64_t maxConcurrentTasks;
+        if (core::Property::StringToInt(procCfg.maxConcurrentTasks,
+                                        maxConcurrentTasks)) {
+          logger_->log_debug("parseProcessorNode: maxConcurrentTasks => [%d]",
+                             maxConcurrentTasks);
+          processor->setMaxConcurrentTasks(maxConcurrentTasks);
+        }
+
+        if (core::Property::StringToInt(procCfg.runDurationNanos,
+                                        runDurationNanos)) {
+          logger_->log_debug("parseProcessorNode: runDurationNanos => [%d]",
+                             runDurationNanos);
+          processor->setRunDurationNano(runDurationNanos);
+        }
+
+        std::set<core::Relationship> autoTerminatedRelationships;
+        for (auto &&relString : procCfg.autoTerminatedRelationships) {
+          core::Relationship relationship(relString, "");
+          logger_->log_debug(
+              "parseProcessorNode: autoTerminatedRelationship  => [%s]",
+              relString.c_str());
+          autoTerminatedRelationships.insert(relationship);
+        }
+
+        processor->setAutoTerminatedRelationships(autoTerminatedRelationships);
+
+        parentGroup->addProcessor(processor);
+      }
+    }
+  } else {
+    throw new std::invalid_argument(
+        "Cannot instantiate a MiNiFi instance without a defined Processors configuration node.");
+  }
+}
+
+void YamlConfiguration::parseRemoteProcessGroupYaml(
+    YAML::Node *rpgNode, core::ProcessGroup * parentGroup) {
+  uuid_t uuid;
+
+  if (!parentGroup) {
+    logger_->log_error("parseRemoteProcessGroupYaml: no parent group exists");
+    return;
+  }
+
+  if (rpgNode) {
+    if (rpgNode->IsSequence()) {
+      for (YAML::const_iterator iter = rpgNode->begin(); iter != rpgNode->end();
+          ++iter) {
+        YAML::Node rpgNode = iter->as<YAML::Node>();
+
+        auto name = rpgNode["name"].as<std::string>();
+        auto id = rpgNode["id"].as<std::string>();
+
+        logger_->log_debug(
+            "parseRemoteProcessGroupYaml: name => [%s], id => [%s]",
+            name.c_str(), id.c_str());
+
+        std::string url = rpgNode["url"].as<std::string>();
+        logger_->log_debug("parseRemoteProcessGroupYaml: url => [%s]",
+                           url.c_str());
+
+        std::string timeout = rpgNode["timeout"].as<std::string>();
+        logger_->log_debug("parseRemoteProcessGroupYaml: timeout => [%s]",
+                           timeout.c_str());
+
+        std::string yieldPeriod = rpgNode["yield period"].as<std::string>();
+        logger_->log_debug("parseRemoteProcessGroupYaml: yield period => [%s]",
+                           yieldPeriod.c_str());
+
+        YAML::Node inputPorts = rpgNode["Input Ports"].as<YAML::Node>();
+        YAML::Node outputPorts = rpgNode["Output Ports"].as<YAML::Node>();
+        core::ProcessGroup *group = NULL;
+
+        uuid_parse(id.c_str(), uuid);
+
+        int64_t timeoutValue = -1;
+        int64_t yieldPeriodValue = -1;
+
+        group = this->createRemoteProcessGroup(name.c_str(), uuid).release();
+        group->setParent(parentGroup);
+        parentGroup->addProcessGroup(group);
+
+        core::TimeUnit unit;
+
+        if (core::Property::StringToTime(yieldPeriod, yieldPeriodValue, unit)
+            && core::Property::ConvertTimeUnitToMS(yieldPeriodValue, unit,
+                                                   yieldPeriodValue) && group) {
+          logger_->log_debug(
+              "parseRemoteProcessGroupYaml: yieldPeriod => [%d] ms",
+              yieldPeriodValue);
+          group->setYieldPeriodMsec(yieldPeriodValue);
+        }
+
+        if (core::Property::StringToTime(timeout, timeoutValue, unit)
+            && core::Property::ConvertTimeUnitToMS(timeoutValue, unit,
+                                                   timeoutValue) && group) {
+          logger_->log_debug(
+              "parseRemoteProcessGroupYaml: timeoutValue => [%d] ms",
+              timeoutValue);
+          group->setTimeOut(timeoutValue);
+        }
+
+        group->setTransmitting(true);
+        group->setURL(url);
+
+        if (inputPorts && inputPorts.IsSequence()) {
+          for (YAML::const_iterator portIter = inputPorts.begin();
+              portIter != inputPorts.end(); ++portIter) {
+            logger_->log_debug("Got a current port, iterating...");
+
+            YAML::Node currPort = portIter->as<YAML::Node>();
+
+            this->parsePortYaml(&currPort, group, SEND);
+          }  // for node
+        }
+        if (outputPorts && outputPorts.IsSequence()) {
+          for (YAML::const_iterator portIter = outputPorts.begin();
+              portIter != outputPorts.end(); ++portIter) {
+            logger_->log_debug("Got a current port, iterating...");
+
+            YAML::Node currPort = portIter->as<YAML::Node>();
+
+            this->parsePortYaml(&currPort, group, RECEIVE);
+          }  // for node
+        }
+
+      }
+    }
+  }
+}
+
+void YamlConfiguration::parseConnectionYaml(YAML::Node *connectionsNode,
+                                            core::ProcessGroup *parent) {
+  uuid_t uuid;
+  std::shared_ptr<minifi::Connection> connection = nullptr;
+
+  if (!parent) {
+    logger_->log_error("parseProcessNode: no parent group was provided");
+    return;
+  }
+
+  if (connectionsNode) {
+
+    if (connectionsNode->IsSequence()) {
+      for (YAML::const_iterator iter = connectionsNode->begin();
+          iter != connectionsNode->end(); ++iter) {
+
+        YAML::Node connectionNode = iter->as<YAML::Node>();
+
+        std::string name = connectionNode["name"].as<std::string>();
+        std::string id = connectionNode["id"].as<std::string>();
+        std::string destId = connectionNode["destination id"].as<std::string>();
+
+        uuid_parse(id.c_str(), uuid);
+
+        logger_->log_debug("Created connection with UUID %s and name %s",
+                           id.c_str(), name.c_str());
+        connection = this->createConnection(name, uuid);
+        auto rawRelationship = connectionNode["source relationship name"]
+            .as<std::string>();
+        core::Relationship relationship(rawRelationship, "");
+        logger_->log_debug("parseConnection: relationship => [%s]",
+                           rawRelationship.c_str());
+        if (connection)
+          connection->setRelationship(relationship);
+        std::string connectionSrcProcId = connectionNode["source id"]
+            .as<std::string>();
+        uuid_t srcUUID;
+        uuid_parse(connectionSrcProcId.c_str(), srcUUID);
+
+        auto srcProcessor = parent->findProcessor(srcUUID);
+
+        if (!srcProcessor) {
+          logger_->log_error(
+              "Could not locate a source with id %s to create a connection",
+              connectionSrcProcId.c_str());
+          throw std::invalid_argument(
+              "Could not locate a source with id %s to create a connection "
+                  + connectionSrcProcId);
+        }
+
+        uuid_t destUUID;
+        uuid_parse(destId.c_str(), destUUID);
+        auto destProcessor = parent->findProcessor(destUUID);
+        // If we could not find name, try by UUID
+        if (!destProcessor) {
+          uuid_t destUuid;
+          uuid_parse(destId.c_str(), destUuid);
+          destProcessor = parent->findProcessor(destUuid);
+        }
+        if (destProcessor) {
+          std::string destUuid = destProcessor->getUUIDStr();
+        }
+
+        uuid_t srcUuid;
+        uuid_t destUuid;
+        srcProcessor->getUUID(srcUuid);
+        connection->setSourceUUID(srcUuid);
+        destProcessor->getUUID(destUuid);
+        connection->setDestinationUUID(destUuid);
+
+        if (connection) {
+          parent->addConnection(connection);
+        }
+      }
+    }
+
+    if (connection)
+      parent->addConnection(connection);
+
+    return;
+  }
+}
+
+void YamlConfiguration::parsePortYaml(YAML::Node *portNode,
+                                      core::ProcessGroup *parent,
+                                      TransferDirection direction) {
+  uuid_t uuid;
+  std::shared_ptr<core::Processor> processor = NULL;
+  minifi::RemoteProcessorGroupPort *port = NULL;
+
+  if (!parent) {
+    logger_->log_error("parseProcessNode: no parent group existed");
+    return;
+  }
+
+  YAML::Node inputPortsObj = portNode->as<YAML::Node>();
+
+  // generate the random UIID
+  uuid_generate(uuid);
+
+  auto portId = inputPortsObj["id"].as<std::string>();
+  auto nameStr = inputPortsObj["name"].as<std::string>();
+  uuid_parse(portId.c_str(), uuid);
+
+  port = new minifi::RemoteProcessorGroupPort(nameStr.c_str(), uuid);
+
+  processor = (std::shared_ptr<core::Processor>) port;
+  port->setDirection(direction);
+  port->setTimeOut(parent->getTimeOut());
+  port->setTransmitting(true);
+  processor->setYieldPeriodMsec(parent->getYieldPeriodMsec());
+  processor->initialize();
+
+  // handle port properties
+  YAML::Node nodeVal = portNode->as<YAML::Node>();
+  YAML::Node propertiesNode = nodeVal["Properties"];
+
+  parsePropertiesNodeYaml(&propertiesNode, processor);
+
+  // add processor to parent
+  parent->addProcessor(processor);
+  processor->setScheduledState(core::RUNNING);
+  auto rawMaxConcurrentTasks = inputPortsObj["max concurrent tasks"]
+      .as<std::string>();
+  int64_t maxConcurrentTasks;
+  if (core::Property::StringToInt(rawMaxConcurrentTasks, maxConcurrentTasks)) {
+    processor->setMaxConcurrentTasks(maxConcurrentTasks);
+  }
+  logger_->log_debug("parseProcessorNode: maxConcurrentTasks => [%d]",
+                     maxConcurrentTasks);
+  processor->setMaxConcurrentTasks(maxConcurrentTasks);
+
+}
+
+void YamlConfiguration::parsePropertiesNodeYaml(
+    YAML::Node *propertiesNode, std::shared_ptr<core::Processor> processor) {
+  // Treat generically as a YAML node so we can perform inspection on entries to ensure they are populated
+  for (YAML::const_iterator propsIter = propertiesNode->begin();
+      propsIter != propertiesNode->end(); ++propsIter) {
+    std::string propertyName = propsIter->first.as<std::string>();
+    YAML::Node propertyValueNode = propsIter->second;
+    if (!propertyValueNode.IsNull() && propertyValueNode.IsDefined()) {
+      std::string rawValueString = propertyValueNode.as<std::string>();
+      if (!processor->setProperty(propertyName, rawValueString)) {
+        logger_->log_warn(
+            "Received property %s with value %s but is not one of the properties for %s",
+            propertyName.c_str(), rawValueString.c_str(),
+            processor->getName().c_str());
+      }
+    }
+  }
+}
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/io/BaseStream.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/io/BaseStream.cpp b/libminifi/src/io/BaseStream.cpp
index cf3fe45..1400a1d 100644
--- a/libminifi/src/io/BaseStream.cpp
+++ b/libminifi/src/io/BaseStream.cpp
@@ -18,6 +18,12 @@
 #include "io/BaseStream.h"
 #include "io/Serializable.h"
 
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace io {
 /**
  * write 4 bytes to stream
  * @param base_value non encoded value
@@ -26,7 +32,7 @@
  * @return resulting write size
  **/
 int BaseStream::write(uint32_t base_value, bool is_little_endian) {
-	return ::Serializable::write(base_value, (DataStream*) this, is_little_endian);
+	return Serializable::write(base_value, (DataStream*) this, is_little_endian);
 }
 
 /**
@@ -37,7 +43,7 @@ int BaseStream::write(uint32_t base_value, bool is_little_endian) {
  * @return resulting write size
  **/
 int BaseStream::write(uint16_t base_value, bool is_little_endian) {
-	return ::Serializable::write(base_value, (DataStream*) this, is_little_endian);
+	return Serializable::write(base_value, (DataStream*) this, is_little_endian);
 }
 
 /**
@@ -48,7 +54,7 @@ int BaseStream::write(uint16_t base_value, bool is_little_endian) {
  * @return resulting write size
  **/
 int BaseStream::write(uint8_t *value, int len) {
-	return ::Serializable::write(value, len, (DataStream*) this);
+	return Serializable::write(value, len, (DataStream*) this);
 }
 
 /**
@@ -59,7 +65,7 @@ int BaseStream::write(uint8_t *value, int len) {
  * @return resulting write size
  **/
 int BaseStream::write(uint64_t base_value, bool is_little_endian) {
-	return ::Serializable::write(base_value, (DataStream*) this, is_little_endian);
+	return Serializable::write(base_value, (DataStream*) this, is_little_endian);
 }
 
 /**
@@ -69,7 +75,7 @@ int BaseStream::write(uint64_t base_value, bool is_little_endian) {
  **/
 int BaseStream::write(bool value) {
 	uint8_t v = value;
-	return ::Serializable::write(v);
+	return Serializable::write(v);
 }
 
 /**
@@ -78,7 +84,7 @@ int BaseStream::write(bool value) {
  * @return resulting write size
  **/
 int BaseStream::writeUTF(std::string str, bool widen) {
-	return ::Serializable::writeUTF(str, (DataStream*) this, widen);
+	return Serializable::writeUTF(str, (DataStream*) this, widen);
 }
 
 /**
@@ -88,7 +94,7 @@ int BaseStream::writeUTF(std::string str, bool widen) {
  * @return resulting read size
  **/
 int BaseStream::read(uint8_t &value) {
-	return ::Serializable::read(value, (DataStream*) this);
+	return Serializable::read(value, (DataStream*) this);
 }
 
 /**
@@ -98,7 +104,7 @@ int BaseStream::read(uint8_t &value) {
  * @return resulting read size
  **/
 int BaseStream::read(uint16_t &base_value, bool is_little_endian) {
-	return ::Serializable::read(base_value, (DataStream*) this);
+	return Serializable::read(base_value, (DataStream*) this);
 }
 
 /**
@@ -108,7 +114,7 @@ int BaseStream::read(uint16_t &base_value, bool is_little_endian) {
  * @return resulting read size
  **/
 int BaseStream::read(char &value) {
-	return ::Serializable::read(value, (DataStream*) this);
+	return Serializable::read(value, (DataStream*) this);
 }
 
 /**
@@ -119,7 +125,7 @@ int BaseStream::read(char &value) {
  * @return resulting read size
  **/
 int BaseStream::read(uint8_t *value, int len) {
-	return ::Serializable::read(value, len, (DataStream*) this);
+	return Serializable::read(value, len, (DataStream*) this);
 }
 
 /**
@@ -129,7 +135,7 @@ int BaseStream::read(uint8_t *value, int len) {
  * @return resulting read size
  **/
 int BaseStream::read(uint32_t &value, bool is_little_endian) {
-	return ::Serializable::read(value, (DataStream*) this, is_little_endian);
+	return Serializable::read(value, (DataStream*) this, is_little_endian);
 }
 
 /**
@@ -139,7 +145,7 @@ int BaseStream::read(uint32_t &value, bool is_little_endian) {
  * @return resulting read size
  **/
 int BaseStream::read(uint64_t &value, bool is_little_endian) {
-	return ::Serializable::read(value, (DataStream*) this, is_little_endian);
+	return Serializable::read(value, (DataStream*) this, is_little_endian);
 }
 
 /**
@@ -149,5 +155,12 @@ int BaseStream::read(uint64_t &value, bool is_little_endian) {
  * @return resulting read size
  **/
 int BaseStream::readUTF(std::string &str, bool widen) {
-	return ::Serializable::readUTF(str, (DataStream*) this, widen);
+	return Serializable::readUTF(str, (DataStream*) this, widen);
 }
+
+
+} /* namespace io */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/io/ClientSocket.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/io/ClientSocket.cpp b/libminifi/src/io/ClientSocket.cpp
index 39b71b4..ad6b04d 100644
--- a/libminifi/src/io/ClientSocket.cpp
+++ b/libminifi/src/io/ClientSocket.cpp
@@ -30,425 +30,424 @@
 #include "io/validation.h"
 #include "io/ClientSocket.h"
 
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace io {
 
 std::string Socket::HOSTNAME = Socket::getMyHostName(0);
 
-
 Socket::Socket(const std::string &hostname, const uint16_t port,
-		const uint16_t listeners = -1) :
-		requested_hostname_(hostname), port_(port), addr_info_(0), socket_file_descriptor_(
-				-1), socket_max_(0), listeners_(listeners), canonical_hostname_("") {
-	logger_ = Logger::getLogger();
-	FD_ZERO(&total_list_);
-	FD_ZERO(&read_fds_);
+               const uint16_t listeners = -1)
+    : requested_hostname_(hostname),
+      port_(port),
+      addr_info_(0),
+      socket_file_descriptor_(-1),
+      socket_max_(0),
+      listeners_(listeners),
+      canonical_hostname_("") {
+  logger_ = logging::Logger::getLogger();
+  FD_ZERO(&total_list_);
+  FD_ZERO(&read_fds_);
 
 }
 
-Socket::Socket(const std::string &hostname, const uint16_t port) :
-		::Socket(hostname, port, 0) {
+Socket::Socket(const std::string &hostname, const uint16_t port)
+    : Socket(hostname, port, 0) {
 
 }
 
-Socket::Socket(const Socket &&other) :
-		requested_hostname_(std::move(other.requested_hostname_)), port_(
-				std::move(other.port_)), addr_info_(std::move(other.addr_info_)), socket_file_descriptor_(
-				other.socket_file_descriptor_), socket_max_(
-				other.socket_max_.load()), listeners_(other.listeners_), total_list_(
-				other.total_list_), read_fds_(other.read_fds_), canonical_hostname_(
-				std::move(other.canonical_hostname_)) {
-	logger_ = Logger::getLogger();
+Socket::Socket(const Socket &&other)
+    : requested_hostname_(std::move(other.requested_hostname_)),
+      port_(std::move(other.port_)),
+      addr_info_(std::move(other.addr_info_)),
+      socket_file_descriptor_(other.socket_file_descriptor_),
+      socket_max_(other.socket_max_.load()),
+      listeners_(other.listeners_),
+      total_list_(other.total_list_),
+      read_fds_(other.read_fds_),
+      canonical_hostname_(std::move(other.canonical_hostname_)) {
+  logger_ = logging::Logger::getLogger();
 
 }
 
 Socket::~Socket() {
-	closeStream();
+  closeStream();
 }
 
-void Socket::closeStream()
-{
-    if (0 != addr_info_) {
-	    freeaddrinfo(addr_info_);
-	    addr_info_=0;
-    }
+void Socket::closeStream() {
+  if (0 != addr_info_) {
+    freeaddrinfo(addr_info_);
+    addr_info_ = 0;
+  }
 
-    if (socket_file_descriptor_ >= 0)
-    {
-	    close(socket_file_descriptor_);
-	    socket_file_descriptor_ = -1;
-    }
+  if (socket_file_descriptor_ >= 0) {
+    close(socket_file_descriptor_);
+    socket_file_descriptor_ = -1;
+  }
 }
 
-int8_t Socket::createConnection(const addrinfo *p,in_addr_t &addr) {
-	if ((socket_file_descriptor_ = socket(p->ai_family, p->ai_socktype,
-			p->ai_protocol)) == -1) {
-		logger_->log_error("error while connecting to server socket");
-		return -1;
-	}
-
-	setSocketOptions(socket_file_descriptor_);
-
-	if (listeners_ > 0) {
-
-		struct sockaddr_in *sa_loc = (struct sockaddr_in*) p->ai_addr;
-		sa_loc->sin_family = AF_INET;
-		sa_loc->sin_port = htons(port_);
-		sa_loc->sin_addr.s_addr = htonl(INADDR_ANY);
-
-		if (bind(socket_file_descriptor_, p->ai_addr, p->ai_addrlen) == -1) {
-			logger_->log_error("Could not bind to socket", strerror(errno));
-			return -1;
-		}
-	}
-	{
-	  if (listeners_ <=0 )
-	  {
-		struct sockaddr_in *sa_loc = (struct sockaddr_in*) p->ai_addr;
-		sa_loc->sin_family = AF_INET;
-		//sa_loc->sin_port = htons(port);
-		sa_loc->sin_port = htons(port_);
-		// use any address if you are connecting to the local machine for testing
-		// otherwise we must use the requested hostname
-		if (IsNullOrEmpty(requested_hostname_) || requested_hostname_=="localhost")
-		{
-		  sa_loc->sin_addr.s_addr = htonl(INADDR_ANY);
-		}
-		else
-		{
-		  sa_loc->sin_addr.s_addr = addr;
-		}
-		if (connect(socket_file_descriptor_, p->ai_addr, p->ai_addrlen) == -1) {
-			  close(socket_file_descriptor_);
-				socket_file_descriptor_ = -1;
-				logger_->log_warn("Could not connect to socket, error:%s", strerror(errno));
-				return -1;
-
-		}
-	  }
-	}
-
-	// listen
-	if (listeners_ > 0) {
-		if (listen(socket_file_descriptor_, listeners_) == -1) {
-			logger_->log_warn("attempted connection, saw %s", strerror(errno));
-			return -1;
-		}
-
-	}
-	// add the listener to the total set
-	FD_SET(socket_file_descriptor_, &total_list_);
-	socket_max_ = socket_file_descriptor_;
-	return 0;
-}
+int8_t Socket::createConnection(const addrinfo *p, in_addr_t &addr) {
+  if ((socket_file_descriptor_ = socket(p->ai_family, p->ai_socktype,
+                                        p->ai_protocol)) == -1) {
+    logger_->log_error("error while connecting to server socket");
+    return -1;
+  }
 
-short Socket::initialize() {
+  setSocketOptions(socket_file_descriptor_);
 
-	struct sockaddr_in servAddr;
+  if (listeners_ > 0) {
 
-	addrinfo hints = { sizeof(addrinfo) };
-	memset(&hints, 0, sizeof hints); // make sure the struct is empty
-	hints.ai_family = AF_UNSPEC;
-	hints.ai_socktype = SOCK_STREAM;
-	hints.ai_flags = AI_CANONNAME;
-	if (listeners_ > 0)
-	    hints.ai_flags |= AI_PASSIVE;
+    struct sockaddr_in *sa_loc = (struct sockaddr_in*) p->ai_addr;
+    sa_loc->sin_family = AF_INET;
+    sa_loc->sin_port = htons(port_);
+    sa_loc->sin_addr.s_addr = htonl(INADDR_ANY);
 
-	hints.ai_protocol = 0; /* any protocol */
+    if (bind(socket_file_descriptor_, p->ai_addr, p->ai_addrlen) == -1) {
+      logger_->log_error("Could not bind to socket", strerror(errno));
+      return -1;
+    }
+  }
+  {
+    if (listeners_ <= 0) {
+      struct sockaddr_in *sa_loc = (struct sockaddr_in*) p->ai_addr;
+      sa_loc->sin_family = AF_INET;
+      //sa_loc->sin_port = htons(port);
+      sa_loc->sin_port = htons(port_);
+      // use any address if you are connecting to the local machine for testing
+      // otherwise we must use the requested hostname
+      if (IsNullOrEmpty(requested_hostname_)
+          || requested_hostname_ == "localhost") {
+        sa_loc->sin_addr.s_addr = htonl(INADDR_ANY);
+      } else {
+        sa_loc->sin_addr.s_addr = addr;
+      }
+      if (connect(socket_file_descriptor_, p->ai_addr, p->ai_addrlen) == -1) {
+        close(socket_file_descriptor_);
+        socket_file_descriptor_ = -1;
+        logger_->log_warn("Could not connect to socket, error:%s",
+                          strerror(errno));
+        return -1;
+
+      }
+    }
+  }
 
-	int errcode = getaddrinfo(requested_hostname_.c_str(), 0, &hints, &addr_info_);
+  // listen
+  if (listeners_ > 0) {
+    if (listen(socket_file_descriptor_, listeners_) == -1) {
+      logger_->log_warn("attempted connection, saw %s", strerror(errno));
+      return -1;
+    }
 
-	if (errcode != 0) {
-		logger_->log_error("Saw error during getaddrinfo, error: %s",strerror(errno));
-		return -1;
-	}
+  }
+  // add the listener to the total set
+  FD_SET(socket_file_descriptor_, &total_list_);
+  socket_max_ = socket_file_descriptor_;
+  return 0;
+}
 
-	socket_file_descriptor_ = -1;
+short Socket::initialize() {
 
-	in_addr_t addr;
-	  struct hostent *h;
-	#ifdef __MACH__
-	  h = gethostbyname(requested_hostname_.c_str());
-	#else
-	  const char *host;
-    uint16_t port;
+  struct sockaddr_in servAddr;
 
-    host = requested_hostname_.c_str();
-    port = port_;
-	  char buf[1024];
-	  struct hostent he;
-	  int hh_errno;
-	  gethostbyname_r(host, &he, buf, sizeof(buf), &h, &hh_errno);
-	#endif
+  addrinfo hints = { sizeof(addrinfo) };
+  memset(&hints, 0, sizeof hints);  // make sure the struct is empty
+  hints.ai_family = AF_UNSPEC;
+  hints.ai_socktype = SOCK_STREAM;
+  hints.ai_flags = AI_CANONNAME;
+  if (listeners_ > 0)
+    hints.ai_flags |= AI_PASSIVE;
 
-	  memcpy((char *) &addr, h->h_addr_list[0], h->h_length);
+  hints.ai_protocol = 0; /* any protocol */
 
+  int errcode = getaddrinfo(requested_hostname_.c_str(), 0, &hints,
+                            &addr_info_);
 
-	auto p = addr_info_;
-	for (; p != NULL; p = p->ai_next) {
-		if (IsNullOrEmpty(canonical_hostname_)) {
-			if (!IsNullOrEmpty(p) && !IsNullOrEmpty(p->ai_canonname))
-				canonical_hostname_ = p->ai_canonname;
-		}
+  if (errcode != 0) {
+    logger_->log_error("Saw error during getaddrinfo, error: %s",
+                       strerror(errno));
+    return -1;
+  }
 
+  socket_file_descriptor_ = -1;
 
-		//we've successfully connected
-		if (port_ > 0 && createConnection(p,addr) >= 0)
-		{
-			return 0;
-			break;
-		}
-	}
+  in_addr_t addr;
+  struct hostent *h;
+#ifdef __MACH__
+  h = gethostbyname(requested_hostname_.c_str());
+#else
+  const char *host;
+  uint16_t port;
+
+  host = requested_hostname_.c_str();
+  port = port_;
+  char buf[1024];
+  struct hostent he;
+  int hh_errno;
+  gethostbyname_r(host, &he, buf, sizeof(buf), &h, &hh_errno);
+#endif
 
-	return -1;
+  memcpy((char *) &addr, h->h_addr_list[0], h->h_length);
+
+  auto p = addr_info_;
+  for (; p != NULL; p = p->ai_next) {
+    if (IsNullOrEmpty(canonical_hostname_)) {
+      if (!IsNullOrEmpty(p) && !IsNullOrEmpty(p->ai_canonname))
+        canonical_hostname_ = p->ai_canonname;
+    }
+
+    //we've successfully connected
+    if (port_ > 0 && createConnection(p, addr) >= 0) {
+      return 0;
+      break;
+    }
+  }
+
+  return -1;
 
 }
 
 short Socket::select_descriptor(const uint16_t msec) {
-	struct timeval tv;
-	int retval;
-
-	read_fds_ = total_list_;
-
-	tv.tv_sec = msec / 1000;
-	tv.tv_usec = (msec % 1000) * 1000;
-
-	std::lock_guard<std::recursive_mutex> guard(selection_mutex_);
-
-	if (msec > 0)
-		retval = select(socket_max_ + 1, &read_fds_, NULL, NULL, &tv);
-	else
-		retval = select(socket_max_ + 1, &read_fds_, NULL, NULL, NULL);
-
-	if (retval < 0) {
-		logger_->log_error("Saw error during selection, error:%i %s",retval,strerror(errno));
-		return retval;
-	}
-
-	for (int i = 0; i <= socket_max_; i++) {
-		if (FD_ISSET(i, &read_fds_)) {
-
-			if (i == socket_file_descriptor_) {
-			  if (listeners_ > 0)
-			  {
-				struct sockaddr_storage remoteaddr; // client address
-				socklen_t addrlen = sizeof remoteaddr;
-				int newfd = accept(socket_file_descriptor_,
-						(struct sockaddr *) &remoteaddr, &addrlen);
-				FD_SET(newfd, &total_list_); // add to master set
-				if (newfd > socket_max_) {    // keep track of the max
-					socket_max_ = newfd;
-				}
-				return newfd;
-
-
-
-			  }
-			  else{
-			    return socket_file_descriptor_;
-			  }
-				// we have a new connection
-			} else {
-				// data to be received on i
-				return i;
-			}
-		}
-
-	}
-
-	return -1;
+  struct timeval tv;
+  int retval;
+
+  read_fds_ = total_list_;
+
+  tv.tv_sec = msec / 1000;
+  tv.tv_usec = (msec % 1000) * 1000;
+
+  std::lock_guard<std::recursive_mutex> guard(selection_mutex_);
+
+  if (msec > 0)
+    retval = select(socket_max_ + 1, &read_fds_, NULL, NULL, &tv);
+  else
+    retval = select(socket_max_ + 1, &read_fds_, NULL, NULL, NULL);
+
+  if (retval < 0) {
+    logger_->log_error("Saw error during selection, error:%i %s", retval,
+                       strerror(errno));
+    return retval;
+  }
+
+  for (int i = 0; i <= socket_max_; i++) {
+    if (FD_ISSET(i, &read_fds_)) {
+
+      if (i == socket_file_descriptor_) {
+        if (listeners_ > 0) {
+          struct sockaddr_storage remoteaddr;  // client address
+          socklen_t addrlen = sizeof remoteaddr;
+          int newfd = accept(socket_file_descriptor_,
+                             (struct sockaddr *) &remoteaddr, &addrlen);
+          FD_SET(newfd, &total_list_);  // add to master set
+          if (newfd > socket_max_) {    // keep track of the max
+            socket_max_ = newfd;
+          }
+          return newfd;
+
+        } else {
+          return socket_file_descriptor_;
+        }
+        // we have a new connection
+      } else {
+        // data to be received on i
+        return i;
+      }
+    }
+
+  }
+
+  return -1;
 }
 
 short Socket::setSocketOptions(const int sock) {
-	int opt = 1;
-	bool nagle_off = true;
+  int opt = 1;
+  bool nagle_off = true;
 #ifndef __MACH__
-	if (nagle_off) {
-		if (setsockopt(sock, SOL_TCP, TCP_NODELAY, (void *) &opt, sizeof(opt))
-				< 0) {
-			logger_->log_error("setsockopt() TCP_NODELAY failed");
-			close(sock);
-			return -1;
-		}
-		if (setsockopt(sock, SOL_SOCKET, SO_REUSEADDR, (char *) &opt,
-						sizeof(opt)) < 0) {
-			logger_->log_error("setsockopt() SO_REUSEADDR failed");
-			close(sock);
-			return -1;
-		}
-	}
-
-	int sndsize = 256 * 1024;
-	if (setsockopt(sock, SOL_SOCKET, SO_SNDBUF, (char *) &sndsize,
-					(int) sizeof(sndsize)) < 0) {
-		logger_->log_error("setsockopt() SO_SNDBUF failed");
-		close(sock);
-		return -1;
-	}
+  if (nagle_off) {
+    if (setsockopt(sock, SOL_TCP, TCP_NODELAY, (void *) &opt, sizeof(opt))
+        < 0) {
+      logger_->log_error("setsockopt() TCP_NODELAY failed");
+      close(sock);
+      return -1;
+    }
+    if (setsockopt(sock, SOL_SOCKET, SO_REUSEADDR, (char *) &opt,
+            sizeof(opt)) < 0) {
+      logger_->log_error("setsockopt() SO_REUSEADDR failed");
+      close(sock);
+      return -1;
+    }
+  }
+
+  int sndsize = 256 * 1024;
+  if (setsockopt(sock, SOL_SOCKET, SO_SNDBUF, (char *) &sndsize,
+          (int) sizeof(sndsize)) < 0) {
+    logger_->log_error("setsockopt() SO_SNDBUF failed");
+    close(sock);
+    return -1;
+  }
 
 #else
-	if (listeners_ > 0) {
-		// lose the pesky "address already in use" error message
-		if (setsockopt(sock, SOL_SOCKET, SO_REUSEADDR, (char *) &opt,
-				sizeof(opt)) < 0) {
-			logger_->log_error("setsockopt() SO_REUSEADDR failed");
-			close(sock);
-			return -1;
-		}
-	}
+  if (listeners_ > 0) {
+    // lose the pesky "address already in use" error message
+    if (setsockopt(sock, SOL_SOCKET, SO_REUSEADDR, (char *) &opt, sizeof(opt))
+        < 0) {
+      logger_->log_error("setsockopt() SO_REUSEADDR failed");
+      close(sock);
+      return -1;
+    }
+  }
 #endif
-	return 0;
+  return 0;
 }
 
 std::string Socket::getHostname() const {
-	return canonical_hostname_;
+  return canonical_hostname_;
 }
 
 int Socket::writeData(std::vector<uint8_t> &buf, int buflen) {
 
-	if (buf.capacity() < buflen)
-		return -1;
-	return writeData((uint8_t*) &buf[0], buflen);
+  if (buf.capacity() < buflen)
+    return -1;
+  return writeData((uint8_t*) &buf[0], buflen);
 }
 
 // data stream overrides
 
 int Socket::writeData(uint8_t *value, int size) {
 
-	int ret = 0, bytes = 0;
+  int ret = 0, bytes = 0;
 
-	while (bytes < size) {
+  while (bytes < size) {
 
+    ret = send(socket_file_descriptor_, value + bytes, size - bytes, 0);
+    //check for errors
+    if (ret <= 0) {
+      close(socket_file_descriptor_);
+      logger_->log_error("Could not send to %d, error: %s",
+                         socket_file_descriptor_, strerror(errno));
+      return ret;
+    }
+    bytes += ret;
 
+  }
 
-		ret = send(socket_file_descriptor_, value + bytes, size - bytes, 0);
-		//check for errors
-		if (ret <= 0) {
-			close(socket_file_descriptor_);
-			logger_->log_error("Could not send to %d, error: %s",socket_file_descriptor_, strerror(errno));
-			return ret;
-		}
-		bytes += ret;
-
-	}
-
-	if (ret)
-		logger_->log_trace("Send data size %d over socket %d", size,
-				socket_file_descriptor_);
+  if (ret)
+    logger_->log_trace("Send data size %d over socket %d", size,
+                       socket_file_descriptor_);
 
-	return bytes;
+  return bytes;
 
 }
 
 template<typename T>
 inline std::vector<uint8_t> Socket::readBuffer(const T& t) {
-	std::vector<uint8_t> buf;
-	buf.resize(sizeof t);
-	readData((uint8_t*) &buf[0], sizeof(t));
-	return buf;
+  std::vector<uint8_t> buf;
+  buf.resize(sizeof t);
+  readData((uint8_t*) &buf[0], sizeof(t));
+  return buf;
 }
 
+int Socket::write(uint64_t base_value, bool is_little_endian) {
 
-int Socket::write(uint64_t base_value, bool is_little_endian){
-
-  return Serializable::write(base_value,this,is_little_endian);
+  return Serializable::write(base_value, this, is_little_endian);
 }
 
-
-int Socket::write(uint32_t base_value, bool is_little_endian){
-  return Serializable::write(base_value,this,is_little_endian);
+int Socket::write(uint32_t base_value, bool is_little_endian) {
+  return Serializable::write(base_value, this, is_little_endian);
 }
 
-int Socket::write(uint16_t base_value, bool is_little_endian){
-  return Serializable::write(base_value,this,is_little_endian);
+int Socket::write(uint16_t base_value, bool is_little_endian) {
+  return Serializable::write(base_value, this, is_little_endian);
 }
 
-
 int Socket::read(uint64_t &value, bool is_little_endian) {
 
-	auto buf = readBuffer(value);
-
-	if (is_little_endian) {
-		value = ((uint64_t) buf[0] << 56) | ((uint64_t) (buf[1] & 255) << 48)
-				| ((uint64_t) (buf[2] & 255) << 40)
-				| ((uint64_t) (buf[3] & 255) << 32)
-				| ((uint64_t) (buf[4] & 255) << 24)
-				| ((uint64_t) (buf[5] & 255) << 16)
-				| ((uint64_t) (buf[6] & 255) << 8)
-				| ((uint64_t) (buf[7] & 255) << 0);
-	} else {
-		value = ((uint64_t) buf[0] << 0) | ((uint64_t) (buf[1] & 255) << 8)
-				| ((uint64_t) (buf[2] & 255) << 16)
-				| ((uint64_t) (buf[3] & 255) << 24)
-				| ((uint64_t) (buf[4] & 255) << 32)
-				| ((uint64_t) (buf[5] & 255) << 40)
-				| ((uint64_t) (buf[6] & 255) << 48)
-				| ((uint64_t) (buf[7] & 255) << 56);
-	}
-	return sizeof(value);
+  auto buf = readBuffer(value);
+
+  if (is_little_endian) {
+    value = ((uint64_t) buf[0] << 56) | ((uint64_t) (buf[1] & 255) << 48)
+        | ((uint64_t) (buf[2] & 255) << 40) | ((uint64_t) (buf[3] & 255) << 32)
+        | ((uint64_t) (buf[4] & 255) << 24) | ((uint64_t) (buf[5] & 255) << 16)
+        | ((uint64_t) (buf[6] & 255) << 8) | ((uint64_t) (buf[7] & 255) << 0);
+  } else {
+    value = ((uint64_t) buf[0] << 0) | ((uint64_t) (buf[1] & 255) << 8)
+        | ((uint64_t) (buf[2] & 255) << 16) | ((uint64_t) (buf[3] & 255) << 24)
+        | ((uint64_t) (buf[4] & 255) << 32) | ((uint64_t) (buf[5] & 255) << 40)
+        | ((uint64_t) (buf[6] & 255) << 48) | ((uint64_t) (buf[7] & 255) << 56);
+  }
+  return sizeof(value);
 }
 
 int Socket::read(uint32_t &value, bool is_little_endian) {
 
-	auto buf = readBuffer(value);
+  auto buf = readBuffer(value);
 
-	if (is_little_endian) {
-		value = (buf[0] << 24) | (buf[1] << 16) | (buf[2] << 8) | buf[3];
-	} else {
-		value = buf[0] | buf[1] << 8 | buf[2] << 16 | buf[3] << 24;
+  if (is_little_endian) {
+    value = (buf[0] << 24) | (buf[1] << 16) | (buf[2] << 8) | buf[3];
+  } else {
+    value = buf[0] | buf[1] << 8 | buf[2] << 16 | buf[3] << 24;
 
-	}
+  }
 
-	return sizeof(value);
+  return sizeof(value);
 }
 
 int Socket::read(uint16_t &value, bool is_little_endian) {
 
-	auto buf = readBuffer(value);
+  auto buf = readBuffer(value);
 
-	if (is_little_endian) {
-		value = (buf[0] << 8) | buf[1];
-	} else {
-		value = buf[0] | buf[1] << 8;
+  if (is_little_endian) {
+    value = (buf[0] << 8) | buf[1];
+  } else {
+    value = buf[0] | buf[1] << 8;
 
-	}
-	return sizeof(value);
+  }
+  return sizeof(value);
 }
 
 int Socket::readData(std::vector<uint8_t> &buf, int buflen) {
 
-	if (buf.capacity() < buflen)
-	{
-	      buf.resize(buflen);
-	}
-	return readData((uint8_t*) &buf[0], buflen);
+  if (buf.capacity() < buflen) {
+    buf.resize(buflen);
+  }
+  return readData((uint8_t*) &buf[0], buflen);
 }
 
 int Socket::readData(uint8_t *buf, int buflen) {
 
-	int total_read = 0;
-	while (buflen) {
-		short fd = select_descriptor(1000);
-		if (fd < 0) {
-
-			logger_->log_info("fd close %i",buflen);
-			close(socket_file_descriptor_);
-			return -1;
-		}
-
-		int bytes_read = recv(fd, buf, buflen, 0);
-		if (bytes_read <= 0) {
-			if (bytes_read == 0)
-				logger_->log_info("Other side hung up on %d", fd);
-			else {
-				logger_->log_error("Could not recv on %d, error: %s",fd, strerror(errno));
-			}
-			return -1;
-		}
-
-
-		buflen -= bytes_read;
-		buf += bytes_read;
-		total_read += bytes_read;
-	}
-
-	return total_read;
+  int total_read = 0;
+  while (buflen) {
+    short fd = select_descriptor(1000);
+    if (fd < 0) {
+
+      logger_->log_info("fd close %i", buflen);
+      close(socket_file_descriptor_);
+      return -1;
+    }
+
+    int bytes_read = recv(fd, buf, buflen, 0);
+    if (bytes_read <= 0) {
+      if (bytes_read == 0)
+        logger_->log_info("Other side hung up on %d", fd);
+      else {
+        logger_->log_error("Could not recv on %d, error: %s", fd,
+                           strerror(errno));
+      }
+      return -1;
+    }
+
+    buflen -= bytes_read;
+    buf += bytes_read;
+    total_read += bytes_read;
+  }
+
+  return total_read;
 }
+
+} /* namespace io */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/io/DataStream.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/io/DataStream.cpp b/libminifi/src/io/DataStream.cpp
index 3ff0a57..7a10bd9 100644
--- a/libminifi/src/io/DataStream.cpp
+++ b/libminifi/src/io/DataStream.cpp
@@ -27,6 +27,11 @@
 #include <algorithm>
 #include "io/DataStream.h"
 
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace io {
 
 
 int DataStream::writeData(uint8_t *value, int size) {
@@ -126,3 +131,10 @@ int DataStream::readData(uint8_t *buf,int buflen) {
     readBuffer += buflen;
     return buflen;
 }
+
+
+} /* namespace io */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/io/EndianCheck.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/io/EndianCheck.cpp b/libminifi/src/io/EndianCheck.cpp
index f1bc98b..1b5020d 100644
--- a/libminifi/src/io/EndianCheck.cpp
+++ b/libminifi/src/io/EndianCheck.cpp
@@ -18,5 +18,16 @@
 
 #include "io/EndianCheck.h"
 
-
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace io {
 bool EndiannessCheck::IS_LITTLE = EndiannessCheck::is_little_endian();
+
+
+} /* namespace io */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/io/Serializable.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/io/Serializable.cpp b/libminifi/src/io/Serializable.cpp
index 78dc63c..f8c623a 100644
--- a/libminifi/src/io/Serializable.cpp
+++ b/libminifi/src/io/Serializable.cpp
@@ -22,7 +22,11 @@
 #include <arpa/inet.h>
 #include "io/DataStream.h"
 #include "io/Serializable.h"
-
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace io {
 
 #define htonll_r(x) ((((uint64_t)htonl(x)) << 32) + htonl((x) >> 32))
 
@@ -216,3 +220,10 @@ int Serializable::writeUTF(std::string str,DataStream *stream, bool widen) {
     }
     return ret;
 }
+
+
+} /* namespace io */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/io/SocketFactory.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/io/SocketFactory.cpp b/libminifi/src/io/SocketFactory.cpp
deleted file mode 100644
index cbfdf96..0000000
--- a/libminifi/src/io/SocketFactory.cpp
+++ /dev/null
@@ -1,24 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#include "io/SocketFactory.h"
-
-#include <atomic>
-#include <mutex> 
-  
-std::atomic<SocketFactory*> SocketFactory::context_instance_;
-std::mutex SocketFactory::context_mutex_;

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/io/StreamFactory.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/io/StreamFactory.cpp b/libminifi/src/io/StreamFactory.cpp
new file mode 100644
index 0000000..e3aa290
--- /dev/null
+++ b/libminifi/src/io/StreamFactory.cpp
@@ -0,0 +1,37 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include "../../include/io/StreamFactory.h"
+
+#include <atomic>
+#include <mutex> 
+  
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace io {
+
+std::atomic<StreamFactory*> StreamFactory::context_instance_;
+std::mutex StreamFactory::context_mutex_;
+
+
+} /* namespace io */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/io/TLSSocket.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/io/TLSSocket.cpp b/libminifi/src/io/TLSSocket.cpp
deleted file mode 100644
index 1c81f6c..0000000
--- a/libminifi/src/io/TLSSocket.cpp
+++ /dev/null
@@ -1,237 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#ifdef OPENSSL_SUPPORT
-#include "Property.h"
-#include "Configure.h"
-#include "io/TLSSocket.h"
-#include "utils/StringUtils.h"
-#include <openssl/ssl.h>
-#include <openssl/err.h>
-
-
-std::atomic<TLSContext*> TLSContext::context_instance;
-std::mutex TLSContext::context_mutex;
-
-TLSContext::TLSContext() :
-		error_value(0), ctx(0), logger_(Logger::getLogger()), configuration(
-				Configure::getConfigure()) {
-
-}
-
-/**
- * The memory barrier is defined by the singleton
- */
-short TLSContext::initialize() {
-	if (ctx != 0) {
-		return error_value;
-	}
-	std::string clientAuthStr;
-	bool needClientCert = true;
-	if (!(configuration->get(Configure::nifi_security_need_ClientAuth,
-			clientAuthStr)
-			&& StringUtils::StringToBool(clientAuthStr, needClientCert))) {
-		needClientCert = true;
-	}
-
-	SSL_library_init();
-	const SSL_METHOD *method;
-
-	OpenSSL_add_all_algorithms();
-	SSL_load_error_strings();
-	method = TLSv1_2_client_method();
-	ctx = SSL_CTX_new(method);
-	if (ctx == NULL) {
-		logger_->log_error("Could not create SSL context, error: %s.",
-				std::strerror(errno));
-		error_value = TLS_ERROR_CONTEXT;
-		return error_value;
-	}
-	if (needClientCert) {
-		std::string certificate;
-		std::string privatekey;
-		std::string passphrase;
-		std::string caCertificate;
-
-		if (!(configuration->get(Configure::nifi_security_client_certificate,
-				certificate)
-				&& configuration->get(
-						Configure::nifi_security_client_private_key, privatekey))) {
-			logger_->log_error(
-					"Certificate and Private Key PEM file not configured, error: %s.",
-					std::strerror(errno));
-			error_value = TLS_ERROR_PEM_MISSING;
-			return error_value;
-		}
-		// load certificates and private key in PEM format
-		if (SSL_CTX_use_certificate_file(ctx, certificate.c_str(),
-				SSL_FILETYPE_PEM) <= 0) {
-			logger_->log_error("Could not create load certificate, error : %s",
-					std::strerror(errno));
-			error_value = TLS_ERROR_CERT_MISSING;
-			return error_value;
-
-		}
-		if (configuration->get(Configure::nifi_security_client_pass_phrase,
-				passphrase)) {
-			// if the private key has passphase
-			SSL_CTX_set_default_passwd_cb(ctx, pemPassWordCb);
-		}
-		
-
-		int retp = SSL_CTX_use_PrivateKey_file(ctx, privatekey.c_str(),
-				SSL_FILETYPE_PEM);
-		if (retp != 1) {
-			logger_->log_error("Could not create load private key,%i on %s error : %s",
-					retp,privatekey.c_str(),std::strerror(errno));
-			error_value = TLS_ERROR_KEY_ERROR;
-			return error_value;
-		}
-		// verify private key
-		if (!SSL_CTX_check_private_key(ctx)) {
-			logger_->log_error(
-					"Private key does not match the public certificate, error : %s",
-					std::strerror(errno));
-			error_value = TLS_ERROR_KEY_ERROR;
-			return error_value;
-		}
-		// load CA certificates
-		if (configuration->get(Configure::nifi_security_client_ca_certificate,
-				caCertificate)) {
-			retp = SSL_CTX_load_verify_locations(ctx, caCertificate.c_str(), 0);
-			if (retp==0) {
-				logger_->log_error(
-						"Can not load CA certificate, Exiting, error : %s",
-						std::strerror(errno));
-				error_value = TLS_ERROR_CERT_ERROR;
-				return error_value;
-			}
-		}
-
-		logger_->log_info("Load/Verify Client Certificate OK.");
-	}
-	return 0;
-}
-
-TLSSocket::~TLSSocket()
-{
-	if (ssl != 0)
-		SSL_free(ssl);
-}
-/**
- * Constructor that accepts host name, port and listeners. With this
- * contructor we will be creating a server socket
- * @param hostname our host name
- * @param port connecting port
- * @param listeners number of listeners in the queue
- */
-TLSSocket::TLSSocket(const std::string &hostname, const uint16_t port,
-		const uint16_t listeners) :
-		::Socket(hostname, port, listeners), ssl(0) {
-}
-
-TLSSocket::TLSSocket(const std::string &hostname, const uint16_t port) :
-		::Socket(hostname, port, 0), ssl(0) {
-}
-
-TLSSocket::TLSSocket(const TLSSocket &&d) :
-		::Socket(std::move(d)), ssl(0) {
-}
-
-short TLSSocket::initialize() {
-	TLSContext *context = TLSContext::getInstance();
-	short ret = context->initialize();
-	Socket::initialize();
-	if (!ret) {
-		// we have s2s secure config
-		ssl = SSL_new(context->getContext());
-		SSL_set_fd(ssl, socket_file_descriptor_);
-		if (SSL_connect(ssl) == -1) {
-			logger_->log_error("SSL socket connect failed to %s %d",
-					requested_hostname_.c_str(), port_);
-			SSL_free(ssl);
-			ssl = NULL;
-			close(socket_file_descriptor_);
-			return -1;
-		} else {
-			logger_->log_info("SSL socket connect success to %s %d",
-					requested_hostname_.c_str(), port_);
-			return 0;
-		}
-	}
-	return ret;
-}
-
-short TLSSocket::select_descriptor(const uint16_t msec) {
-	if (ssl && SSL_pending(ssl))
-		return 1;
-	return Socket::select_descriptor(msec);
-}
-
-int TLSSocket::writeData(std::vector< uint8_t>& buf, int buflen)
-{
- return Socket::writeData(buf,buflen);
-}
-
-int TLSSocket::writeData(uint8_t *value, int size) {
-	if (IsNullOrEmpty(ssl))
-	  return -1;
-	// for SSL, wait for the TLS IO is completed
-	int bytes = 0;
-	int sent = 0;
-	while (bytes < size) {
-
-		sent = SSL_write(ssl, value + bytes, size - bytes);
-		//check for errors
-		if (sent < 0) {
-			logger_->log_error("Site2Site Peer socket %d send failed %s",
-					socket_file_descriptor_, strerror(errno));
-			return sent;
-		}
-		bytes += sent;
-	}
-	return size;
-}
-
-int TLSSocket::readData(uint8_t *buf, int buflen) {
-
-	if (IsNullOrEmpty(ssl))
-	  return -1;
-	int total_read = 0;
-	int status = 0;
-	while (buflen) {
-		short fd = select_descriptor(1000);
-		if (fd <= 0) {
-
-			close(socket_file_descriptor_);
-			return -1;
-		}
-
-		int sslStatus;
-		do {
-			status = SSL_read(ssl, buf, buflen);
-			sslStatus = SSL_get_error(ssl, status);
-		} while (status < 0 && sslStatus == SSL_ERROR_WANT_READ);
-
-		buflen -= status;
-		buf += status;
-		total_read += status;
-	}
-
-	return total_read;
-}
-#endif // #ifdef OPENSSL_SUPPORT

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/io/tls/TLSSocket.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/io/tls/TLSSocket.cpp b/libminifi/src/io/tls/TLSSocket.cpp
new file mode 100644
index 0000000..b2df394
--- /dev/null
+++ b/libminifi/src/io/tls/TLSSocket.cpp
@@ -0,0 +1,249 @@
+/**
+ *
+ * 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 "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "utils/StringUtils.h"
+
+#include "core/Property.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace io {
+
+#include <openssl/ssl.h>
+#include <openssl/err.h>
+
+std::atomic<TLSContext*> TLSContext::context_instance;
+std::mutex TLSContext::context_mutex;
+
+TLSContext::TLSContext() :
+		error_value(0), ctx(0), logger_(logging::Logger::getLogger()), configuration(
+				Configure::getConfigure()) {
+
+}
+
+/**
+ * The memory barrier is defined by the singleton
+ */
+short TLSContext::initialize() {
+	if (ctx != 0) {
+		return error_value;
+	}
+	std::string clientAuthStr;
+	bool needClientCert = true;
+	if (!(configuration->get(Configure::nifi_security_need_ClientAuth,
+			clientAuthStr)
+			&& org::apache::nifi::minifi::utils::StringUtils::StringToBool(clientAuthStr, needClientCert))) {
+		needClientCert = true;
+	}
+
+	SSL_library_init();
+	const SSL_METHOD *method;
+
+	OpenSSL_add_all_algorithms();
+	SSL_load_error_strings();
+	method = TLSv1_2_client_method();
+	ctx = SSL_CTX_new(method);
+	if (ctx == NULL) {
+		logger_->log_error("Could not create SSL context, error: %s.",
+				std::strerror(errno));
+		error_value = TLS_ERROR_CONTEXT;
+		return error_value;
+	}
+	if (needClientCert) {
+		std::string certificate;
+		std::string privatekey;
+		std::string passphrase;
+		std::string caCertificate;
+
+		if (!(configuration->get(Configure::nifi_security_client_certificate,
+				certificate)
+				&& configuration->get(
+						Configure::nifi_security_client_private_key, privatekey))) {
+			logger_->log_error(
+					"Certificate and Private Key PEM file not configured, error: %s.",
+					std::strerror(errno));
+			error_value = TLS_ERROR_PEM_MISSING;
+			return error_value;
+		}
+		// load certificates and private key in PEM format
+		if (SSL_CTX_use_certificate_file(ctx, certificate.c_str(),
+				SSL_FILETYPE_PEM) <= 0) {
+			logger_->log_error("Could not create load certificate, error : %s",
+					std::strerror(errno));
+			error_value = TLS_ERROR_CERT_MISSING;
+			return error_value;
+
+		}
+		if (configuration->get(Configure::nifi_security_client_pass_phrase,
+				passphrase)) {
+			// if the private key has passphase
+			SSL_CTX_set_default_passwd_cb(ctx, pemPassWordCb);
+		}
+		
+
+		int retp = SSL_CTX_use_PrivateKey_file(ctx, privatekey.c_str(),
+				SSL_FILETYPE_PEM);
+		if (retp != 1) {
+			logger_->log_error("Could not create load private key,%i on %s error : %s",
+					retp,privatekey.c_str(),std::strerror(errno));
+			error_value = TLS_ERROR_KEY_ERROR;
+			return error_value;
+		}
+		// verify private key
+		if (!SSL_CTX_check_private_key(ctx)) {
+			logger_->log_error(
+					"Private key does not match the public certificate, error : %s",
+					std::strerror(errno));
+			error_value = TLS_ERROR_KEY_ERROR;
+			return error_value;
+		}
+		// load CA certificates
+		if (configuration->get(Configure::nifi_security_client_ca_certificate,
+				caCertificate)) {
+			retp = SSL_CTX_load_verify_locations(ctx, caCertificate.c_str(), 0);
+			if (retp==0) {
+				logger_->log_error(
+						"Can not load CA certificate, Exiting, error : %s",
+						std::strerror(errno));
+				error_value = TLS_ERROR_CERT_ERROR;
+				return error_value;
+			}
+		}
+
+		logger_->log_info("Load/Verify Client Certificate OK.");
+	}
+	return 0;
+}
+
+TLSSocket::~TLSSocket()
+{
+	if (ssl != 0)
+		SSL_free(ssl);
+}
+/**
+ * Constructor that accepts host name, port and listeners. With this
+ * contructor we will be creating a server socket
+ * @param hostname our host name
+ * @param port connecting port
+ * @param listeners number of listeners in the queue
+ */
+TLSSocket::TLSSocket(const std::string &hostname, const uint16_t port,
+		const uint16_t listeners) :
+		Socket(hostname, port, listeners), ssl(0) {
+}
+
+TLSSocket::TLSSocket(const std::string &hostname, const uint16_t port) :
+		Socket(hostname, port, 0), ssl(0) {
+}
+
+TLSSocket::TLSSocket(const TLSSocket &&d) :
+		Socket(std::move(d)), ssl(0) {
+}
+
+short TLSSocket::initialize() {
+	TLSContext *context = TLSContext::getInstance();
+	short ret = context->initialize();
+	Socket::initialize();
+	if (!ret) {
+		// we have s2s secure config
+		ssl = SSL_new(context->getContext());
+		SSL_set_fd(ssl, socket_file_descriptor_);
+		if (SSL_connect(ssl) == -1) {
+			logger_->log_error("SSL socket connect failed to %s %d",
+					requested_hostname_.c_str(), port_);
+			SSL_free(ssl);
+			ssl = NULL;
+			close(socket_file_descriptor_);
+			return -1;
+		} else {
+			logger_->log_info("SSL socket connect success to %s %d",
+					requested_hostname_.c_str(), port_);
+			return 0;
+		}
+	}
+	return ret;
+}
+
+short TLSSocket::select_descriptor(const uint16_t msec) {
+	if (ssl && SSL_pending(ssl))
+		return 1;
+	return Socket::select_descriptor(msec);
+}
+
+int TLSSocket::writeData(std::vector< uint8_t>& buf, int buflen)
+{
+ return Socket::writeData(buf,buflen);
+}
+
+int TLSSocket::writeData(uint8_t *value, int size) {
+	if (IsNullOrEmpty(ssl))
+	  return -1;
+	// for SSL, wait for the TLS IO is completed
+	int bytes = 0;
+	int sent = 0;
+	while (bytes < size) {
+
+		sent = SSL_write(ssl, value + bytes, size - bytes);
+		//check for errors
+		if (sent < 0) {
+			logger_->log_error("Site2Site Peer socket %d send failed %s",
+					socket_file_descriptor_, strerror(errno));
+			return sent;
+		}
+		bytes += sent;
+	}
+	return size;
+}
+
+int TLSSocket::readData(uint8_t *buf, int buflen) {
+
+	if (IsNullOrEmpty(ssl))
+	  return -1;
+	int total_read = 0;
+	int status = 0;
+	while (buflen) {
+		short fd = select_descriptor(1000);
+		if (fd <= 0) {
+
+			close(socket_file_descriptor_);
+			return -1;
+		}
+
+		int sslStatus;
+		do {
+			status = SSL_read(ssl, buf, buflen);
+			sslStatus = SSL_get_error(ssl, status);
+		} while (status < 0 && sslStatus == SSL_ERROR_WANT_READ);
+
+		buflen -= status;
+		buf += status;
+		total_read += status;
+	}
+
+	return total_read;
+}
+
+
+} /* namespace io */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/processors/AppendHostInfo.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/AppendHostInfo.cpp b/libminifi/src/processors/AppendHostInfo.cpp
new file mode 100644
index 0000000..24ccc9a
--- /dev/null
+++ b/libminifi/src/processors/AppendHostInfo.cpp
@@ -0,0 +1,124 @@
+/**
+ * @file AppendHostInfo.cpp
+ * AppendHostInfo class implementation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <set>
+#include <sys/time.h>
+#include <string.h>
+#include "processors/AppendHostInfo.h"
+#include "core/ProcessContext.h"
+#include "core/Property.h"
+#include "core/ProcessSession.h"
+
+#include <netdb.h>
+#include <netinet/in.h>
+#include <sys/socket.h>
+#include <sys/ioctl.h>
+#include <net/if.h>
+#include <arpa/inet.h>
+
+#include "../../include/core/FlowFile.h"
+#include "io/ClientSocket.h"
+
+#define __USE_POSIX
+#include <limits.h>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+#ifndef HOST_NAME_MAX
+#define HOST_NAME_MAX 255
+#endif
+
+const std::string AppendHostInfo::ProcessorName("AppendHostInfo");
+core::Property AppendHostInfo::InterfaceName(
+    "Network Interface Name",
+    "Network interface from which to read an IP v4 address", "eth0");
+core::Property AppendHostInfo::HostAttribute(
+    "Hostname Attribute",
+    "Flowfile attribute to used to record the agent's hostname",
+    "source.hostname");
+core::Property AppendHostInfo::IPAttribute(
+    "IP Attribute",
+    "Flowfile attribute to used to record the agent's IP address",
+    "source.ipv4");
+core::Relationship AppendHostInfo::Success(
+    "success", "success operational on the flow record");
+
+void AppendHostInfo::initialize() {
+  // Set the supported properties
+  std::set<core::Property> properties;
+  properties.insert(InterfaceName);
+  properties.insert(HostAttribute);
+  properties.insert(IPAttribute);
+  setSupportedProperties(properties);
+
+  // Set the supported relationships
+  std::set<core::Relationship> relationships;
+  relationships.insert(Success);
+  setSupportedRelationships(relationships);
+}
+
+void AppendHostInfo::onTrigger(
+    core::ProcessContext *context,
+    core::ProcessSession *session) {
+  std::shared_ptr<core::FlowFile> flow =
+      session->get();
+  if (!flow)
+    return;
+
+  //Get Hostname
+
+  std::string hostAttribute = "";
+  context->getProperty(HostAttribute.getName(), hostAttribute);
+  flow->addAttribute(hostAttribute.c_str(),
+                     org::apache::nifi::minifi::io::Socket::getMyHostName());
+
+  //Get IP address for the specified interface
+  std::string iface;
+  context->getProperty(InterfaceName.getName(), iface);
+  //Confirm the specified interface name exists on this device
+  if (if_nametoindex(iface.c_str()) != 0) {
+    struct ifreq ifr;
+    int fd = socket(AF_INET, SOCK_DGRAM, 0);
+    //Type of address to retrieve - IPv4 IP address
+    ifr.ifr_addr.sa_family = AF_INET;
+    //Copy the interface name in the ifreq structure
+    strncpy(ifr.ifr_name, iface.c_str(), IFNAMSIZ - 1);
+    ioctl(fd, SIOCGIFADDR, &ifr);
+    close(fd);
+
+    std::string ipAttribute;
+    context->getProperty(IPAttribute.getName(), ipAttribute);
+    flow->addAttribute(
+        ipAttribute.c_str(),
+        inet_ntoa(((struct sockaddr_in *) &ifr.ifr_addr)->sin_addr));
+  }
+
+  // Transfer to the relationship
+  session->transfer(flow, Success);
+}
+
+} /* namespace processors */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */


[10/16] nifi-minifi-cpp git commit: MINIFI-217: Updates namespaces and removes use of raw pointers for user facing API.

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/provenance/Provenance.h
----------------------------------------------------------------------
diff --git a/libminifi/include/provenance/Provenance.h b/libminifi/include/provenance/Provenance.h
new file mode 100644
index 0000000..3d5d19e
--- /dev/null
+++ b/libminifi/include/provenance/Provenance.h
@@ -0,0 +1,560 @@
+/**
+ *
+ * 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 __PROVENANCE_H__
+#define __PROVENANCE_H__
+
+#include <ftw.h>
+#include <uuid/uuid.h>
+#include <atomic>
+#include <cstdint>
+#include <cstring>
+#include <iostream>
+#include <map>
+#include <set>
+#include <string>
+#include <thread>
+#include <vector>
+
+
+#include "core/Repository.h"
+#include "core/Property.h"
+#include "properties/Configure.h"
+#include "Connection.h"
+#include "FlowFileRecord.h"
+#include "core/logging/Logger.h"
+#include "ResourceClaim.h"
+#include "io/Serializable.h"
+#include "utils/TimeUtil.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace provenance {
+// Provenance Event Record Serialization Seg Size
+#define PROVENANCE_EVENT_RECORD_SEG_SIZE 2048
+
+// Provenance Event Record
+class ProvenanceEventRecord :
+    protected org::apache::nifi::minifi::io::Serializable {
+ public:
+  enum ProvenanceEventType {
+
+    /**
+     * A CREATE event is used when a FlowFile is generated from data that was
+     * not received from a remote system or external process
+     */
+    CREATE,
+
+    /**
+     * Indicates a provenance event for receiving data from an external process. This Event Type
+     * is expected to be the first event for a FlowFile. As such, a Processor that receives data
+     * from an external source and uses that data to replace the content of an existing FlowFile
+     * should use the {@link #FETCH} event type, rather than the RECEIVE event type.
+     */
+    RECEIVE,
+
+    /**
+     * Indicates that the contents of a FlowFile were overwritten using the contents of some
+     * external resource. This is similar to the {@link #RECEIVE} event but varies in that
+     * RECEIVE events are intended to be used as the event that introduces the FlowFile into
+     * the system, whereas FETCH is used to indicate that the contents of an existing FlowFile
+     * were overwritten.
+     */
+    FETCH,
+
+    /**
+     * Indicates a provenance event for sending data to an external process
+     */
+    SEND,
+
+    /**
+     * Indicates that the contents of a FlowFile were downloaded by a user or external entity.
+     */
+    DOWNLOAD,
+
+    /**
+     * Indicates a provenance event for the conclusion of an object's life for
+     * some reason other than object expiration
+     */
+    DROP,
+
+    /**
+     * Indicates a provenance event for the conclusion of an object's life due
+     * to the fact that the object could not be processed in a timely manner
+     */
+    EXPIRE,
+
+    /**
+     * FORK is used to indicate that one or more FlowFile was derived from a
+     * parent FlowFile.
+     */
+    FORK,
+
+    /**
+     * JOIN is used to indicate that a single FlowFile is derived from joining
+     * together multiple parent FlowFiles.
+     */
+    JOIN,
+
+    /**
+     * CLONE is used to indicate that a FlowFile is an exact duplicate of its
+     * parent FlowFile.
+     */
+    CLONE,
+
+    /**
+     * CONTENT_MODIFIED is used to indicate that a FlowFile's content was
+     * modified in some way. When using this Event Type, it is advisable to
+     * provide details about how the content is modified.
+     */
+    CONTENT_MODIFIED,
+
+    /**
+     * ATTRIBUTES_MODIFIED is used to indicate that a FlowFile's attributes were
+     * modified in some way. This event is not needed when another event is
+     * reported at the same time, as the other event will already contain all
+     * FlowFile attributes.
+     */
+    ATTRIBUTES_MODIFIED,
+
+    /**
+     * ROUTE is used to show that a FlowFile was routed to a specified
+     * {@link org.apache.nifi.processor.Relationship Relationship} and should provide
+     * information about why the FlowFile was routed to this relationship.
+     */
+    ROUTE,
+
+    /**
+     * Indicates a provenance event for adding additional information such as a
+     * new linkage to a new URI or UUID
+     */
+    ADDINFO,
+
+    /**
+     * Indicates a provenance event for replaying a FlowFile. The UUID of the
+     * event will indicate the UUID of the original FlowFile that is being
+     * replayed. The event will contain exactly one Parent UUID that is also the
+     * UUID of the FlowFile that is being replayed and exactly one Child UUID
+     * that is the UUID of the a newly created FlowFile that will be re-queued
+     * for processing.
+     */
+    REPLAY
+  };
+ public:
+  // Constructor
+  /*!
+   * Create a new provenance event record
+   */
+  ProvenanceEventRecord(ProvenanceEventType event, std::string componentId,
+                        std::string componentType) {
+    _eventType = event;
+    _componentId = componentId;
+    _componentType = componentType;
+    _eventTime = getTimeMillis();
+    char eventIdStr[37];
+    // Generate the global UUID for th event
+    uuid_generate(_eventId);
+    uuid_unparse_lower(_eventId, eventIdStr);
+    _eventIdStr = eventIdStr;
+    logger_ = logging::Logger::getLogger();
+  }
+
+  ProvenanceEventRecord() {
+    _eventTime = getTimeMillis();
+    logger_ = logging::Logger::getLogger();
+  }
+
+  // Destructor
+  virtual ~ProvenanceEventRecord() {
+  }
+  // Get the Event ID
+  std::string getEventId() {
+    return _eventIdStr;
+  }
+  // Get Attributes
+  std::map<std::string, std::string> getAttributes() {
+    return _attributes;
+  }
+  // Get Size
+  uint64_t getFileSize() {
+    return _size;
+  }
+  // ! Get Offset
+  uint64_t getFileOffset() {
+    return _offset;
+  }
+  // ! Get Entry Date
+  uint64_t getFlowFileEntryDate() {
+    return _entryDate;
+  }
+  // ! Get Lineage Start Date
+  uint64_t getlineageStartDate() {
+    return _lineageStartDate;
+  }
+  // ! Get Event Time
+  uint64_t getEventTime() {
+    return _eventTime;
+  }
+  // ! Get Event Duration
+  uint64_t getEventDuration() {
+    return _eventDuration;
+  }
+  // Set Event Duration
+  void setEventDuration(uint64_t duration) {
+    _eventDuration = duration;
+  }
+  // ! Get Event Type
+  ProvenanceEventType getEventType() {
+    return _eventType;
+  }
+  // Get Component ID
+  std::string getComponentId() {
+    return _componentId;
+  }
+  // Get Component Type
+  std::string getComponentType() {
+    return _componentType;
+  }
+  // Get FlowFileUuid
+  std::string getFlowFileUuid() {
+    return uuid_;
+  }
+  // Get content full path
+  std::string getContentFullPath() {
+    return _contentFullPath;
+  }
+  // Get LineageIdentifiers
+  std::set<std::string> getLineageIdentifiers() {
+    return _lineageIdentifiers;
+  }
+  // Get Details
+  std::string getDetails() {
+    return _details;
+  }
+  // Set Details
+  void setDetails(std::string details) {
+    _details = details;
+  }
+  // Get TransitUri
+  std::string getTransitUri() {
+    return _transitUri;
+  }
+  // Set TransitUri
+  void setTransitUri(std::string uri) {
+    _transitUri = uri;
+  }
+  // Get SourceSystemFlowFileIdentifier
+  std::string getSourceSystemFlowFileIdentifier() {
+    return _sourceSystemFlowFileIdentifier;
+  }
+  // Set SourceSystemFlowFileIdentifier
+  void setSourceSystemFlowFileIdentifier(std::string identifier) {
+    _sourceSystemFlowFileIdentifier = identifier;
+  }
+  // Get Parent UUIDs
+  std::vector<std::string> getParentUuids() {
+    return _parentUuids;
+  }
+  // Add Parent UUID
+  void addParentUuid(std::string uuid) {
+    if (std::find(_parentUuids.begin(), _parentUuids.end(), uuid)
+        != _parentUuids.end())
+      return;
+    else
+      _parentUuids.push_back(uuid);
+  }
+  // Add Parent Flow File
+  void addParentFlowFile(std::shared_ptr<core::FlowFile> flow) {
+    addParentUuid(flow->getUUIDStr());
+    return;
+  }
+  // Remove Parent UUID
+  void removeParentUuid(std::string uuid) {
+    _parentUuids.erase(
+        std::remove(_parentUuids.begin(), _parentUuids.end(), uuid),
+        _parentUuids.end());
+  }
+  // Remove Parent Flow File
+  void removeParentFlowFile(std::shared_ptr<core::FlowFile> flow) {
+    removeParentUuid(flow->getUUIDStr());
+    return;
+  }
+  // Get Children UUIDs
+  std::vector<std::string> getChildrenUuids() {
+    return _childrenUuids;
+  }
+  // Add Child UUID
+  void addChildUuid(std::string uuid) {
+    if (std::find(_childrenUuids.begin(), _childrenUuids.end(), uuid)
+        != _childrenUuids.end())
+      return;
+    else
+      _childrenUuids.push_back(uuid);
+  }
+  // Add Child Flow File
+  void addChildFlowFile(std::shared_ptr<core::FlowFile> flow) {
+    addChildUuid(flow->getUUIDStr());
+    return;
+  }
+  // Remove Child UUID
+  void removeChildUuid(std::string uuid) {
+    _childrenUuids.erase(
+        std::remove(_childrenUuids.begin(), _childrenUuids.end(), uuid),
+        _childrenUuids.end());
+  }
+  // Remove Child Flow File
+  void removeChildFlowFile(std::shared_ptr<core::FlowFile> flow) {
+    removeChildUuid(flow->getUUIDStr());
+    return;
+  }
+  // Get AlternateIdentifierUri
+  std::string getAlternateIdentifierUri() {
+    return _alternateIdentifierUri;
+  }
+  // Set AlternateIdentifierUri
+  void setAlternateIdentifierUri(std::string uri) {
+    _alternateIdentifierUri = uri;
+  }
+  // Get Relationship
+  std::string getRelationship() {
+    return _relationship;
+  }
+  // Set Relationship
+  void setRelationship(std::string relation) {
+    _relationship = relation;
+  }
+  // Get sourceQueueIdentifier
+  std::string getSourceQueueIdentifier() {
+    return _sourceQueueIdentifier;
+  }
+  // Set sourceQueueIdentifier
+  void setSourceQueueIdentifier(std::string identifier) {
+    _sourceQueueIdentifier = identifier;
+  }
+  // fromFlowFile
+  void fromFlowFile(std::shared_ptr<core::FlowFile> &flow) {
+    _entryDate = flow->getEntryDate();
+    _lineageStartDate = flow->getlineageStartDate();
+    _lineageIdentifiers = flow->getlineageIdentifiers();
+    uuid_ = flow->getUUIDStr();
+    _attributes = flow->getAttributes();
+    _size = flow->getSize();
+    _offset = flow->getOffset();
+    if (flow->getOriginalConnection())
+      _sourceQueueIdentifier = flow->getOriginalConnection()->getName();
+    if (flow->getResourceClaim()) {
+      _contentFullPath = flow->getResourceClaim()->getContentFullPath();
+    }
+  }
+  // Serialize and Persistent to the repository
+  bool Serialize(const std::shared_ptr<core::Repository> &repo);
+  // DeSerialize
+  bool DeSerialize(const uint8_t *buffer, const int 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);
+
+ protected:
+
+  // Event type
+  ProvenanceEventType _eventType;
+  // Date at which the event was created
+  uint64_t _eventTime;
+  // Date at which the flow file entered the flow
+  uint64_t _entryDate;
+  // Date at which the origin of this flow file entered the flow
+  uint64_t _lineageStartDate;
+  // Event Duration
+  uint64_t _eventDuration;
+  // Component ID
+  std::string _componentId;
+  // Component Type
+  std::string _componentType;
+  // Size in bytes of the data corresponding to this flow file
+  uint64_t _size;
+  // flow uuid
+  std::string 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
+  std::string _transitUri;
+  // sourceSystemFlowFileIdentifier
+  std::string _sourceSystemFlowFileIdentifier;
+  // parent UUID
+  std::vector<std::string> _parentUuids;
+  // child UUID
+  std::vector<std::string> _childrenUuids;
+  // detail
+  std::string _details;
+  // sourceQueueIdentifier
+  std::string _sourceQueueIdentifier;
+  // event ID Str
+  std::string _eventIdStr;
+  // relationship
+  std::string _relationship;
+  // alternateIdentifierUri;
+  std::string _alternateIdentifierUri;
+
+ private:
+
+  // Logger
+  std::shared_ptr<logging::Logger> logger_;
+
+  // Prevent default copy constructor and assignment operation
+  // Only support pass by reference or pointer
+  ProvenanceEventRecord(const ProvenanceEventRecord &parent);
+  ProvenanceEventRecord &operator=(const ProvenanceEventRecord &parent);
+
+};
+
+// Provenance Reporter
+class ProvenanceReporter {
+ public:
+  // Constructor
+  /*!
+   * Create a new provenance reporter associated with the process session
+   */
+  ProvenanceReporter(std::shared_ptr<core::Repository> repo,
+                     std::string componentId, std::string componentType) {
+    logger_ = logging::Logger::getLogger();
+    _componentId = componentId;
+    _componentType = componentType;
+    repo_ = repo;
+  }
+
+  // Destructor
+  virtual ~ProvenanceReporter() {
+    clear();
+  }
+  // Get events
+  std::set<ProvenanceEventRecord *> getEvents() {
+    return _events;
+  }
+  // Add event
+  void add(ProvenanceEventRecord *event) {
+    _events.insert(event);
+  }
+  // Remove event
+  void remove(ProvenanceEventRecord *event) {
+    if (_events.find(event) != _events.end()) {
+      _events.erase(event);
+    }
+  }
+  //
+  // clear
+  void clear() {
+    for (auto it : _events) {
+      delete it;
+    }
+    _events.clear();
+  }
+  // allocate
+  ProvenanceEventRecord *allocate(
+      ProvenanceEventRecord::ProvenanceEventType eventType,
+      std::shared_ptr<core::FlowFile> flow) {
+    ProvenanceEventRecord *event = new ProvenanceEventRecord(eventType,
+                                                             _componentId,
+                                                             _componentType);
+    if (event)
+      event->fromFlowFile(flow);
+
+    return event;
+  }
+  // commit
+  void commit();
+  // create
+  void create(std::shared_ptr<core::FlowFile> flow, std::string detail);
+  // route
+  void route(std::shared_ptr<core::FlowFile> flow, core::Relationship relation,
+             std::string detail, uint64_t processingDuration);
+  // modifyAttributes
+  void modifyAttributes(std::shared_ptr<core::FlowFile> flow,
+                        std::string detail);
+  // modifyContent
+  void modifyContent(std::shared_ptr<core::FlowFile> flow, std::string detail,
+                     uint64_t processingDuration);
+  // clone
+  void clone(std::shared_ptr<core::FlowFile> parent,
+             std::shared_ptr<core::FlowFile> child);
+  // join
+  void join(std::vector<std::shared_ptr<core::FlowFile> > parents,
+            std::shared_ptr<core::FlowFile> child, std::string detail,
+            uint64_t processingDuration);
+  // fork
+  void fork(std::vector<std::shared_ptr<core::FlowFile> > child,
+            std::shared_ptr<core::FlowFile> parent, std::string detail,
+            uint64_t processingDuration);
+  // expire
+  void expire(std::shared_ptr<core::FlowFile> flow, std::string detail);
+  // drop
+  void drop(std::shared_ptr<core::FlowFile> flow, std::string reason);
+  // send
+  void send(std::shared_ptr<core::FlowFile> flow, std::string transitUri,
+            std::string detail, uint64_t processingDuration, bool force);
+  // fetch
+  void fetch(std::shared_ptr<core::FlowFile> flow, std::string transitUri,
+             std::string detail, uint64_t processingDuration);
+  // receive
+  void receive(std::shared_ptr<core::FlowFile> flow, std::string transitUri,
+               std::string sourceSystemFlowFileIdentifier, std::string detail,
+               uint64_t processingDuration);
+
+ protected:
+
+  // Component ID
+  std::string _componentId;
+  // Component Type
+  std::string _componentType;
+
+ private:
+
+  // Incoming connection Iterator
+  std::set<ProvenanceEventRecord *> _events;
+  // Logger
+  std::shared_ptr<logging::Logger> logger_;
+  // provenance repository.
+  std::shared_ptr<core::Repository> repo_;
+
+  // Prevent default copy constructor and assignment operation
+  // Only support pass by reference or pointer
+  ProvenanceReporter(const ProvenanceReporter &parent);
+  ProvenanceReporter &operator=(const ProvenanceReporter &parent);
+};
+
+// Provenance Repository
+
+} /* namespace provenance */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/provenance/ProvenanceRepository.h
----------------------------------------------------------------------
diff --git a/libminifi/include/provenance/ProvenanceRepository.h b/libminifi/include/provenance/ProvenanceRepository.h
new file mode 100644
index 0000000..0f8ee5d
--- /dev/null
+++ b/libminifi/include/provenance/ProvenanceRepository.h
@@ -0,0 +1,166 @@
+/**
+ * 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_PROVENANCE_PROVENANCEREPOSITORY_H_
+#define LIBMINIFI_INCLUDE_PROVENANCE_PROVENANCEREPOSITORY_H_
+
+#include "leveldb/db.h"
+#include "leveldb/options.h"
+#include "leveldb/slice.h"
+#include "leveldb/status.h"
+#include "core/Repository.h"
+#include "core/core.h"
+#include "provenance/Provenance.h"
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace provenance {
+
+#define PROVENANCE_DIRECTORY "./provenance_repository"
+#define MAX_PROVENANCE_STORAGE_SIZE (10*1024*1024) // 10M
+#define MAX_PROVENANCE_ENTRY_LIFE_TIME (60000) // 1 minute
+#define PROVENANCE_PURGE_PERIOD (2500) // 2500 msec
+
+class ProvenanceRepository : public core::Repository,
+    public std::enable_shared_from_this<ProvenanceRepository> {
+ public:
+  // Constructor
+  /*!
+   * Create a new provenance repository
+   */
+  ProvenanceRepository(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(core::getClassName<ProvenanceRepository>(), directory,
+                   maxPartitionMillis, maxPartitionBytes, purgePeriod) {
+
+    db_ = NULL;
+  }
+
+  // Destructor
+  virtual ~ProvenanceRepository() {
+    if (db_)
+      delete db_;
+  }
+
+  // initialize
+  virtual bool initialize() {
+    std::string value;
+    if (configure_->get(Configure::nifi_provenance_repository_directory_default,
+                        value)) {
+      directory_ = value;
+    }
+    logger_->log_info("NiFi Provenance Repository Directory %s",
+                      directory_.c_str());
+    if (configure_->get(Configure::nifi_provenance_repository_max_storage_size,
+                        value)) {
+      core::Property::StringToInt(value, max_partition_bytes_);
+    }
+    logger_->log_info("NiFi Provenance Max Partition Bytes %d",
+                      max_partition_bytes_);
+    if (configure_->get(Configure::nifi_provenance_repository_max_storage_time,
+                        value)) {
+      core::TimeUnit unit;
+      if (core::Property::StringToTime(value, max_partition_millis_, unit)
+          && core::Property::ConvertTimeUnitToMS(max_partition_millis_, unit,
+                                                 max_partition_millis_)) {
+      }
+    }
+    logger_->log_info("NiFi Provenance Max Storage Time: [%d] ms",
+                      max_partition_millis_);
+    leveldb::Options options;
+    options.create_if_missing = true;
+    leveldb::Status status = leveldb::DB::Open(options, directory_.c_str(),
+                                               &db_);
+    if (status.ok()) {
+      logger_->log_info("NiFi Provenance Repository database open %s success",
+                        directory_.c_str());
+    } else {
+      logger_->log_error("NiFi Provenance Repository database open %s fail",
+                         directory_.c_str());
+      return false;
+    }
+
+    return true;
+  }
+  // Put
+  virtual bool Put(std::string key, uint8_t *buf, int bufLen) {
+
+    // persistent to the DB
+    leveldb::Slice value((const char *) buf, bufLen);
+    leveldb::Status status;
+    status = db_->Put(leveldb::WriteOptions(), key, value);
+    if (status.ok())
+      return true;
+    else
+      return false;
+  }
+  // Delete
+  virtual bool Delete(std::string key) {
+    leveldb::Status status;
+    status = db_->Delete(leveldb::WriteOptions(), key);
+    if (status.ok())
+      return true;
+    else
+      return false;
+  }
+  // Get
+  virtual bool Get(std::string key, std::string &value) {
+    leveldb::Status status;
+    status = db_->Get(leveldb::ReadOptions(), key, &value);
+    if (status.ok())
+      return true;
+    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());
+  }
+  // destroy
+  void destroy() {
+    if (db_) {
+      delete db_;
+      db_ = NULL;
+    }
+  }
+  // Run function for the thread
+   void run();
+
+  // Prevent default copy constructor and assignment operation
+  // Only support pass by reference or pointer
+  ProvenanceRepository(const ProvenanceRepository &parent) = delete;
+  ProvenanceRepository &operator=(const ProvenanceRepository &parent) = delete;
+
+ private:
+  leveldb::DB* db_;
+
+};
+
+} /* namespace provenance */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+#endif /* LIBMINIFI_INCLUDE_PROVENANCE_PROVENANCEREPOSITORY_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/utils/FailurePolicy.h
----------------------------------------------------------------------
diff --git a/libminifi/include/utils/FailurePolicy.h b/libminifi/include/utils/FailurePolicy.h
index a4a7f9e..98ec18a 100644
--- a/libminifi/include/utils/FailurePolicy.h
+++ b/libminifi/include/utils/FailurePolicy.h
@@ -17,6 +17,12 @@
 #ifndef LIBMINIFI_INCLUDE_UTILS_FAILUREPOLICY_H_
 #define LIBMINIFI_INCLUDE_UTILS_FAILUREPOLICY_H_
 
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
 /**
  * Basic failure policy enumeration
  *
@@ -42,4 +48,11 @@ enum FailurePolicy {
 	EXIT
 };
 
+} /* namespace utils */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+
 #endif /* LIBMINIFI_INCLUDE_UTILS_FAILUREPOLICY_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/include/utils/StringUtils.h
----------------------------------------------------------------------
diff --git a/libminifi/include/utils/StringUtils.h b/libminifi/include/utils/StringUtils.h
index 30858c8..82459db 100644
--- a/libminifi/include/utils/StringUtils.h
+++ b/libminifi/include/utils/StringUtils.h
@@ -19,7 +19,13 @@
 
 #include <algorithm>
 #include <sstream>
-#include "../utils/FailurePolicy.h"
+#include "utils/FailurePolicy.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
 
 /**
  * Stateless String utility class.
@@ -122,4 +128,11 @@ public:
 
 };
 
+} /* namespace utils */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */
+
+
 #endif /* LIBMINIFI_INCLUDE_IO_STRINGUTILS_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/AppendHostInfo.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/AppendHostInfo.cpp b/libminifi/src/AppendHostInfo.cpp
deleted file mode 100644
index d0769c1..0000000
--- a/libminifi/src/AppendHostInfo.cpp
+++ /dev/null
@@ -1,97 +0,0 @@
-/**
- * @file AppendHostInfo.cpp
- * AppendHostInfo class implementation
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#include <set>
-#include <sys/time.h>
-#include <string.h>
-#include "AppendHostInfo.h"
-#include "ProcessContext.h"
-#include "ProcessSession.h"
-
-#include <netdb.h>
-#include <netinet/in.h>
-#include <sys/socket.h>
-#include <sys/ioctl.h>
-#include <net/if.h>
-#include <arpa/inet.h>
-
-#include "io/ClientSocket.h"
-
-#define __USE_POSIX
-#include <limits.h>
-
-#ifndef HOST_NAME_MAX
-#define HOST_NAME_MAX 255
-#endif
-
-const std::string AppendHostInfo::ProcessorName("AppendHostInfo");
-Property AppendHostInfo::InterfaceName("Network Interface Name", "Network interface from which to read an IP v4 address", "eth0");
-Property AppendHostInfo::HostAttribute("Hostname Attribute", "Flowfile attribute to used to record the agent's hostname", "source.hostname");
-Property AppendHostInfo::IPAttribute("IP Attribute", "Flowfile attribute to used to record the agent's IP address", "source.ipv4");
-Relationship AppendHostInfo::Success("success", "success operational on the flow record");
-
-void AppendHostInfo::initialize()
-{
-	//! Set the supported properties
-	std::set<Property> properties;
-	properties.insert(InterfaceName);
-	properties.insert(HostAttribute);
-	properties.insert(IPAttribute);
-	setSupportedProperties(properties);
-
-	//! Set the supported relationships
-	std::set<Relationship> relationships;
-	relationships.insert(Success);
-	setSupportedRelationships(relationships);
-}
-
-void AppendHostInfo::onTrigger(ProcessContext *context, ProcessSession *session)
-{
-	FlowFileRecord *flow = session->get();
-	if (!flow)
-	  return;
-
-	//Get Hostname
-
-	std::string hostAttribute = "";
-	context->getProperty(HostAttribute.getName(), hostAttribute);
-	flow->addAttribute(hostAttribute.c_str(), Socket::getMyHostName());
-
-	//Get IP address for the specified interface
-  std::string iface;
-	context->getProperty(InterfaceName.getName(), iface);
-  //Confirm the specified interface name exists on this device
-  if (if_nametoindex(iface.c_str()) != 0){
-    struct ifreq ifr;
-    int fd = socket(AF_INET, SOCK_DGRAM, 0);
-    //Type of address to retrieve - IPv4 IP address
-    ifr.ifr_addr.sa_family = AF_INET;
-    //Copy the interface name in the ifreq structure
-    strncpy(ifr.ifr_name , iface.c_str(), IFNAMSIZ-1);
-    ioctl(fd, SIOCGIFADDR, &ifr);
-    close(fd);
-
-    std::string ipAttribute;
-    context->getProperty(IPAttribute.getName(), ipAttribute);
-    flow->addAttribute(ipAttribute.c_str(), inet_ntoa(((struct sockaddr_in *)&ifr.ifr_addr)->sin_addr));
-  }
-
-	// Transfer to the relationship
-	session->transfer(flow, Success);
-}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/BaseLogger.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/BaseLogger.cpp b/libminifi/src/BaseLogger.cpp
deleted file mode 100644
index 1b3b2fd..0000000
--- a/libminifi/src/BaseLogger.cpp
+++ /dev/null
@@ -1,153 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#include "BaseLogger.h"
-
-// Logger related configuration items.
-const char *BaseLogger::nifi_log_level = "nifi.log.level";
-const char *BaseLogger::nifi_log_appender = "nifi.log.appender";
-
-/**
- * @brief Log error message
- * @param format format string ('man printf' for syntax)
- * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
- */
- void BaseLogger::log_error(const char * const format, ...) {
-	if (logger_ == NULL || !logger_->should_log(spdlog::level::level_enum::err))
-		return;
-	FILL_BUFFER
-	log_str(err,buffer);
-}
-/**
- * @brief Log warn message
- * @param format format string ('man printf' for syntax)
- * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
- */
- void BaseLogger::log_warn(const char * const format, ...) {
-	if (logger_ == NULL
-			|| !logger_->should_log(spdlog::level::level_enum::warn))
-		return;
-	FILL_BUFFER
-	log_str(warn,buffer);
-}
-/**
- * @brief Log info message
- * @param format format string ('man printf' for syntax)
- * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
- */
- void BaseLogger::log_info(const char * const format, ...) {
-	if (logger_ == NULL
-			|| !logger_->should_log(spdlog::level::level_enum::info))
-		return;
-	FILL_BUFFER
-	log_str(info,buffer);
-}
-/**
- * @brief Log debug message
- * @param format format string ('man printf' for syntax)
- * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
- */
- void BaseLogger::log_debug(const char * const format, ...) {
-
-	if (logger_ == NULL
-			|| !logger_->should_log(spdlog::level::level_enum::debug))
-		return;
-	FILL_BUFFER
-	log_str(debug,buffer);
-}
-/**
- * @brief Log trace message
- * @param format format string ('man printf' for syntax)
- * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
- */
- void BaseLogger::log_trace(const char * const format, ...) {
-
-	if (logger_ == NULL
-			|| !logger_->should_log(spdlog::level::level_enum::trace))
-		return;
-	FILL_BUFFER
-	log_str(debug,buffer);
-}
-
-// overridables
-
-/**
- * @brief Log error message
- * @param format format string ('man printf' for syntax)
- * @warning does not check @p log or @p format for null. Caller must ensure parameters and format string lengths match
- */
-void BaseLogger::log_str(LOG_LEVEL_E level, const std::string &buffer) {
-	switch (level) {
-	case err:
-	case critical:
-		if (stderr_ != nullptr) {
-			stderr_->error(buffer);
-		} else {
-			logger_->error(buffer);
-		}
-		break;
-	case warn:
-		logger_->warn(buffer);
-		break;
-	case info:
-		logger_->info(buffer);
-		break;
-	case debug:
-		logger_->debug(buffer);
-		break;
-	case trace:
-		logger_->trace(buffer);
-		break;
-	case off:
-		break;
-	default:
-		logger_->info(buffer);
-		break;
-	}
-
-}
-
-void BaseLogger::setLogLevel(const std::string &level,
-		LOG_LEVEL_E defaultLevel) {
-	std::string logLevel = level;
-	std::transform(logLevel.begin(), logLevel.end(), logLevel.begin(),
-			::tolower);
-
-	if (logLevel == "trace") {
-		setLogLevel(trace);
-	} else if (logLevel == "debug") {
-		setLogLevel(debug);
-	} else if (logLevel == "info") {
-		setLogLevel(info);
-	} else if (logLevel == "warn") {
-		setLogLevel(warn);
-	} else if (logLevel == "error") {
-		setLogLevel(err);
-	} else if (logLevel == "critical") {
-		setLogLevel(critical);
-	} else if (logLevel == "off") {
-		setLogLevel(off);
-	} else {
-		setLogLevel(defaultLevel);
-	}
-}
-
-void BaseLogger::set_error_logger(std::shared_ptr<spdlog::logger> other) {
-	stderr_ = std::move(other);
-}
-

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/Configure.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/Configure.cpp b/libminifi/src/Configure.cpp
index 6f5c08d..96ed7c7 100644
--- a/libminifi/src/Configure.cpp
+++ b/libminifi/src/Configure.cpp
@@ -1,6 +1,4 @@
 /**
- * @file Configure.cpp
- * Configure class implementation
  *
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -17,26 +15,33 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-#include "Configure.h"
+#include "properties/Configure.h"
 #include "utils/StringUtils.h"
+#include "core/core.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
 
 Configure *Configure::configure_(NULL);
 const char *Configure::nifi_flow_configuration_file = "nifi.flow.configuration.file";
 const char *Configure::nifi_administrative_yield_duration = "nifi.administrative.yield.duration";
 const char *Configure::nifi_bored_yield_duration = "nifi.bored.yield.duration";
-const char *Configure::nifi_graceful_shutdown_seconds  = "nifi.graceful.shutdown.seconds";
+const char *Configure::nifi_graceful_shutdown_seconds  = "nifi.flowcontroller.graceful.shutdown.period";
 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_provenance_repository_class_name = "nifi.provenance.repository.class.name";
 const char *Configure::nifi_server_port = "nifi.server.port";
 const char *Configure::nifi_server_report_interval= "nifi.server.report.interval";
 const char *Configure::nifi_provenance_repository_max_storage_size = "nifi.provenance.repository.max.storage.size";
 const char *Configure::nifi_provenance_repository_max_storage_time = "nifi.provenance.repository.max.storage.time";
 const char *Configure::nifi_provenance_repository_directory_default = "nifi.provenance.repository.directory.default";
-const char *Configure::nifi_provenance_repository_enable = "nifi.provenance.repository.enable";
 const char *Configure::nifi_flowfile_repository_max_storage_size = "nifi.flowfile.repository.max.storage.size";
 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_flowfile_repository_enable = "nifi.flowfile.repository.enable";
 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";
@@ -44,13 +49,13 @@ const char *Configure::nifi_security_client_private_key = "nifi.security.client.
 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";
 
-//! Get the config value
+// Get the config value
 bool Configure::get(std::string key, std::string &value)
 {
-	std::lock_guard<std::mutex> lock(_mtx);
-	auto it = _properties.find(key);
+	std::lock_guard<std::mutex> lock(mutex_);
+	auto it = properties_.find(key);
 
-	if (it != _properties.end())
+	if (it != properties_.end())
 	{
 		value = it->second;
 		return true;
@@ -62,7 +67,7 @@ bool Configure::get(std::string key, std::string &value)
 }
 
 
-//! Parse one line in configure file like key=value
+// Parse one line in configure file like key=value
 void Configure::parseConfigureFileLine(char *buf)
 {
 	char *line = buf;
@@ -96,12 +101,12 @@ void Configure::parseConfigureFileLine(char *buf)
     }
 
     std::string value = equal;
-    key = StringUtils::trimRight(key);
-    value = StringUtils::trimRight(value);
+    key = org::apache::nifi::minifi::utils::StringUtils::trimRight(key);
+    value = org::apache::nifi::minifi::utils::StringUtils::trimRight(value);
     set(key, value);
 }
 
-//! Load Configure File
+// Load Configure File
 void Configure::loadConfigureFile(const char *fileName)
 {
 
@@ -138,7 +143,7 @@ void Configure::loadConfigureFile(const char *fileName)
     }
 }
 
-//! Parse Command Line
+// Parse Command Line
 void Configure::parseCommandLine(int argc, char **argv)
 {
 	int i;
@@ -162,3 +167,8 @@ void Configure::parseCommandLine(int argc, char **argv)
 	}
 	return;
 }
+
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/Connection.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/Connection.cpp b/libminifi/src/Connection.cpp
index 42dbfe4..6f64ff3 100644
--- a/libminifi/src/Connection.cpp
+++ b/libminifi/src/Connection.cpp
@@ -27,185 +27,161 @@
 #include <thread>
 #include <iostream>
 
+#include "core/FlowFile.h"
 #include "Connection.h"
-#include "Processor.h"
-#include "FlowFileRepository.h"
-#include "FlowController.h"
-
-Connection::Connection(std::string name, uuid_t uuid, uuid_t srcUUID, uuid_t destUUID)
-: _name(name)
-{
-	if (!uuid)
-		// Generate the global UUID for the flow record
-		uuid_generate(_uuid);
-	else
-		uuid_copy(_uuid, uuid);
-
-	if (srcUUID)
-		uuid_copy(_srcUUID, srcUUID);
-	if (destUUID)
-		uuid_copy(_destUUID, destUUID);
-
-	_srcProcessor = NULL;
-	_destProcessor = NULL;
-	_maxQueueSize = 0;
-	_maxQueueDataSize = 0;
-	_expiredDuration = 0;
-	_queuedDataSize = 0;
-
-	logger_ = Logger::getLogger();
-
-	char uuidStr[37];
-	uuid_unparse_lower(_uuid, uuidStr);
-	_uuidStr = uuidStr;
-
-	logger_->log_info("Connection %s created", _name.c_str());
+#include "core/Processor.h"
+
+namespace org {
+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)
+    : core::Connectable(name, uuid),
+      flow_repository_(flow_repository) {
+
+  if (srcUUID)
+    uuid_copy(src_uuid_, srcUUID);
+  if (destUUID)
+    uuid_copy(dest_uuid_, destUUID);
+
+  source_connectable_ = nullptr;
+  dest_connectable_ = nullptr;
+  max_queue_size_ = 0;
+  max_data_queue_size_ = 0;
+  expired_duration_ = 0;
+  queued_data_size_ = 0;
+
+  logger_ = logging::Logger::getLogger();
+
+  logger_->log_info("Connection %s created", name_.c_str());
 }
 
-bool Connection::isEmpty()
-{
-	std::lock_guard<std::mutex> lock(_mtx);
+bool Connection::isEmpty() {
+  std::lock_guard<std::mutex> lock(mutex_);
 
-	return _queue.empty();
+  return queue_.empty();
 }
 
-bool Connection::isFull()
-{
-	std::lock_guard<std::mutex> lock(_mtx);
+bool Connection::isFull() {
+  std::lock_guard<std::mutex> lock(mutex_);
 
-	if (_maxQueueSize <= 0 && _maxQueueDataSize <= 0)
-		// No back pressure setting
-		return false;
+  if (max_queue_size_ <= 0 && max_data_queue_size_ <= 0)
+    // No back pressure setting
+    return false;
 
-	if (_maxQueueSize > 0 && _queue.size() >= _maxQueueSize)
-		return true;
+  if (max_queue_size_ > 0 && queue_.size() >= max_queue_size_)
+    return true;
 
-	if (_maxQueueDataSize > 0 && _queuedDataSize >= _maxQueueDataSize)
-		return true;
+  if (max_data_queue_size_ > 0 && queued_data_size_ >= max_data_queue_size_)
+    return true;
 
-	return false;
+  return false;
 }
 
-void Connection::put(FlowFileRecord *flow)
-{
-	{
-		std::lock_guard<std::mutex> lock(_mtx);
-	
-		_queue.push(flow);
-	
-		_queuedDataSize += flow->getSize();
-	
-		logger_->log_debug("Enqueue flow file UUID %s to connection %s",
-				flow->getUUIDStr().c_str(), _name.c_str());
-	}
-
-
-	if (FlowControllerFactory::getFlowController()->getFlowFileRepository() &&
-			FlowControllerFactory::getFlowController()->getFlowFileRepository()->isEnable() &&
-			!flow->isStoredToRepository())
-	{
-		// Save to the flowfile repo
-		FlowFileEventRecord event;
-		event.fromFlowFile(flow, this->_uuidStr);
-		if (event.Serialize(
-				FlowControllerFactory::getFlowController()->getFlowFileRepository()))
-		{
-			flow->setStoredToRepository(true);
-		}
-	}
-
-	// Notify receiving processor that work may be available
-	if(_destProcessor)
-	{
-		_destProcessor->notifyWork();
-	}
+void Connection::put(std::shared_ptr<core::FlowFile> flow) {
+  {
+    std::lock_guard<std::mutex> lock(mutex_);
+
+    queue_.push(flow);
+
+    queued_data_size_ += flow->getSize();
+
+    logger_->log_debug("Enqueue flow file UUID %s to connection %s",
+                       flow->getUUIDStr().c_str(), name_.c_str());
+  }
+
+  if (!flow->isStored()) {
+    // Save to the flowfile repo
+    FlowFileRecord event(flow_repository_,flow,this->uuidStr_);
+    if (event.Serialize()) {
+      flow->setStoredToRepository(true);
+    }
+  }
+
+  // Notify receiving processor that work may be available
+  if (dest_connectable_) {
+    dest_connectable_->notifyWork();
+  }
 }
 
-FlowFileRecord* Connection::poll(std::set<FlowFileRecord *> &expiredFlowRecords)
-{
-	std::lock_guard<std::mutex> lock(_mtx);
-
-	while (!_queue.empty())
-	{
-		FlowFileRecord *item = _queue.front();
-		_queue.pop();
-		_queuedDataSize -= item->getSize();
-
-		if (_expiredDuration > 0)
-		{
-			// We need to check for flow expiration
-			if (getTimeMillis() > (item->getEntryDate() + _expiredDuration))
-			{
-				// Flow record expired
-				expiredFlowRecords.insert(item);
-				if (FlowControllerFactory::getFlowController()->getFlowFileRepository() &&
-						FlowControllerFactory::getFlowController()->getFlowFileRepository()->isEnable())
-				{
-					// delete from the flowfile repo
-					FlowControllerFactory::getFlowController()->getFlowFileRepository()->Delete(item->getUUIDStr());
-					item->setStoredToRepository(false);
-				}
-			}
-			else
-			{
-				// Flow record not expired
-				if (item->isPenalized())
-				{
-					// Flow record was penalized
-					_queue.push(item);
-					_queuedDataSize += item->getSize();
-					break;
-				}
-				item->setOriginalConnection(this);
-				logger_->log_debug("Dequeue flow file UUID %s from connection %s",
-						item->getUUIDStr().c_str(), _name.c_str());
-				if (FlowControllerFactory::getFlowController()->getFlowFileRepository() &&
-						FlowControllerFactory::getFlowController()->getFlowFileRepository()->isEnable())
-				{
-					// delete from the flowfile repo
-					FlowControllerFactory::getFlowController()->getFlowFileRepository()->Delete(item->getUUIDStr());
-					item->setStoredToRepository(false);
-				}
-				return item;
-			}
-		}
-		else
-		{
-			// Flow record not expired
-			if (item->isPenalized())
-			{
-				// Flow record was penalized
-				_queue.push(item);
-				_queuedDataSize += item->getSize();
-				break;
-			}
-			item->setOriginalConnection(this);
-			logger_->log_debug("Dequeue flow file UUID %s from connection %s",
-					item->getUUIDStr().c_str(), _name.c_str());
-			if (FlowControllerFactory::getFlowController()->getFlowFileRepository() &&
-					FlowControllerFactory::getFlowController()->getFlowFileRepository()->isEnable())
-			{
-				// delete from the flowfile repo
-				FlowControllerFactory::getFlowController()->getFlowFileRepository()->Delete(item->getUUIDStr());
-				item->setStoredToRepository(false);
-			}
-			return item;
-		}
-	}
-
-	return NULL;
+std::shared_ptr<core::FlowFile> Connection::poll(
+    std::set<std::shared_ptr<core::FlowFile>> &expiredFlowRecords) {
+  std::lock_guard<std::mutex> lock(mutex_);
+
+  while (!queue_.empty()) {
+    std::shared_ptr<core::FlowFile> item = queue_.front();
+    queue_.pop();
+    queued_data_size_ -= item->getSize();
+
+    if (expired_duration_ > 0) {
+      // We need to check for flow expiration
+      if (getTimeMillis() > (item->getEntryDate() + expired_duration_)) {
+        // Flow record expired
+        expiredFlowRecords.insert(item);
+        if (flow_repository_->Delete(item->getUUIDStr())) {
+          item->setStoredToRepository(false);
+        }
+      } else {
+        // Flow record not expired
+        if (item->isPenalized()) {
+          // Flow record was penalized
+          queue_.push(item);
+          queued_data_size_ += item->getSize();
+          break;
+        }
+        std::shared_ptr<Connectable> connectable = std::static_pointer_cast<
+            Connectable>(shared_from_this());
+        item->setOriginalConnection(connectable);
+        logger_->log_debug("Dequeue flow file UUID %s from connection %s",
+                           item->getUUIDStr().c_str(), name_.c_str());
+
+        // delete from the flowfile repo
+        if (flow_repository_->Delete(item->getUUIDStr())) {
+          item->setStoredToRepository(false);
+        }
+
+        return item;
+      }
+    } else {
+      // Flow record not expired
+      if (item->isPenalized()) {
+        // Flow record was penalized
+        queue_.push(item);
+        queued_data_size_ += item->getSize();
+        break;
+      }
+      std::shared_ptr<Connectable> connectable = std::static_pointer_cast<
+          Connectable>(shared_from_this());
+      item->setOriginalConnection(connectable);
+      logger_->log_debug("Dequeue flow file UUID %s from connection %s",
+                         item->getUUIDStr().c_str(), name_.c_str());
+      // delete from the flowfile repo
+      if (flow_repository_->Delete(item->getUUIDStr())) {
+        item->setStoredToRepository(false);
+      }
+
+      return item;
+    }
+  }
+
+  return NULL;
 }
 
-void Connection::drain()
-{
-	std::lock_guard<std::mutex> lock(_mtx);
+void Connection::drain() {
+  std::lock_guard<std::mutex> lock(mutex_);
 
-	while (!_queue.empty())
-	{
-		FlowFileRecord *item = _queue.front();
-		_queue.pop();
-		delete item;
-	}
+  while (!queue_.empty()) {
+    auto &&item = queue_.front();
+    queue_.pop();
+  }
 
-	logger_->log_debug("Drain connection %s", _name.c_str());
+  logger_->log_debug("Drain connection %s", name_.c_str());
 }
+
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/EventDrivenSchedulingAgent.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/EventDrivenSchedulingAgent.cpp b/libminifi/src/EventDrivenSchedulingAgent.cpp
index 53bde4e..0484139 100644
--- a/libminifi/src/EventDrivenSchedulingAgent.cpp
+++ b/libminifi/src/EventDrivenSchedulingAgent.cpp
@@ -20,28 +20,43 @@
 #include <chrono>
 #include <thread>
 #include <iostream>
-#include "Property.h"
 #include "EventDrivenSchedulingAgent.h"
+#include "core/Processor.h"
+#include "core/ProcessContext.h"
+#include "core/ProcessSessionFactory.h"
+#include "core/Property.h"
 
-void EventDrivenSchedulingAgent::run(Processor *processor, ProcessContext *processContext, ProcessSessionFactory *sessionFactory)
-{
-	while (this->_running)
-	{
-		bool shouldYield = this->onTrigger(processor, processContext, sessionFactory);
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
 
-		if (processor->isYield())
-		{
-			// Honor the yield
-			std::this_thread::sleep_for(std::chrono::milliseconds(processor->getYieldTime()));
-		}
-		else if (shouldYield && this->_boredYieldDuration > 0)
-		{
-			// No work to do or need to apply back pressure
-			std::this_thread::sleep_for(std::chrono::milliseconds(this->_boredYieldDuration));
-		}
 
-		// Block until work is available
-		processor->waitForWork(1000);
-	}
-	return;
+void EventDrivenSchedulingAgent::run(
+    std::shared_ptr<core::Processor> processor,
+    core::ProcessContext *processContext,
+    core::ProcessSessionFactory *sessionFactory) {
+  while (this->running_) {
+    bool shouldYield = this->onTrigger(processor, processContext,
+                                       sessionFactory);
+
+    if (processor->isYield()) {
+      // Honor the yield
+      std::this_thread::sleep_for(
+          std::chrono::milliseconds(processor->getYieldTime()));
+    } else if (shouldYield && this->_boredYieldDuration > 0) {
+      // No work to do or need to apply back pressure
+      std::this_thread::sleep_for(
+          std::chrono::milliseconds(this->_boredYieldDuration));
+    }
+
+    // Block until work is available
+    processor->waitForWork(1000);
+  }
+  return;
 }
+
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/ExecuteProcess.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/ExecuteProcess.cpp b/libminifi/src/ExecuteProcess.cpp
deleted file mode 100644
index 61f96d5..0000000
--- a/libminifi/src/ExecuteProcess.cpp
+++ /dev/null
@@ -1,251 +0,0 @@
-/**
- * @file ExecuteProcess.cpp
- * ExecuteProcess class implementation
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#include "ExecuteProcess.h"
-#include "ProcessContext.h"
-#include "ProcessSession.h"
-#include <cstring>
-#include "utils/StringUtils.h"
-#include "utils/TimeUtil.h"
-
-const std::string ExecuteProcess::ProcessorName("ExecuteProcess");
-Property ExecuteProcess::Command("Command", "Specifies the command to be executed; if just the name of an executable is provided, it must be in the user's environment PATH.", "");
-Property ExecuteProcess::CommandArguments("Command Arguments",
-		"The arguments to supply to the executable delimited by white space. White space can be escaped by enclosing it in double-quotes.", "");
-Property ExecuteProcess::WorkingDir("Working Directory",
-		"The directory to use as the current working directory when executing the command", "");
-Property ExecuteProcess::BatchDuration("Batch Duration",
-		"If the process is expected to be long-running and produce textual output, a batch duration can be specified.", "0");
-Property ExecuteProcess::RedirectErrorStream("Redirect Error Stream",
-		"If true will redirect any error stream output of the process to the output stream.", "false");
-Relationship ExecuteProcess::Success("success", "All created FlowFiles are routed to this relationship.");
-
-void ExecuteProcess::initialize()
-{
-	//! Set the supported properties
-	std::set<Property> properties;
-	properties.insert(Command);
-	properties.insert(CommandArguments);
-	properties.insert(WorkingDir);
-	properties.insert(BatchDuration);
-	properties.insert(RedirectErrorStream);
-	setSupportedProperties(properties);
-	//! Set the supported relationships
-	std::set<Relationship> relationships;
-	relationships.insert(Success);
-	setSupportedRelationships(relationships);
-}
-
-
-void ExecuteProcess::onTrigger(ProcessContext *context, ProcessSession *session)
-{
-	std::string value;
-	if (context->getProperty(Command.getName(), value))
-	{
-		this->_command = value;
-	}
-	if (context->getProperty(CommandArguments.getName(), value))
-	{
-		this->_commandArgument = value;
-	}
-	if (context->getProperty(WorkingDir.getName(), value))
-	{
-		this->_workingDir = value;
-	}
-	if (context->getProperty(BatchDuration.getName(), value))
-	{
-		TimeUnit unit;
-		if (Property::StringToTime(value, _batchDuration, unit) &&
-			Property::ConvertTimeUnitToMS(_batchDuration, unit, _batchDuration))
-		{
-
-		}
-	}
-	if (context->getProperty(RedirectErrorStream.getName(), value))
-	{
-		StringUtils::StringToBool(value, _redirectErrorStream);
-	}
-	this->_fullCommand = _command + " " + _commandArgument;
-	if (_fullCommand.length() == 0)
-	{
-		yield();
-		return;
-	}
-	if (_workingDir.length() > 0 && _workingDir != ".")
-	{
-		// change to working directory
-		if (chdir(_workingDir.c_str()) != 0)
-		{
-			logger_->log_error("Execute Command can not chdir %s", _workingDir.c_str());
-			yield();
-			return;
-		}
-	}
-	logger_->log_info("Execute Command %s", _fullCommand.c_str());
-	// split the command into array
-	char cstr[_fullCommand.length()+1];
-	std::strcpy(cstr, _fullCommand.c_str());
-	char *p = std::strtok (cstr, " ");
-	int argc = 0;
-	char *argv[64];
-	while (p != 0 && argc < 64)
-	{
-		argv[argc] = p;
-		p = std::strtok(NULL, " ");
-		argc++;
-	}
-	argv[argc] = NULL;
-	int status, died;
-	if (!_processRunning)
-	{
-		_processRunning = true;
-		// if the process has not launched yet
-		// create the pipe
-		if (pipe(_pipefd) == -1)
-		{
-			_processRunning = false;
-			yield();
-			return;
-		}
-		switch (_pid = fork())
-		{
-		case -1:
-			logger_->log_error("Execute Process fork failed");
-			_processRunning = false;
-			close(_pipefd[0]);
-			close(_pipefd[1]);
-			yield();
-			break;
-		case 0 : // this is the code the child runs
-			close(1);      // close stdout
-			dup(_pipefd[1]); // points pipefd at file descriptor
-			if (_redirectErrorStream)
-				// redirect stderr
-				dup2(_pipefd[1], 2);
-			close(_pipefd[0]);
-			execvp(argv[0], argv);
-			exit(1);
-			break;
-		default: // this is the code the parent runs
-			// the parent isn't going to write to the pipe
-			close(_pipefd[1]);
-			if (_batchDuration > 0)
-			{
-				while (1)
-				{
-					std::this_thread::sleep_for(std::chrono::milliseconds(_batchDuration));
-					char buffer[4096];
-					int numRead = read(_pipefd[0], buffer, sizeof(buffer));
-					if (numRead <= 0)
-						break;
-					logger_->log_info("Execute Command Respond %d", numRead);
-					ExecuteProcess::WriteCallback callback(buffer, numRead);
-					FlowFileRecord *flowFile = session->create();
-					if (!flowFile)
-						continue;
-          flowFile->addAttribute("command", _command.c_str());
-          flowFile->addAttribute("command.arguments", _commandArgument.c_str());
-					session->write(flowFile, &callback);
-					session->transfer(flowFile, Success);
-					session->commit();
-				}
-			}
-			else
-			{
-				char buffer[4096];
-				char *bufPtr = buffer;
-				int totalRead = 0;
-				FlowFileRecord *flowFile = NULL;
-				while (1)
-				{
-					int numRead = read(_pipefd[0], bufPtr, (sizeof(buffer) - totalRead));
-					if (numRead <= 0)
-					{
-						if (totalRead > 0)
-						{
-							logger_->log_info("Execute Command Respond %d", totalRead);
-							// child exits and close the pipe
-							ExecuteProcess::WriteCallback callback(buffer, totalRead);
-							if (!flowFile)
-							{
-								flowFile = session->create();
-								if (!flowFile)
-									break;
-								flowFile->addAttribute("command", _command.c_str());
-								flowFile->addAttribute("command.arguments", _commandArgument.c_str());
-								session->write(flowFile, &callback);
-							}
-							else
-							{
-								session->append(flowFile, &callback);
-							}
-							session->transfer(flowFile, Success);
-						}
-						break;
-					}
-					else
-					{
-						if (numRead == (sizeof(buffer) - totalRead))
-						{
-							// we reach the max buffer size
-							logger_->log_info("Execute Command Max Respond %d", sizeof(buffer));
-							ExecuteProcess::WriteCallback callback(buffer, sizeof(buffer));
-							if (!flowFile)
-							{
-								flowFile = session->create();
-								if (!flowFile)
-									continue;
-								flowFile->addAttribute("command", _command.c_str());
-								flowFile->addAttribute("command.arguments", _commandArgument.c_str());
-								session->write(flowFile, &callback);
-							}
-							else
-							{
-								session->append(flowFile, &callback);
-							}
-							// Rewind
-							totalRead = 0;
-							bufPtr = buffer;
-						}
-						else
-						{
-							totalRead += numRead;
-							bufPtr += numRead;
-						}
-					}
-				}
-			}
-
-			died= wait(&status);
-			if (WIFEXITED(status))
-			{
-				logger_->log_info("Execute Command Complete %s status %d pid %d", _fullCommand.c_str(), WEXITSTATUS(status), _pid);
-			}
-			else
-			{
-				logger_->log_info("Execute Command Complete %s status %d pid %d", _fullCommand.c_str(), WTERMSIG(status), _pid);
-			}
-
-			close(_pipefd[0]);
-			_processRunning = false;
-			break;
-		}
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/FlowControlProtocol.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/FlowControlProtocol.cpp b/libminifi/src/FlowControlProtocol.cpp
index 22ef1f9..50fc0e2 100644
--- a/libminifi/src/FlowControlProtocol.cpp
+++ b/libminifi/src/FlowControlProtocol.cpp
@@ -27,493 +27,471 @@
 #include <iostream>
 #include "FlowController.h"
 #include "FlowControlProtocol.h"
-
-int FlowControlProtocol::connectServer(const char *host, uint16_t port)
-{
-	in_addr_t addr;
-	int sock = 0;
-	struct hostent *h;
+#include "core/core.h"
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+int FlowControlProtocol::connectServer(const char *host, uint16_t port) {
+  in_addr_t addr;
+  int sock = 0;
+  struct hostent *h;
 #ifdef __MACH__
-	h = gethostbyname(host);
+  h = gethostbyname(host);
 #else
-	char buf[1024];
-	struct hostent he;
-	int hh_errno;
-	gethostbyname_r(host, &he, buf, sizeof(buf), &h, &hh_errno);
+  char buf[1024];
+  struct hostent he;
+  int hh_errno;
+  gethostbyname_r(host, &he, buf, sizeof(buf), &h, &hh_errno);
 #endif
-	memcpy((char *) &addr, h->h_addr_list[0], h->h_length);
-	sock = socket(AF_INET, SOCK_STREAM, 0);
-	if (sock < 0)
-	{
-		logger_->log_error("Could not create socket to hostName %s", host);
-		return 0;
-	}
+  memcpy((char *) &addr, h->h_addr_list[0], h->h_length);
+  sock = socket(AF_INET, SOCK_STREAM, 0);
+  if (sock < 0) {
+    logger_->log_error("Could not create socket to hostName %s", host);
+    return 0;
+  }
 
 #ifndef __MACH__
-	int opt = 1;
-	bool nagle_off = true;
-
-	if (nagle_off)
-	{
-		if (setsockopt(sock, SOL_TCP, TCP_NODELAY, (void *)&opt, sizeof(opt)) < 0)
-		{
-			logger_->log_error("setsockopt() TCP_NODELAY failed");
-			close(sock);
-			return 0;
-		}
-		if (setsockopt(sock, SOL_SOCKET, SO_REUSEADDR,
-				(char *)&opt, sizeof(opt)) < 0)
-		{
-			logger_->log_error("setsockopt() SO_REUSEADDR failed");
-			close(sock);
-			return 0;
-		}
-	}
-
-	int sndsize = 256*1024;
-	if (setsockopt(sock, SOL_SOCKET, SO_SNDBUF, (char *)&sndsize, (int)sizeof(sndsize)) < 0)
-	{
-		logger_->log_error("setsockopt() SO_SNDBUF failed");
-		close(sock);
-		return 0;
-	}
+  int opt = 1;
+  bool nagle_off = true;
+
+  if (nagle_off)
+  {
+    if (setsockopt(sock, SOL_TCP, TCP_NODELAY, (void *)&opt, sizeof(opt)) < 0)
+    {
+      logger_->log_error("setsockopt() TCP_NODELAY failed");
+      close(sock);
+      return 0;
+    }
+    if (setsockopt(sock, SOL_SOCKET, SO_REUSEADDR,
+            (char *)&opt, sizeof(opt)) < 0)
+    {
+      logger_->log_error("setsockopt() SO_REUSEADDR failed");
+      close(sock);
+      return 0;
+    }
+  }
+
+  int sndsize = 256*1024;
+  if (setsockopt(sock, SOL_SOCKET, SO_SNDBUF, (char *)&sndsize, (int)sizeof(sndsize)) < 0)
+  {
+    logger_->log_error("setsockopt() SO_SNDBUF failed");
+    close(sock);
+    return 0;
+  }
 #endif
 
-	struct sockaddr_in sa;
-	socklen_t socklen;
-	int status;
-
-	memset(&sa, 0, sizeof(sa));
-	sa.sin_family = AF_INET;
-	sa.sin_addr.s_addr = htonl(INADDR_ANY);
-	sa.sin_port = htons(0);
-	socklen = sizeof(sa);
-	if (bind(sock, (struct sockaddr *)&sa, socklen) < 0)
-	{
-		logger_->log_error("socket bind failed");
-		close(sock);
-		return 0;
-	}
-
-	memset(&sa, 0, sizeof(sa));
-	sa.sin_family = AF_INET;
-	sa.sin_addr.s_addr = addr;
-	sa.sin_port = htons(port);
-	socklen = sizeof(sa);
-
-	status = connect(sock, (struct sockaddr *)&sa, socklen);
-
-	if (status < 0)
-	{
-		logger_->log_error("socket connect failed to %s %d", host, port);
-		close(sock);
-		return 0;
-	}
-
-	logger_->log_info("Flow Control Protocol socket %d connect to server %s port %d success", sock, host, port);
-
-	return sock;
+  struct sockaddr_in sa;
+  socklen_t socklen;
+  int status;
+
+  memset(&sa, 0, sizeof(sa));
+  sa.sin_family = AF_INET;
+  sa.sin_addr.s_addr = htonl(INADDR_ANY);
+  sa.sin_port = htons(0);
+  socklen = sizeof(sa);
+  if (bind(sock, (struct sockaddr *) &sa, socklen) < 0) {
+    logger_->log_error("socket bind failed");
+    close(sock);
+    return 0;
+  }
+
+  memset(&sa, 0, sizeof(sa));
+  sa.sin_family = AF_INET;
+  sa.sin_addr.s_addr = addr;
+  sa.sin_port = htons(port);
+  socklen = sizeof(sa);
+
+  status = connect(sock, (struct sockaddr *) &sa, socklen);
+
+  if (status < 0) {
+    logger_->log_error("socket connect failed to %s %d", host, port);
+    close(sock);
+    return 0;
+  }
+
+  logger_->log_info(
+      "Flow Control Protocol socket %d connect to server %s port %d success",
+      sock, host, port);
+
+  return sock;
 }
 
-int FlowControlProtocol::sendData(uint8_t *buf, int buflen)
-{
-	int ret = 0, bytes = 0;
-
-	while (bytes < buflen)
-	{
-		ret = send(_socket, buf+bytes, buflen-bytes, 0);
-		//check for errors
-		if (ret == -1)
-		{
-			return ret;
-		}
-		bytes+=ret;
-	}
-
-	return bytes;
+int FlowControlProtocol::sendData(uint8_t *buf, int buflen) {
+  int ret = 0, bytes = 0;
+
+  while (bytes < buflen) {
+    ret = send(_socket, buf + bytes, buflen - bytes, 0);
+    //check for errors
+    if (ret == -1) {
+      return ret;
+    }
+    bytes += ret;
+  }
+
+  return bytes;
 }
 
-int FlowControlProtocol::selectClient(int msec)
-{
-	fd_set fds;
-	struct timeval tv;
-    int retval;
-    int fd = _socket;
-
-    FD_ZERO(&fds);
-    FD_SET(fd, &fds);
-
-    tv.tv_sec = msec/1000;
-    tv.tv_usec = (msec % 1000) * 1000;
-
-    if (msec > 0)
-       retval = select(fd+1, &fds, NULL, NULL, &tv);
-    else
-       retval = select(fd+1, &fds, NULL, NULL, NULL);
-
-    if (retval <= 0)
-      return retval;
-    if (FD_ISSET(fd, &fds))
-      return retval;
-    else
-      return 0;
+int FlowControlProtocol::selectClient(int msec) {
+  fd_set fds;
+  struct timeval tv;
+  int retval;
+  int fd = _socket;
+
+  FD_ZERO(&fds);
+  FD_SET(fd, &fds);
+
+  tv.tv_sec = msec / 1000;
+  tv.tv_usec = (msec % 1000) * 1000;
+
+  if (msec > 0)
+    retval = select(fd + 1, &fds, NULL, NULL, &tv);
+  else
+    retval = select(fd + 1, &fds, NULL, NULL, NULL);
+
+  if (retval <= 0)
+    return retval;
+  if (FD_ISSET(fd, &fds))
+    return retval;
+  else
+    return 0;
 }
 
-int FlowControlProtocol::readData(uint8_t *buf, int buflen)
-{
-	int sendSize = buflen;
-
-	while (buflen)
-	{
-		int status;
-		status = selectClient(MAX_READ_TIMEOUT);
-		if (status <= 0)
-		{
-			return status;
-		}
+int FlowControlProtocol::readData(uint8_t *buf, int buflen) {
+  int sendSize = buflen;
+
+  while (buflen) {
+    int status;
+    status = selectClient(MAX_READ_TIMEOUT);
+    if (status <= 0) {
+      return status;
+    }
 #ifndef __MACH__
-		status = read(_socket, buf, buflen);
+    status = read(_socket, buf, buflen);
 #else
-		status = recv(_socket, buf, buflen, 0);
+    status = recv(_socket, buf, buflen, 0);
 #endif
-		if (status <= 0)
-		{
-			return status;
-		}
-		buflen -= status;
-		buf += status;
-	}
-
-	return sendSize;
+    if (status <= 0) {
+      return status;
+    }
+    buflen -= status;
+    buf += status;
+  }
+
+  return sendSize;
 }
 
-int FlowControlProtocol::readHdr(FlowControlProtocolHeader *hdr)
-{
-	uint8_t buffer[sizeof(FlowControlProtocolHeader)];
+int FlowControlProtocol::readHdr(FlowControlProtocolHeader *hdr) {
+  uint8_t buffer[sizeof(FlowControlProtocolHeader)];
 
-	uint8_t *data = buffer;
+  uint8_t *data = buffer;
 
-	int status = readData(buffer, sizeof(FlowControlProtocolHeader));
-	if (status <= 0)
-		return status;
+  int status = readData(buffer, sizeof(FlowControlProtocolHeader));
+  if (status <= 0)
+    return status;
 
-	uint32_t value;
-	data = this->decode(data, value);
-	hdr->msgType = value;
+  uint32_t value;
+  data = this->decode(data, value);
+  hdr->msgType = value;
 
-	data = this->decode(data, value);
-	hdr->seqNumber = value;
+  data = this->decode(data, value);
+  hdr->seqNumber = value;
 
-	data = this->decode(data, value);
-	hdr->status = value;
+  data = this->decode(data, value);
+  hdr->status = value;
 
-	data = this->decode(data, value);
-	hdr->payloadLen = value;
+  data = this->decode(data, value);
+  hdr->payloadLen = value;
 
-	return sizeof(FlowControlProtocolHeader);
+  return sizeof(FlowControlProtocolHeader);
 }
 
-void FlowControlProtocol::start()
-{
-	if (_reportInterval <= 0)
-		return;
-	if (_running)
-		return;
-	_running = true;
-	logger_->log_info("FlowControl Protocol Start");
-	_thread = new std::thread(run, this);
-	_thread->detach();
+void FlowControlProtocol::start() {
+  if (_reportInterval <= 0)
+    return;
+  if (running_)
+    return;
+  running_ = true;
+  logger_->log_info("FlowControl Protocol Start");
+  _thread = new std::thread(run, this);
+  _thread->detach();
 }
 
-void FlowControlProtocol::stop()
-{
-	if (!_running)
-		return;
-	_running = false;
-	logger_->log_info("FlowControl Protocol Stop");
+void FlowControlProtocol::stop() {
+  if (!running_)
+    return;
+  running_ = false;
+  logger_->log_info("FlowControl Protocol Stop");
 }
 
-void FlowControlProtocol::run(FlowControlProtocol *protocol)
-{
-	while (protocol->_running)
-	{
-		std::this_thread::sleep_for(std::chrono::milliseconds(protocol->_reportInterval));
-		if (!protocol->_registered)
-		{
-			// if it is not register yet
-			protocol->sendRegisterReq();
-		}
-		else
-			protocol->sendReportReq();
-	}
-	return;
+void FlowControlProtocol::run(FlowControlProtocol *protocol) {
+  while (protocol->running_) {
+    std::this_thread::sleep_for(
+        std::chrono::milliseconds(protocol->_reportInterval));
+    if (!protocol->_registered) {
+      // if it is not register yet
+      protocol->sendRegisterReq();
+    } else
+      protocol->sendReportReq();
+  }
+  return;
 }
 
-int FlowControlProtocol::sendRegisterReq()
-{
-	if (_registered)
-	{
-		logger_->log_info("Already registered");
-		return -1;
-	}
-
-	uint16_t port = this->_serverPort;
-
-	if (this->_socket <= 0)
-		this->_socket = connectServer(_serverName.c_str(), port);
-
-	if (this->_socket <= 0)
-		return -1;
-
-	// Calculate the total payload msg size
-	uint32_t payloadSize = FlowControlMsgIDEncodingLen(FLOW_SERIAL_NUMBER, 0) +
-			FlowControlMsgIDEncodingLen(FLOW_YML_NAME, this->_controller->getName().size()+1);
-	uint32_t size = sizeof(FlowControlProtocolHeader) + payloadSize;
-
-	uint8_t *data = new uint8_t[size];
-	uint8_t *start = data;
-
-	// encode the HDR
-	FlowControlProtocolHeader hdr;
-	hdr.msgType = REGISTER_REQ;
-	hdr.payloadLen = payloadSize;
-	hdr.seqNumber  = this->_seqNumber;
-	hdr.status = RESP_SUCCESS;
-	data = this->encode(data, hdr.msgType);
-	data = this->encode(data, hdr.seqNumber);
-	data = this->encode(data, hdr.status);
-	data = this->encode(data, hdr.payloadLen);
-
-	// encode the serial number
-	data = this->encode(data, FLOW_SERIAL_NUMBER);
-	data = this->encode(data, this->_serialNumber, 8);
-
-	// encode the YAML name
-	data = this->encode(data, FLOW_YML_NAME);
-	data = this->encode(data, this->_controller->getName());
-
-	// send it
-	int status = sendData(start, size);
-	delete[] start;
-	if (status <= 0)
-	{
-		close(_socket);
-		_socket = 0;
-		logger_->log_error("Flow Control Protocol Send Register Req failed");
-		return -1;
-	}
-
-	// Looking for register respond
-	status = readHdr(&hdr);
-
-	if (status <= 0)
-	{
-		close(_socket);
-		_socket = 0;
-		logger_->log_error("Flow Control Protocol Read Register Resp header failed");
-		return -1;
-	}
-	logger_->log_info("Flow Control Protocol receive MsgType %s", FlowControlMsgTypeToStr((FlowControlMsgType) hdr.msgType));
-	logger_->log_info("Flow Control Protocol receive Seq Num %d", hdr.seqNumber);
-	logger_->log_info("Flow Control Protocol receive Resp Code %s", FlowControlRespCodeToStr((FlowControlRespCode) hdr.status));
-	logger_->log_info("Flow Control Protocol receive Payload len %d", hdr.payloadLen);
-
-	if (hdr.status == RESP_SUCCESS && hdr.seqNumber == this->_seqNumber)
-	{
-		this->_registered = true;
-		this->_seqNumber++;
-		logger_->log_info("Flow Control Protocol Register success");
-		uint8_t *payload = new uint8_t[hdr.payloadLen];
-		uint8_t *payloadPtr = payload;
-		status = readData(payload, hdr.payloadLen);
-		if (status <= 0)
-		{
-			delete[] payload;
-			logger_->log_info("Flow Control Protocol Register Read Payload fail");
-			close(_socket);
-			_socket = 0;
-			return -1;
-		}
-		while (payloadPtr < (payload + hdr.payloadLen))
-		{
-			uint32_t msgID;
-			payloadPtr = this->decode(payloadPtr, msgID);
-			if (((FlowControlMsgID) msgID) == REPORT_INTERVAL)
-			{
-				// Fixed 4 bytes
-				uint32_t reportInterval;
-				payloadPtr = this->decode(payloadPtr, reportInterval);
-				logger_->log_info("Flow Control Protocol receive report interval %d ms", reportInterval);
-				this->_reportInterval = reportInterval;
-			}
-			else
-			{
-				break;
-			}
-		}
-		delete[] payload;
-		close(_socket);
-		_socket = 0;
-		return 0;
-	}
-	else
-	{
-		logger_->log_info("Flow Control Protocol Register fail");
-		close(_socket);
-		_socket = 0;
-		return -1;
-	}
+int FlowControlProtocol::sendRegisterReq() {
+  if (_registered) {
+    logger_->log_info("Already registered");
+    return -1;
+  }
+
+  uint16_t port = this->_serverPort;
+
+  if (this->_socket <= 0)
+    this->_socket = connectServer(_serverName.c_str(), port);
+
+  if (this->_socket <= 0)
+    return -1;
+
+  // Calculate the total payload msg size
+  uint32_t payloadSize = FlowControlMsgIDEncodingLen(FLOW_SERIAL_NUMBER, 0)
+      + FlowControlMsgIDEncodingLen(FLOW_YML_NAME,
+                                    this->_controller->getName().size() + 1);
+  uint32_t size = sizeof(FlowControlProtocolHeader) + payloadSize;
+
+  uint8_t *data = new uint8_t[size];
+  uint8_t *start = data;
+
+  // encode the HDR
+  FlowControlProtocolHeader hdr;
+  hdr.msgType = REGISTER_REQ;
+  hdr.payloadLen = payloadSize;
+  hdr.seqNumber = this->_seqNumber;
+  hdr.status = RESP_SUCCESS;
+  data = this->encode(data, hdr.msgType);
+  data = this->encode(data, hdr.seqNumber);
+  data = this->encode(data, hdr.status);
+  data = this->encode(data, hdr.payloadLen);
+
+  // encode the serial number
+  data = this->encode(data, FLOW_SERIAL_NUMBER);
+  data = this->encode(data, this->_serialNumber, 8);
+
+  // encode the YAML name
+  data = this->encode(data, FLOW_YML_NAME);
+  data = this->encode(data, this->_controller->getName());
+
+  // send it
+  int status = sendData(start, size);
+  delete[] start;
+  if (status <= 0) {
+    close(_socket);
+    _socket = 0;
+    logger_->log_error("Flow Control Protocol Send Register Req failed");
+    return -1;
+  }
+
+  // Looking for register respond
+  status = readHdr(&hdr);
+
+  if (status <= 0) {
+    close(_socket);
+    _socket = 0;
+    logger_->log_error(
+        "Flow Control Protocol Read Register Resp header failed");
+    return -1;
+  }
+  logger_->log_info("Flow Control Protocol receive MsgType %s",
+                    FlowControlMsgTypeToStr((FlowControlMsgType) hdr.msgType));
+  logger_->log_info("Flow Control Protocol receive Seq Num %d", hdr.seqNumber);
+  logger_->log_info("Flow Control Protocol receive Resp Code %s",
+                    FlowControlRespCodeToStr((FlowControlRespCode) hdr.status));
+  logger_->log_info("Flow Control Protocol receive Payload len %d",
+                    hdr.payloadLen);
+
+  if (hdr.status == RESP_SUCCESS && hdr.seqNumber == this->_seqNumber) {
+    this->_registered = true;
+    this->_seqNumber++;
+    logger_->log_info("Flow Control Protocol Register success");
+    uint8_t *payload = new uint8_t[hdr.payloadLen];
+    uint8_t *payloadPtr = payload;
+    status = readData(payload, hdr.payloadLen);
+    if (status <= 0) {
+      delete[] payload;
+      logger_->log_info("Flow Control Protocol Register Read Payload fail");
+      close(_socket);
+      _socket = 0;
+      return -1;
+    }
+    while (payloadPtr < (payload + hdr.payloadLen)) {
+      uint32_t msgID;
+      payloadPtr = this->decode(payloadPtr, msgID);
+      if (((FlowControlMsgID) msgID) == REPORT_INTERVAL) {
+        // Fixed 4 bytes
+        uint32_t reportInterval;
+        payloadPtr = this->decode(payloadPtr, reportInterval);
+        logger_->log_info("Flow Control Protocol receive report interval %d ms",
+                          reportInterval);
+        this->_reportInterval = reportInterval;
+      } else {
+        break;
+      }
+    }
+    delete[] payload;
+    close(_socket);
+    _socket = 0;
+    return 0;
+  } else {
+    logger_->log_info("Flow Control Protocol Register fail");
+    close(_socket);
+    _socket = 0;
+    return -1;
+  }
 }
 
-
-int FlowControlProtocol::sendReportReq()
-{
-	uint16_t port = this->_serverPort;
-
-	if (this->_socket <= 0)
-		this->_socket = connectServer(_serverName.c_str(), port);
-
-	if (this->_socket <= 0)
-		return -1;
-
-	// Calculate the total payload msg size
-	uint32_t payloadSize =
-			FlowControlMsgIDEncodingLen(FLOW_YML_NAME, this->_controller->getName().size()+1);
-	uint32_t size = sizeof(FlowControlProtocolHeader) + payloadSize;
-
-	uint8_t *data = new uint8_t[size];
-	uint8_t *start = data;
-
-	// encode the HDR
-	FlowControlProtocolHeader hdr;
-	hdr.msgType = REPORT_REQ;
-	hdr.payloadLen = payloadSize;
-	hdr.seqNumber  = this->_seqNumber;
-	hdr.status = RESP_SUCCESS;
-	data = this->encode(data, hdr.msgType);
-	data = this->encode(data, hdr.seqNumber);
-	data = this->encode(data, hdr.status);
-	data = this->encode(data, hdr.payloadLen);
-
-	// encode the YAML name
-	data = this->encode(data, FLOW_YML_NAME);
-	data = this->encode(data, this->_controller->getName());
-
-	// send it
-	int status = sendData(start, size);
-	delete[] start;
-	if (status <= 0)
-	{
-		close(_socket);
-		_socket = 0;
-		logger_->log_error("Flow Control Protocol Send Report Req failed");
-		return -1;
-	}
-
-	// Looking for report respond
-	status = readHdr(&hdr);
-
-	if (status <= 0)
-	{
-		close(_socket);
-		_socket = 0;
-		logger_->log_error("Flow Control Protocol Read Report Resp header failed");
-		return -1;
-	}
-	logger_->log_info("Flow Control Protocol receive MsgType %s", FlowControlMsgTypeToStr((FlowControlMsgType) hdr.msgType));
-	logger_->log_info("Flow Control Protocol receive Seq Num %d", hdr.seqNumber);
-	logger_->log_info("Flow Control Protocol receive Resp Code %s", FlowControlRespCodeToStr((FlowControlRespCode) hdr.status));
-	logger_->log_info("Flow Control Protocol receive Payload len %d", hdr.payloadLen);
-
-	if (hdr.status == RESP_SUCCESS && hdr.seqNumber == this->_seqNumber)
-	{
-		this->_seqNumber++;
-		uint8_t *payload = new uint8_t[hdr.payloadLen];
-		uint8_t *payloadPtr = payload;
-		status = readData(payload, hdr.payloadLen);
-		if (status <= 0)
-		{
-			delete[] payload;
-			logger_->log_info("Flow Control Protocol Report Resp Read Payload fail");
-			close(_socket);
-			_socket = 0;
-			return -1;
-		}
-		std::string processor;
-		std::string propertyName;
-		std::string propertyValue;
-		while (payloadPtr < (payload + hdr.payloadLen))
-		{
-			uint32_t msgID;
-			payloadPtr = this->decode(payloadPtr, msgID);
-			if (((FlowControlMsgID) msgID) == PROCESSOR_NAME)
-			{
-				uint32_t len;
-				payloadPtr = this->decode(payloadPtr, len);
-				processor = (const char *) payloadPtr;
-				payloadPtr += len;
-				logger_->log_info("Flow Control Protocol receive report resp processor %s", processor.c_str());
-			}
-			else if (((FlowControlMsgID) msgID) == PROPERTY_NAME)
-			{
-				uint32_t len;
-				payloadPtr = this->decode(payloadPtr, len);
-				propertyName = (const char *) payloadPtr;
-				payloadPtr += len;
-				logger_->log_info("Flow Control Protocol receive report resp property name %s", propertyName.c_str());
-			}
-			else if (((FlowControlMsgID) msgID) == PROPERTY_VALUE)
-			{
-				uint32_t len;
-				payloadPtr = this->decode(payloadPtr, len);
-				propertyValue = (const char *) payloadPtr;
-				payloadPtr += len;
-				logger_->log_info("Flow Control Protocol receive report resp property value %s", propertyValue.c_str());
-				this->_controller->updatePropertyValue(processor, propertyName, propertyValue);
-			}
-			else
-			{
-				break;
-			}
-		}
-		delete[] payload;
-		close(_socket);
-		_socket = 0;
-		return 0;
-	}
-	else if (hdr.status == RESP_TRIGGER_REGISTER && hdr.seqNumber == this->_seqNumber)
-	{
-		logger_->log_info("Flow Control Protocol trigger reregister");
-		this->_registered = false;
-		this->_seqNumber++;
-		close(_socket);
-		_socket = 0;
-		return 0;
-	}
-	else if (hdr.status == RESP_STOP_FLOW_CONTROLLER && hdr.seqNumber == this->_seqNumber)
-	{
-		logger_->log_info("Flow Control Protocol stop flow controller");
-		this->_controller->stop(true);
-		this->_seqNumber++;
-		close(_socket);
-		_socket = 0;
-		return 0;
-	}
-	else if (hdr.status == RESP_START_FLOW_CONTROLLER && hdr.seqNumber == this->_seqNumber)
-	{
-		logger_->log_info("Flow Control Protocol start flow controller");
-		this->_controller->start();
-		this->_seqNumber++;
-		close(_socket);
-		_socket = 0;
-		return 0;
-	}
-	else
-	{
-		logger_->log_info("Flow Control Protocol Report fail");
-		close(_socket);
-		_socket = 0;
-		return -1;
-	}
+int FlowControlProtocol::sendReportReq() {
+  uint16_t port = this->_serverPort;
+
+  if (this->_socket <= 0)
+    this->_socket = connectServer(_serverName.c_str(), port);
+
+  if (this->_socket <= 0)
+    return -1;
+
+  // Calculate the total payload msg size
+  uint32_t payloadSize = FlowControlMsgIDEncodingLen(
+      FLOW_YML_NAME, this->_controller->getName().size() + 1);
+  uint32_t size = sizeof(FlowControlProtocolHeader) + payloadSize;
+
+  uint8_t *data = new uint8_t[size];
+  uint8_t *start = data;
+
+  // encode the HDR
+  FlowControlProtocolHeader hdr;
+  hdr.msgType = REPORT_REQ;
+  hdr.payloadLen = payloadSize;
+  hdr.seqNumber = this->_seqNumber;
+  hdr.status = RESP_SUCCESS;
+  data = this->encode(data, hdr.msgType);
+  data = this->encode(data, hdr.seqNumber);
+  data = this->encode(data, hdr.status);
+  data = this->encode(data, hdr.payloadLen);
+
+  // encode the YAML name
+  data = this->encode(data, FLOW_YML_NAME);
+  data = this->encode(data, this->_controller->getName());
+
+  // send it
+  int status = sendData(start, size);
+  delete[] start;
+  if (status <= 0) {
+    close(_socket);
+    _socket = 0;
+    logger_->log_error("Flow Control Protocol Send Report Req failed");
+    return -1;
+  }
+
+  // Looking for report respond
+  status = readHdr(&hdr);
+
+  if (status <= 0) {
+    close(_socket);
+    _socket = 0;
+    logger_->log_error("Flow Control Protocol Read Report Resp header failed");
+    return -1;
+  }
+  logger_->log_info("Flow Control Protocol receive MsgType %s",
+                    FlowControlMsgTypeToStr((FlowControlMsgType) hdr.msgType));
+  logger_->log_info("Flow Control Protocol receive Seq Num %d", hdr.seqNumber);
+  logger_->log_info("Flow Control Protocol receive Resp Code %s",
+                    FlowControlRespCodeToStr((FlowControlRespCode) hdr.status));
+  logger_->log_info("Flow Control Protocol receive Payload len %d",
+                    hdr.payloadLen);
+
+  if (hdr.status == RESP_SUCCESS && hdr.seqNumber == this->_seqNumber) {
+    this->_seqNumber++;
+    uint8_t *payload = new uint8_t[hdr.payloadLen];
+    uint8_t *payloadPtr = payload;
+    status = readData(payload, hdr.payloadLen);
+    if (status <= 0) {
+      delete[] payload;
+      logger_->log_info("Flow Control Protocol Report Resp Read Payload fail");
+      close(_socket);
+      _socket = 0;
+      return -1;
+    }
+    std::string processor;
+    std::string propertyName;
+    std::string propertyValue;
+    while (payloadPtr < (payload + hdr.payloadLen)) {
+      uint32_t msgID;
+      payloadPtr = this->decode(payloadPtr, msgID);
+      if (((FlowControlMsgID) msgID) == PROCESSOR_NAME) {
+        uint32_t len;
+        payloadPtr = this->decode(payloadPtr, len);
+        processor = (const char *) payloadPtr;
+        payloadPtr += len;
+        logger_->log_info(
+            "Flow Control Protocol receive report resp processor %s",
+            processor.c_str());
+      } else if (((FlowControlMsgID) msgID) == PROPERTY_NAME) {
+        uint32_t len;
+        payloadPtr = this->decode(payloadPtr, len);
+        propertyName = (const char *) payloadPtr;
+        payloadPtr += len;
+        logger_->log_info(
+            "Flow Control Protocol receive report resp property name %s",
+            propertyName.c_str());
+      } else if (((FlowControlMsgID) msgID) == PROPERTY_VALUE) {
+        uint32_t len;
+        payloadPtr = this->decode(payloadPtr, len);
+        propertyValue = (const char *) payloadPtr;
+        payloadPtr += len;
+        logger_->log_info(
+            "Flow Control Protocol receive report resp property value %s",
+            propertyValue.c_str());
+        this->_controller->updatePropertyValue(processor, propertyName,
+                                               propertyValue);
+      } else {
+        break;
+      }
+    }
+    delete[] payload;
+    close(_socket);
+    _socket = 0;
+    return 0;
+  } else if (hdr.status == RESP_TRIGGER_REGISTER
+      && hdr.seqNumber == this->_seqNumber) {
+    logger_->log_info("Flow Control Protocol trigger reregister");
+    this->_registered = false;
+    this->_seqNumber++;
+    close(_socket);
+    _socket = 0;
+    return 0;
+  } else if (hdr.status == RESP_STOP_FLOW_CONTROLLER
+      && hdr.seqNumber == this->_seqNumber) {
+    logger_->log_info("Flow Control Protocol stop flow controller");
+    this->_controller->stop(true);
+    this->_seqNumber++;
+    close(_socket);
+    _socket = 0;
+    return 0;
+  } else if (hdr.status == RESP_START_FLOW_CONTROLLER
+      && hdr.seqNumber == this->_seqNumber) {
+    logger_->log_info("Flow Control Protocol start flow controller");
+    this->_controller->start();
+    this->_seqNumber++;
+    close(_socket);
+    _socket = 0;
+    return 0;
+  } else {
+    logger_->log_info("Flow Control Protocol Report fail");
+    close(_socket);
+    _socket = 0;
+    return -1;
+  }
 }
 
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */


[08/16] nifi-minifi-cpp git commit: MINIFI-217: Updates namespaces and removes use of raw pointers for user facing API.

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/ListenSyslog.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/ListenSyslog.cpp b/libminifi/src/ListenSyslog.cpp
deleted file mode 100644
index f2901e0..0000000
--- a/libminifi/src/ListenSyslog.cpp
+++ /dev/null
@@ -1,343 +0,0 @@
-/**
- * @file ListenSyslog.cpp
- * ListenSyslog class implementation
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#include <queue>
-#include <stdio.h>
-#include <string>
-#include "utils/TimeUtil.h"
-#include "utils/StringUtils.h"
-#include "ListenSyslog.h"
-#include "ProcessContext.h"
-#include "ProcessSession.h"
-
-const std::string ListenSyslog::ProcessorName("ListenSyslog");
-Property ListenSyslog::RecvBufSize("Receive Buffer Size", "The size of each buffer used to receive Syslog messages.", "65507 B");
-Property ListenSyslog::MaxSocketBufSize("Max Size of Socket Buffer", "The maximum size of the socket buffer that should be used.", "1 MB");
-Property ListenSyslog::MaxConnections("Max Number of TCP Connections", "The maximum number of concurrent connections to accept Syslog messages in TCP mode.", "2");
-Property ListenSyslog::MaxBatchSize("Max Batch Size",
-		"The maximum number of Syslog events to add to a single FlowFile.", "1");
-Property ListenSyslog::MessageDelimiter("Message Delimiter",
-		"Specifies the delimiter to place between Syslog messages when multiple messages are bundled together (see <Max Batch Size> property).", "\n");
-Property ListenSyslog::ParseMessages("Parse Messages",
-		"Indicates if the processor should parse the Syslog messages. If set to false, each outgoing FlowFile will only.", "false");
-Property ListenSyslog::Protocol("Protocol", "The protocol for Syslog communication.", "UDP");
-Property ListenSyslog::Port("Port", "The port for Syslog communication.", "514");
-Relationship ListenSyslog::Success("success", "All files are routed to success");
-Relationship ListenSyslog::Invalid("invalid", "SysLog message format invalid");
-
-void ListenSyslog::initialize()
-{
-	//! Set the supported properties
-	std::set<Property> properties;
-	properties.insert(RecvBufSize);
-	properties.insert(MaxSocketBufSize);
-	properties.insert(MaxConnections);
-	properties.insert(MaxBatchSize);
-	properties.insert(MessageDelimiter);
-	properties.insert(ParseMessages);
-	properties.insert(Protocol);
-	properties.insert(Port);
-	setSupportedProperties(properties);
-	//! Set the supported relationships
-	std::set<Relationship> relationships;
-	relationships.insert(Success);
-	relationships.insert(Invalid);
-	setSupportedRelationships(relationships);
-}
-
-void ListenSyslog::startSocketThread()
-{
-	if (_thread != NULL)
-		return;
-
-	logger_->log_info("ListenSysLog Socket Thread Start");
-	_serverTheadRunning = true;
-	_thread = new std::thread(run, this);
-	_thread->detach();
-}
-
-void ListenSyslog::run(ListenSyslog *process)
-{
-	process->runThread();
-}
-
-void ListenSyslog::runThread()
-{
-	while (_serverTheadRunning)
-	{
-		if (_resetServerSocket)
-		{
-			_resetServerSocket = false;
-			// need to reset the socket
-			std::vector<int>::iterator it;
-			for (it = _clientSockets.begin(); it != _clientSockets.end(); ++it)
-			{
-				int clientSocket = *it;
-				close(clientSocket);
-			}
-			_clientSockets.clear();
-			if (_serverSocket > 0)
-			{
-				close(_serverSocket);
-				_serverSocket = 0;
-			}
-		}
-
-		if (_serverSocket <= 0)
-		{
-			uint16_t portno = _port;
-			struct sockaddr_in serv_addr;
-			int sockfd;
-			if (_protocol == "TCP")
-				sockfd = socket(AF_INET, SOCK_STREAM, 0);
-			else
-				sockfd = socket(AF_INET, SOCK_DGRAM, 0);
-			if (sockfd < 0)
-			{
-				logger_->log_info("ListenSysLog Server socket creation failed");
-				break;
-			}
-			bzero((char *) &serv_addr, sizeof(serv_addr));
-			serv_addr.sin_family = AF_INET;
-			serv_addr.sin_addr.s_addr = INADDR_ANY;
-			serv_addr.sin_port = htons(portno);
-			if (bind(sockfd, (struct sockaddr *) &serv_addr,
-					sizeof(serv_addr)) < 0)
-			{
-				logger_->log_error("ListenSysLog Server socket bind failed");
-				break;
-			}
-			if (_protocol == "TCP")
-				listen(sockfd,5);
-			_serverSocket = sockfd;
-			logger_->log_error("ListenSysLog Server socket %d bind OK to port %d", _serverSocket, portno);
-		}
-		FD_ZERO(&_readfds);
-		FD_SET(_serverSocket, &_readfds);
-		_maxFds = _serverSocket;
-		std::vector<int>::iterator it;
-		for (it = _clientSockets.begin(); it != _clientSockets.end(); ++it)
-		{
-			int clientSocket = *it;
-			if (clientSocket >= _maxFds)
-				_maxFds = clientSocket;
-			FD_SET(clientSocket, &_readfds);
-		}
-		fd_set fds;
-		struct timeval tv;
-		int retval;
-		fds = _readfds;
-		tv.tv_sec = 0;
-		// 100 msec
-		tv.tv_usec = 100000;
-		retval = select(_maxFds+1, &fds, NULL, NULL, &tv);
-		if (retval < 0)
-			break;
-		if (retval == 0)
-			continue;
-		if (FD_ISSET(_serverSocket, &fds))
-		{
-			// server socket, either we have UDP datagram or TCP connection request
-			if (_protocol == "TCP")
-			{
-				socklen_t clilen;
-				struct sockaddr_in cli_addr;
-				clilen = sizeof(cli_addr);
-				int newsockfd = accept(_serverSocket,
-						(struct sockaddr *) &cli_addr,
-						&clilen);
-				if (newsockfd > 0)
-				{
-					if (_clientSockets.size() < _maxConnections)
-					{
-						_clientSockets.push_back(newsockfd);
-						logger_->log_info("ListenSysLog new client socket %d connection", newsockfd);
-						continue;
-					}
-					else
-					{
-						close(newsockfd);
-					}
-				}
-			}
-			else
-			{
-				socklen_t clilen;
-				struct sockaddr_in cli_addr;
-				clilen = sizeof(cli_addr);
-				int recvlen = recvfrom(_serverSocket, _buffer, sizeof(_buffer), 0,
-						(struct sockaddr *)&cli_addr, &clilen);
-				if (recvlen > 0 && (recvlen + getEventQueueByteSize()) <= _recvBufSize)
-				{
-					uint8_t *payload = new uint8_t[recvlen];
-					memcpy(payload, _buffer, recvlen);
-					putEvent(payload, recvlen);
-				}
-			}
-		}
-		it = _clientSockets.begin();
-		while (it != _clientSockets.end())
-		{
-			int clientSocket = *it;
-			if (FD_ISSET(clientSocket, &fds))
-			{
-				int recvlen = readline(clientSocket, (char *)_buffer, sizeof(_buffer));
-				if (recvlen <= 0)
-				{
-					close(clientSocket);
-					logger_->log_info("ListenSysLog client socket %d close", clientSocket);
-					it = _clientSockets.erase(it);
-				}
-				else
-				{
-					if ((recvlen + getEventQueueByteSize()) <= _recvBufSize)
-					{
-						uint8_t *payload = new uint8_t[recvlen];
-						memcpy(payload, _buffer, recvlen);
-						putEvent(payload, recvlen);
-					}
-					++it;
-				}
-			}
-		}
-	}
-	return;
-}
-
-
-int ListenSyslog::readline( int fd, char *bufptr, size_t len )
-{
-	char *bufx = bufptr;
-	static char *bp;
-	static int cnt = 0;
-	static char b[ 2048 ];
-	char c;
-
-	while ( --len > 0 )
-    {
-      if ( --cnt <= 0 )
-      {
-    	  cnt = recv( fd, b, sizeof( b ), 0 );
-    	  if ( cnt < 0 )
-    	  {
-    		  if ( errno == EINTR )
-    		  {
-    			  len++;		/* the while will decrement */
-    			  continue;
-    		  }
-    		  return -1;
-    	  }
-    	  if ( cnt == 0 )
-    		  return 0;
-    	  bp = b;
-      }
-      c = *bp++;
-      *bufptr++ = c;
-      if ( c == '\n' )
-      {
-    	  *bufptr = '\n';
-    	  return bufptr - bufx + 1;
-      }
-    }
-	return -1;
-}
-
-void ListenSyslog::onTrigger(ProcessContext *context, ProcessSession *session)
-{
-	std::string value;
-	bool needResetServerSocket = false;
-	if (context->getProperty(Protocol.getName(), value))
-	{
-		if (_protocol != value)
-			needResetServerSocket = true;
-		_protocol = value;
-	}
-	if (context->getProperty(RecvBufSize.getName(), value))
-	{
-		Property::StringToInt(value, _recvBufSize);
-	}
-	if (context->getProperty(MaxSocketBufSize.getName(), value))
-	{
-		Property::StringToInt(value, _maxSocketBufSize);
-	}
-	if (context->getProperty(MaxConnections.getName(), value))
-	{
-		Property::StringToInt(value, _maxConnections);
-	}
-	if (context->getProperty(MessageDelimiter.getName(), value))
-	{
-		_messageDelimiter = value;
-	}
-	if (context->getProperty(ParseMessages.getName(), value))
-	{
-		StringUtils::StringToBool(value, _parseMessages);
-	}
-	if (context->getProperty(Port.getName(), value))
-	{
-		int64_t oldPort = _port;
-		Property::StringToInt(value, _port);
-		if (_port != oldPort)
-			needResetServerSocket = true;
-	}
-	if (context->getProperty(MaxBatchSize.getName(), value))
-	{
-		Property::StringToInt(value, _maxBatchSize);
-	}
-
-	if (needResetServerSocket)
-		_resetServerSocket = true;
-
-	startSocketThread();
-
-	// read from the event queue
-	if (isEventQueueEmpty())
-	{
-		context->yield();
-		return;
-	}
-
-	std::queue<SysLogEvent> eventQueue;
-	pollEvent(eventQueue, _maxBatchSize);
-	bool firstEvent = true;
-	FlowFileRecord *flowFile = NULL;
-	while(!eventQueue.empty())
-	{
-		SysLogEvent event = eventQueue.front();
-		eventQueue.pop();
-		if (firstEvent)
-		{
-			flowFile = session->create();
-			if (!flowFile)
-				return;
-			ListenSyslog::WriteCallback callback((char *)event.payload, event.len);
-			session->write(flowFile, &callback);
-			delete[] event.payload;
-			firstEvent = false;
-		}
-		else
-		{
-			ListenSyslog::WriteCallback callback((char *)event.payload, event.len);
-			session->append(flowFile, &callback);
-			delete[] event.payload;
-		}
-	}
-	flowFile->addAttribute("syslog.protocol", _protocol);
-	flowFile->addAttribute("syslog.port", std::to_string(_port));
-	session->transfer(flowFile, Success);
-}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/LogAppenders.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/LogAppenders.cpp b/libminifi/src/LogAppenders.cpp
deleted file mode 100644
index c90588d..0000000
--- a/libminifi/src/LogAppenders.cpp
+++ /dev/null
@@ -1,25 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#include "../include/LogAppenders.h"
-
-const char *OutputStreamAppender::nifi_log_output_stream_error_stderr="nifi.log.outputstream.appender.error.stderr";
-
-const char *RollingAppender::nifi_log_rolling_apender_file = "nifi.log.rolling.appender.file";
-const char *RollingAppender::nifi_log_rolling_appender_max_files = "nifi.log.rolling.appender.max.files";
-const char *RollingAppender::nifi_log_rolling_appender_max_file_size = "nifi.log.rolling.appender.max.file_size";

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/LogAttribute.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/LogAttribute.cpp b/libminifi/src/LogAttribute.cpp
deleted file mode 100644
index 345eb69..0000000
--- a/libminifi/src/LogAttribute.cpp
+++ /dev/null
@@ -1,159 +0,0 @@
-/**
- * @file LogAttribute.cpp
- * LogAttribute class implementation
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#include <vector>
-#include <queue>
-#include <map>
-#include <set>
-#include <sys/time.h>
-#include <time.h>
-#include <sstream>
-#include <string.h>
-#include <iostream>
-
-#include "utils/TimeUtil.h"
-#include "utils/StringUtils.h"
-#include "LogAttribute.h"
-#include "ProcessContext.h"
-#include "ProcessSession.h"
-
-const std::string LogAttribute::ProcessorName("LogAttribute");
-Property LogAttribute::LogLevel("Log Level", "The Log Level to use when logging the Attributes", "info");
-Property LogAttribute::AttributesToLog("Attributes to Log", "A comma-separated list of Attributes to Log. If not specified, all attributes will be logged.", "");
-Property LogAttribute::AttributesToIgnore("Attributes to Ignore", "A comma-separated list of Attributes to ignore. If not specified, no attributes will be ignored.", "");
-Property LogAttribute::LogPayload("Log Payload",
-		"If true, the FlowFile's payload will be logged, in addition to its attributes; otherwise, just the Attributes will be logged.", "false");
-Property LogAttribute::LogPrefix("Log prefix",
-		"Log prefix appended to the log lines. It helps to distinguish the output of multiple LogAttribute processors.", "");
-Relationship LogAttribute::Success("success", "success operational on the flow record");
-
-void LogAttribute::initialize()
-{
-	//! Set the supported properties
-	std::set<Property> properties;
-	properties.insert(LogLevel);
-	properties.insert(AttributesToLog);
-	properties.insert(AttributesToIgnore);
-	properties.insert(LogPayload);
-	properties.insert(LogPrefix);
-	setSupportedProperties(properties);
-	//! Set the supported relationships
-	std::set<Relationship> relationships;
-	relationships.insert(Success);
-	setSupportedRelationships(relationships);
-}
-
-void LogAttribute::onTrigger(ProcessContext *context, ProcessSession *session)
-{
-	std::string dashLine = "--------------------------------------------------";
-	LogAttrLevel level = LogAttrLevelInfo;
-	bool logPayload = false;
-	std::ostringstream message;
-
-	FlowFileRecord *flow = session->get();
-
-	if (!flow)
-		return;
-
-	std::string value;
-	if (context->getProperty(LogLevel.getName(), value))
-	{
-		logLevelStringToEnum(value, level);
-	}
-	if (context->getProperty(LogPrefix.getName(), value))
-	{
-		dashLine = "-----" + value + "-----";
-	}
-	if (context->getProperty(LogPayload.getName(), value))
-	{
-		StringUtils::StringToBool(value, logPayload);
-	}
-
-	message << "Logging for flow file " << "\n";
-	message << dashLine;
-	message << "\nStandard FlowFile Attributes";
-	message << "\n" << "UUID:" << flow->getUUIDStr();
-	message << "\n" << "EntryDate:" << getTimeStr(flow->getEntryDate());
-	message << "\n" << "lineageStartDate:" << getTimeStr(flow->getlineageStartDate());
-	message << "\n" << "Size:" << flow->getSize() << " Offset:" << flow->getOffset();
-	message << "\nFlowFile Attributes Map Content";
-	std::map<std::string, std::string> attrs = flow->getAttributes();
-    std::map<std::string, std::string>::iterator it;
-    for (it = attrs.begin(); it!= attrs.end(); it++)
-    {
-    	message << "\n" << "key:" << it->first << " value:" << it->second;
-    }
-    message << "\nFlowFile Resource Claim Content";
-    ResourceClaim *claim = flow->getResourceClaim();
-    if (claim)
-    {
-    	message << "\n" << "Content Claim:" << claim->getContentFullPath();
-    }
-    if (logPayload && flow->getSize() <= 1024*1024)
-    {
-    	message << "\n" << "Payload:" << "\n";
-    	ReadCallback callback(flow->getSize());
-    	session->read(flow, &callback);
-    	for (unsigned int i = 0, j = 0; i < callback._readSize; i++)
-    	{
-    		char temp[8];
-    		sprintf(temp, "%02x ", (unsigned char) (callback._buffer[i]));
-    		message << temp;
-    		j++;
-    		if (j == 16)
-    		{
-    			message << '\n';
-    			j = 0;
-    		}
-    	}
-    }
-    message << "\n" << dashLine << std::ends;
-    std::string output = message.str();
-
-    switch (level)
-    {
-    case LogAttrLevelInfo:
-    	logger_->log_info("%s", output.c_str());
-		break;
-    case LogAttrLevelDebug:
-    	logger_->log_debug("%s", output.c_str());
-		break;
-    case LogAttrLevelError:
-    	logger_->log_error("%s", output.c_str());
-		break;
-    case LogAttrLevelTrace:
-    	logger_->log_trace("%s", output.c_str());
-    	break;
-    case LogAttrLevelWarn:
-    	logger_->log_warn("%s", output.c_str());
-    	break;
-    default:
-    	break;
-    }
-
-    // Test Import
-    /*
-    FlowFileRecord *importRecord = session->create();
-    session->import(claim->getContentFullPath(), importRecord);
-    session->transfer(importRecord, Success); */
-
-
-    // Transfer to the relationship
-    session->transfer(flow, Success);
-}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/Logger.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/Logger.cpp b/libminifi/src/Logger.cpp
deleted file mode 100644
index e90667d..0000000
--- a/libminifi/src/Logger.cpp
+++ /dev/null
@@ -1,28 +0,0 @@
-/**
- * @file Logger.cpp
- * Logger class implementation
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#include "../include/Logger.h"
-
-#include <vector>
-#include <queue>
-#include <map>
-
-
-std::shared_ptr<Logger> Logger::singleton_logger_(nullptr);
-

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/ProcessGroup.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/ProcessGroup.cpp b/libminifi/src/ProcessGroup.cpp
deleted file mode 100644
index 7e3527e..0000000
--- a/libminifi/src/ProcessGroup.cpp
+++ /dev/null
@@ -1,307 +0,0 @@
-/**
- * @file ProcessGroup.cpp
- * ProcessGroup class implementation
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#include <vector>
-#include <queue>
-#include <map>
-#include <set>
-#include <sys/time.h>
-#include <time.h>
-#include <chrono>
-#include <thread>
-
-#include "ProcessGroup.h"
-#include "Processor.h"
-
-ProcessGroup::ProcessGroup(ProcessGroupType type, std::string name, uuid_t uuid,
-		ProcessGroup *parent) :
-		name_(name), type_(type), parent_process_group_(parent) {
-	if (!uuid)
-		// Generate the global UUID for the flow record
-		uuid_generate(uuid_);
-	else
-		uuid_copy(uuid_, uuid);
-
-	yield_period_msec_ = 0;
-	transmitting_ = false;
-
-	logger_ = Logger::getLogger();
-	logger_->log_info("ProcessGroup %s created", name_.c_str());
-}
-
-ProcessGroup::~ProcessGroup() {
-	for (std::set<Connection *>::iterator it = connections_.begin();
-			it != connections_.end(); ++it) {
-		Connection *connection = *it;
-		connection->drain();
-		delete connection;
-	}
-
-	for (std::set<ProcessGroup *>::iterator it = child_process_groups_.begin();
-			it != child_process_groups_.end(); ++it) {
-		ProcessGroup *processGroup(*it);
-		delete processGroup;
-	}
-
-	for (std::set<Processor *>::iterator it = processors_.begin();
-			it != processors_.end(); ++it) {
-		Processor *processor(*it);
-		delete processor;
-	}
-}
-
-void ProcessGroup::getConnections(std::map<std::string, Connection*> *connectionMap)
-{
-	for (auto connection : connections_)
-	{
-		(*connectionMap)[connection->getUUIDStr()] = connection;
-	}
-
-	for (auto processGroup: child_process_groups_) {
-		processGroup->getConnections(connectionMap);
-	}
-}
-
-bool ProcessGroup::isRootProcessGroup() {
-	std::lock_guard<std::mutex> lock(mtx_);
-	return (type_ == ROOT_PROCESS_GROUP);
-}
-
-void ProcessGroup::addProcessor(Processor *processor) {
-	std::lock_guard<std::mutex> lock(mtx_);
-
-	if (processors_.find(processor) == processors_.end()) {
-		// We do not have the same processor in this process group yet
-		processors_.insert(processor);
-		logger_->log_info("Add processor %s into process group %s",
-				processor->getName().c_str(), name_.c_str());
-	}
-}
-
-void ProcessGroup::removeProcessor(Processor *processor) {
-	std::lock_guard<std::mutex> lock(mtx_);
-
-	if (processors_.find(processor) != processors_.end()) {
-		// We do have the same processor in this process group yet
-		processors_.erase(processor);
-		logger_->log_info("Remove processor %s from process group %s",
-				processor->getName().c_str(), name_.c_str());
-	}
-}
-
-void ProcessGroup::addProcessGroup(ProcessGroup *child) {
-	std::lock_guard<std::mutex> lock(mtx_);
-
-	if (child_process_groups_.find(child) == child_process_groups_.end()) {
-		// We do not have the same child process group in this process group yet
-		child_process_groups_.insert(child);
-		logger_->log_info("Add child process group %s into process group %s",
-				child->getName().c_str(), name_.c_str());
-	}
-}
-
-void ProcessGroup::removeProcessGroup(ProcessGroup *child) {
-	std::lock_guard<std::mutex> lock(mtx_);
-
-	if (child_process_groups_.find(child) != child_process_groups_.end()) {
-		// We do have the same child process group in this process group yet
-		child_process_groups_.erase(child);
-		logger_->log_info("Remove child process group %s from process group %s",
-				child->getName().c_str(), name_.c_str());
-	}
-}
-
-void ProcessGroup::startProcessing(TimerDrivenSchedulingAgent *timeScheduler,
-		EventDrivenSchedulingAgent *eventScheduler) {
-	std::lock_guard<std::mutex> lock(mtx_);
-
-	try {
-		// Start all the processor node, input and output ports
-		for (auto processor : processors_) {
-			logger_->log_debug("Starting %s", processor->getName().c_str());
-
-			if (!processor->isRunning()
-					&& processor->getScheduledState() != DISABLED) {
-				if (processor->getSchedulingStrategy() == TIMER_DRIVEN)
-					timeScheduler->schedule(processor);
-				else if (processor->getSchedulingStrategy() == EVENT_DRIVEN)
-					eventScheduler->schedule(processor);
-			}
-		}
-		// Start processing the group
-		for (auto processGroup : child_process_groups_) {
-			processGroup->startProcessing(timeScheduler, eventScheduler);
-		}
-	} catch (std::exception &exception) {
-		logger_->log_debug("Caught Exception %s", exception.what());
-		throw;
-	} catch (...) {
-		logger_->log_debug(
-				"Caught Exception during process group start processing");
-		throw;
-	}
-}
-
-void ProcessGroup::stopProcessing(TimerDrivenSchedulingAgent *timeScheduler,
-		EventDrivenSchedulingAgent *eventScheduler) {
-	std::lock_guard<std::mutex> lock(mtx_);
-
-	try {
-		// Stop all the processor node, input and output ports
-		for (std::set<Processor *>::iterator it = processors_.begin();
-				it != processors_.end(); ++it) {
-			Processor *processor(*it);
-			if (processor->getSchedulingStrategy() == TIMER_DRIVEN)
-				timeScheduler->unschedule(processor);
-			else if (processor->getSchedulingStrategy() == EVENT_DRIVEN)
-				eventScheduler->unschedule(processor);
-		}
-
-		for (std::set<ProcessGroup *>::iterator it =
-				child_process_groups_.begin(); it != child_process_groups_.end();
-				++it) {
-			ProcessGroup *processGroup(*it);
-			processGroup->stopProcessing(timeScheduler, eventScheduler);
-		}
-	} catch (std::exception &exception) {
-		logger_->log_debug("Caught Exception %s", exception.what());
-		throw;
-	} catch (...) {
-		logger_->log_debug(
-				"Caught Exception during process group stop processing");
-		throw;
-	}
-}
-
-Processor *ProcessGroup::findProcessor(uuid_t uuid) {
-
-	Processor *ret = NULL;
-	// std::lock_guard<std::mutex> lock(_mtx);
-
-	for(auto processor : processors_){
-		logger_->log_info("find processor %s", processor->getName().c_str());
-		uuid_t processorUUID;
-
-		if (processor->getUUID(processorUUID)) {
-
-			char uuid_str[37]; // ex. "1b4e28ba-2fa1-11d2-883f-0016d3cca427" + "\0"
-			uuid_unparse_lower(processorUUID, uuid_str);
-			std::string processorUUIDstr = uuid_str;
-			uuid_unparse_lower(uuid, uuid_str);
-			std::string uuidStr = uuid_str;
-			if (processorUUIDstr == uuidStr) {
-				return processor;
-			}
-		}
-
-	}
-	for(auto processGroup : child_process_groups_){
-
-		logger_->log_info("find processor child %s",
-				processGroup->getName().c_str());
-		Processor *processor = processGroup->findProcessor(uuid);
-		if (processor)
-			return processor;
-	}
-
-	return ret;
-}
-
-Processor *ProcessGroup::findProcessor(std::string processorName) {
-	Processor *ret = NULL;
-
-	for(auto processor : processors_){
-		logger_->log_debug("Current processor is %s",
-				processor->getName().c_str());
-		if (processor->getName() == processorName)
-			return processor;
-	}
-
-	for(auto processGroup : child_process_groups_){
-		Processor *processor = processGroup->findProcessor(processorName);
-		if (processor)
-			return processor;
-	}
-
-	return ret;
-}
-
-void ProcessGroup::updatePropertyValue(std::string processorName,
-		std::string propertyName, std::string propertyValue) {
-	std::lock_guard<std::mutex> lock(mtx_);
-
-	for(auto processor : processors_){
-		if (processor->getName() == processorName) {
-			processor->setProperty(propertyName, propertyValue);
-		}
-	}
-
-	for(auto processGroup : child_process_groups_){
-		processGroup->updatePropertyValue(processorName, propertyName,
-				propertyValue);
-	}
-
-	return;
-}
-
-void ProcessGroup::addConnection(Connection *connection) {
-	std::lock_guard<std::mutex> lock(mtx_);
-
-	if (connections_.find(connection) == connections_.end()) {
-		// We do not have the same connection in this process group yet
-		connections_.insert(connection);
-		logger_->log_info("Add connection %s into process group %s",
-				connection->getName().c_str(), name_.c_str());
-		uuid_t sourceUUID;
-		Processor *source = NULL;
-		connection->getSourceProcessorUUID(sourceUUID);
-		source = this->findProcessor(sourceUUID);
-		if (source)
-			source->addConnection(connection);
-		Processor *destination = NULL;
-		uuid_t destinationUUID;
-		connection->getDestinationProcessorUUID(destinationUUID);
-		destination = this->findProcessor(destinationUUID);
-		if (destination && destination != source)
-			destination->addConnection(connection);
-	}
-}
-
-void ProcessGroup::removeConnection(Connection *connection) {
-	std::lock_guard<std::mutex> lock(mtx_);
-
-	if (connections_.find(connection) != connections_.end()) {
-		// We do not have the same connection in this process group yet
-		connections_.erase(connection);
-		logger_->log_info("Remove connection %s into process group %s",
-				connection->getName().c_str(), name_.c_str());
-		uuid_t sourceUUID;
-		Processor *source = NULL;
-		connection->getSourceProcessorUUID(sourceUUID);
-		source = this->findProcessor(sourceUUID);
-		if (source)
-			source->removeConnection(connection);
-		Processor *destination = NULL;
-		uuid_t destinationUUID;
-		connection->getDestinationProcessorUUID(destinationUUID);
-		destination = this->findProcessor(destinationUUID);
-		if (destination && destination != source)
-			destination->removeConnection(connection);
-	}
-}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/ProcessSession.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/ProcessSession.cpp b/libminifi/src/ProcessSession.cpp
deleted file mode 100644
index 3b3eb64..0000000
--- a/libminifi/src/ProcessSession.cpp
+++ /dev/null
@@ -1,790 +0,0 @@
-/**
- * @file ProcessSession.cpp
- * ProcessSession class implementation
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#include <vector>
-#include <queue>
-#include <map>
-#include <set>
-#include <sys/time.h>
-#include <time.h>
-#include <chrono>
-#include <thread>
-#include <iostream>
-
-#include "ProcessSession.h"
-#include "FlowController.h"
-
-ProcessSession::ProcessSession(ProcessContext *processContext) : _processContext(processContext) {
-	logger_ = Logger::getLogger();
-	logger_->log_trace("ProcessSession created for %s", _processContext->getProcessor()->getName().c_str());
-	_provenanceReport = NULL;
-	if (FlowControllerFactory::getFlowController()->getProvenanceRepository()->isEnable())
-	{
-		_provenanceReport = new ProvenanceReporter(_processContext->getProcessor()->getUUIDStr(),
-					_processContext->getProcessor()->getName());
-	}
-}
-
-FlowFileRecord* ProcessSession::create()
-{
-	std::map<std::string, std::string> empty;
-	FlowFileRecord *record = new FlowFileRecord(empty);
-
-	if (record)
-	{
-		_addedFlowFiles[record->getUUIDStr()] = record;
-		logger_->log_debug("Create FlowFile with UUID %s", record->getUUIDStr().c_str());
-		std::string details = _processContext->getProcessor()->getName() + " creates flow record " +  record->getUUIDStr();
-		if (_provenanceReport)
-			_provenanceReport->create(record, details);
-	}
-
-	return record;
-}
-
-FlowFileRecord* ProcessSession::create(FlowFileRecord *parent)
-{
-	std::map<std::string, std::string> empty;
-	FlowFileRecord *record = new FlowFileRecord(empty);
-
-	if (record)
-	{
-		_addedFlowFiles[record->getUUIDStr()] = record;
-		logger_->log_debug("Create FlowFile with UUID %s", record->getUUIDStr().c_str());
-	}
-
-	if (record)
-	{
-		// Copy attributes
-		std::map<std::string, std::string> parentAttributes = parent->getAttributes();
-	    std::map<std::string, std::string>::iterator it;
-	    for (it = parentAttributes.begin(); it!= parentAttributes.end(); it++)
-	    {
-	    	if (it->first == FlowAttributeKey(ALTERNATE_IDENTIFIER) ||
-	    			it->first == FlowAttributeKey(DISCARD_REASON) ||
-					it->first == FlowAttributeKey(UUID))
-	    		// Do not copy special attributes from parent
-	    		continue;
-	    	record->setAttribute(it->first, it->second);
-	    }
-	    record->_lineageStartDate = parent->_lineageStartDate;
-	    record->_lineageIdentifiers = parent->_lineageIdentifiers;
-	    record->_lineageIdentifiers.insert(parent->_uuidStr);
-
-	}
-	return record;
-}
-
-FlowFileRecord* ProcessSession::clone(FlowFileRecord *parent)
-{
-	FlowFileRecord *record = this->create(parent);
-	if (record)
-	{
-		// Copy Resource Claim
-		record->_claim = parent->_claim;
-		if (record->_claim)
-		{
-			record->_offset = parent->_offset;
-			record->_size = parent->_size;
-			record->_claim->increaseFlowFileRecordOwnedCount();
-		}
-		if (_provenanceReport)
-			_provenanceReport->clone(parent, record);
-	}
-	return record;
-}
-
-FlowFileRecord* ProcessSession::cloneDuringTransfer(FlowFileRecord *parent)
-{
-	std::map<std::string, std::string> empty;
-	FlowFileRecord *record = new FlowFileRecord(empty);
-
-	if (record)
-	{
-		this->_clonedFlowFiles[record->getUUIDStr()] = record;
-		logger_->log_debug("Clone FlowFile with UUID %s during transfer", record->getUUIDStr().c_str());
-		// Copy attributes
-		std::map<std::string, std::string> parentAttributes = parent->getAttributes();
-		std::map<std::string, std::string>::iterator it;
-		for (it = parentAttributes.begin(); it!= parentAttributes.end(); it++)
-		{
-			if (it->first == FlowAttributeKey(ALTERNATE_IDENTIFIER) ||
-	    			it->first == FlowAttributeKey(DISCARD_REASON) ||
-					it->first == FlowAttributeKey(UUID))
-	    		// Do not copy special attributes from parent
-	    		continue;
-	    	record->setAttribute(it->first, it->second);
-	    }
-	    record->_lineageStartDate = parent->_lineageStartDate;
-	    record->_lineageIdentifiers = parent->_lineageIdentifiers;
-	    record->_lineageIdentifiers.insert(parent->_uuidStr);
-
-	    // Copy Resource Claim
-	    record->_claim = parent->_claim;
-	    if (record->_claim)
-	    {
-	    	record->_offset = parent->_offset;
-	    	record->_size = parent->_size;
-	    	record->_claim->increaseFlowFileRecordOwnedCount();
-	    }
-	    if (_provenanceReport)
-	    	_provenanceReport->clone(parent, record);
-	}
-
-	return record;
-}
-
-FlowFileRecord* ProcessSession::clone(FlowFileRecord *parent, long offset, long size)
-{
-	FlowFileRecord *record = this->create(parent);
-	if (record)
-	{
-		if (parent->_claim)
-		{
-			if ((offset + size) > (long) parent->_size)
-			{
-				// Set offset and size
-				logger_->log_error("clone offset %d and size %d exceed parent size %d",
-						offset, size, parent->_size);
-				// Remove the Add FlowFile for the session
-				std::map<std::string, FlowFileRecord *>::iterator it =
-						this->_addedFlowFiles.find(record->getUUIDStr());
-				if (it != this->_addedFlowFiles.end())
-					this->_addedFlowFiles.erase(record->getUUIDStr());
-				delete record;
-				return NULL;
-			}
-			record->_offset = parent->_offset + parent->_offset;
-			record->_size = size;
-			// Copy Resource Claim
-			record->_claim = parent->_claim;
-			record->_claim->increaseFlowFileRecordOwnedCount();
-		}
-		if (_provenanceReport)
-			_provenanceReport->clone(parent, record);
-	}
-	return record;
-}
-
-void ProcessSession::remove(FlowFileRecord *flow)
-{
-	flow->_markedDelete = true;
-	_deletedFlowFiles[flow->getUUIDStr()] = flow;
-	std::string reason = _processContext->getProcessor()->getName() + " drop flow record " +  flow->getUUIDStr();
-	if (_provenanceReport)
-		_provenanceReport->drop(flow, reason);
-}
-
-void ProcessSession::putAttribute(FlowFileRecord *flow, std::string key, std::string value)
-{
-	flow->setAttribute(key, value);
-	std::string details = _processContext->getProcessor()->getName() + " modify flow record " +  flow->getUUIDStr() +
-			" attribute " + key + ":" + value;
-	if (_provenanceReport)
-		_provenanceReport->modifyAttributes(flow, details);
-}
-
-void ProcessSession::removeAttribute(FlowFileRecord *flow, std::string key)
-{
-	flow->removeAttribute(key);
-	std::string details = _processContext->getProcessor()->getName() + " remove flow record " +  flow->getUUIDStr() +
-				" attribute " + key;
-	if (_provenanceReport)
-		_provenanceReport->modifyAttributes(flow, details);
-}
-
-void ProcessSession::penalize(FlowFileRecord *flow)
-{
-	flow->_penaltyExpirationMs = getTimeMillis() + this->_processContext->getProcessor()->getPenalizationPeriodMsec();
-}
-
-void ProcessSession::transfer(FlowFileRecord *flow, Relationship relationship)
-{
-	_transferRelationship[flow->getUUIDStr()] = relationship;
-}
-
-void ProcessSession::write(FlowFileRecord *flow, OutputStreamCallback *callback)
-{
-	ResourceClaim *claim = NULL;
-
-	claim = new ResourceClaim();
-
-	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->_size = fs.tellp();
-				flow->_offset = 0;
-				if (flow->_claim)
-				{
-					// Remove the old claim
-					flow->_claim->decreaseFlowFileRecordOwnedCount();
-					flow->_claim = NULL;
-				}
-				flow->_claim = 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 = _processContext->getProcessor()->getName() + " modify flow record content " +  flow->getUUIDStr();
-				uint64_t endTime = getTimeMillis();
-				if (_provenanceReport)
-					_provenanceReport->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");
-		}
-	}
-	catch (std::exception &exception)
-	{
-		if (flow && flow->_claim == claim)
-		{
-			flow->_claim->decreaseFlowFileRecordOwnedCount();
-			flow->_claim = NULL;
-		}
-		if (claim)
-			delete claim;
-		logger_->log_debug("Caught Exception %s", exception.what());
-		throw;
-	}
-	catch (...)
-	{
-		if (flow && flow->_claim == claim)
-		{
-			flow->_claim->decreaseFlowFileRecordOwnedCount();
-			flow->_claim = NULL;
-		}
-		if (claim)
-			delete claim;
-		logger_->log_debug("Caught Exception during process session write");
-		throw;
-	}
-}
-
-void ProcessSession::append(FlowFileRecord *flow, OutputStreamCallback *callback)
-{
-	ResourceClaim *claim = NULL;
-
-	if (flow->_claim == NULL)
-	{
-		// No existed claim for append, we need to create new claim
-		return write(flow, callback);
-	}
-
-	claim = flow->_claim;
-
-	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->_size += 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 = _processContext->getProcessor()->getName() + " modify flow record content " +  flow->getUUIDStr();
-				uint64_t endTime = getTimeMillis();
-				if (_provenanceReport)
-					_provenanceReport->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");
-		}
-	}
-	catch (std::exception &exception)
-	{
-		logger_->log_debug("Caught Exception %s", exception.what());
-		throw;
-	}
-	catch (...)
-	{
-		logger_->log_debug("Caught Exception during process session append");
-		throw;
-	}
-}
-
-void ProcessSession::read(FlowFileRecord *flow, InputStreamCallback *callback)
-{
-	try
-	{
-		ResourceClaim *claim = NULL;
-		if (flow->_claim == NULL)
-		{
-			// No existed claim for read, we throw exception
-			throw Exception(FILE_OPERATION_EXCEPTION, "No Content Claim existed for read");
-		}
-
-		claim = flow->_claim;
-		std::ifstream fs;
-		fs.open(claim->getContentFullPath().c_str(), std::fstream::in | std::fstream::binary);
-		if (fs.is_open())
-		{
-			fs.seekg(flow->_offset, 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");
-		}
-	}
-	catch (std::exception &exception)
-	{
-		logger_->log_debug("Caught Exception %s", exception.what());
-		throw;
-	}
-	catch (...)
-	{
-		logger_->log_debug("Caught Exception during process session read");
-		throw;
-	}
-}
-
-void ProcessSession::import(std::string source, FlowFileRecord *flow, bool keepSource, uint64_t offset)
-{
-	ResourceClaim *claim = NULL;
-
-	claim = new ResourceClaim();
-	char *buf = NULL;
-	int size = 4096;
-	buf = new char [size];
-
-	try
-	{
-		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())
-		{
-			// Open the source file and stream to the flow file
-			input.seekg(offset, fs.beg);
-			while (input.good())
-			{
-				input.read(buf, size);
-				if (input)
-					fs.write(buf, size);
-				else
-					fs.write(buf, input.gcount());
-			}
-
-			if (fs.good() && fs.tellp() >= 0)
-			{
-				flow->_size = fs.tellp();
-				flow->_offset = 0;
-				if (flow->_claim)
-				{
-					// Remove the old claim
-					flow->_claim->decreaseFlowFileRecordOwnedCount();
-					flow->_claim = NULL;
-				}
-				flow->_claim = claim;
-				claim->increaseFlowFileRecordOwnedCount();
-
-				logger_->log_debug("Import 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();
-				input.close();
-				if (!keepSource)
-					std::remove(source.c_str());
-				std::string details = _processContext->getProcessor()->getName() + " modify flow record content " +  flow->getUUIDStr();
-				uint64_t endTime = getTimeMillis();
-				if (_provenanceReport)
-					_provenanceReport->modifyContent(flow, details, endTime - startTime);
-			}
-			else
-			{
-				fs.close();
-				input.close();
-				throw Exception(FILE_OPERATION_EXCEPTION, "File Import Error");
-			}
-		}
-		else
-		{
-			throw Exception(FILE_OPERATION_EXCEPTION, "File Import Error");
-		}
-
-		delete[] buf;
-	}
-	catch (std::exception &exception)
-	{
-		if (flow && flow->_claim == claim)
-		{
-			flow->_claim->decreaseFlowFileRecordOwnedCount();
-			flow->_claim = NULL;
-		}
-		if (claim)
-			delete claim;
-		logger_->log_debug("Caught Exception %s", exception.what());
-		delete[] buf;
-		throw;
-	}
-	catch (...)
-	{
-		if (flow && flow->_claim == claim)
-		{
-			flow->_claim->decreaseFlowFileRecordOwnedCount();
-			flow->_claim = NULL;
-		}
-		if (claim)
-			delete claim;
-		logger_->log_debug("Caught Exception during process session write");
-		delete[] buf;
-		throw;
-	}
-}
-
-void ProcessSession::commit()
-{
-	try
-	{
-		// First we clone the flow record based on the transfered relationship for updated flow record
-		for (auto && it : _updatedFlowFiles)
-		{
-			FlowFileRecord *record = it.second;
-			if (record->_markedDelete)
-				continue;
-			std::map<std::string, Relationship>::iterator itRelationship =
-					this->_transferRelationship.find(record->getUUIDStr());
-			if (itRelationship != _transferRelationship.end())
-			{
-				Relationship relationship = itRelationship->second;
-				// Find the relationship, we need to find the connections for that relationship
-				std::set<Connection *> connections =
-						_processContext->getProcessor()->getOutGoingConnections(relationship.getName());
-				if (connections.empty())
-				{
-					// No connection
-					if (!_processContext->getProcessor()->isAutoTerminated(relationship))
-					{
-						// Not autoterminate, we should have the connect
-						std::string message = "Connect empty for non auto terminated relationship" + relationship.getName();
-						throw Exception(PROCESS_SESSION_EXCEPTION, message.c_str());
-					}
-					else
-					{
-						// Autoterminated
-						remove(record);
-					}
-				}
-				else
-				{
-					// We connections, clone the flow and assign the connection accordingly
-					for (std::set<Connection *>::iterator itConnection = connections.begin(); itConnection != connections.end(); ++itConnection)
-					{
-						Connection *connection(*itConnection);
-						if (itConnection == connections.begin())
-						{
-							// First connection which the flow need be routed to
-							record->_connection = connection;
-						}
-						else
-						{
-							// Clone the flow file and route to the connection
-							FlowFileRecord *cloneRecord;
-							cloneRecord = this->cloneDuringTransfer(record);
-							if (cloneRecord)
-								cloneRecord->_connection = connection;
-							else
-								throw Exception(PROCESS_SESSION_EXCEPTION, "Can not clone the flow for transfer");
-						}
-					}
-				}
-			}
-			else
-			{
-				// Can not find relationship for the flow
-				throw Exception(PROCESS_SESSION_EXCEPTION, "Can not find the transfer relationship for the flow");
-			}
-		}
-		// Do the samething for added flow file
-		for(const auto it : _addedFlowFiles)
-		{
-			FlowFileRecord *record = it.second;
-			if (record->_markedDelete)
-				continue;
-			std::map<std::string, Relationship>::iterator itRelationship =
-					this->_transferRelationship.find(record->getUUIDStr());
-			if (itRelationship != _transferRelationship.end())
-			{
-				Relationship relationship = itRelationship->second;
-				// Find the relationship, we need to find the connections for that relationship
-				std::set<Connection *> connections =
-						_processContext->getProcessor()->getOutGoingConnections(relationship.getName());
-				if (connections.empty())
-				{
-					// No connection
-					if (!_processContext->getProcessor()->isAutoTerminated(relationship))
-					{
-						// Not autoterminate, we should have the connect
-						std::string message = "Connect empty for non auto terminated relationship " + relationship.getName();
-						throw Exception(PROCESS_SESSION_EXCEPTION, message.c_str());
-					}
-					else
-					{
-						// Autoterminated
-						remove(record);
-					}
-				}
-				else
-				{
-					// We connections, clone the flow and assign the connection accordingly
-					for (std::set<Connection *>::iterator itConnection = connections.begin(); itConnection != connections.end(); ++itConnection)
-					{
-						Connection *connection(*itConnection);
-						if (itConnection == connections.begin())
-						{
-							// First connection which the flow need be routed to
-							record->_connection = connection;
-						}
-						else
-						{
-							// Clone the flow file and route to the connection
-							FlowFileRecord *cloneRecord;
-							cloneRecord = this->cloneDuringTransfer(record);
-							if (cloneRecord)
-								cloneRecord->_connection = connection;
-							else
-								throw Exception(PROCESS_SESSION_EXCEPTION, "Can not clone the flow for transfer");
-						}
-					}
-				}
-			}
-			else
-			{
-				// Can not find relationship for the flow
-				throw Exception(PROCESS_SESSION_EXCEPTION, "Can not find the transfer relationship for the flow");
-			}
-		}
-		// Complete process the added and update flow files for the session, send the flow file to its queue
-		for(const auto &it : _updatedFlowFiles)
-		{
-			FlowFileRecord *record = it.second;
-			if (record->_markedDelete)
-			{
-				continue;
-			}
-			if (record->_connection)
-				record->_connection->put(record);
-			else
-				delete record;
-		}
-		for(const auto &it : _addedFlowFiles)
-		{
-			FlowFileRecord *record = it.second;
-			if (record->_markedDelete)
-			{
-				continue;
-			}
-			if (record->_connection)
-				record->_connection->put(record);
-			else
-				delete record;
-		}
-		// Process the clone flow files
-		for(const auto &it : _clonedFlowFiles)
-		{
-			FlowFileRecord *record = it.second;
-			if (record->_markedDelete)
-			{
-				continue;
-			}
-			if (record->_connection)
-				record->_connection->put(record);
-			else
-				delete record;
-		}
-		// Delete the deleted flow files
-		for(const auto &it : _deletedFlowFiles)
-		{
-			FlowFileRecord *record = it.second;
-			delete record;
-		}
-		// Delete the snapshot
-		for(const auto &it : _originalFlowFiles)
-		{
-			FlowFileRecord *record = it.second;
-			delete record;
-		}
-		// All done
-		_updatedFlowFiles.clear();
-		_addedFlowFiles.clear();
-		_clonedFlowFiles.clear();
-		_deletedFlowFiles.clear();
-		_originalFlowFiles.clear();
-		// persistent the provenance report
-		if (this->_provenanceReport)
-			this->_provenanceReport->commit();
-		logger_->log_trace("ProcessSession committed for %s", _processContext->getProcessor()->getName().c_str());
-	}
-	catch (std::exception &exception)
-	{
-		logger_->log_debug("Caught Exception %s", exception.what());
-		throw;
-	}
-	catch (...)
-	{
-		logger_->log_debug("Caught Exception during process session commit");
-		throw;
-	}
-}
-
-
-void ProcessSession::rollback()
-{
-	try
-	{
-		// Requeue the snapshot of the flowfile back
-		for(const auto &it : _originalFlowFiles)
-		{
-			FlowFileRecord *record = it.second;
-			if (record->_orginalConnection)
-			{
-				record->_snapshot = false;
-				record->_orginalConnection->put(record);
-			}
-			else
-				delete record;
-		}
-		_originalFlowFiles.clear();
-		// Process the clone flow files
-		for(const auto &it : _clonedFlowFiles)
-		{
-			FlowFileRecord *record = it.second;
-			delete record;
-		}
-		_clonedFlowFiles.clear();
-		for(const auto &it : _addedFlowFiles)
-		{
-			FlowFileRecord *record = it.second;
-			delete record;
-		}
-		_addedFlowFiles.clear();
-		for(const auto &it : _updatedFlowFiles)
-		{
-			FlowFileRecord *record = it.second;
-			delete record;
-		}
-		_updatedFlowFiles.clear();
-		_deletedFlowFiles.clear();
-		logger_->log_trace("ProcessSession rollback for %s", _processContext->getProcessor()->getName().c_str());
-	}
-	catch (std::exception &exception)
-	{
-		logger_->log_debug("Caught Exception %s", exception.what());
-		throw;
-	}
-	catch (...)
-	{
-		logger_->log_debug("Caught Exception during process session roll back");
-		throw;
-	}
-}
-
-FlowFileRecord *ProcessSession::get()
-{
-	Connection *first = _processContext->getProcessor()->getNextIncomingConnection();
-
-	if (first == NULL)
-		return NULL;
-
-	Connection *current = first;
-
-	do
-	{
-		std::set<FlowFileRecord *> expired;
-		FlowFileRecord *ret = current->poll(expired);
-		if (expired.size() > 0)
-		{
-			// Remove expired flow record
-			for (std::set<FlowFileRecord *>::iterator it = expired.begin(); it != expired.end(); ++it)
-			{
-				FlowFileRecord *record = *it;
-				std::string details = _processContext->getProcessor()->getName() + " expire flow record " +  record->getUUIDStr();
-				if (_provenanceReport)
-					_provenanceReport->expire(record, details);
-				delete (record);
-			}
-		}
-		if (ret)
-		{
-			// add the flow record to the current process session update map
-			ret->_markedDelete = false;
-			_updatedFlowFiles[ret->getUUIDStr()] = ret;
-			std::map<std::string, std::string> empty;
-			FlowFileRecord *snapshot = new FlowFileRecord(empty);
-			logger_->log_debug("Create Snapshot FlowFile with UUID %s", snapshot->getUUIDStr().c_str());
-			snapshot->duplicate(ret);
-			// save a snapshot
-			_originalFlowFiles[snapshot->getUUIDStr()] = snapshot;
-			return ret;
-		}
-		current = _processContext->getProcessor()->getNextIncomingConnection();
-	}
-	while (current != NULL && current != first);
-
-	return NULL;
-}
-

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/ProcessSessionFactory.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/ProcessSessionFactory.cpp b/libminifi/src/ProcessSessionFactory.cpp
deleted file mode 100644
index a105b1c..0000000
--- a/libminifi/src/ProcessSessionFactory.cpp
+++ /dev/null
@@ -1,28 +0,0 @@
-/**
- * @file ProcessSessionFactory.cpp
- * ProcessSessionFactory class implementation
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#include "ProcessSessionFactory.h"
-
-#include <memory>
-
-std::unique_ptr<ProcessSession> ProcessSessionFactory::createSession()
-{
-	return std::unique_ptr<ProcessSession>(new ProcessSession(_processContext));
-}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/Processor.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/Processor.cpp b/libminifi/src/Processor.cpp
deleted file mode 100644
index 94aaa20..0000000
--- a/libminifi/src/Processor.cpp
+++ /dev/null
@@ -1,526 +0,0 @@
-/**
- * @file Processor.cpp
- * Processor class implementation
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#include <vector>
-#include <queue>
-#include <map>
-#include <set>
-#include <sys/time.h>
-#include <time.h>
-#include <chrono>
-#include <thread>
-#include <memory>
-#include <functional>
-
-#include "Processor.h"
-#include "ProcessContext.h"
-#include "ProcessSession.h"
-#include "ProcessSessionFactory.h"
-
-Processor::Processor(std::string name, uuid_t uuid)
-: _name(name)
-{
-	if (!uuid)
-		// Generate the global UUID for the flow record
-		uuid_generate(_uuid);
-	else
-		uuid_copy(_uuid, uuid);
-
-	char uuidStr[37];
-	uuid_unparse_lower(_uuid, uuidStr);
-	_uuidStr = uuidStr;
-	_hasWork.store(false);
-	// Setup the default values
-	_state = DISABLED;
-	_strategy = TIMER_DRIVEN;
-	_lossTolerant = false;
-	_triggerWhenEmpty = false;
-	_schedulingPeriodNano = MINIMUM_SCHEDULING_NANOS;
-	_runDurantionNano = 0;
-	_yieldPeriodMsec = DEFAULT_YIELD_PERIOD_SECONDS * 1000;
-	_penalizationPeriodMsec = DEFAULT_PENALIZATION_PERIOD_SECONDS * 1000;
-	_maxConcurrentTasks = 1;
-	_activeTasks = 0;
-	_yieldExpiration = 0;
-	_incomingConnectionsIter = this->_incomingConnections.begin();
-	logger_ = Logger::getLogger();
-	logger_->log_info("Processor %s created UUID %s", _name.c_str(), _uuidStr.c_str());
-}
-
-Processor::~Processor()
-{
-
-}
-
-bool Processor::isRunning()
-{
-	return (_state == RUNNING && _activeTasks > 0);
-}
-
-void Processor::setScheduledState(ScheduledState state)
-{
-	_state = state;
-}
-
-bool Processor::setSupportedProperties(std::set<Property> properties)
-{
-	if (isRunning())
-	{
-		logger_->log_info("Can not set processor property while the process %s is running",
-				_name.c_str());
-		return false;
-	}
-
-	std::lock_guard<std::mutex> lock(_mtx);
-
-	_properties.clear();
-	for (auto item : properties)
-	{
-		_properties[item.getName()] = item;
-		logger_->log_info("Processor %s supported property name %s", _name.c_str(), item.getName().c_str());
-	}
-
-	return true;
-}
-
-bool Processor::setSupportedRelationships(std::set<Relationship> relationships)
-{
-	if (isRunning())
-	{
-		logger_->log_info("Can not set processor supported relationship while the process %s is running",
-				_name.c_str());
-		return false;
-	}
-
-	std::lock_guard<std::mutex> lock(_mtx);
-
-	_relationships.clear();
-	for(auto item : relationships)
-	{
-		_relationships[item.getName()] = item;
-		logger_->log_info("Processor %s supported relationship name %s", _name.c_str(), item.getName().c_str());
-	}
-
-	return true;
-}
-
-bool Processor::setAutoTerminatedRelationships(std::set<Relationship> relationships)
-{
-	if (isRunning())
-	{
-		logger_->log_info("Can not set processor auto terminated relationship while the process %s is running",
-				_name.c_str());
-		return false;
-	}
-
-	std::lock_guard<std::mutex> lock(_mtx);
-
-	_autoTerminatedRelationships.clear();
-	for(auto item : relationships)
-	{
-		_autoTerminatedRelationships[item.getName()] = item;
-		logger_->log_info("Processor %s auto terminated relationship name %s", _name.c_str(), item.getName().c_str());
-	}
-
-	return true;
-}
-
-bool Processor::isAutoTerminated(Relationship relationship)
-{
-	bool isRun = isRunning();
-		
-	auto conditionalLock = !isRun ? 
-			  std::unique_lock<std::mutex>() 
-			: std::unique_lock<std::mutex>(_mtx);
-
-	const auto &it = _autoTerminatedRelationships.find(relationship.getName());
-	if (it != _autoTerminatedRelationships.end())
-	{
-		return true;
-	}
-	else
-	{
-		return false;
-	}
-}
-
-bool Processor::isSupportedRelationship(Relationship relationship)
-{
-	bool isRun = isRunning();
-
-	auto conditionalLock = !isRun ? 
-			  std::unique_lock<std::mutex>() 
-			: std::unique_lock<std::mutex>(_mtx);
-
-	const auto &it = _relationships.find(relationship.getName());
-	if (it != _relationships.end())
-	{
-		return true;
-	}
-	else
-	{
-		return false;
-	}
-}
-
-bool Processor::getProperty(std::string name, std::string &value)
-{
-	bool isRun = isRunning();
-
-	
-	 auto conditionalLock = !isRun ? 
-                           std::unique_lock<std::mutex>() 
-                         : std::unique_lock<std::mutex>(_mtx);
-			 
-	const auto &it = _properties.find(name);
-	if (it != _properties.end())
-	{
-		Property item = it->second;
-		value = item.getValue();
-		return true;
-	}
-	else
-	{
-		return false;
-	}
-}
-
-bool Processor::setProperty(std::string name, std::string value)
-{
-
-	std::lock_guard<std::mutex> lock(_mtx);
-	auto &&it = _properties.find(name);
-
-	if (it != _properties.end())
-	{
-		Property item = it->second;
-		item.setValue(value);
-		_properties[item.getName()] = item;
-		logger_->log_info("Processor %s property name %s value %s", _name.c_str(), item.getName().c_str(), value.c_str());
-		return true;
-	}
-	else
-	{
-		return false;
-	}
-}
-
-bool Processor::setProperty(Property prop, std::string value) {
-
-	std::lock_guard<std::mutex> lock(_mtx);
-	auto it = _properties.find(
-			prop.getName());
-
-	if (it != _properties.end()) {
-		Property item = it->second;
-		item.setValue(value);
-		_properties[item.getName()] = item;
-		logger_->log_info("Processor %s property name %s value %s",
-				_name.c_str(), item.getName().c_str(), value.c_str());
-		return true;
-	} else {
-		Property newProp(prop);
-		newProp.setValue(value);
-		_properties.insert(
-				std::pair<std::string, Property>(prop.getName(), newProp));
-		return true;
-
-		return false;
-	}
-}
-
-std::set<Connection *> Processor::getOutGoingConnections(std::string relationship)
-{
-	std::set<Connection *> empty;
-
-	auto  &&it = _outGoingConnections.find(relationship);
-	if (it != _outGoingConnections.end())
-	{
-		return _outGoingConnections[relationship];
-	}
-	else
-	{
-		return empty;
-	}
-}
-
-bool Processor::addConnection(Connection *connection)
-{
-	bool ret = false;
-
-
-	if (isRunning())
-	{
-		logger_->log_info("Can not add connection while the process %s is running",
-				_name.c_str());
-		return false;
-	}
-
-
-	std::lock_guard<std::mutex> lock(_mtx);
-
-	uuid_t srcUUID;
-	uuid_t destUUID;
-
-	connection->getSourceProcessorUUID(srcUUID);
-	connection->getDestinationProcessorUUID(destUUID);
-	char uuid_str[37];
-
-
-	uuid_unparse_lower(_uuid, uuid_str);
-	std::string my_uuid = uuid_str;
-	uuid_unparse_lower(destUUID, uuid_str);
-	std::string destination_uuid = uuid_str;
-	if (my_uuid == destination_uuid)
-	{
-		// Connection is destination to the current processor
-		if (_incomingConnections.find(connection) == _incomingConnections.end())
-		{
-			_incomingConnections.insert(connection);
-			connection->setDestinationProcessor(this);
-			logger_->log_info("Add connection %s into Processor %s incoming connection",
-					connection->getName().c_str(), _name.c_str());
-			_incomingConnectionsIter = this->_incomingConnections.begin();
-			ret = true;
-		}
-	}
-	uuid_unparse_lower(srcUUID, uuid_str);
-	std::string source_uuid = uuid_str;
-	if (my_uuid == source_uuid)
-	{
-		std::string relationship = connection->getRelationship().getName();
-		// Connection is source from the current processor
-		auto &&it =
-				_outGoingConnections.find(relationship);
-		if (it != _outGoingConnections.end())
-		{
-			// We already has connection for this relationship
-			std::set<Connection *> existedConnection = it->second;
-			if (existedConnection.find(connection) == existedConnection.end())
-			{
-				// We do not have the same connection for this relationship yet
-				existedConnection.insert(connection);
-				connection->setSourceProcessor(this);
-				_outGoingConnections[relationship] = existedConnection;
-				logger_->log_info("Add connection %s into Processor %s outgoing connection for relationship %s",
-												connection->getName().c_str(), _name.c_str(), relationship.c_str());
-				ret = true;
-			}
-		}
-		else
-		{
-
-			// We do not have any outgoing connection for this relationship yet
-			std::set<Connection *> newConnection;
-			newConnection.insert(connection);
-			connection->setSourceProcessor(this);
-			_outGoingConnections[relationship] = newConnection;
-			logger_->log_info("Add connection %s into Processor %s outgoing connection for relationship %s",
-								connection->getName().c_str(), _name.c_str(), relationship.c_str());
-			ret = true;
-		}
-	}
-	
-
-	return ret;
-}
-
-void Processor::removeConnection(Connection *connection)
-{
-	if (isRunning())
-	{
-		logger_->log_info("Can not remove connection while the process %s is running",
-				_name.c_str());
-		return;
-	}
-
-	std::lock_guard<std::mutex> lock(_mtx);
-
-	uuid_t srcUUID;
-	uuid_t destUUID;
-
-	connection->getSourceProcessorUUID(srcUUID);
-	connection->getDestinationProcessorUUID(destUUID);
-
-	if (uuid_compare(_uuid, destUUID) == 0)
-	{
-		// Connection is destination to the current processor
-		if (_incomingConnections.find(connection) != _incomingConnections.end())
-		{
-			_incomingConnections.erase(connection);
-			connection->setDestinationProcessor(NULL);
-			logger_->log_info("Remove connection %s into Processor %s incoming connection",
-					connection->getName().c_str(), _name.c_str());
-			_incomingConnectionsIter = this->_incomingConnections.begin();
-		}
-	}
-
-	if (uuid_compare(_uuid, srcUUID) == 0)
-	{
-		std::string relationship = connection->getRelationship().getName();
-		// Connection is source from the current processor
-		auto &&it =
-				_outGoingConnections.find(relationship);
-		if (it == _outGoingConnections.end())
-		{
-			return;
-		}
-		else
-		{
-			if (_outGoingConnections[relationship].find(connection) != _outGoingConnections[relationship].end())
-			{
-				_outGoingConnections[relationship].erase(connection);
-				connection->setSourceProcessor(NULL);
-				logger_->log_info("Remove connection %s into Processor %s outgoing connection for relationship %s",
-								connection->getName().c_str(), _name.c_str(), relationship.c_str());
-			}
-		}
-	}
-}
-
-Connection *Processor::getNextIncomingConnection()
-{
-	std::lock_guard<std::mutex> lock(_mtx);
-
-	if (_incomingConnections.size() == 0)
-		return NULL;
-
-	if (_incomingConnectionsIter == _incomingConnections.end())
-		_incomingConnectionsIter = _incomingConnections.begin();
-
-	Connection *ret = *_incomingConnectionsIter;
-	_incomingConnectionsIter++;
-
-	if (_incomingConnectionsIter == _incomingConnections.end())
-		_incomingConnectionsIter = _incomingConnections.begin();
-
-	return ret;
-}
-
-bool Processor::flowFilesQueued()
-{
-	std::lock_guard<std::mutex> lock(_mtx);
-
-	if (_incomingConnections.size() == 0)
-		return false;
-
-	for(auto &&connection : _incomingConnections)
-	{
-		if (connection->getQueueSize() > 0)
-			return true;
-	}
-
-	return false;
-}
-
-bool Processor::flowFilesOutGoingFull()
-{
-	std::lock_guard<std::mutex> lock(_mtx);
-
-	 for(auto &&connection : _outGoingConnections)
-	{
-		// We already has connection for this relationship
-		std::set<Connection *> existedConnection = connection.second;
-		for(const auto connection : existedConnection)
-		{
-			if (connection->isFull())
-				return true;
-		}
-	}
-
-	return false;
-}
-
-void Processor::onTrigger(ProcessContext *context, ProcessSessionFactory *sessionFactory)
-{
-	auto session = sessionFactory->createSession();
-
-	try {
-		// Call the virtual trigger function
-		onTrigger(context, session.get());
-		session->commit();
-	}
-	catch (std::exception &exception)
-	{
-		logger_->log_debug("Caught Exception %s", exception.what());
-		session->rollback();
-		throw;
-	}
-	catch (...)
-	{
-		logger_->log_debug("Caught Exception Processor::onTrigger");
-		session->rollback();
-		throw;
-	}
-}
-
-void Processor::waitForWork(uint64_t timeoutMs)
-{
-	_hasWork.store( isWorkAvailable() );
-
-	if (!_hasWork.load())
-	{
-	    std::unique_lock<std::mutex> lock(_workAvailableMtx);
-	    _hasWorkCondition.wait_for(lock, std::chrono::milliseconds(timeoutMs), [&] { return _hasWork.load(); });
-	}
-
-}
-
-void Processor::notifyWork()
-{
-	// Do nothing if we are not event-driven
-	if (_strategy != EVENT_DRIVEN)
-	{
-		return;
-	}
-
-	{
-		_hasWork.store( isWorkAvailable() );
-
-
-		if (_hasWork.load())
-		{
-		      _hasWorkCondition.notify_one();
-		}
-	}
-}
-
-bool Processor::isWorkAvailable()
-{
-	// We have work if any incoming connection has work
-	bool hasWork = false;
-
-	try
-	{
-		for (const auto &conn : getIncomingConnections())
-		{
-			if (conn->getQueueSize() > 0)
-			{
-				hasWork = true;
-				break;
-			}
-		}
-	}
-	catch (...)
-	{
-		logger_->log_error("Caught an exception while checking if work is available; unless it was positively determined that work is available, assuming NO work is available!");
-	}
-
-	return hasWork;
-}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/Provenance.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/Provenance.cpp b/libminifi/src/Provenance.cpp
deleted file mode 100644
index 58cf730..0000000
--- a/libminifi/src/Provenance.cpp
+++ /dev/null
@@ -1,566 +0,0 @@
-/**
- * @file Provenance.cpp
- * Provenance implemenatation 
- *
- * 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 <cstdint>
-#include <vector>
-#include <arpa/inet.h>
-#include "io/DataStream.h"
-#include "io/Serializable.h"
-#include "Provenance.h"
-#include "Relationship.h"
-#include "Logger.h"
-#include "FlowController.h"
-
-//! DeSerialize
-bool ProvenanceEventRecord::DeSerialize(ProvenanceRepository *repo,
-		std::string key) {
-	std::string value;
-	bool ret;
-
-	ret = repo->Get(key, value);
-
-	if (!ret) {
-		logger_->log_error("NiFi Provenance Store event %s can not found",
-				key.c_str());
-		return false;
-	} else
-		logger_->log_debug("NiFi Provenance Read event %s length %d",
-				key.c_str(), value.length());
-
-
-	DataStream stream((const uint8_t*)value.data(),value.length());
-
-	ret = DeSerialize(stream);
-
-	if (ret) {
-		logger_->log_debug(
-				"NiFi Provenance retrieve event %s size %d eventType %d success",
-				_eventIdStr.c_str(), stream.getSize(), _eventType);
-	} else {
-		logger_->log_debug(
-				"NiFi Provenance retrieve event %s size %d eventType %d fail",
-				_eventIdStr.c_str(), stream.getSize(), _eventType);
-	}
-
-	return ret;
-}
-
-bool ProvenanceEventRecord::Serialize(ProvenanceRepository *repo) {
-
-	DataStream outStream;
-
-	int ret;
-
-	ret = writeUTF(this->_eventIdStr,&outStream);
-	if (ret <= 0) {
-
-		return false;
-	}
-
-	uint32_t eventType = this->_eventType;
-	ret = write(eventType,&outStream);
-	if (ret != 4) {
-
-		return false;
-	}
-
-	ret = write(this->_eventTime,&outStream);
-	if (ret != 8) {
-
-		return false;
-	}
-
-	ret = write(this->_entryDate,&outStream);
-	if (ret != 8) {
-		return false;
-	}
-
-	ret = write(this->_eventDuration,&outStream);
-	if (ret != 8) {
-
-		return false;
-	}
-
-	ret = write(this->_lineageStartDate,&outStream);
-	if (ret != 8) {
-
-		return false;
-	}
-
-	ret = writeUTF(this->_componentId,&outStream);
-	if (ret <= 0) {
-
-		return false;
-	}
-
-	ret = writeUTF(this->_componentType,&outStream);
-	if (ret <= 0) {
-
-		return false;
-	}
-
-	ret = writeUTF(this->_uuid,&outStream);
-	if (ret <= 0) {
-
-		return false;
-	}
-
-	ret = writeUTF(this->_details,&outStream);
-	if (ret <= 0) {
-
-		return false;
-	}
-
-	// write flow attributes
-	uint32_t numAttributes = this->_attributes.size();
-	ret = write(numAttributes,&outStream);
-	if (ret != 4) {
-
-		return false;
-	}
-
-	for (auto itAttribute : _attributes) {
-		ret = writeUTF(itAttribute.first,&outStream, true);
-		if (ret <= 0) {
-
-			return false;
-		}
-		ret = writeUTF(itAttribute.second,&outStream, true);
-		if (ret <= 0) {
-
-			return false;
-		}
-	}
-
-	ret = writeUTF(this->_contentFullPath,&outStream);
-	if (ret <= 0) {
-
-		return false;
-	}
-
-	ret = write(this->_size,&outStream);
-	if (ret != 8) {
-
-		return false;
-	}
-
-	ret = write(this->_offset,&outStream);
-	if (ret != 8) {
-
-		return false;
-	}
-
-	ret = writeUTF(this->_sourceQueueIdentifier,&outStream);
-	if (ret <= 0) {
-
-		return false;
-	}
-
-	if (this->_eventType == ProvenanceEventRecord::FORK
-			|| this->_eventType == ProvenanceEventRecord::CLONE
-			|| this->_eventType == ProvenanceEventRecord::JOIN) {
-		// write UUIDs
-		uint32_t number = this->_parentUuids.size();
-		ret = write(number,&outStream);
-		if (ret != 4) {
-
-			return false;
-		}
-		for (auto parentUUID : _parentUuids) {
-			ret = writeUTF(parentUUID,&outStream);
-			if (ret <= 0) {
-
-				return false;
-			}
-		}
-		number = this->_childrenUuids.size();
-		ret = write(number,&outStream);
-		if (ret != 4) {
-			return false;
-		}
-		for (auto childUUID : _childrenUuids) {
-			ret = writeUTF(childUUID,&outStream);
-			if (ret <= 0) {
-
-				return false;
-			}
-		}
-	} else if (this->_eventType == ProvenanceEventRecord::SEND
-			|| this->_eventType == ProvenanceEventRecord::FETCH) {
-		ret = writeUTF(this->_transitUri,&outStream);
-		if (ret <= 0) {
-
-			return false;
-		}
-	} else if (this->_eventType == ProvenanceEventRecord::RECEIVE) {
-		ret = writeUTF(this->_transitUri,&outStream);
-		if (ret <= 0) {
-
-			return false;
-		}
-		ret = writeUTF(this->_sourceSystemFlowFileIdentifier,&outStream);
-		if (ret <= 0) {
-
-			return false;
-		}
-	}
-
-	// 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());
-		return true;
-	} else {
-		logger_->log_error("NiFi Provenance Store event %s size %d fail",
-				_eventIdStr.c_str(), outStream.getSize());
-		return false;
-	}
-
-	// cleanup
-
-	return true;
-}
-
-bool ProvenanceEventRecord::DeSerialize(const uint8_t *buffer, const int bufferSize) {
-
-	int ret;
-
-	DataStream outStream(buffer,bufferSize);
-
-	ret = readUTF(this->_eventIdStr,&outStream);
-
-	if (ret <= 0) {
-		return false;
-	}
-
-	uint32_t eventType;
-	ret = read(eventType,&outStream);
-	if (ret != 4) {
-		return false;
-	}
-	this->_eventType = (ProvenanceEventRecord::ProvenanceEventType) eventType;
-
-	ret = read(this->_eventTime,&outStream);
-	if (ret != 8) {
-		return false;
-	}
-
-	ret = read(this->_entryDate,&outStream);
-	if (ret != 8) {
-		return false;
-	}
-
-	ret = read(this->_eventDuration,&outStream);
-	if (ret != 8) {
-		return false;
-	}
-
-	ret = read(this->_lineageStartDate,&outStream);
-	if (ret != 8) {
-		return false;
-	}
-
-	ret = readUTF(this->_componentId,&outStream);
-	if (ret <= 0) {
-		return false;
-	}
-
-	ret = readUTF(this->_componentType,&outStream);
-	if (ret <= 0) {
-		return false;
-	}
-
-	ret = readUTF(this->_uuid,&outStream);
-	if (ret <= 0) {
-		return false;
-	}
-
-	ret = readUTF(this->_details,&outStream);
-
-	if (ret <= 0) {
-		return false;
-	}
-
-	// read flow attributes
-	uint32_t numAttributes = 0;
-	ret = read(numAttributes,&outStream);
-	if (ret != 4) {
-		return false;
-	}
-
-	for (uint32_t i = 0; i < numAttributes; i++) {
-		std::string key;
-		ret = readUTF(key,&outStream, true);
-		if (ret <= 0) {
-			return false;
-		}
-		std::string value;
-		ret = readUTF(value,&outStream, true);
-		if (ret <= 0) {
-			return false;
-		}
-		this->_attributes[key] = value;
-	}
-
-	ret = readUTF(this->_contentFullPath,&outStream);
-	if (ret <= 0) {
-		return false;
-	}
-
-	ret = read(this->_size,&outStream);
-	if (ret != 8) {
-		return false;
-	}
-
-	ret = read(this->_offset,&outStream);
-	if (ret != 8) {
-		return false;
-	}
-
-	ret = readUTF(this->_sourceQueueIdentifier,&outStream);
-	if (ret <= 0) {
-		return false;
-	}
-
-	if (this->_eventType == ProvenanceEventRecord::FORK
-			|| this->_eventType == ProvenanceEventRecord::CLONE
-			|| this->_eventType == ProvenanceEventRecord::JOIN) {
-		// read UUIDs
-		uint32_t number = 0;
-		ret = read(number,&outStream);
-		if (ret != 4) {
-			return false;
-		}
-
-
-		for (uint32_t i = 0; i < number; i++) {
-			std::string parentUUID;
-			ret = readUTF(parentUUID,&outStream);
-			if (ret <= 0) {
-				return false;
-			}
-			this->addParentUuid(parentUUID);
-		}
-		number = 0;
-		ret = read(number,&outStream);
-		if (ret != 4) {
-			return false;
-		}
-		for (uint32_t i = 0; i < number; i++) {
-			std::string childUUID;
-			ret = readUTF(childUUID,&outStream);
-			if (ret <= 0) {
-				return false;
-			}
-			this->addChildUuid(childUUID);
-		}
-	} else if (this->_eventType == ProvenanceEventRecord::SEND
-			|| this->_eventType == ProvenanceEventRecord::FETCH) {
-		ret = readUTF(this->_transitUri,&outStream);
-		if (ret <= 0) {
-			return false;
-		}
-	} else if (this->_eventType == ProvenanceEventRecord::RECEIVE) {
-		ret = readUTF(this->_transitUri,&outStream);
-		if (ret <= 0) {
-			return false;
-		}
-		ret = readUTF(this->_sourceSystemFlowFileIdentifier,&outStream);
-		if (ret <= 0) {
-			return false;
-		}
-	}
-
-	return true;
-}
-
-void ProvenanceReporter::commit() {
-	if (!FlowControllerFactory::getFlowController()->getProvenanceRepository()->isEnable())
-		return;
-	for (auto event : _events) {
-		if (!FlowControllerFactory::getFlowController()->getProvenanceRepository()->isFull()) {
-			event->Serialize(
-					FlowControllerFactory::getFlowController()->getProvenanceRepository());
-		} else {
-			logger_->log_debug("Provenance Repository is full");
-		}
-	}
-}
-
-void ProvenanceReporter::create(FlowFileRecord *flow, std::string detail) {
-	ProvenanceEventRecord *event = allocate(ProvenanceEventRecord::CREATE,
-			flow);
-
-	if (event) {
-		event->setDetails(detail);
-		add(event);
-	}
-}
-
-void ProvenanceReporter::route(FlowFileRecord *flow, Relationship relation,
-		std::string detail, uint64_t processingDuration) {
-	ProvenanceEventRecord *event = allocate(ProvenanceEventRecord::ROUTE, flow);
-
-	if (event) {
-		event->setDetails(detail);
-		event->setRelationship(relation.getName());
-		event->setEventDuration(processingDuration);
-		add(event);
-	}
-}
-
-void ProvenanceReporter::modifyAttributes(FlowFileRecord *flow,
-		std::string detail) {
-	ProvenanceEventRecord *event = allocate(
-			ProvenanceEventRecord::ATTRIBUTES_MODIFIED, flow);
-
-	if (event) {
-		event->setDetails(detail);
-		add(event);
-	}
-}
-
-void ProvenanceReporter::modifyContent(FlowFileRecord *flow, std::string detail,
-		uint64_t processingDuration) {
-	ProvenanceEventRecord *event = allocate(
-			ProvenanceEventRecord::CONTENT_MODIFIED, flow);
-
-	if (event) {
-		event->setDetails(detail);
-		event->setEventDuration(processingDuration);
-		add(event);
-	}
-}
-
-void ProvenanceReporter::clone(FlowFileRecord *parent, FlowFileRecord *child) {
-	ProvenanceEventRecord *event = allocate(ProvenanceEventRecord::CLONE,
-			parent);
-
-	if (event) {
-		event->addChildFlowFile(child);
-		event->addParentFlowFile(parent);
-		add(event);
-	}
-}
-
-void ProvenanceReporter::join(std::vector<FlowFileRecord *> parents,
-		FlowFileRecord *child, std::string detail,
-		uint64_t processingDuration) {
-	ProvenanceEventRecord *event = allocate(ProvenanceEventRecord::JOIN, child);
-
-	if (event) {
-		event->addChildFlowFile(child);
-		std::vector<FlowFileRecord *>::iterator it;
-		for (it = parents.begin(); it != parents.end(); it++) {
-			FlowFileRecord *record = *it;
-			event->addParentFlowFile(record);
-		}
-		event->setDetails(detail);
-		event->setEventDuration(processingDuration);
-		add(event);
-	}
-}
-
-void ProvenanceReporter::fork(std::vector<FlowFileRecord *> child,
-		FlowFileRecord *parent, std::string detail,
-		uint64_t processingDuration) {
-	ProvenanceEventRecord *event = allocate(ProvenanceEventRecord::FORK,
-			parent);
-
-	if (event) {
-		event->addParentFlowFile(parent);
-		std::vector<FlowFileRecord *>::iterator it;
-		for (it = child.begin(); it != child.end(); it++) {
-			FlowFileRecord *record = *it;
-			event->addChildFlowFile(record);
-		}
-		event->setDetails(detail);
-		event->setEventDuration(processingDuration);
-		add(event);
-	}
-}
-
-void ProvenanceReporter::expire(FlowFileRecord *flow, std::string detail) {
-	ProvenanceEventRecord *event = allocate(ProvenanceEventRecord::EXPIRE,
-			flow);
-
-	if (event) {
-		event->setDetails(detail);
-		add(event);
-	}
-}
-
-void ProvenanceReporter::drop(FlowFileRecord *flow, std::string reason) {
-	ProvenanceEventRecord *event = allocate(ProvenanceEventRecord::DROP, flow);
-
-	if (event) {
-		std::string dropReason = "Discard reason: " + reason;
-		event->setDetails(dropReason);
-		add(event);
-	}
-}
-
-void ProvenanceReporter::send(FlowFileRecord *flow, std::string transitUri,
-		std::string detail, uint64_t processingDuration, bool force) {
-	ProvenanceEventRecord *event = allocate(ProvenanceEventRecord::SEND, flow);
-
-	if (event) {
-		event->setTransitUri(transitUri);
-		event->setDetails(detail);
-		event->setEventDuration(processingDuration);
-		if (!force) {
-			add(event);
-		} else {
-			if (!FlowControllerFactory::getFlowController()->getProvenanceRepository()->isFull())
-				event->Serialize(
-						FlowControllerFactory::getFlowController()->getProvenanceRepository());
-			delete event;
-		}
-	}
-}
-
-void ProvenanceReporter::receive(FlowFileRecord *flow, std::string transitUri,
-		std::string sourceSystemFlowFileIdentifier, std::string detail,
-		uint64_t processingDuration) {
-	ProvenanceEventRecord *event = allocate(ProvenanceEventRecord::RECEIVE,
-			flow);
-
-	if (event) {
-		event->setTransitUri(transitUri);
-		event->setDetails(detail);
-		event->setEventDuration(processingDuration);
-		event->setSourceSystemFlowFileIdentifier(
-				sourceSystemFlowFileIdentifier);
-		add(event);
-	}
-}
-
-void ProvenanceReporter::fetch(FlowFileRecord *flow, std::string transitUri,
-		std::string detail, uint64_t processingDuration) {
-	ProvenanceEventRecord *event = allocate(ProvenanceEventRecord::FETCH, flow);
-
-	if (event) {
-		event->setTransitUri(transitUri);
-		event->setDetails(detail);
-		event->setEventDuration(processingDuration);
-		add(event);
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/44704b36/libminifi/src/PutFile.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/PutFile.cpp b/libminifi/src/PutFile.cpp
deleted file mode 100644
index d7cc83a..0000000
--- a/libminifi/src/PutFile.cpp
+++ /dev/null
@@ -1,203 +0,0 @@
-/**
- * @file PutFile.cpp
- * PutFile class implementation
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#include <sstream>
-#include <stdio.h>
-#include <string>
-#include <iostream>
-#include <fstream>
-#include <uuid/uuid.h>
-
-#include "utils/StringUtils.h"
-#include "utils/TimeUtil.h"
-#include "PutFile.h"
-#include "ProcessContext.h"
-#include "ProcessSession.h"
-
-const std::string PutFile::CONFLICT_RESOLUTION_STRATEGY_REPLACE("replace");
-const std::string PutFile::CONFLICT_RESOLUTION_STRATEGY_IGNORE("ignore");
-const std::string PutFile::CONFLICT_RESOLUTION_STRATEGY_FAIL("fail");
-
-const std::string PutFile::ProcessorName("PutFile");
-
-Property PutFile::Directory("Output Directory", "The output directory to which to put files", ".");
-Property PutFile::ConflictResolution("Conflict Resolution Strategy", "Indicates what should happen when a file with the same name already exists in the output directory", CONFLICT_RESOLUTION_STRATEGY_FAIL);
-
-Relationship PutFile::Success("success", "All files are routed to success");
-Relationship PutFile::Failure("failure", "Failed files (conflict, write failure, etc.) are transferred to failure");
-
-void PutFile::initialize()
-{
-	//! Set the supported properties
-	std::set<Property> properties;
-	properties.insert(Directory);
-	properties.insert(ConflictResolution);
-	setSupportedProperties(properties);
-	//! Set the supported relationships
-	std::set<Relationship> relationships;
-	relationships.insert(Success);
-	relationships.insert(Failure);
-	setSupportedRelationships(relationships);
-}
-
-void PutFile::onTrigger(ProcessContext *context, ProcessSession *session)
-{
-	std::string directory;
-
-	if (!context->getProperty(Directory.getName(), directory))
-	{
-		logger_->log_error("Directory attribute is missing or invalid");
-		return;
-	}
-
-	std::string conflictResolution;
-
-	if (!context->getProperty(ConflictResolution.getName(), conflictResolution))
-	{
-		logger_->log_error("Conflict Resolution Strategy attribute is missing or invalid");
-		return;
-	}
-
-	FlowFileRecord *flowFile = session->get();
-
-	// Do nothing if there are no incoming files
-	if (!flowFile)
-	{
-		return;
-	}
-
-	std::string filename;
-	flowFile->getAttribute(FILENAME, filename);
-
-	// Generate a safe (universally-unique) temporary filename on the same partition 
-	char tmpFileUuidStr[37];
-	uuid_t tmpFileUuid;
-	uuid_generate(tmpFileUuid);
-	uuid_unparse_lower(tmpFileUuid, tmpFileUuidStr);
-	std::stringstream tmpFileSs;
-	tmpFileSs << directory << "/." << filename << "." << tmpFileUuidStr;
-	std::string tmpFile = tmpFileSs.str();
-	logger_->log_info("PutFile using temporary file %s", tmpFile.c_str());
-
-	// Determine dest full file paths
-	std::stringstream destFileSs;
-	destFileSs << directory << "/" << filename;
-	std::string destFile = destFileSs.str();
-
-	logger_->log_info("PutFile writing file %s into directory %s", filename.c_str(), directory.c_str());
-
-	// If file exists, apply conflict resolution strategy
-	struct stat statResult;
-
-	if (stat(destFile.c_str(), &statResult) == 0)
-	{
-		logger_->log_info("Destination file %s exists; applying Conflict Resolution Strategy: %s", destFile.c_str(), conflictResolution.c_str());
-
-		if (conflictResolution == CONFLICT_RESOLUTION_STRATEGY_REPLACE)
-		{
-			putFile(session, flowFile, tmpFile, destFile);
-		}
-		else if (conflictResolution == CONFLICT_RESOLUTION_STRATEGY_IGNORE)
-		{
-			session->transfer(flowFile, Success);
-		}
-		else
-		{
-			session->transfer(flowFile, Failure);
-		}
-	}
-	else
-	{
-		putFile(session, flowFile, tmpFile, destFile);
-	}
-}
-
-bool PutFile::putFile(ProcessSession *session, FlowFileRecord *flowFile, const std::string &tmpFile, const std::string &destFile)
-{
-
-	ReadCallback cb(tmpFile, destFile);
-    	session->read(flowFile, &cb);
-
-	if (cb.commit())
-	{
-		session->transfer(flowFile, Success);
-		return true;
-	}
-	else
-	{
-		session->transfer(flowFile, Failure);
-	}
-	return false;
-}
-
-PutFile::ReadCallback::ReadCallback(const std::string &tmpFile, const std::string &destFile)
-: _tmpFile(tmpFile)
-, _tmpFileOs(tmpFile)
-, _destFile(destFile)
-{
-	logger_ = Logger::getLogger();
-}
-
-// Copy the entire file contents to the temporary file
-void PutFile::ReadCallback::process(std::ifstream *stream)
-{
-	// Copy file contents into tmp file
-	_writeSucceeded = false;
-	_tmpFileOs << stream->rdbuf();
-	_writeSucceeded = true;
-}
-
-// Renames tmp file to final destination
-// Returns true if commit succeeded
-bool PutFile::ReadCallback::commit()
-{
-	bool success = false;
-
-	logger_->log_info("PutFile committing put file operation to %s", _destFile.c_str());
-
-	if (_writeSucceeded)
-	{
-		_tmpFileOs.close();
-
-		if (rename(_tmpFile.c_str(), _destFile.c_str()))
-		{
-			logger_->log_info("PutFile commit put file operation to %s failed because rename() call failed", _destFile.c_str());
-		}
-		else
-		{
-			success = true;
-			logger_->log_info("PutFile commit put file operation to %s succeeded", _destFile.c_str());
-		}
-	}
-	else
-	{
-		logger_->log_error("PutFile commit put file operation to %s failed because write failed", _destFile.c_str());
-	}
-
-	return success;
-}
-
-// Clean up resources
-PutFile::ReadCallback::~ReadCallback() {
-	// Close tmp file
-	_tmpFileOs.close();
-
-	// Clean up tmp file, if necessary
-	unlink(_tmpFile.c_str());
-}