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/04/05 16:11:45 UTC

[1/4] nifi-minifi-cpp git commit: MINIFI-254: Incremental update for linter changes

Repository: nifi-minifi-cpp
Updated Branches:
  refs/heads/master 3676468fc -> be3f2ffea


http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/processors/ListenSyslog.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/ListenSyslog.cpp b/libminifi/src/processors/ListenSyslog.cpp
index 2dd223c..e7d2e7b 100644
--- a/libminifi/src/processors/ListenSyslog.cpp
+++ b/libminifi/src/processors/ListenSyslog.cpp
@@ -17,12 +17,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-#include <queue>
+#include "processors/ListenSyslog.h"
 #include <stdio.h>
+#include <memory>
 #include <string>
+#include <vector>
+#include <set>
+#include <queue>
 #include "utils/TimeUtil.h"
 #include "utils/StringUtils.h"
-#include "processors/ListenSyslog.h"
 #include "core/ProcessContext.h"
 #include "core/ProcessSession.h"
 
@@ -32,7 +35,6 @@ 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");
@@ -48,20 +50,22 @@ core::Property ListenSyslog::MaxBatchSize(
     "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).",
+    "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");
+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
@@ -125,7 +129,7 @@ void ListenSyslog::runThread() {
         logger_->log_info("ListenSysLog Server socket creation failed");
         break;
       }
-      bzero((char *) &serv_addr, sizeof(serv_addr));
+      bzero(reinterpret_cast<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);
@@ -167,7 +171,8 @@ void ListenSyslog::runThread() {
         socklen_t clilen;
         struct sockaddr_in cli_addr;
         clilen = sizeof(cli_addr);
-        int newsockfd = accept(_serverSocket, (struct sockaddr *) &cli_addr,
+        int newsockfd = accept(_serverSocket,
+                               reinterpret_cast<struct sockaddr *>(&cli_addr),
                                &clilen);
         if (newsockfd > 0) {
           if (_clientSockets.size() < _maxConnections) {
@@ -197,7 +202,7 @@ void ListenSyslog::runThread() {
     while (it != _clientSockets.end()) {
       int clientSocket = *it;
       if (FD_ISSET(clientSocket, &fds)) {
-        int recvlen = readline(clientSocket, (char *) _buffer, sizeof(_buffer));
+        int recvlen = readline(clientSocket, _buffer, sizeof(_buffer));
         if (recvlen <= 0) {
           close(clientSocket);
           logger_->log_info("ListenSysLog client socket %d close",
@@ -228,7 +233,7 @@ int ListenSyslog::readline(int fd, char *bufptr, size_t len) {
     if (--cnt <= 0) {
       cnt = recv(fd, b, sizeof(b), 0);
       if (cnt < 0) {
-        if ( errno == EINTR) {
+        if (errno == EINTR) {
           len++; /* the while will decrement */
           continue;
         }
@@ -248,9 +253,8 @@ int ListenSyslog::readline(int fd, char *bufptr, size_t len) {
   return -1;
 }
 
-void ListenSyslog::onTrigger(
-    core::ProcessContext *context,
-    core::ProcessSession *session) {
+void ListenSyslog::onTrigger(core::ProcessContext *context,
+                             core::ProcessSession *session) {
   std::string value;
   bool needResetServerSocket = false;
   if (context->getProperty(Protocol.getName(), value)) {
@@ -262,12 +266,10 @@ void ListenSyslog::onTrigger(
     core::Property::StringToInt(value, _recvBufSize);
   }
   if (context->getProperty(MaxSocketBufSize.getName(), value)) {
-    core::Property::StringToInt(value,
-                                                           _maxSocketBufSize);
+    core::Property::StringToInt(value, _maxSocketBufSize);
   }
   if (context->getProperty(MaxConnections.getName(), value)) {
-    core::Property::StringToInt(value,
-                                                           _maxConnections);
+    core::Property::StringToInt(value, _maxConnections);
   }
   if (context->getProperty(MessageDelimiter.getName(), value)) {
     _messageDelimiter = value;
@@ -283,8 +285,7 @@ void ListenSyslog::onTrigger(
       needResetServerSocket = true;
   }
   if (context->getProperty(MaxBatchSize.getName(), value)) {
-    core::Property::StringToInt(value,
-                                                           _maxBatchSize);
+    core::Property::StringToInt(value, _maxBatchSize);
   }
 
   if (needResetServerSocket)
@@ -309,12 +310,12 @@ void ListenSyslog::onTrigger(
       flowFile = std::static_pointer_cast<FlowFileRecord>(session->create());
       if (!flowFile)
         return;
-      ListenSyslog::WriteCallback callback((char *) event.payload, event.len);
+      ListenSyslog::WriteCallback callback(event.payload, event.len);
       session->write(flowFile, &callback);
       delete[] event.payload;
       firstEvent = false;
     } else {
-      ListenSyslog::WriteCallback callback((char *) event.payload, event.len);
+      ListenSyslog::WriteCallback callback(event.payload, event.len);
       session->append(flowFile, &callback);
       delete[] event.payload;
     }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/processors/LogAttribute.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/LogAttribute.cpp b/libminifi/src/processors/LogAttribute.cpp
index e2cf16c..d2dcd10 100644
--- a/libminifi/src/processors/LogAttribute.cpp
+++ b/libminifi/src/processors/LogAttribute.cpp
@@ -17,19 +17,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+#include "processors/LogAttribute.h"
+#include <sys/time.h>
+#include <time.h>
+#include <string.h>
+#include <memory>
+#include <string>
 #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"
 
@@ -38,7 +39,6 @@ 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(
@@ -51,7 +51,8 @@ core::Property LogAttribute::AttributesToIgnore(
     "");
 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.",
+    "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",
@@ -75,16 +76,14 @@ void LogAttribute::initialize() {
   setSupportedRelationships(relationships);
 }
 
-void LogAttribute::onTrigger(
-    core::ProcessContext *context,
-    core::ProcessSession *session) {
+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();
+  std::shared_ptr<core::FlowFile> flow = session->get();
 
   if (!flow)
     return;
@@ -97,7 +96,8 @@ void LogAttribute::onTrigger(
     dashLine = "-----" + value + "-----";
   }
   if (context->getProperty(LogPayload.getName(), value)) {
-    org::apache::nifi::minifi::utils::StringUtils::StringToBool(value, logPayload);
+    org::apache::nifi::minifi::utils::StringUtils::StringToBool(value,
+                                                                logPayload);
   }
 
   message << "Logging for flow file " << "\n";
@@ -125,9 +125,7 @@ void LogAttribute::onTrigger(
     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;
+      message << std::hex << callback._buffer[i];
       j++;
       if (j == 16) {
         message << '\n';
@@ -168,7 +166,6 @@ void LogAttribute::onTrigger(
   session->transfer(flow, Success);
 }
 
-
 } /* namespace processors */
 } /* namespace minifi */
 } /* namespace nifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/processors/PutFile.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/PutFile.cpp b/libminifi/src/processors/PutFile.cpp
index 51fbb6f..bd08877 100644
--- a/libminifi/src/processors/PutFile.cpp
+++ b/libminifi/src/processors/PutFile.cpp
@@ -17,17 +17,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-#include <sstream>
+#include "processors/PutFile.h"
 #include <stdio.h>
+#include <uuid/uuid.h>
+#include <sstream>
 #include <string>
 #include <iostream>
+#include <memory>
+#include <set>
 #include <fstream>
-#include <uuid/uuid.h>
-
 #include "io/validation.h"
 #include "utils/StringUtils.h"
 #include "utils/TimeUtil.h"
-#include "processors/PutFile.h"
 #include "core/ProcessContext.h"
 #include "core/ProcessSession.h"
 
@@ -37,12 +38,6 @@ 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",
                                   ".");
@@ -72,7 +67,6 @@ void PutFile::initialize() {
 
 void PutFile::onSchedule(core::ProcessContext *context,
                          core::ProcessSessionFactory *sessionFactory) {
-
   if (!context->getProperty(Directory.getName(), directory_)) {
     logger_->log_error("Directory attribute is missing or invalid");
   }
@@ -82,11 +76,10 @@ void PutFile::onSchedule(core::ProcessContext *context,
     logger_->log_error(
         "Conflict Resolution Strategy attribute is missing or invalid");
   }
-
 }
+
 void PutFile::onTrigger(core::ProcessContext *context,
                         core::ProcessSession *session) {
-
   if (IsNullOrEmpty(directory_) || IsNullOrEmpty(conflict_resolution_)) {
     context->yield();
     return;
@@ -144,7 +137,6 @@ void PutFile::onTrigger(core::ProcessContext *context,
 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);
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/processors/RealTimeDataCollector.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/RealTimeDataCollector.cpp b/libminifi/src/processors/RealTimeDataCollector.cpp
deleted file mode 100644
index 922835d..0000000
--- a/libminifi/src/processors/RealTimeDataCollector.cpp
+++ /dev/null
@@ -1,480 +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 <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 */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/processors/TailFile.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/TailFile.cpp b/libminifi/src/processors/TailFile.cpp
index bcdd8fd..abb02ca 100644
--- a/libminifi/src/processors/TailFile.cpp
+++ b/libminifi/src/processors/TailFile.cpp
@@ -17,22 +17,23 @@
  * 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 <vector>
+#include <queue>
+#include <map>
+#include <set>
+#include <memory>
+#include <algorithm>
+#include <sstream>
+#include <string>
+#include <iostream>
 #include "utils/TimeUtil.h"
 #include "utils/StringUtils.h"
 #include "processors/TailFile.h"
@@ -45,16 +46,16 @@ 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",
+    "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");
+core::Relationship TailFile::Success("success",
+                                     "All files are routed to success");
 
 void TailFile::initialize() {
   // Set the supported properties
@@ -123,9 +124,9 @@ void TailFile::recoverState() {
     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)) {
+  char buf[BUFFER_SIZE];
+  for (file.getline(buf, BUFFER_SIZE); file.good();
+      file.getline(buf, BUFFER_SIZE)) {
     parseStateFileLine(buf);
   }
 }
@@ -145,7 +146,8 @@ static bool sortTailMatchedFileItem(TailMatchedFileItem i,
                                     TailMatchedFileItem j) {
   return (i.modifiedTime < j.modifiedTime);
 }
-void TailFile::checkRollOver(const std::string &fileLocation, const std::string &fileName) {
+void TailFile::checkRollOver(const std::string &fileLocation,
+                             const std::string &fileName) {
   struct stat statbuf;
   std::vector<TailMatchedFileItem> matchedFiles;
   std::string fullPath = fileLocation + "/" + _currentTailFileName;
@@ -208,18 +210,17 @@ void TailFile::checkRollOver(const std::string &fileLocation, const std::string
         break;
       }
     }
-  } else
+  } else {
     return;
+  }
 }
 
-void TailFile::onTrigger(
-    core::ProcessContext *context,
-    core::ProcessSession *session) {
-
+void TailFile::onTrigger(core::ProcessContext *context,
+                         core::ProcessSession *session) {
   std::lock_guard<std::mutex> tail_lock(tail_file_mutex_);
   std::string value;
-  std::string fileLocation="";
-  std::string fileName="";
+  std::string fileLocation = "";
+  std::string fileName = "";
   if (context->getProperty(FileName.getName(), value)) {
     std::size_t found = value.find_last_of("/\\");
     fileLocation = value.substr(0, found);
@@ -235,17 +236,17 @@ void TailFile::onTrigger(
     // recover the state if we have not done so
     this->recoverState();
   }
-  checkRollOver(fileLocation,fileName);
+  checkRollOver(fileLocation, fileName);
   std::string fullPath = 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
-        {
+    if (statbuf.st_size <= this->_currentTailFilePosition) {
+      // there are no new input for the current tail fil
       context->yield();
       return;
     }
-    std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast<FlowFileRecord>(session->create());
+    std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast<
+        FlowFileRecord>(session->create());
     if (!flowFile)
       return;
     std::size_t found = _currentTailFileName.find_last_of(".");
@@ -267,7 +268,6 @@ void TailFile::onTrigger(
   }
 }
 
-
 } /* namespace processors */
 } /* namespace minifi */
 } /* namespace nifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/provenance/Provenance.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/provenance/Provenance.cpp b/libminifi/src/provenance/Provenance.cpp
index 289f026..b1db9a8 100644
--- a/libminifi/src/provenance/Provenance.cpp
+++ b/libminifi/src/provenance/Provenance.cpp
@@ -15,13 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
+#include <arpa/inet.h>
 #include <cstdint>
+#include <memory>
+#include <string>
 #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"
@@ -44,9 +46,10 @@ bool ProvenanceEventRecord::DeSerialize(
     logger_->log_error("NiFi Provenance Store event %s can not found",
                        key.c_str());
     return false;
-  } else
+  } 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());
@@ -75,20 +78,17 @@ bool ProvenanceEventRecord::Serialize(
 
   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;
   }
 
@@ -99,37 +99,31 @@ bool ProvenanceEventRecord::Serialize(
 
   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;
   }
 
@@ -137,44 +131,37 @@ bool ProvenanceEventRecord::Serialize(
   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;
   }
 
@@ -185,13 +172,11 @@ bool ProvenanceEventRecord::Serialize(
     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;
       }
     }
@@ -203,7 +188,6 @@ bool ProvenanceEventRecord::Serialize(
     for (auto childUUID : _childrenUuids) {
       ret = writeUTF(childUUID, &outStream);
       if (ret <= 0) {
-
         return false;
       }
     }
@@ -211,24 +195,19 @@ bool ProvenanceEventRecord::Serialize(
       || 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",
@@ -237,16 +216,11 @@ bool ProvenanceEventRecord::Serialize(
     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);

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/provenance/ProvenanceRepository.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/provenance/ProvenanceRepository.cpp b/libminifi/src/provenance/ProvenanceRepository.cpp
index 6fe332b..77de5ba 100644
--- a/libminifi/src/provenance/ProvenanceRepository.cpp
+++ b/libminifi/src/provenance/ProvenanceRepository.cpp
@@ -16,22 +16,21 @@
  * limitations under the License.
  */
 
-#include "provenance/Provenance.h"
 #include "provenance/ProvenanceRepository.h"
-
+#include <string>
+#include <vector>
+#include "provenance/Provenance.h"
 namespace org {
 namespace apache {
 namespace nifi {
 namespace minifi {
-namespace provenance{
-
-
+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_));
+    std::this_thread::sleep_for(std::chrono::milliseconds(purge_period_));
     std::this_thread::sleep_for(std::chrono::milliseconds(purge_period_));
     uint64_t curTime = getTimeMillis();
     uint64_t size = repoSize();
@@ -41,24 +40,24 @@ void ProvenanceRepository::run() {
       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_)
+        if (eventRead.DeSerialize(
+            reinterpret_cast<uint8_t*>(const_cast<char*>(it->value().data())),
+            it->value().size())) {
+          if ((curTime - eventRead.getEventTime()) > max_partition_millis_)
             purgeList.push_back(key);
         } else {
           logger_->log_debug("NiFi Provenance retrieve event %s fail",
-                                   key.c_str());
+                             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());
+                          eventId.c_str());
         Delete(eventId);
       }
     }
@@ -66,7 +65,6 @@ void ProvenanceRepository::run() {
       repo_full_ = true;
     else
       repo_full_ = false;
-    
   }
   return;
 }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/test/TestBase.h
----------------------------------------------------------------------
diff --git a/libminifi/test/TestBase.h b/libminifi/test/TestBase.h
index 4f926d3..585c8cd 100644
--- a/libminifi/test/TestBase.h
+++ b/libminifi/test/TestBase.h
@@ -26,7 +26,7 @@
 #include <vector>
 #include "core/logging/LogAppenders.h"
 #include "core/logging/Logger.h"
-#include "core/core.h"
+#include "core/Core.h"
 
 class LogTestController {
  public:
@@ -48,7 +48,7 @@ class TestController {
 
   TestController()
       : log("info") {
-    minifi::ResourceClaim::default_directory_path = "./";
+    minifi::ResourceClaim::default_directory_path = const_cast<char*>("./");
   }
 
   ~TestController() {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/test/nodefs/NoLevelDB.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/nodefs/NoLevelDB.cpp b/libminifi/test/nodefs/NoLevelDB.cpp
index 00c9212..09b4916 100644
--- a/libminifi/test/nodefs/NoLevelDB.cpp
+++ b/libminifi/test/nodefs/NoLevelDB.cpp
@@ -18,7 +18,7 @@
 
 #include "../TestBase.h"
 
-#include "core/core.h"
+#include "core/Core.h"
 #include "core/RepositoryFactory.h"
 
 TEST_CASE("NoLevelDBTest1", "[NoLevelDBTest]") {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/test/nodefs/NoYamlConfiguration.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/nodefs/NoYamlConfiguration.cpp b/libminifi/test/nodefs/NoYamlConfiguration.cpp
index 9a9b10e..5f3fce4 100644
--- a/libminifi/test/nodefs/NoYamlConfiguration.cpp
+++ b/libminifi/test/nodefs/NoYamlConfiguration.cpp
@@ -17,7 +17,7 @@
  */
 
 
-#include "core/core.h"
+#include "core/Core.h"
 #include "core/RepositoryFactory.h"
 
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/test/unit/ProcessorTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/ProcessorTests.cpp b/libminifi/test/unit/ProcessorTests.cpp
index 91a55f7..87f190c 100644
--- a/libminifi/test/unit/ProcessorTests.cpp
+++ b/libminifi/test/unit/ProcessorTests.cpp
@@ -23,7 +23,7 @@
 #include "core/logging/LogAppenders.h"
 #include "core/logging/BaseLogger.h"
 #include "processors/GetFile.h"
-#include "core/core.h"
+#include "core/Core.h"
 #include "../../include/core/FlowFile.h"
 #include "core/Processor.h"
 #include "core/ProcessContext.h"
@@ -189,6 +189,7 @@ TEST_CASE("Test GetFileLikeIt'sThreaded", "[getfileCreate3]") {
   processor->setScheduledState(core::ScheduledState::RUNNING);
   processor->onSchedule(&context, &factory);
 
+  int prev = 0;
   for (int i = 0; i < 10; i++) {
 
     core::ProcessSession session(&context);
@@ -216,7 +217,6 @@ TEST_CASE("Test GetFileLikeIt'sThreaded", "[getfileCreate3]") {
     processor->setScheduledState(core::ScheduledState::RUNNING);
     processor->onTrigger(&context, &session);
     unlink(ss.str().c_str());
-    rmdir(dir);
     reporter = session.getProvenanceReporter();
 
     REQUIRE(processor->getName() == "getfileCreate2");
@@ -229,14 +229,10 @@ TEST_CASE("Test GetFileLikeIt'sThreaded", "[getfileCreate3]") {
     session.commit();
     std::shared_ptr<core::FlowFile> ffr = session.get();
 
-    REQUIRE(2 == repo->getRepoMap().size());
+    REQUIRE((repo->getRepoMap().size()%2) == 0);
+    REQUIRE(repo->getRepoMap().size() == (prev+2));
+    prev+=2;
 
-    for (auto entry : repo->getRepoMap()) {
-      provenance::ProvenanceEventRecord newRecord;
-      newRecord.DeSerialize((uint8_t*) entry.second.data(),
-                            entry.second.length());
-
-    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/test/unit/ProvenanceTestHelper.h
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/ProvenanceTestHelper.h b/libminifi/test/unit/ProvenanceTestHelper.h
index 1e16aa6..80d8642 100644
--- a/libminifi/test/unit/ProvenanceTestHelper.h
+++ b/libminifi/test/unit/ProvenanceTestHelper.h
@@ -21,7 +21,7 @@
 #include "provenance/Provenance.h"
 #include "FlowController.h"
 #include "core/Repository.h"
-#include "core/core.h"
+#include "core/Core.h"
 /**
  * Test repository
  */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/test/unit/ProvenanceTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/ProvenanceTests.cpp b/libminifi/test/unit/ProvenanceTests.cpp
index c73cef2..2e41cc8 100644
--- a/libminifi/test/unit/ProvenanceTests.cpp
+++ b/libminifi/test/unit/ProvenanceTests.cpp
@@ -21,7 +21,7 @@
 #include "ProvenanceTestHelper.h"
 #include "provenance/Provenance.h"
 #include "FlowFileRecord.h"
-#include "core/core.h"
+#include "core/Core.h"
 #include "core/repository/FlowFileRepository.h"
 
 TEST_CASE("Test Provenance record create", "[Testprovenance::ProvenanceEventRecord]") {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/test/unit/RepoTests.cpp
----------------------------------------------------------------------
diff --git a/libminifi/test/unit/RepoTests.cpp b/libminifi/test/unit/RepoTests.cpp
index 9237e7e..21fae45 100644
--- a/libminifi/test/unit/RepoTests.cpp
+++ b/libminifi/test/unit/RepoTests.cpp
@@ -21,7 +21,7 @@
 #include "ProvenanceTestHelper.h"
 #include "provenance/Provenance.h"
 #include "FlowFileRecord.h"
-#include "core/core.h"
+#include "core/Core.h"
 #include "core/repository/FlowFileRepository.h"
 
 TEST_CASE("Test Repo Empty Value Attribute", "[TestFFR1]") {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/main/MiNiFiMain.cpp
----------------------------------------------------------------------
diff --git a/main/MiNiFiMain.cpp b/main/MiNiFiMain.cpp
index 6bfd9c9..9e6a37f 100644
--- a/main/MiNiFiMain.cpp
+++ b/main/MiNiFiMain.cpp
@@ -28,7 +28,7 @@
 #include <yaml-cpp/yaml.h>
 #include <iostream>
 
-#include "core/core.h"
+#include "core/Core.h"
 
 #include "core/logging/BaseLogger.h"
 #include "core/logging/LogAppenders.h"

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/thirdparty/google-styleguide/cpplint.py
----------------------------------------------------------------------
diff --git a/thirdparty/google-styleguide/cpplint.py b/thirdparty/google-styleguide/cpplint.py
index fafc243..eda78bd 100644
--- a/thirdparty/google-styleguide/cpplint.py
+++ b/thirdparty/google-styleguide/cpplint.py
@@ -5766,14 +5766,8 @@ def FlagCxx11Features(filename, clean_lines, linenum, error):
 
   # Flag unapproved C++11 headers.
   if include and include.group(1) in ('cfenv',
-                                      'condition_variable',
                                       'fenv.h',
-                                      'future',
-                                      'mutex',
-                                      'thread',
-                                      'chrono',
                                       'ratio',
-                                      'regex',
                                       'system_error',
                                      ):
     error(filename, linenum, 'build/c++11', 5,

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/thirdparty/google-styleguide/run_linter.sh
----------------------------------------------------------------------
diff --git a/thirdparty/google-styleguide/run_linter.sh b/thirdparty/google-styleguide/run_linter.sh
index e04d0aa..fbf8730 100755
--- a/thirdparty/google-styleguide/run_linter.sh
+++ b/thirdparty/google-styleguide/run_linter.sh
@@ -16,9 +16,14 @@
 #
 # ./run_linter <includedir> <srcdir>
 #!/bin/bash
+if [ "$(uname)" == "Darwin" ]; then
 SCRIPT_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
+else
+SCRIPT=$(readlink -f $0)
+SCRIPT_DIR=`dirname $SCRIPT`
+fi
 HEADERS=`find ${1} -name '*.h' | tr '\n' ','`
 SOURCES=`find ${2} -name  '*.cpp' | tr '\n' ' '`
 echo ${HEADERS}
 echo ${SOURCES}
-python ${SCRIPT_DIR}/cpplint.py --headers=${HEADERS} ${SOURCES}
+python ${SCRIPT_DIR}/cpplint.py --linelength=128 --headers=${HEADERS} ${SOURCES}


[2/4] nifi-minifi-cpp git commit: MINIFI-254: Incremental update for linter changes

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/core/Record.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/Record.cpp b/libminifi/src/core/Record.cpp
deleted file mode 100644
index 6f33300..0000000
--- a/libminifi/src/core/Record.cpp
+++ /dev/null
@@ -1,225 +0,0 @@
-/*
- * 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_;
-}
-
-}
-}
-}
-}
-}

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/core/Repository.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/Repository.cpp b/libminifi/src/core/Repository.cpp
index 9a27785..50e8cd2 100644
--- a/libminifi/src/core/Repository.cpp
+++ b/libminifi/src/core/Repository.cpp
@@ -15,15 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+#include "core/Repository.h"
+#include <arpa/inet.h>
 #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"
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/core/RepositoryFactory.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/RepositoryFactory.cpp b/libminifi/src/core/RepositoryFactory.cpp
index 9bdc7c3..c24a2af 100644
--- a/libminifi/src/core/RepositoryFactory.cpp
+++ b/libminifi/src/core/RepositoryFactory.cpp
@@ -1,6 +1,24 @@
+/**
+ * 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/RepositoryFactory.h"
+#include <memory>
+#include <string>
+#include <algorithm>
 #include "core/Repository.h"
-#include "core/Repository.h"
-
 #ifdef LEVELDB_SUPPORT
 #include "core/repository/FlowFileRepository.h"
 #include "provenance/ProvenanceRepository.h"
@@ -11,57 +29,51 @@ namespace apache {
 namespace nifi {
 namespace minifi {
 #ifndef LEVELDB_SUPPORT
-  namespace provenance{
-  class ProvenanceRepository;
-  }
+namespace provenance {
+class ProvenanceRepository;
+}
 #endif
 namespace core {
 
 #ifndef LEVELDB_SUPPORT
-  class FlowFileRepository;
+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 = instantiate<core::repository::FlowFileRepository>();
-      } else if (class_name_lc == "provenancerepository") {
-
-
-	return_obj = instantiate<provenance::ProvenanceRepository>();//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);
-      }
+std::shared_ptr<core::Repository> createRepository(
+    const std::string configuration_class_name, bool fail_safe) {
+  std::shared_ptr<core::Repository> return_obj = nullptr;
+  std::string class_name_lc = configuration_class_name;
+  std::transform(class_name_lc.begin(), class_name_lc.end(),
+                 class_name_lc.begin(), ::tolower);
+  try {
+    std::shared_ptr<core::Repository> return_obj = nullptr;
+    if (class_name_lc == "flowfilerepository") {
+      return_obj = instantiate<core::repository::FlowFileRepository>();
+    } else if (class_name_lc == "provenancerepository") {
+      return_obj = instantiate<provenance::ProvenanceRepository>();
     }
 
-    throw std::runtime_error(
-        "Support for the provided configuration class could not be found");
+    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 */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/core/logging/BaseLogger.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/logging/BaseLogger.cpp b/libminifi/src/core/logging/BaseLogger.cpp
index a6b43a8..d4774df 100644
--- a/libminifi/src/core/logging/BaseLogger.cpp
+++ b/libminifi/src/core/logging/BaseLogger.cpp
@@ -17,6 +17,10 @@
  */
 
 #include "core/logging/BaseLogger.h"
+#include <utility>
+#include <memory>
+#include <algorithm>
+#include <string>
 
 namespace org {
 namespace apache {
@@ -68,7 +72,6 @@ void BaseLogger::log_info(const char * const format, ...) {
  * @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
@@ -80,7 +83,6 @@ void BaseLogger::log_debug(const char * const format, ...) {
  * @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
@@ -122,7 +124,6 @@ void BaseLogger::log_str(LOG_LEVEL_E level, const std::string &buffer) {
       logger_->info(buffer);
       break;
   }
-
 }
 
 void BaseLogger::setLogLevel(const std::string &level,

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/core/logging/LogAppenders.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/logging/LogAppenders.cpp b/libminifi/src/core/logging/LogAppenders.cpp
index 5d92334..1918a0d 100644
--- a/libminifi/src/core/logging/LogAppenders.cpp
+++ b/libminifi/src/core/logging/LogAppenders.cpp
@@ -24,12 +24,15 @@ 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";
+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 */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/core/logging/Logger.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/logging/Logger.cpp b/libminifi/src/core/logging/Logger.cpp
index d8cadf3..5c08fa8 100644
--- a/libminifi/src/core/logging/Logger.cpp
+++ b/libminifi/src/core/logging/Logger.cpp
@@ -21,6 +21,7 @@
 
 #include <vector>
 #include <queue>
+#include <memory>
 #include <map>
 
 namespace org {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/core/repository/FlowFileRepository.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/repository/FlowFileRepository.cpp b/libminifi/src/core/repository/FlowFileRepository.cpp
index 8f13f39..5f62f83 100644
--- a/libminifi/src/core/repository/FlowFileRepository.cpp
+++ b/libminifi/src/core/repository/FlowFileRepository.cpp
@@ -1,4 +1,24 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 #include "core/repository/FlowFileRepository.h"
+#include <memory>
+#include <string>
+#include <vector>
 #include "FlowFileRecord.h"
 
 namespace org {
@@ -20,10 +40,12 @@ void FlowFileRepository::run() {
       leveldb::Iterator* it = db_->NewIterator(leveldb::ReadOptions());
 
       for (it->SeekToFirst(); it->Valid(); it->Next()) {
-        std::shared_ptr<FlowFileRecord> eventRead = std::make_shared<FlowFileRecord>(shared_from_this());
+        std::shared_ptr<FlowFileRecord> eventRead = std::make_shared<
+            FlowFileRecord>(shared_from_this());
         std::string key = it->key().ToString();
-        if (eventRead->DeSerialize((uint8_t *) it->value().data(),
-                                  (int) it->value().size())) {
+        if (eventRead->DeSerialize(
+            reinterpret_cast<const uint8_t *>(it->value().data()),
+            it->value().size())) {
           if ((curTime - eventRead->getEventTime()) > max_partition_millis_)
             purgeList.push_back(key);
         } else {
@@ -47,54 +69,44 @@ void FlowFileRepository::run() {
   return;
 }
 
-void FlowFileRepository::loadComponent()
- {
-
+void FlowFileRepository::loadComponent() {
   std::vector<std::string> purgeList;
-  leveldb::Iterator* it = db_->NewIterator(
-            leveldb::ReadOptions());
+  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());
+  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 (eventRead->DeSerialize(
+        reinterpret_cast<const uint8_t *>(it->value().data()),
+        it->value().size())) {
       auto search = connectionMap.find(eventRead->getConnectionUuid());
-      if (search != connectionMap.end())
-      {
+      if (search != connectionMap.end()) {
         // we find the connection for the persistent flowfile, create the flowfile and enqueue that
-	std::shared_ptr<core::FlowFile> flow_file_ref = std::static_pointer_cast<core::FlowFile>(eventRead);
-        std::shared_ptr<FlowFileRecord> record = std::make_shared<FlowFileRecord>(shared_from_this(),flow_file_ref);
+        std::shared_ptr<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)
-        {
+      } else {
+        if (eventRead->getContentFullPath().length() > 0) {
           std::remove(eventRead->getContentFullPath().c_str());
         }
         purgeList.push_back(key);
       }
-    }
-    else
-    {
+    } else {
       purgeList.push_back(key);
     }
   }
 
   delete it;
   std::vector<std::string>::iterator itPurge;
-  for (itPurge = purgeList.begin(); itPurge != purgeList.end();
-            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());
+    logger_->log_info("Repository Repo %s Purge %s", name_.c_str(),
+                      eventId.c_str());
     Delete(eventId);
   }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/core/yaml/YamlConfiguration.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/yaml/YamlConfiguration.cpp b/libminifi/src/core/yaml/YamlConfiguration.cpp
index 8e59363..4e736f8 100644
--- a/libminifi/src/core/yaml/YamlConfiguration.cpp
+++ b/libminifi/src/core/yaml/YamlConfiguration.cpp
@@ -17,7 +17,10 @@
  */
 
 #include "core/yaml/YamlConfiguration.h"
-
+#include <memory>
+#include <string>
+#include <vector>
+#include <set>
 namespace org {
 namespace apache {
 namespace nifi {
@@ -29,18 +32,17 @@ core::ProcessGroup *YamlConfiguration::parseRootProcessGroupYaml(
   uuid_t uuid;
 
   std::string flowName = rootFlowNode["name"].as<std::string>();
-  std::string id ;
-  
+  std::string id;
+
   try {
     rootFlowNode["id"].as<std::string>();
 
     uuid_parse(id.c_str(), uuid);
-  }catch(...)
-  {
+  } catch (...) {
     logger_->log_warn("Generating random ID for root node");
     uuid_generate(uuid);
     char uuid_str[37];
-    uuid_unparse(uuid,uuid_str);
+    uuid_unparse(uuid, uuid_str);
     id = uuid_str;
   }
 
@@ -69,7 +71,6 @@ void YamlConfiguration::parseProcessorNodeYaml(
   }
 
   if (processorsNode) {
-
     if (processorsNode.IsSequence()) {
       // Evaluate sequence of processors
       int numProcessors = processorsNode.size();
@@ -196,7 +197,6 @@ void YamlConfiguration::parseProcessorNodeYaml(
           processor->setSchedulingStrategy(core::CRON_DRIVEN);
           logger_->log_debug("setting scheduling strategy as %s",
                              procCfg.schedulingStrategy.c_str());
-
         }
 
         int64_t maxConcurrentTasks;
@@ -324,7 +324,6 @@ void YamlConfiguration::parseRemoteProcessGroupYaml(
             this->parsePortYaml(&currPort, group, RECEIVE);
           }  // for node
         }
-
       }
     }
   }
@@ -341,11 +340,9 @@ void YamlConfiguration::parseConnectionYaml(YAML::Node *connectionsNode,
   }
 
   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>();
@@ -461,7 +458,6 @@ void YamlConfiguration::parsePortYaml(YAML::Node *portNode,
   logger_->log_debug("parseProcessorNode: maxConcurrentTasks => [%d]",
                      maxConcurrentTasks);
   processor->setMaxConcurrentTasks(maxConcurrentTasks);
-
 }
 
 void YamlConfiguration::parsePropertiesNodeYaml(

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/io/BaseStream.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/io/BaseStream.cpp b/libminifi/src/io/BaseStream.cpp
index 1400a1d..8070c38 100644
--- a/libminifi/src/io/BaseStream.cpp
+++ b/libminifi/src/io/BaseStream.cpp
@@ -16,9 +16,9 @@
  * limitations under the License.
  */
 #include "io/BaseStream.h"
+#include <string>
 #include "io/Serializable.h"
 
-
 namespace org {
 namespace apache {
 namespace nifi {
@@ -32,7 +32,8 @@ namespace io {
  * @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, reinterpret_cast<DataStream*>(this),
+                             is_little_endian);
 }
 
 /**
@@ -43,7 +44,8 @@ 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, reinterpret_cast<DataStream*>(this),
+                             is_little_endian);
 }
 
 /**
@@ -54,7 +56,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, reinterpret_cast<DataStream*>(this));
 }
 
 /**
@@ -65,7 +67,8 @@ 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, reinterpret_cast<DataStream*>(this),
+                             is_little_endian);
 }
 
 /**
@@ -74,8 +77,8 @@ int BaseStream::write(uint64_t base_value, bool is_little_endian) {
  * @return resulting write size
  **/
 int BaseStream::write(bool value) {
-	uint8_t v = value;
-	return Serializable::write(v);
+  uint8_t v = value;
+  return Serializable::write(v);
 }
 
 /**
@@ -84,7 +87,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, reinterpret_cast<DataStream*>(this), widen);
 }
 
 /**
@@ -94,7 +97,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, reinterpret_cast<DataStream*>(this));
 }
 
 /**
@@ -104,7 +107,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, reinterpret_cast<DataStream*>(this));
 }
 
 /**
@@ -114,7 +117,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, reinterpret_cast<DataStream*>(this));
 }
 
 /**
@@ -125,7 +128,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, reinterpret_cast<DataStream*>(this));
 }
 
 /**
@@ -135,7 +138,8 @@ 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, reinterpret_cast<DataStream*>(this),
+                            is_little_endian);
 }
 
 /**
@@ -145,7 +149,8 @@ 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, reinterpret_cast<DataStream*>(this),
+                            is_little_endian);
 }
 
 /**
@@ -155,10 +160,9 @@ 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, reinterpret_cast<DataStream*>(this), widen);
 }
 
-
 } /* namespace io */
 } /* namespace minifi */
 } /* namespace nifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/io/CRCStream.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/io/CRCStream.cpp b/libminifi/src/io/CRCStream.cpp
index 47b45b5..e06a8f5 100644
--- a/libminifi/src/io/CRCStream.cpp
+++ b/libminifi/src/io/CRCStream.cpp
@@ -20,5 +20,3 @@
 #include <memory>
 #include "io/CRCStream.h"
 
-
-

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/io/ClientSocket.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/io/ClientSocket.cpp b/libminifi/src/io/ClientSocket.cpp
index ad6b04d..e62d4f1 100644
--- a/libminifi/src/io/ClientSocket.cpp
+++ b/libminifi/src/io/ClientSocket.cpp
@@ -15,20 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+#include "io/ClientSocket.h"
 #include <netinet/tcp.h>
 #include <sys/types.h>
+#include <netdb.h>
 #include <sys/socket.h>
 #include <netinet/in.h>
 #include <arpa/inet.h>
 #include <unistd.h>
 #include <cstdio>
+#include <utility>
+#include <vector>
 #include <cerrno>
-#include <netdb.h>
 #include <iostream>
 #include <string>
-
 #include "io/validation.h"
-#include "io/ClientSocket.h"
 
 namespace org {
 namespace apache {
@@ -36,7 +37,7 @@ namespace nifi {
 namespace minifi {
 namespace io {
 
-std::string Socket::HOSTNAME = Socket::getMyHostName(0);
+char *Socket::HOSTNAME = const_cast<char*>(Socket::getMyHostName(0).c_str());
 
 Socket::Socket(const std::string &hostname, const uint16_t port,
                const uint16_t listeners = -1)
@@ -50,12 +51,10 @@ Socket::Socket(const std::string &hostname, const uint16_t port,
   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 Socket &&other)
@@ -69,7 +68,6 @@ Socket::Socket(const Socket &&other)
       read_fds_(other.read_fds_),
       canonical_hostname_(std::move(other.canonical_hostname_)) {
   logger_ = logging::Logger::getLogger();
-
 }
 
 Socket::~Socket() {
@@ -81,7 +79,6 @@ void Socket::closeStream() {
     freeaddrinfo(addr_info_);
     addr_info_ = 0;
   }
-
   if (socket_file_descriptor_ >= 0) {
     close(socket_file_descriptor_);
     socket_file_descriptor_ = -1;
@@ -98,12 +95,10 @@ int8_t Socket::createConnection(const addrinfo *p, in_addr_t &addr) {
   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;
@@ -113,7 +108,6 @@ int8_t Socket::createConnection(const addrinfo *p, in_addr_t &addr) {
     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
@@ -129,7 +123,6 @@ int8_t Socket::createConnection(const addrinfo *p, in_addr_t &addr) {
         logger_->log_warn("Could not connect to socket, error:%s",
                           strerror(errno));
         return -1;
-
       }
     }
   }
@@ -140,7 +133,6 @@ int8_t Socket::createConnection(const addrinfo *p, in_addr_t &addr) {
       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_);
@@ -148,8 +140,7 @@ int8_t Socket::createConnection(const addrinfo *p, in_addr_t &addr) {
   return 0;
 }
 
-short Socket::initialize() {
-
+int16_t Socket::initialize() {
   struct sockaddr_in servAddr;
 
   addrinfo hints = { sizeof(addrinfo) };
@@ -159,7 +150,6 @@ short Socket::initialize() {
   hints.ai_flags = AI_CANONNAME;
   if (listeners_ > 0)
     hints.ai_flags |= AI_PASSIVE;
-
   hints.ai_protocol = 0; /* any protocol */
 
   int errcode = getaddrinfo(requested_hostname_.c_str(), 0, &hints,
@@ -188,8 +178,7 @@ short Socket::initialize() {
   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);
+  memcpy(reinterpret_cast<char*>(&addr), h->h_addr_list[0], h->h_length);
 
   auto p = addr_info_;
   for (; p != NULL; p = p->ai_next) {
@@ -197,8 +186,7 @@ short Socket::initialize() {
       if (!IsNullOrEmpty(p) && !IsNullOrEmpty(p->ai_canonname))
         canonical_hostname_ = p->ai_canonname;
     }
-
-    //we've successfully connected
+    // we've successfully connected
     if (port_ > 0 && createConnection(p, addr) >= 0) {
       return 0;
       break;
@@ -206,10 +194,9 @@ short Socket::initialize() {
   }
 
   return -1;
-
 }
 
-short Socket::select_descriptor(const uint16_t msec) {
+int16_t Socket::select_descriptor(const uint16_t msec) {
   struct timeval tv;
   int retval;
 
@@ -233,7 +220,6 @@ short Socket::select_descriptor(const uint16_t msec) {
 
   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
@@ -255,24 +241,23 @@ short Socket::select_descriptor(const uint16_t msec) {
         return i;
       }
     }
-
   }
 
   return -1;
 }
 
-short Socket::setSocketOptions(const int sock) {
+int16_t Socket::setSocketOptions(const int sock) {
   int opt = 1;
   bool nagle_off = true;
 #ifndef __MACH__
   if (nagle_off) {
-    if (setsockopt(sock, SOL_TCP, TCP_NODELAY, (void *) &opt, sizeof(opt))
+    if (setsockopt(sock, SOL_TCP, TCP_NODELAY, static_cast<void*>(&opt), sizeof(opt))
         < 0) {
       logger_->log_error("setsockopt() TCP_NODELAY failed");
       close(sock);
       return -1;
     }
-    if (setsockopt(sock, SOL_SOCKET, SO_REUSEADDR, (char *) &opt,
+    if (setsockopt(sock, SOL_SOCKET, SO_REUSEADDR, reinterpret_cast<char*>(&opt),
             sizeof(opt)) < 0) {
       logger_->log_error("setsockopt() SO_REUSEADDR failed");
       close(sock);
@@ -281,8 +266,8 @@ short Socket::setSocketOptions(const int sock) {
   }
 
   int sndsize = 256 * 1024;
-  if (setsockopt(sock, SOL_SOCKET, SO_SNDBUF, (char *) &sndsize,
-          (int) sizeof(sndsize)) < 0) {
+  if (setsockopt(sock, SOL_SOCKET, SO_SNDBUF, reinterpret_cast<char *>( &sndsize),
+          sizeof(sndsize)) < 0) {
     logger_->log_error("setsockopt() SO_SNDBUF failed");
     close(sock);
     return -1;
@@ -291,8 +276,8 @@ short Socket::setSocketOptions(const int sock) {
 #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) {
+    if (setsockopt(sock, SOL_SOCKET, SO_REUSEADDR,
+                   reinterpret_cast<char *>(&opt), sizeof(opt)) < 0) {
       logger_->log_error("setsockopt() SO_REUSEADDR failed");
       close(sock);
       return -1;
@@ -307,22 +292,19 @@ std::string Socket::getHostname() const {
 }
 
 int Socket::writeData(std::vector<uint8_t> &buf, int buflen) {
-
   if (buf.capacity() < buflen)
     return -1;
-  return writeData((uint8_t*) &buf[0], buflen);
+  return writeData(reinterpret_cast<uint8_t *>(&buf[0]), buflen);
 }
 
 // data stream overrides
 
 int Socket::writeData(uint8_t *value, int size) {
-
   int ret = 0, bytes = 0;
 
   while (bytes < size) {
-
     ret = send(socket_file_descriptor_, value + bytes, size - bytes, 0);
-    //check for errors
+    // check for errors
     if (ret <= 0) {
       close(socket_file_descriptor_);
       logger_->log_error("Could not send to %d, error: %s",
@@ -330,27 +312,23 @@ int Socket::writeData(uint8_t *value, int size) {
       return ret;
     }
     bytes += ret;
-
   }
 
   if (ret)
     logger_->log_trace("Send data size %d over socket %d", size,
                        socket_file_descriptor_);
-
   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));
+  readData(reinterpret_cast<uint8_t *>(&buf[0]), sizeof(t));
   return buf;
 }
 
 int Socket::write(uint64_t base_value, bool is_little_endian) {
-
   return Serializable::write(base_value, this, is_little_endian);
 }
 
@@ -363,7 +341,6 @@ int Socket::write(uint16_t base_value, bool is_little_endian) {
 }
 
 int Socket::read(uint64_t &value, bool is_little_endian) {
-
   auto buf = readBuffer(value);
 
   if (is_little_endian) {
@@ -381,68 +358,57 @@ int Socket::read(uint64_t &value, bool is_little_endian) {
 }
 
 int Socket::read(uint32_t &value, bool is_little_endian) {
-
   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;
-
   }
-
   return sizeof(value);
 }
 
 int Socket::read(uint16_t &value, bool is_little_endian) {
-
   auto buf = readBuffer(value);
 
   if (is_little_endian) {
     value = (buf[0] << 8) | buf[1];
   } else {
     value = buf[0] | buf[1] << 8;
-
   }
   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);
+  return readData(reinterpret_cast<uint8_t*>(&buf[0]), buflen);
 }
 
 int Socket::readData(uint8_t *buf, int buflen) {
-
-  int total_read = 0;
+  int32_t total_read = 0;
   while (buflen) {
-    short fd = select_descriptor(1000);
+    int16_t 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)
+      if (bytes_read == 0) {
         logger_->log_info("Other side hung up on %d", fd);
-      else {
+      } 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;
 }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/io/DataStream.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/io/DataStream.cpp b/libminifi/src/io/DataStream.cpp
index 7a10bd9..9e0dfce 100644
--- a/libminifi/src/io/DataStream.cpp
+++ b/libminifi/src/io/DataStream.cpp
@@ -15,17 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-
+#include "io/DataStream.h"
+#include <arpa/inet.h>
 #include <vector>
 #include <iostream>
 #include <cstdint>
 #include <cstdio>
 #include <cstring>
 #include <string>
-#include <arpa/inet.h>
 #include <algorithm>
-#include "io/DataStream.h"
 
 namespace org {
 namespace apache {
@@ -33,106 +31,92 @@ namespace nifi {
 namespace minifi {
 namespace io {
 
-
 int DataStream::writeData(uint8_t *value, int size) {
-
-    std::copy(value,value+size,std::back_inserter(buffer));
-
-    return size;
+  std::copy(value, value + size, std::back_inserter(buffer));
+  return size;
 }
 
 int DataStream::read(uint64_t &value, bool is_little_endian) {
-    if ((8 + readBuffer) > buffer.size()) {
-        // if read exceed
-        return -1;
-    }
-    uint8_t *buf = &buffer[readBuffer];
-
-    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);
-    }
-    readBuffer += 8;
-    return 8;
+  if ((8 + readBuffer) > buffer.size()) {
+    // if read exceed
+    return -1;
+  }
+  uint8_t *buf = &buffer[readBuffer];
+
+  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);
+  }
+  readBuffer += 8;
+  return 8;
 }
 
 int DataStream::read(uint32_t &value, bool is_little_endian) {
-    if ((4 + readBuffer) > buffer.size()) {
-        // if read exceed
-        return -1;
-    }
-    uint8_t *buf = &buffer[readBuffer];
-
-    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;
-
-    }
-    readBuffer += 4;
-    return 4;
+  if ((4 + readBuffer) > buffer.size()) {
+    // if read exceed
+    return -1;
+  }
+  uint8_t *buf = &buffer[readBuffer];
+
+  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;
+  }
+  readBuffer += 4;
+  return 4;
 }
 
 int DataStream::read(uint16_t &value, bool is_little_endian) {
-
-    if ((2 + readBuffer) > buffer.size()) {
-        // if read exceed
-        return -1;
-    }
-    uint8_t *buf = &buffer[readBuffer];
-
-    if (is_little_endian) {
-        value = (buf[0] << 8) | buf[1];
-    } else {
-        value = buf[0] | buf[1] << 8;
-
-    }
-    readBuffer += 2;
-    return 2;
+  if ((2 + readBuffer) > buffer.size()) {
+    // if read exceed
+    return -1;
+  }
+  uint8_t *buf = &buffer[readBuffer];
+
+  if (is_little_endian) {
+    value = (buf[0] << 8) | buf[1];
+  } else {
+    value = buf[0] | buf[1] << 8;
+  }
+  readBuffer += 2;
+  return 2;
 }
 
-int DataStream::readData(std::vector<uint8_t> &buf,int buflen) {
-    if ((buflen + readBuffer) > buffer.size()) {
-        // if read exceed
-        return -1;
-    }
+int DataStream::readData(std::vector<uint8_t> &buf, int buflen) {
+  if ((buflen + readBuffer) > buffer.size()) {
+    // if read exceed
+    return -1;
+  }
 
-    if (buf.capacity() < buflen)
-    	buf.resize(buflen);
+  if (buf.capacity() < buflen)
+    buf.resize(buflen);
 
-    buf.insert(buf.begin(),&buffer[readBuffer],&buffer[readBuffer+buflen]);
+  buf.insert(buf.begin(), &buffer[readBuffer], &buffer[readBuffer + buflen]);
 
-    readBuffer += buflen;
-    return buflen;
+  readBuffer += buflen;
+  return buflen;
 }
 
+int DataStream::readData(uint8_t *buf, int buflen) {
+  if ((buflen + readBuffer) > buffer.size()) {
+    // if read exceed
+    return -1;
+  }
 
-int DataStream::readData(uint8_t *buf,int buflen) {
-    if ((buflen + readBuffer) > buffer.size()) {
-        // if read exceed
-        return -1;
-    }
-
-    std::copy(&buffer[readBuffer],&buffer[readBuffer+buflen],buf);
+  std::copy(&buffer[readBuffer], &buffer[readBuffer + buflen], buf);
 
-    readBuffer += buflen;
-    return buflen;
+  readBuffer += buflen;
+  return buflen;
 }
 
-
 } /* namespace io */
 } /* namespace minifi */
 } /* namespace nifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/io/EndianCheck.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/io/EndianCheck.cpp b/libminifi/src/io/EndianCheck.cpp
index 1b5020d..fd754c4 100644
--- a/libminifi/src/io/EndianCheck.cpp
+++ b/libminifi/src/io/EndianCheck.cpp
@@ -25,7 +25,6 @@ namespace minifi {
 namespace io {
 bool EndiannessCheck::IS_LITTLE = EndiannessCheck::is_little_endian();
 
-
 } /* namespace io */
 } /* namespace minifi */
 } /* namespace nifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/io/Serializable.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/io/Serializable.cpp b/libminifi/src/io/Serializable.cpp
index 7b7f2bd..c3f74c7 100644
--- a/libminifi/src/io/Serializable.cpp
+++ b/libminifi/src/io/Serializable.cpp
@@ -15,13 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+#include "io/Serializable.h"
+#include <arpa/inet.h>
+#include <cstdio>
 #include <iostream>
 #include <vector>
 #include <string>
-#include <cstdio>
-#include <arpa/inet.h>
+#include <algorithm>
 #include "io/DataStream.h"
-#include "io/Serializable.h"
 namespace org {
 namespace apache {
 namespace nifi {
@@ -29,8 +30,7 @@ namespace minifi {
 namespace io {
 
 #define htonll_r(x) ((((uint64_t)htonl(x)) << 32) + htonl((x) >> 32))
-
-#define IS_ASCII(c) __builtin_expect(!!((c >= 1) && (c <= 127)),1)
+#define IS_ASCII(c) __builtin_expect(!!((c >= 1) && (c <= 127)), 1)
 
 template<typename T>
 int Serializable::writeData(const T &t, DataStream *stream) {
@@ -63,7 +63,7 @@ int Serializable::write(uint8_t value, DataStream *stream) {
   return stream->writeData(&value, 1);
 }
 int Serializable::write(char value, DataStream *stream) {
-  return stream->writeData((uint8_t *) &value, 1);
+  return stream->writeData(reinterpret_cast<uint8_t *>(&value), 1);
 }
 
 int Serializable::write(uint8_t *value, int len, DataStream *stream) {
@@ -89,7 +89,7 @@ int Serializable::read(char &value, DataStream *stream) {
 
   int ret = stream->readData(&buf, 1);
   if (ret == 1)
-    value = (char) buf;
+    value = buf;
   return ret;
 }
 
@@ -105,17 +105,14 @@ int Serializable::read(uint16_t &value, DataStream *stream,
 int Serializable::read(uint32_t &value, DataStream *stream,
                        bool is_little_endian) {
   return stream->read(value, is_little_endian);
-
 }
 int Serializable::read(uint64_t &value, DataStream *stream,
                        bool is_little_endian) {
   return stream->read(value, is_little_endian);
-
 }
 
 int Serializable::write(uint32_t base_value, DataStream *stream,
                         bool is_little_endian) {
-
   const uint32_t value = is_little_endian ? htonl(base_value) : base_value;
 
   return writeData(value, stream);
@@ -123,7 +120,6 @@ int Serializable::write(uint32_t base_value, DataStream *stream,
 
 int Serializable::write(uint64_t base_value, DataStream *stream,
                         bool is_little_endian) {
-
   const uint64_t value =
       is_little_endian == 1 ? htonll_r(base_value) : base_value;
   return writeData(value, stream);
@@ -131,7 +127,6 @@ int Serializable::write(uint64_t base_value, DataStream *stream,
 
 int Serializable::write(uint16_t base_value, DataStream *stream,
                         bool is_little_endian) {
-
   const uint16_t value = is_little_endian == 1 ? htons(base_value) : base_value;
 
   return writeData(value, stream);
@@ -150,7 +145,6 @@ int Serializable::readUTF(std::string &str, DataStream *stream, bool widen) {
   } else {
     uint32_t len;
     ret = read(len, stream);
-
     if (ret <= 0)
       return ret;
     utflen = len;
@@ -166,7 +160,6 @@ int Serializable::readUTF(std::string &str, DataStream *stream, bool widen) {
 
   // The number of chars produced may be less than utflen
   str = std::string((const char*) &buf[0], utflen);
-
   return utflen;
 }
 
@@ -181,7 +174,6 @@ int Serializable::writeUTF(std::string str, DataStream *stream, bool widen) {
     return -1;
 
   if (utflen == 0) {
-
     if (!widen) {
       uint16_t shortLen = utflen;
       write(shortLen, stream);
@@ -207,12 +199,10 @@ int Serializable::writeUTF(std::string str, DataStream *stream, bool widen) {
   int ret;
 
   if (!widen) {
-
     uint16_t short_length = utflen;
     write(short_length, stream);
     ret = stream->writeData(utf_to_write.data(), utflen);
   } else {
-    //utflen += 4;
     write(utflen, stream);
     ret = stream->writeData(utf_to_write.data(), utflen);
   }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/io/StreamFactory.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/io/StreamFactory.cpp b/libminifi/src/io/StreamFactory.cpp
index e3aa290..1cf419e 100644
--- a/libminifi/src/io/StreamFactory.cpp
+++ b/libminifi/src/io/StreamFactory.cpp
@@ -15,11 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-#include "../../include/io/StreamFactory.h"
-
+#include "io/StreamFactory.h"
 #include <atomic>
-#include <mutex> 
-  
+#include <mutex>
+
 namespace org {
 namespace apache {
 namespace nifi {
@@ -29,7 +28,6 @@ namespace io {
 std::atomic<StreamFactory*> StreamFactory::context_instance_;
 std::mutex StreamFactory::context_mutex_;
 
-
 } /* namespace io */
 } /* namespace minifi */
 } /* namespace nifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/io/tls/TLSSocket.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/io/tls/TLSSocket.cpp b/libminifi/src/io/tls/TLSSocket.cpp
index b2df394..1499840 100644
--- a/libminifi/src/io/tls/TLSSocket.cpp
+++ b/libminifi/src/io/tls/TLSSocket.cpp
@@ -15,10 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-#include "properties/Configure.h"
 #include "io/tls/TLSSocket.h"
+#include <openssl/ssl.h>
+#include <openssl/err.h>
+#include <utility>
+#include <string>
+#include <vector>
+#include "properties/Configure.h"
 #include "utils/StringUtils.h"
-
 #include "core/Property.h"
 
 namespace org {
@@ -27,116 +31,111 @@ 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()) {
-
+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;
+int16_t 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);
+TLSSocket::~TLSSocket() {
+  if (ssl != 0)
+    SSL_free(ssl);
 }
 /**
  * Constructor that accepts host name, port and listeners. With this
@@ -146,102 +145,100 @@ TLSSocket::~TLSSocket()
  * @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) {
+                     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 std::string &hostname, const uint16_t port)
+    : Socket(hostname, port, 0),
+      ssl(0) {
 }
 
-TLSSocket::TLSSocket(const TLSSocket &&d) :
-		Socket(std::move(d)), 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;
+int16_t TLSSocket::initialize() {
+  TLSContext *context = TLSContext::getInstance();
+  int16_t 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);
+int16_t 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(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;
+  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;
+  if (IsNullOrEmpty(ssl))
+    return -1;
+  int total_read = 0;
+  int status = 0;
+  while (buflen) {
+    int16_t 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 */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/processors/AppendHostInfo.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/AppendHostInfo.cpp b/libminifi/src/processors/AppendHostInfo.cpp
index 24ccc9a..b3c76db 100644
--- a/libminifi/src/processors/AppendHostInfo.cpp
+++ b/libminifi/src/processors/AppendHostInfo.cpp
@@ -17,27 +17,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-#include <set>
+#include "processors/AppendHostInfo.h"
+#define __USE_POSIX
+#include <limits.h>
 #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 <memory>
+#include <string>
+#include <set>
+#include "core/ProcessContext.h"
+#include "core/Property.h"
+#include "core/ProcessSession.h"
+#include "core/FlowFile.h"
 #include "io/ClientSocket.h"
-
-#define __USE_POSIX
-#include <limits.h>
-
 namespace org {
 namespace apache {
 namespace nifi {
@@ -48,7 +46,6 @@ namespace processors {
 #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");
@@ -77,31 +74,29 @@ void AppendHostInfo::initialize() {
   setSupportedRelationships(relationships);
 }
 
-void AppendHostInfo::onTrigger(
-    core::ProcessContext *context,
-    core::ProcessSession *session) {
-  std::shared_ptr<core::FlowFile> flow =
-      session->get();
+void AppendHostInfo::onTrigger(core::ProcessContext *context,
+                               core::ProcessSession *session) {
+  std::shared_ptr<core::FlowFile> flow = session->get();
   if (!flow)
     return;
 
-  //Get Hostname
+  // 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
+  // Get IP address for the specified interface
   std::string iface;
   context->getProperty(InterfaceName.getName(), iface);
-  //Confirm the specified interface name exists on this device
+  // 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
+    // Type of address to retrieve - IPv4 IP address
     ifr.ifr_addr.sa_family = AF_INET;
-    //Copy the interface name in the ifreq structure
+    // Copy the interface name in the ifreq structure
     strncpy(ifr.ifr_name, iface.c_str(), IFNAMSIZ - 1);
     ioctl(fd, SIOCGIFADDR, &ifr);
     close(fd);

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/processors/ExecuteProcess.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/ExecuteProcess.cpp b/libminifi/src/processors/ExecuteProcess.cpp
index 3cbbc1b..701c645 100644
--- a/libminifi/src/processors/ExecuteProcess.cpp
+++ b/libminifi/src/processors/ExecuteProcess.cpp
@@ -18,9 +18,12 @@
  * limitations under the License.
  */
 #include "processors/ExecuteProcess.h"
+#include <cstring>
+#include <memory>
+#include <string>
+#include <set>
 #include "core/ProcessContext.h"
 #include "core/ProcessSession.h"
-#include <cstring>
 #include "utils/StringUtils.h"
 #include "utils/TimeUtil.h"
 
@@ -30,14 +33,15 @@ 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.",
+    "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.",
+    "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",
@@ -45,7 +49,8 @@ core::Property ExecuteProcess::WorkingDir(
     "");
 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.",
+    "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",
@@ -69,9 +74,8 @@ void ExecuteProcess::initialize() {
   setSupportedRelationships(relationships);
 }
 
-void ExecuteProcess::onTrigger(
-    core::ProcessContext *context,
-    core::ProcessSession *session) {
+void ExecuteProcess::onTrigger(core::ProcessContext *context,
+                               core::ProcessSession *session) {
   std::string value;
   if (context->getProperty(Command.getName(), value)) {
     this->_command = value;
@@ -84,12 +88,10 @@ void ExecuteProcess::onTrigger(
   }
   if (context->getProperty(BatchDuration.getName(), value)) {
     core::TimeUnit unit;
-    if (core::Property::StringToTime(value,
-                                                                _batchDuration,
-                                                                unit)
-        && core::Property::ConvertTimeUnitToMS(
-            _batchDuration, unit, _batchDuration)) {
-
+    if (core::Property::StringToTime(value, _batchDuration, unit)
+        && core::Property::ConvertTimeUnitToMS(_batchDuration, unit,
+                                               _batchDuration)) {
+      logger_->log_info("Setting _batchDuration");
     }
   }
   if (context->getProperty(RedirectErrorStream.getName(), value)) {
@@ -112,9 +114,7 @@ void ExecuteProcess::onTrigger(
   }
   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, " ");
+  char *p = std::strtok(const_cast<char*>(_fullCommand.c_str()), " ");
   int argc = 0;
   char *argv[64];
   while (p != 0 && argc < 64) {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/processors/GenerateFlowFile.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/GenerateFlowFile.cpp b/libminifi/src/processors/GenerateFlowFile.cpp
index ebdaaa3..34c0ae2 100644
--- a/libminifi/src/processors/GenerateFlowFile.cpp
+++ b/libminifi/src/processors/GenerateFlowFile.cpp
@@ -17,18 +17,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+#include "processors/GenerateFlowFile.h"
+#include <sys/time.h>
+#include <time.h>
 #include <vector>
 #include <queue>
 #include <map>
+#include <memory>
+#include <string>
 #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"
 
@@ -39,104 +40,101 @@ 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");
+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::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;
+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);
-	}
+  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);
-		}
-	}
+  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();
+      *(reinterpret_cast<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();
+        *(reinterpret_cast<int*>(current)) = randValue;
+        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 */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/processors/GetFile.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/GetFile.cpp b/libminifi/src/processors/GetFile.cpp
index 652caf7..2740793 100644
--- a/libminifi/src/processors/GetFile.cpp
+++ b/libminifi/src/processors/GetFile.cpp
@@ -15,30 +15,32 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-#include <vector>
-#include <queue>
-#include <map>
-#include <set>
+#include "processors/GetFile.h"
 #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__ >= 4)
 #if (__GNUC_MINOR__ < 9)
 #include <regex.h>
+#else
+#include <regex>
 #endif
 #endif
+#include <vector>
+#include <queue>
+#include <map>
+#include <memory>
+#include <set>
+#include <sstream>
+#include <string>
+#include <iostream>
 #include "utils/StringUtils.h"
-#include <regex>
 #include "utils/TimeUtil.h"
-#include "processors/GetFile.h"
 #include "core/ProcessContext.h"
 #include "core/ProcessSession.h"
 
@@ -47,7 +49,6 @@ 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");
@@ -62,11 +63,13 @@ core::Property GetFile::KeepSourceFile(
     "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",
+    "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",
+    "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",
@@ -113,7 +116,6 @@ void GetFile::onSchedule(core::ProcessContext *context,
                          core::ProcessSessionFactory *sessionFactory) {
   std::string value;
 
-  logger_->log_info("onTrigger GetFile");
   if (context->getProperty(Directory.getName(), value)) {
     request_.directory = value;
   }
@@ -129,13 +131,12 @@ void GetFile::onSchedule(core::ProcessContext *context,
         value, request_.keepSourceFile);
   }
 
-  logger_->log_info("onTrigger GetFile");
   if (context->getProperty(MaxAge.getName(), value)) {
     core::TimeUnit unit;
     if (core::Property::StringToTime(value, request_.maxAge, unit)
         && core::Property::ConvertTimeUnitToMS(request_.maxAge, unit,
                                                request_.maxAge)) {
-
+      logger_->log_debug("successfully applied _maxAge");
     }
   }
   if (context->getProperty(MinAge.getName(), value)) {
@@ -143,7 +144,7 @@ void GetFile::onSchedule(core::ProcessContext *context,
     if (core::Property::StringToTime(value, request_.minAge, unit)
         && core::Property::ConvertTimeUnitToMS(request_.minAge, unit,
                                                request_.minAge)) {
-
+      logger_->log_debug("successfully applied _minAge");
     }
   }
   if (context->getProperty(MaxSize.getName(), value)) {
@@ -157,7 +158,7 @@ void GetFile::onSchedule(core::ProcessContext *context,
     if (core::Property::StringToTime(value, request_.pollInterval, unit)
         && core::Property::ConvertTimeUnitToMS(request_.pollInterval, unit,
                                                request_.pollInterval)) {
-
+      logger_->log_debug("successfully applied _pollInterval");
     }
   }
   if (context->getProperty(Recurse.getName(), value)) {
@@ -172,11 +173,9 @@ void GetFile::onSchedule(core::ProcessContext *context,
 
 void GetFile::onTrigger(core::ProcessContext *context,
                         core::ProcessSession *session) {
-
   // Perform directory list
   logger_->log_info("Is listing empty %i", isListingEmpty());
   if (isListingEmpty()) {
-
     if (request_.pollInterval == 0
         || (getTimeMillis() - last_listing_time_) > request_.pollInterval) {
       performListing(request_.directory, request_);
@@ -190,7 +189,6 @@ void GetFile::onTrigger(core::ProcessContext *context,
       std::queue<std::string> list;
       pollListing(list, request_);
       while (!list.empty()) {
-
         std::string fileName = list.front();
         list.pop();
         logger_->log_info("GetFile process %s", fileName.c_str());
@@ -214,7 +212,6 @@ void GetFile::onTrigger(core::ProcessContext *context,
       throw;
     }
   }
-
 }
 
 bool GetFile::isListingEmpty() {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/processors/ListenHTTP.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/processors/ListenHTTP.cpp b/libminifi/src/processors/ListenHTTP.cpp
index 36c743e..5c4a6bb 100644
--- a/libminifi/src/processors/ListenHTTP.cpp
+++ b/libminifi/src/processors/ListenHTTP.cpp
@@ -1,5 +1,6 @@
 /**
  * @file ListenHTTP.cpp
+
  * ListenHTTP class implementation
  *
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -17,17 +18,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-#include <sstream>
+#include "processors/ListenHTTP.h"
+#include <uuid/uuid.h>
+#include <CivetServer.h>
 #include <stdio.h>
+#include <sstream>
+#include <utility>
+#include <memory>
 #include <string>
 #include <iostream>
 #include <fstream>
-#include <uuid/uuid.h>
-
-#include <CivetServer.h>
-
-#include "processors/ListenHTTP.h"
-
+#include <set>
+#include <vector>
 #include "utils/TimeUtil.h"
 #include "core/ProcessContext.h"
 #include "core/ProcessSession.h"
@@ -39,15 +41,15 @@ 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::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.",
+    "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",
@@ -65,17 +67,18 @@ core::Property ListenHTTP::SSLMinimumVersion(
     "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",
+    "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");
+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;
+  std::set<core::Property> properties;
   properties.insert(BasePath);
   properties.insert(Port);
   properties.insert(AuthorizedDNPattern);
@@ -84,17 +87,15 @@ void ListenHTTP::initialize() {
   properties.insert(SSLVerifyPeer);
   properties.insert(SSLMinimumVersion);
   properties.insert(HeadersAsAttributesRegex);
-  setSupportedProperties (properties);
+  setSupportedProperties(properties);
   // Set the supported relationships
-  std::set < core::Relationship > relationships;
+  std::set<core::Relationship> relationships;
   relationships.insert(Success);
-  setSupportedRelationships (relationships);
+  setSupportedRelationships(relationships);
 }
 
-void ListenHTTP::onSchedule(
-    core::ProcessContext *context,
-    core::ProcessSessionFactory *sessionFactory) {
-
+void ListenHTTP::onSchedule(core::ProcessContext *context,
+                            core::ProcessSessionFactory *sessionFactory) {
   std::string basePath;
 
   if (!context->getProperty(BasePath.getName(), basePath)) {
@@ -178,7 +179,7 @@ void ListenHTTP::onSchedule(
       listeningPort.c_str(), basePath.c_str(), numThreads);
 
   // Initialize web server
-  std::vector < std::string > options;
+  std::vector<std::string> options;
   options.push_back("enable_keep_alive");
   options.push_back("yes");
   options.push_back("keep_alive_timeout_ms");
@@ -238,12 +239,10 @@ void ListenHTTP::onSchedule(
 ListenHTTP::~ListenHTTP() {
 }
 
-void ListenHTTP::onTrigger(
-    core::ProcessContext *context,
-    core::ProcessSession *session) {
-
-  std::shared_ptr < FlowFileRecord > flowFile = std::static_pointer_cast
-      < FlowFileRecord > (session->get());
+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) {
@@ -251,10 +250,10 @@ void ListenHTTP::onTrigger(
   }
 }
 
-ListenHTTP::Handler::Handler(
-    core::ProcessContext *context,
-    core::ProcessSessionFactory *sessionFactory,
-    std::string &&authDNPattern, std::string &&headersAsAttributesPattern)
+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;
@@ -292,8 +291,7 @@ bool ListenHTTP::Handler::handlePost(CivetServer *server,
 
   auto session = _processSessionFactory->createSession();
   ListenHTTP::WriteCallback callback(conn, req_info);
-  auto flowFile = std::static_pointer_cast < FlowFileRecord
-      > (session->create());
+  auto flowFile = std::static_pointer_cast<FlowFileRecord>(session->create());
 
   if (!flowFile) {
     sendErrorResponse(conn);
@@ -347,9 +345,9 @@ ListenHTTP::WriteCallback::WriteCallback(
 }
 
 void ListenHTTP::WriteCallback::process(std::ofstream *stream) {
-  long long rlen;
-  long long nlen = 0;
-  long long tlen = _reqInfo->content_length;
+  int64_t rlen;
+  int64_t nlen = 0;
+  int64_t tlen = _reqInfo->content_length;
   char buf[16384];
 
   while (nlen < tlen) {


[4/4] nifi-minifi-cpp git commit: MINIFI-254: Incremental update for linter changes

Posted by al...@apache.org.
MINIFI-254: Incremental update for linter changes

Update YAML to run linter at build time.
Tests will run before linter so that we can give
contributors a chance to see tests run in the event
there are linter failures.

This closes #73.

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

Branch: refs/heads/master
Commit: be3f2ffea3173b46d02a7d484cd901bf36d51e63
Parents: 3676468
Author: Marc Parisi <ph...@apache.org>
Authored: Mon Apr 3 16:27:24 2017 -0400
Committer: Aldrin Piri <al...@apache.org>
Committed: Wed Apr 5 17:48:30 2017 +0200

----------------------------------------------------------------------
 .travis.yml                                     |   2 +-
 libminifi/include/Connection.h                  |   2 +-
 libminifi/include/Exception.h                   |  93 ++--
 libminifi/include/FlowControlProtocol.h         |  11 +-
 libminifi/include/FlowController.h              |   5 +-
 libminifi/include/RemoteProcessorGroupPort.h    |  21 +-
 libminifi/include/ResourceClaim.h               | 109 ++---
 libminifi/include/SchedulingAgent.h             |  20 +-
 libminifi/include/Site2SiteClientProtocol.h     |  11 +-
 libminifi/include/Site2SitePeer.h               |   2 -
 libminifi/include/core/ConfigurationFactory.h   |  24 +-
 libminifi/include/core/Connectable.h            |   2 +-
 libminifi/include/core/Core.h                   | 178 +++++++
 libminifi/include/core/FlowConfiguration.h      |   4 +-
 libminifi/include/core/FlowFile.h               |   9 +-
 libminifi/include/core/ProcessSession.h         |  17 +-
 libminifi/include/core/Processor.h              |   9 +-
 libminifi/include/core/ProcessorConfig.h        |   6 +-
 libminifi/include/core/Repository.h             |  19 +-
 libminifi/include/core/RepositoryFactory.h      |   9 +-
 libminifi/include/core/core.h                   | 180 -------
 .../core/repository/FlowFileRepository.h        | 116 +++--
 libminifi/include/io/BaseStream.h               |   1 -
 libminifi/include/io/ClientSocket.h             |  14 +-
 libminifi/include/io/StreamFactory.h            | 157 +++---
 libminifi/include/io/tls/TLSSocket.h            |  17 +-
 libminifi/include/io/validation.h               |  10 +-
 libminifi/include/processors/AppendHostInfo.h   |   9 +-
 libminifi/include/processors/ExecuteProcess.h   |   9 +-
 libminifi/include/processors/GenerateFlowFile.h |   9 +-
 libminifi/include/processors/GetFile.h          |  55 ++-
 libminifi/include/processors/ListenHTTP.h       |  20 +-
 libminifi/include/processors/ListenSyslog.h     |  15 +-
 libminifi/include/processors/LogAttribute.h     |   9 +-
 libminifi/include/processors/PutFile.h          |  17 +-
 .../include/processors/RealTimeDataCollector.h  | 145 ------
 libminifi/include/processors/TailFile.h         |   6 +-
 libminifi/include/properties/Configure.h        |   2 +-
 libminifi/include/provenance/Provenance.h       |   1 -
 .../include/provenance/ProvenanceRepository.h   |   4 +-
 libminifi/include/utils/FailurePolicy.h         |  35 +-
 libminifi/include/utils/StringUtils.h           | 182 +++----
 libminifi/include/utils/TimeUtil.h              |  32 +-
 libminifi/src/Configure.cpp                     | 253 +++++-----
 libminifi/src/Connection.cpp                    |  10 +-
 libminifi/src/EventDrivenSchedulingAgent.cpp    |   4 +-
 libminifi/src/FlowControlProtocol.cpp           |  46 +-
 libminifi/src/FlowController.cpp                |  31 +-
 libminifi/src/FlowFileRecord.cpp                |  54 +--
 libminifi/src/RemoteProcessorGroupPort.cpp      |  20 +-
 libminifi/src/ResourceClaim.cpp                 |   3 +-
 libminifi/src/SchedulingAgent.cpp               |  16 +-
 libminifi/src/Site2SiteClientProtocol.cpp       |  57 ++-
 libminifi/src/Site2SitePeer.cpp                 |  31 +-
 libminifi/src/ThreadedSchedulingAgent.cpp       |  42 +-
 libminifi/src/TimerDrivenSchedulingAgent.cpp    |   4 +-
 libminifi/src/core/ConfigurableComponent.cpp    |  21 +-
 libminifi/src/core/ConfigurationFactory.cpp     |  72 +--
 libminifi/src/core/Connectable.cpp              |  35 +-
 libminifi/src/core/Core.cpp                     |  34 +-
 libminifi/src/core/FlowConfiguration.cpp        |   8 +-
 libminifi/src/core/FlowFile.cpp                 | 223 +++++++++
 libminifi/src/core/ProcessGroup.cpp             |  26 +-
 libminifi/src/core/ProcessSession.cpp           |  74 ++-
 libminifi/src/core/ProcessSessionFactory.cpp    |  10 +-
 libminifi/src/core/Processor.cpp                |  36 +-
 libminifi/src/core/ProcessorNode.cpp            |  24 +-
 libminifi/src/core/Property.cpp                 |   5 +-
 libminifi/src/core/Record.cpp                   | 225 ---------
 libminifi/src/core/Repository.cpp               |   4 +-
 libminifi/src/core/RepositoryFactory.cpp        |  98 ++--
 libminifi/src/core/logging/BaseLogger.cpp       |   7 +-
 libminifi/src/core/logging/LogAppenders.cpp     |  13 +-
 libminifi/src/core/logging/Logger.cpp           |   1 +
 .../src/core/repository/FlowFileRepository.cpp  |  76 +--
 libminifi/src/core/yaml/YamlConfiguration.cpp   |  20 +-
 libminifi/src/io/BaseStream.cpp                 |  36 +-
 libminifi/src/io/CRCStream.cpp                  |   2 -
 libminifi/src/io/ClientSocket.cpp               |  82 +---
 libminifi/src/io/DataStream.cpp                 | 150 +++---
 libminifi/src/io/EndianCheck.cpp                |   1 -
 libminifi/src/io/Serializable.cpp               |  24 +-
 libminifi/src/io/StreamFactory.cpp              |   8 +-
 libminifi/src/io/tls/TLSSocket.cpp              | 359 +++++++-------
 libminifi/src/processors/AppendHostInfo.cpp     |  41 +-
 libminifi/src/processors/ExecuteProcess.cpp     |  34 +-
 libminifi/src/processors/GenerateFlowFile.cpp   | 192 ++++----
 libminifi/src/processors/GetFile.cpp            |  41 +-
 libminifi/src/processors/ListenHTTP.cpp         |  76 ++-
 libminifi/src/processors/ListenSyslog.cpp       |  55 +--
 libminifi/src/processors/LogAttribute.cpp       |  31 +-
 libminifi/src/processors/PutFile.cpp            |  20 +-
 .../src/processors/RealTimeDataCollector.cpp    | 480 -------------------
 libminifi/src/processors/TailFile.cpp           |  58 +--
 libminifi/src/provenance/Provenance.cpp         |  38 +-
 .../src/provenance/ProvenanceRepository.cpp     |  26 +-
 libminifi/test/TestBase.h                       |   4 +-
 libminifi/test/nodefs/NoLevelDB.cpp             |   2 +-
 libminifi/test/nodefs/NoYamlConfiguration.cpp   |   2 +-
 libminifi/test/unit/ProcessorTests.cpp          |  14 +-
 libminifi/test/unit/ProvenanceTestHelper.h      |   2 +-
 libminifi/test/unit/ProvenanceTests.cpp         |   2 +-
 libminifi/test/unit/RepoTests.cpp               |   2 +-
 main/MiNiFiMain.cpp                             |   2 +-
 thirdparty/google-styleguide/cpplint.py         |   6 -
 thirdparty/google-styleguide/run_linter.sh      |   7 +-
 106 files changed, 2084 insertions(+), 2894 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index faa291b..cc9c8e2 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -47,4 +47,4 @@ matrix:
         - package='openssl'; [[ $(brew ls --versions ${package}) ]] && { brew outdated ${package} || brew upgrade ${package}; } || brew install ${package}
 
 script:
-  - mkdir ./build && cd ./build && cmake .. && make && make test
+  - mkdir ./build && cd ./build && cmake .. && make && make test && make linter

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/Connection.h
----------------------------------------------------------------------
diff --git a/libminifi/include/Connection.h b/libminifi/include/Connection.h
index 1c7b9a4..8fe42d0 100644
--- a/libminifi/include/Connection.h
+++ b/libminifi/include/Connection.h
@@ -27,7 +27,7 @@
 #include <mutex>
 #include <atomic>
 #include <algorithm>
-#include "core/core.h"
+#include "core/Core.h"
 #include "core/Connectable.h"
 #include "core/logging/Logger.h"
 #include "core/Relationship.h"

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/Exception.h
----------------------------------------------------------------------
diff --git a/libminifi/include/Exception.h b/libminifi/include/Exception.h
index a0c70e6..88a3ed2 100644
--- a/libminifi/include/Exception.h
+++ b/libminifi/include/Exception.h
@@ -26,74 +26,65 @@
 #include <errno.h>
 #include <string.h>
 
-
 namespace org {
 namespace apache {
 namespace nifi {
 namespace minifi {
 
 // ExceptionType 
-enum ExceptionType 
-{
-	FILE_OPERATION_EXCEPTION = 0,
-	FLOW_EXCEPTION,
-	PROCESSOR_EXCEPTION,
-	PROCESS_SESSION_EXCEPTION,
-	PROCESS_SCHEDULE_EXCEPTION,
-	SITE2SITE_EXCEPTION,
-	GENERAL_EXCEPTION,
-	MAX_EXCEPTION
+enum ExceptionType {
+  FILE_OPERATION_EXCEPTION = 0,
+  FLOW_EXCEPTION,
+  PROCESSOR_EXCEPTION,
+  PROCESS_SESSION_EXCEPTION,
+  PROCESS_SCHEDULE_EXCEPTION,
+  SITE2SITE_EXCEPTION,
+  GENERAL_EXCEPTION,
+  MAX_EXCEPTION
 };
 
 // Exception String 
-static const char *ExceptionStr[MAX_EXCEPTION] =
-{
-		"File Operation",
-		"Flow File Operation",
-		"Processor Operation",
-		"Process Session Operation",
-		"Process Schedule Operation",
-		"Site2Site Protocol",
-		"General Operation"
-};
+static const char *ExceptionStr[MAX_EXCEPTION] = { "File Operation",
+    "Flow File Operation", "Processor Operation", "Process Session Operation",
+    "Process Schedule Operation", "Site2Site Protocol", "General Operation" };
 
 // Exception Type to String 
-inline const char *ExceptionTypeToString(ExceptionType type)
-{
-	if (type < MAX_EXCEPTION)
-		return ExceptionStr[type];
-	else
-		return NULL;
+inline const char *ExceptionTypeToString(ExceptionType type) {
+  if (type < MAX_EXCEPTION)
+    return ExceptionStr[type];
+  else
+    return NULL;
 }
 
 // Exception Class
-class Exception : public std::exception
-{
-public:
-	// Constructor
-	/*!
-	 * Create a new flow record
-	 */
-	Exception(ExceptionType type, const char *errorMsg) : _type(type), _errorMsg(errorMsg) {
-	}
-	// Destructor
-	virtual ~Exception() throw () {}
-	virtual const char * what() const throw () {
-
-		_whatStr = ExceptionTypeToString(_type);
+class Exception : public std::exception {
+ public:
+  // Constructor
+  /*!
+   * Create a new flow record
+   */
+  Exception(ExceptionType type, const char *errorMsg)
+      : _type(type),
+        _errorMsg(errorMsg) {
+  }
+  // Destructor
+  virtual ~Exception() throw () {
+  }
+  virtual const char * what() const throw () {
 
-		_whatStr += ":" + _errorMsg;
-		return _whatStr.c_str();
-	}
+    _whatStr = ExceptionTypeToString(_type);
 
+    _whatStr += ":" + _errorMsg;
+    return _whatStr.c_str();
+  }
 
-private:
-	// Exception type
-	ExceptionType _type;
-	// Exception detailed information
-	std::string _errorMsg;
-	// Hold the what result
-	mutable std::string _whatStr;
+ private:
+  // Exception type
+  ExceptionType _type;
+  // Exception detailed information
+  std::string _errorMsg;
+  // Hold the what result
+  mutable std::string _whatStr;
 
 };
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/FlowControlProtocol.h
----------------------------------------------------------------------
diff --git a/libminifi/include/FlowControlProtocol.h b/libminifi/include/FlowControlProtocol.h
index c37c8f8..826f999 100644
--- a/libminifi/include/FlowControlProtocol.h
+++ b/libminifi/include/FlowControlProtocol.h
@@ -37,7 +37,6 @@
 #include "properties/Configure.h"
 #include "core/logging/Logger.h"
 
-
 namespace org {
 namespace apache {
 namespace nifi {
@@ -179,16 +178,14 @@ class FlowControlProtocol {
       logger_->log_info("NiFi Server Name %s", _serverName.c_str());
     }
     if (configure_->get(Configure::nifi_server_port, value)
-        && core::Property::StringToInt(
-            value, _serverPort)) {
+        && 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)) {
+      if (core::Property::StringToTime(value, _reportInterval, unit)
+          && core::Property::ConvertTimeUnitToMS(_reportInterval, unit,
+                                                 _reportInterval)) {
         logger_->log_info("NiFi server report interval: [%d] ms",
                           _reportInterval);
       }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/FlowController.h
----------------------------------------------------------------------
diff --git a/libminifi/include/FlowController.h b/libminifi/include/FlowController.h
index 0475623..e007f80 100644
--- a/libminifi/include/FlowController.h
+++ b/libminifi/include/FlowController.h
@@ -67,7 +67,8 @@ class FlowController : public core::CoreComponent {
   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);
+                 const std::string name = DEFAULT_ROOT_GROUP_NAME,
+                 bool headless_mode = false);
 
   // Destructor
   virtual ~FlowController();
@@ -122,7 +123,7 @@ class FlowController : public core::CoreComponent {
   // update property value
   void updatePropertyValue(std::string processorName, std::string propertyName,
                            std::string propertyValue) {
-    if (root_  != nullptr)
+    if (root_ != nullptr)
       root_->updatePropertyValue(processorName, propertyName, propertyValue);
   }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/RemoteProcessorGroupPort.h
----------------------------------------------------------------------
diff --git a/libminifi/include/RemoteProcessorGroupPort.h b/libminifi/include/RemoteProcessorGroupPort.h
index e9a4228..8667519 100644
--- a/libminifi/include/RemoteProcessorGroupPort.h
+++ b/libminifi/include/RemoteProcessorGroupPort.h
@@ -28,14 +28,12 @@
 #include "core/ProcessSession.h"
 #include "Site2SiteClientProtocol.h"
 
-
 namespace org {
 namespace apache {
 namespace nifi {
 namespace minifi {
 // RemoteProcessorGroupPort Class
-class RemoteProcessorGroupPort :
-    public core::Processor {
+class RemoteProcessorGroupPort : public core::Processor {
  public:
   // Constructor
   /*!
@@ -44,9 +42,9 @@ class RemoteProcessorGroupPort :
   RemoteProcessorGroupPort(std::string name, uuid_t uuid = NULL)
       : core::Processor(name, uuid),
         direction_(SEND),
-        transmitting_(false){
+        transmitting_(false) {
     logger_ = logging::Logger::getLogger();
-    uuid_copy(protocol_uuid_,uuid);
+    uuid_copy(protocol_uuid_, uuid);
   }
   // Destructor
   virtual ~RemoteProcessorGroupPort() {
@@ -61,9 +59,8 @@ class RemoteProcessorGroupPort :
   static core::Relationship relation;
  public:
   // OnTrigger method, implemented by NiFi RemoteProcessorGroupPort
-  virtual void onTrigger(
-      core::ProcessContext *context,
-      core::ProcessSession *session);
+  virtual void onTrigger(core::ProcessContext *context,
+                         core::ProcessSession *session);
   // Initialize, over write by NiFi RemoteProcessorGroupPort
   virtual void initialize(void);
   // Set Direction
@@ -84,10 +81,10 @@ class RemoteProcessorGroupPort :
  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
@@ -98,9 +95,9 @@ class RemoteProcessorGroupPort :
   bool transmitting_;
   // timeout
   uint64_t timeout_;
-  
+
   uuid_t protocol_uuid_;
- 
+
 };
 
 } /* namespace minifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/ResourceClaim.h
----------------------------------------------------------------------
diff --git a/libminifi/include/ResourceClaim.h b/libminifi/include/ResourceClaim.h
index 4c5438c..1f5d17f 100644
--- a/libminifi/include/ResourceClaim.h
+++ b/libminifi/include/ResourceClaim.h
@@ -34,74 +34,67 @@ namespace apache {
 namespace nifi {
 namespace minifi {
 
-
 // Default content directory
 #define DEFAULT_CONTENT_DIRECTORY "./content_repository"
 
-
-
 // ResourceClaim Class
 class ResourceClaim {
 
-public:
-  
-	static std::string default_directory_path;
-	// Constructor
-	/*!
-	 * Create a new resource claim
-	 */
-	ResourceClaim(const std::string contentDirectory = default_directory_path);
-	// Destructor
-	virtual ~ResourceClaim() {}
-	// increaseFlowFileRecordOwnedCount
-	void increaseFlowFileRecordOwnedCount()
-	{
-		++_flowFileRecordOwnedCount;
-	}
-	// decreaseFlowFileRecordOwenedCount
-	void decreaseFlowFileRecordOwnedCount()
-	{
-		--_flowFileRecordOwnedCount;
-	}
-	// getFlowFileRecordOwenedCount
-	uint64_t getFlowFileRecordOwnedCount()
-	{
-		return _flowFileRecordOwnedCount;
-	}
-	// Get the content full path
-	std::string getContentFullPath()
-	{
-		return _contentFullPath;
-	}
-	// Set the content full path
-	void setContentFullPath(std::string path)
-	{
-		_contentFullPath = path;
-	}
+ public:
+
+  static char *default_directory_path;
+  // Constructor
+  /*!
+   * Create a new resource claim
+   */
+  ResourceClaim(const std::string contentDirectory = default_directory_path);
+  // Destructor
+  virtual ~ResourceClaim() {
+  }
+  // increaseFlowFileRecordOwnedCount
+  void increaseFlowFileRecordOwnedCount() {
+    ++_flowFileRecordOwnedCount;
+  }
+  // decreaseFlowFileRecordOwenedCount
+  void decreaseFlowFileRecordOwnedCount() {
+    --_flowFileRecordOwnedCount;
+  }
+  // getFlowFileRecordOwenedCount
+  uint64_t getFlowFileRecordOwnedCount() {
+    return _flowFileRecordOwnedCount;
+  }
+  // Get the content full path
+  std::string getContentFullPath() {
+    return _contentFullPath;
+  }
+  // Set the content full path
+  void setContentFullPath(std::string path) {
+    _contentFullPath = path;
+  }
 
-protected:
-	// A global unique identifier
-	uuid_t _uuid;
-	// A local unique identifier
-	uint64_t _id;
-	// Full path to the content
-	std::string _contentFullPath;
+ protected:
+  // A global unique identifier
+  uuid_t _uuid;
+  // A local unique identifier
+  uint64_t _id;
+  // Full path to the content
+  std::string _contentFullPath;
 
-	// How many FlowFileRecord Own this cliam
-	std::atomic<uint64_t> _flowFileRecordOwnedCount;
+  // How many FlowFileRecord Own this cliam
+  std::atomic<uint64_t> _flowFileRecordOwnedCount;
 
-private:
-	// Configure
-	Configure *configure_;
-	// 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);
+ private:
+  // Configure
+  Configure *configure_;
+  // 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
-	static std::atomic<uint64_t> _localResourceClaimNumber;
+  // Local resource claim number
+  static std::atomic<uint64_t> _localResourceClaimNumber;
 };
 
 } /* namespace minifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/SchedulingAgent.h
----------------------------------------------------------------------
diff --git a/libminifi/include/SchedulingAgent.h b/libminifi/include/SchedulingAgent.h
index 0493640..e7d1e58 100644
--- a/libminifi/include/SchedulingAgent.h
+++ b/libminifi/include/SchedulingAgent.h
@@ -28,7 +28,7 @@
 #include <algorithm>
 #include <thread>
 #include "utils/TimeUtil.h"
-#include "core/core.h"
+#include "core/Core.h"
 #include "core/logging/Logger.h"
 #include "properties/Configure.h"
 #include "FlowFileRecord.h"
@@ -37,13 +37,11 @@
 #include "core/ProcessContext.h"
 #include "provenance/ProvenanceRepository.h"
 
-
 namespace org {
 namespace apache {
 namespace nifi {
 namespace minifi {
 
-
 // SchedulingAgent Class
 class SchedulingAgent {
  public:
@@ -62,15 +60,13 @@ class SchedulingAgent {
 
   }
   // onTrigger, return whether the yield is need
-  bool onTrigger(
-      std::shared_ptr<core::Processor> processor,
-      core::ProcessContext *processContext,
-      core::ProcessSessionFactory *sessionFactory);
+  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);
+  bool hasTooMuchOutGoing(std::shared_ptr<core::Processor> processor);
   // start
   void start() {
     running_ = true;
@@ -82,11 +78,9 @@ class SchedulingAgent {
 
  public:
   // schedule, overwritten by different DrivenSchedulingAgent
-  virtual void schedule(
-      std::shared_ptr<core::Processor> processor) = 0;
+  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;
+  virtual void unschedule(std::shared_ptr<core::Processor> processor) = 0;
 
   SchedulingAgent(const SchedulingAgent &parent) = delete;
   SchedulingAgent &operator=(const SchedulingAgent &parent) = delete;

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/Site2SiteClientProtocol.h
----------------------------------------------------------------------
diff --git a/libminifi/include/Site2SiteClientProtocol.h b/libminifi/include/Site2SiteClientProtocol.h
index 6120e3e..78673d8 100644
--- a/libminifi/include/Site2SiteClientProtocol.h
+++ b/libminifi/include/Site2SiteClientProtocol.h
@@ -44,7 +44,6 @@
 #include "core/ProcessSession.h"
 #include "io/CRCStream.h"
 
-
 namespace org {
 namespace apache {
 namespace nifi {
@@ -530,13 +529,11 @@ class Site2SiteClientProtocol {
   // Error the transaction
   void error(std::string transactionID);
   // Receive flow files for the process session
-  void receiveFlowFiles(
-      core::ProcessContext *context,
-      core::ProcessSession *session);
+  void receiveFlowFiles(core::ProcessContext *context,
+                        core::ProcessSession *session);
   // Transfer flow files for the process session
-  void transferFlowFiles(
-      core::ProcessContext *context,
-      core::ProcessSession *session);
+  void transferFlowFiles(core::ProcessContext *context,
+                         core::ProcessSession *session);
   // deleteTransaction
   void deleteTransaction(std::string transactionID);
   // Nest Callback Class for write stream

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/Site2SitePeer.h
----------------------------------------------------------------------
diff --git a/libminifi/include/Site2SitePeer.h b/libminifi/include/Site2SitePeer.h
index de3a42f..a315293 100644
--- a/libminifi/include/Site2SitePeer.h
+++ b/libminifi/include/Site2SitePeer.h
@@ -37,7 +37,6 @@
 #include "io/BaseStream.h"
 #include "utils/TimeUtil.h"
 
-
 namespace org {
 namespace apache {
 namespace nifi {
@@ -281,7 +280,6 @@ class Site2SitePeer : public org::apache::nifi::minifi::io::BaseStream {
 
 };
 
-
 } /* namespace minifi */
 } /* namespace nifi */
 } /* namespace apache */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/core/ConfigurationFactory.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ConfigurationFactory.h b/libminifi/include/core/ConfigurationFactory.h
index 19ed5f4..b25faff 100644
--- a/libminifi/include/core/ConfigurationFactory.h
+++ b/libminifi/include/core/ConfigurationFactory.h
@@ -28,33 +28,29 @@ 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 ) {
+    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);
+    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);
+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 */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/core/Connectable.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/Connectable.h b/libminifi/include/core/Connectable.h
index 15e618f..f7e425e 100644
--- a/libminifi/include/core/Connectable.h
+++ b/libminifi/include/core/Connectable.h
@@ -20,7 +20,7 @@
 #define LIBMINIFI_INCLUDE_CORE_CONNECTABLE_H_
 
 #include <set>
-#include "core.h"
+#include "Core.h"
 #include <condition_variable>
 #include "core/logging/Logger.h"
 #include "Relationship.h"

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/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..1010be7
--- /dev/null
+++ b/libminifi/include/core/Core.h
@@ -0,0 +1,178 @@
+/**
+ *
+ * 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, std::shared_ptr<T>>::type instantiate() {
+  throw std::runtime_error("Cannot instantiate class");
+}
+
+template<typename T>
+typename std::enable_if<class_operations<T>::value, std::shared_ptr<T>>::type instantiate() {
+  return std::make_shared<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_;
+  }
+  
+  void loadComponent(){
+  }
+
+ 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/be3f2ffe/libminifi/include/core/FlowConfiguration.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/FlowConfiguration.h b/libminifi/include/core/FlowConfiguration.h
index e95e684..de8ceb4 100644
--- a/libminifi/include/core/FlowConfiguration.h
+++ b/libminifi/include/core/FlowConfiguration.h
@@ -18,7 +18,7 @@
 #ifndef LIBMINIFI_INCLUDE_CORE_FLOWCONFIGURATION_H_
 #define LIBMINIFI_INCLUDE_CORE_FLOWCONFIGURATION_H_
 
-#include "core/core.h"
+#include "core/Core.h"
 #include "Connection.h"
 #include "RemoteProcessorGroupPort.h"
 #include "provenance/Provenance.h"
@@ -28,7 +28,7 @@
 #include "processors/ListenHTTP.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"

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/core/FlowFile.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/FlowFile.h b/libminifi/include/core/FlowFile.h
index 247ad26..394b9d4 100644
--- a/libminifi/include/core/FlowFile.h
+++ b/libminifi/include/core/FlowFile.h
@@ -183,10 +183,9 @@ class FlowFile {
   std::string getUUIDStr() {
     return uuid_str_;
   }
-  
-  bool getUUID(uuid_t other)
-  {
-    uuid_copy(other,uuid_);
+
+  bool getUUID(uuid_t other) {
+    uuid_copy(other, uuid_);
     return true;
   }
 
@@ -266,7 +265,7 @@ class FlowFile {
 
   // 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

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/core/ProcessSession.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ProcessSession.h b/libminifi/include/core/ProcessSession.h
index a80769e..70805e9 100644
--- a/libminifi/include/core/ProcessSession.h
+++ b/libminifi/include/core/ProcessSession.h
@@ -71,17 +71,17 @@ class ProcessSession {
   provenance::ProvenanceReporter *getProvenanceReporter() {
     return provenance_report_;
   }
-//
-// Get the FlowFile from the highest priority queue
+  //
+  // 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
+  // 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
+  // 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);
-
+  // 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){
+      std::shared_ptr<core::FlowFile> &parent) {
     return create(parent);
   }
 // Clone a new UUID FlowFile from parent both for content resource claim and attributes
@@ -89,7 +89,7 @@ class ProcessSession {
       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);
+                                        int64_t offset, int64_t 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);
@@ -133,7 +133,8 @@ class ProcessSession {
    * @param stream incoming data stream that contains the data to store into a file
    * @param flow flow file
    */
-  void importFrom(io::DataStream &stream, std::shared_ptr<core::FlowFile> &&flow);
+  void importFrom(io::DataStream &stream,
+                  std::shared_ptr<core::FlowFile> &&flow);
   // import from the data source.
   void import(std::string source, std::shared_ptr<core::FlowFile> &flow,
               bool keepSource = true, uint64_t offset = 0);

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/core/Processor.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/Processor.h b/libminifi/include/core/Processor.h
index 4a71816..2b540ec 100644
--- a/libminifi/include/core/Processor.h
+++ b/libminifi/include/core/Processor.h
@@ -123,8 +123,7 @@ class Processor : public Connectable, public ConfigurableComponent,
   void setPenalizationPeriodMsec(uint64_t period) {
     _penalizationPeriodMsec = period;
   }
-  
-  
+
   // Set Processor Maximum Concurrent Tasks
   void setMaxConcurrentTasks(uint8_t tasks) {
     max_concurrent_tasks_ = tasks;
@@ -211,7 +210,6 @@ class Processor : public Connectable, public ConfigurableComponent,
 
  public:
 
-
   // OnTrigger method, implemented by NiFi Processor Designer
   virtual void onTrigger(ProcessContext *context, ProcessSession *session) = 0;
   // Initialize, overridden by NiFi Process Designer
@@ -235,19 +233,18 @@ class Processor : public Connectable, public ConfigurableComponent,
   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:
+ private:
 
   // Mutex for protection
   std::mutex mutex_;
   // Yield Expiration
   std::atomic<uint64_t> yield_expiration_;
-  
 
   // Check all incoming connections for work
   bool isWorkAvailable();

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/core/ProcessorConfig.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ProcessorConfig.h b/libminifi/include/core/ProcessorConfig.h
index 6b4a00a..c1d563e 100644
--- a/libminifi/include/core/ProcessorConfig.h
+++ b/libminifi/include/core/ProcessorConfig.h
@@ -17,7 +17,7 @@
 #ifndef LIBMINIFI_INCLUDE_CORE_PROCESSORCONFIG_H_
 #define LIBMINIFI_INCLUDE_CORE_PROCESSORCONFIG_H_
 
-#include "core/core.h"
+#include "core/Core.h"
 #include "core/Property.h"
 
 namespace org {
@@ -26,7 +26,6 @@ namespace nifi {
 namespace minifi {
 namespace core {
 
-
 struct ProcessorConfig {
   std::string id;
   std::string name;
@@ -41,13 +40,10 @@ struct ProcessorConfig {
   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/be3f2ffe/libminifi/include/core/Repository.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/Repository.h b/libminifi/include/core/Repository.h
index e096023..6209b83 100644
--- a/libminifi/include/core/Repository.h
+++ b/libminifi/include/core/Repository.h
@@ -39,7 +39,7 @@
 #include "io/Serializable.h"
 #include "utils/TimeUtil.h"
 #include "utils/StringUtils.h"
-#include "core.h"
+#include "Core.h"
 
 namespace org {
 namespace apache {
@@ -72,15 +72,15 @@ class Repository : public CoreComponent {
   }
 
   // initialize
-  virtual bool initialize(){
+  virtual bool initialize() {
     return true;
   }
   // Put
-  virtual bool Put(std::string key, uint8_t *buf, int bufLen){
+  virtual bool Put(std::string key, uint8_t *buf, int bufLen) {
     return true;
   }
   // Delete
-  virtual bool Delete(std::string key){
+  virtual bool Delete(std::string key) {
     return true;
   }
 
@@ -89,7 +89,7 @@ class Repository : public CoreComponent {
   }
 
   // Run function for the thread
-  virtual  void run(){
+  virtual void run() {
     // no op
   }
   // Start the repository monitor thread
@@ -138,12 +138,9 @@ class Repository : public CoreComponent {
   // size of the directory
   std::atomic<uint64_t> repo_size_;
   // Run function for the thread
-  void threadExecutor(){
-      run();
-    }
-
-  
-    
+  void threadExecutor() {
+    run();
+  }
 };
 
 } /* namespace core */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/core/RepositoryFactory.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/RepositoryFactory.h b/libminifi/include/core/RepositoryFactory.h
index 03ed524..ed9a026 100644
--- a/libminifi/include/core/RepositoryFactory.h
+++ b/libminifi/include/core/RepositoryFactory.h
@@ -19,9 +19,8 @@
 #ifndef LIBMINIFI_INCLUDE_CORE_REPOSITORYFACTORY_H_
 #define LIBMINIFI_INCLUDE_CORE_REPOSITORYFACTORY_H_
 
-
 #include "core/Repository.h"
-#include "core.h"
+#include "Core.h"
 
 namespace org {
 namespace apache {
@@ -30,10 +29,8 @@ namespace minifi {
 
 namespace core {
 
-  std::shared_ptr<core::Repository> createRepository(
-      const std::string configuration_class_name, bool fail_safe = false);
-
-
+std::shared_ptr<core::Repository> createRepository(
+    const std::string configuration_class_name, bool fail_safe = false);
 
 } /* namespace core */
 } /* namespace minifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/core/core.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/core.h b/libminifi/include/core/core.h
deleted file mode 100644
index a70dbd4..0000000
--- a/libminifi/include/core/core.h
+++ /dev/null
@@ -1,180 +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_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, std::shared_ptr<T>>::type instantiate() {
-  throw std::runtime_error("Cannot instantiate class");
-}
-
-template<typename T>
-typename std::enable_if<class_operations<T>::value, std::shared_ptr<T>>::type instantiate() {
-  return std::make_shared<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_;
-  }
-  
-  void loadComponent(){
-  }
-
- 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/be3f2ffe/libminifi/include/core/repository/FlowFileRepository.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/repository/FlowFileRepository.h b/libminifi/include/core/repository/FlowFileRepository.h
index 0115588..eed1975 100644
--- a/libminifi/include/core/repository/FlowFileRepository.h
+++ b/libminifi/include/core/repository/FlowFileRepository.h
@@ -23,7 +23,7 @@
 #include "leveldb/slice.h"
 #include "leveldb/status.h"
 #include "core/Repository.h"
-#include "core/core.h"
+#include "core/Core.h"
 #include "Connection.h"
 
 namespace org {
@@ -33,8 +33,6 @@ 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
@@ -44,12 +42,11 @@ namespace repository {
  * 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> {
+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,
@@ -58,10 +55,12 @@ class FlowFileRepository : public core::Repository, public std::enable_shared_fr
   {
     db_ = NULL;
   }
-  
-  FlowFileRepository() : FlowFileRepository(FLOWFILE_REPOSITORY_DIRECTORY,
-			MAX_FLOWFILE_REPOSITORY_ENTRY_LIFE_TIME, MAX_FLOWFILE_REPOSITORY_STORAGE_SIZE, FLOWFILE_REPOSITORY_PURGE_PERIOD)
-  {
+
+  FlowFileRepository()
+      : FlowFileRepository(FLOWFILE_REPOSITORY_DIRECTORY,
+      MAX_FLOWFILE_REPOSITORY_ENTRY_LIFE_TIME,
+                           MAX_FLOWFILE_REPOSITORY_STORAGE_SIZE,
+                           FLOWFILE_REPOSITORY_PURGE_PERIOD) {
   }
 
   // Destructor
@@ -112,63 +111,60 @@ class FlowFileRepository : public core::Repository, public std::enable_shared_fr
   }
 
   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;
+
+  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;
+   * 
+   * 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;
+   * 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 setConnectionMap(std::map<std::string, std::shared_ptr<minifi::Connection>> &connectionMap)
-  {
-    this->connectionMap=connectionMap;
+
+  void setConnectionMap(
+      std::map<std::string, std::shared_ptr<minifi::Connection>> &connectionMap) {
+    this->connectionMap = connectionMap;
   }
   void loadComponent();
-  
-   void start() {
-  if (this->purge_period_ <= 0)
-    return;
-  if (running_)
-    return;
-  thread_ = std::thread(&FlowFileRepository::run, shared_from_this());
-  thread_.detach();
-  running_ = true;
-  logger_->log_info("%s Repository Monitor Thread Start", name_.c_str());
-}
+
+  void start() {
+    if (this->purge_period_ <= 0)
+      return;
+    if (running_)
+      return;
+    thread_ = std::thread(&FlowFileRepository::run, shared_from_this());
+    thread_.detach();
+    running_ = true;
+    logger_->log_info("%s Repository Monitor Thread Start", name_.c_str());
+  }
 
  private:
   std::map<std::string, std::shared_ptr<minifi::Connection>> connectionMap;

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/io/BaseStream.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/BaseStream.h b/libminifi/include/io/BaseStream.h
index b0b3589..e2c0474 100644
--- a/libminifi/include/io/BaseStream.h
+++ b/libminifi/include/io/BaseStream.h
@@ -30,7 +30,6 @@ namespace nifi {
 namespace minifi {
 namespace io {
 
-
 class BaseStream : public DataStream, public Serializable {
 
  public:

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/io/ClientSocket.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/ClientSocket.h b/libminifi/include/io/ClientSocket.h
index 97cace2..6998950 100644
--- a/libminifi/include/io/ClientSocket.h
+++ b/libminifi/include/io/ClientSocket.h
@@ -26,7 +26,7 @@
 #include <mutex>
 #include <atomic>
 #include "io/BaseStream.h"
-#include "core/core.h"
+#include "core/Core.h"
 #include "core/logging/Logger.h"
 
 #include "io/validation.h"
@@ -70,14 +70,14 @@ class Socket : public BaseStream {
    */
   explicit Socket(const Socket &&);
 
-  static std::string HOSTNAME;
+  static char *HOSTNAME;
 
   /**
    * Static function to return the current machine's host name
    */
-  static std::string getMyHostName(std::string *str = &HOSTNAME) {
+  static std::string getMyHostName(const char *str = HOSTNAME) {
     if (__builtin_expect(!IsNullOrEmpty(str), 0))
-      return *str;
+      return str;
     else {
       char hostname[1024];
       gethostname(hostname, 1024);
@@ -98,7 +98,7 @@ class Socket : public BaseStream {
    * Initializes the socket
    * @return result of the creation operation.
    */
-  virtual short initialize();
+  virtual int16_t initialize();
 
   std::string getHostname() const;
 
@@ -217,14 +217,14 @@ class Socket : public BaseStream {
    * Sets socket options depending on the instance.
    * @param sock socket file descriptor.
    */
-  virtual short setSocketOptions(const int sock);
+  virtual int16_t 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);
+  virtual int16_t select_descriptor(const uint16_t msec);
 
   std::shared_ptr<logging::Logger> logger_;
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/io/StreamFactory.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/StreamFactory.h b/libminifi/include/io/StreamFactory.h
index faa10b5..f942f63 100644
--- a/libminifi/include/io/StreamFactory.h
+++ b/libminifi/include/io/StreamFactory.h
@@ -39,18 +39,18 @@ namespace io {
 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);
-	}
+  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);
+  }
 
 };
 
@@ -60,73 +60,72 @@ public:
  
  **/
 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_;
+ 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 */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/io/tls/TLSSocket.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/tls/TLSSocket.h b/libminifi/include/io/tls/TLSSocket.h
index f86f8bc..2762ba8 100644
--- a/libminifi/include/io/tls/TLSSocket.h
+++ b/libminifi/include/io/tls/TLSSocket.h
@@ -18,10 +18,12 @@
 #ifndef LIBMINIFI_INCLUDE_IO_TLSSOCKET_H_
 #define LIBMINIFI_INCLUDE_IO_TLSSOCKET_H_
 
+#include <openssl/ssl.h>
+#include <openssl/err.h>
 #include <cstdint>
-#include "../ClientSocket.h"
 #include <atomic>
 #include <mutex>
+#include "../ClientSocket.h"
 
 #include "properties/Configure.h"
 
@@ -31,9 +33,6 @@ 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
@@ -75,11 +74,11 @@ class TLSContext {
     return ctx;
   }
 
-  short getError() {
+  int16_t getError() {
     return error_value;
   }
 
-  short initialize();
+  int16_t initialize();
 
  private:
 
@@ -113,7 +112,7 @@ class TLSContext {
   Configure *configuration;
   SSL_CTX *ctx;
 
-  short error_value;
+  int16_t error_value;
 
   static std::atomic<TLSContext*> context_instance;
   static std::mutex context_mutex;
@@ -151,14 +150,14 @@ class TLSSocket : public Socket {
    * Initializes the socket
    * @return result of the creation operation.
    */
-  short initialize();
+  int16_t 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);
+  virtual int16_t select_descriptor(const uint16_t msec);
 
   /**
    * Reads data and places it into buf

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/io/validation.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/validation.h b/libminifi/include/io/validation.h
index c66c412..9dd1b8a 100644
--- a/libminifi/include/io/validation.h
+++ b/libminifi/include/io/validation.h
@@ -22,7 +22,6 @@
 #include <string>
 #include <cstring>
 
-
 /**
  * A checker that will, at compile time, tell us
  * if the declared type has a size method.
@@ -42,7 +41,6 @@ class size_function_functor_checker {
   };
 };
 
-
 /**
  * Determines if the variable is null or ::size() == 0
  */
@@ -72,9 +70,15 @@ static auto IsNullOrEmpty(
 /**
  * Determines if the variable is null or strlen(str) == 0
  */
-static auto IsNullOrEmpty(char *str)-> decltype(NULL !=str, bool()) {
+static auto IsNullOrEmpty(const char *str)-> decltype(NULL !=str, bool()) {
   return (NULL == str || strlen(str) == 0);
 }
 
+/**
+ * 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);
+}
 
 #endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/processors/AppendHostInfo.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/AppendHostInfo.h b/libminifi/include/processors/AppendHostInfo.h
index 6515918..a16dff3 100644
--- a/libminifi/include/processors/AppendHostInfo.h
+++ b/libminifi/include/processors/AppendHostInfo.h
@@ -24,7 +24,7 @@
 #include "FlowFileRecord.h"
 #include "core/Processor.h"
 #include "core/ProcessSession.h"
-#include "core/core.h"
+#include "core/Core.h"
 
 namespace org {
 namespace apache {
@@ -47,7 +47,7 @@ class AppendHostInfo : public core::Processor {
   virtual ~AppendHostInfo() {
   }
   // Processor Name
-  static const std::string ProcessorName;
+  static constexpr char const* ProcessorName = "AppendHostInfo";
   // Supported Properties
   static core::Property InterfaceName;
   static core::Property HostAttribute;
@@ -58,9 +58,8 @@ class AppendHostInfo : public core::Processor {
 
  public:
   // OnTrigger method, implemented by NiFi AppendHostInfo
-  virtual void onTrigger(
-      core::ProcessContext *context,
-      core::ProcessSession *session);
+  virtual void onTrigger(core::ProcessContext *context,
+                         core::ProcessSession *session);
   // Initialize, over write by NiFi AppendHostInfo
   virtual void initialize(void);
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/processors/ExecuteProcess.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/ExecuteProcess.h b/libminifi/include/processors/ExecuteProcess.h
index 123eed3..f74f489 100644
--- a/libminifi/include/processors/ExecuteProcess.h
+++ b/libminifi/include/processors/ExecuteProcess.h
@@ -34,7 +34,7 @@
 #include "FlowFileRecord.h"
 #include "core/Processor.h"
 #include "core/ProcessSession.h"
-#include "core/core.h"
+#include "core/Core.h"
 
 namespace org {
 namespace apache {
@@ -64,7 +64,7 @@ class ExecuteProcess : public core::Processor {
       kill(_pid, SIGTERM);
   }
   // Processor Name
-  static const std::string ProcessorName;
+  static constexpr char const* ProcessorName = "ExecuteProcess";
   // Supported Properties
   static core::Property Command;
   static core::Property CommandArguments;
@@ -91,9 +91,8 @@ class ExecuteProcess : public core::Processor {
 
  public:
   // OnTrigger method, implemented by NiFi ExecuteProcess
-  virtual void onTrigger(
-      core::ProcessContext *context,
-      core::ProcessSession *session);
+  virtual void onTrigger(core::ProcessContext *context,
+                         core::ProcessSession *session);
   // Initialize, over write by NiFi ExecuteProcess
   virtual void initialize(void);
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/processors/GenerateFlowFile.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/GenerateFlowFile.h b/libminifi/include/processors/GenerateFlowFile.h
index c4ab6fe..d15a02c 100644
--- a/libminifi/include/processors/GenerateFlowFile.h
+++ b/libminifi/include/processors/GenerateFlowFile.h
@@ -23,7 +23,7 @@
 #include "FlowFileRecord.h"
 #include "core/Processor.h"
 #include "core/ProcessSession.h"
-#include "core/core.h"
+#include "core/Core.h"
 
 namespace org {
 namespace apache {
@@ -48,7 +48,7 @@ class GenerateFlowFile : public core::Processor {
       delete[] _data;
   }
   // Processor Name
-  static const std::string ProcessorName;
+  static constexpr char const* ProcessorName = "GenerateFlowFile";
   // Supported Properties
   static core::Property FileSize;
   static core::Property BatchSize;
@@ -75,9 +75,8 @@ class GenerateFlowFile : public core::Processor {
 
  public:
   // OnTrigger method, implemented by NiFi GenerateFlowFile
-  virtual void onTrigger(
-      core::ProcessContext *context,
-      core::ProcessSession *session);
+  virtual void onTrigger(core::ProcessContext *context,
+                         core::ProcessSession *session);
   // Initialize, over write by NiFi GenerateFlowFile
   virtual void initialize(void);
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/processors/GetFile.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/GetFile.h b/libminifi/include/processors/GetFile.h
index cc3beaa..5345404 100644
--- a/libminifi/include/processors/GetFile.h
+++ b/libminifi/include/processors/GetFile.h
@@ -22,7 +22,7 @@
 #include "FlowFileRecord.h"
 #include "core/Processor.h"
 #include "core/ProcessSession.h"
-#include "core/core.h"
+#include "core/Core.h"
 
 namespace org {
 namespace apache {
@@ -30,19 +30,19 @@ namespace nifi {
 namespace minifi {
 namespace processors {
 
-  struct GetFileRequest{
-    std::string directory = ".";
-    bool recursive = true;
-    bool keepSourceFile = false;
-    int64_t minAge = 0;
-    int64_t maxAge = 0;
-    int64_t minSize = 0;
-    int64_t maxSize = 0;
-    bool ignoreHiddenFile = true;
-    int64_t pollInterval = 0;
-    int64_t batchSize = 10;
-    std::string fileFilter= "[^\\.].*";
-  };
+struct GetFileRequest {
+  std::string directory = ".";
+  bool recursive = true;
+  bool keepSourceFile = false;
+  int64_t minAge = 0;
+  int64_t maxAge = 0;
+  int64_t minSize = 0;
+  int64_t maxSize = 0;
+  bool ignoreHiddenFile = true;
+  int64_t pollInterval = 0;
+  int64_t batchSize = 10;
+  std::string fileFilter = "[^\\.].*";
+};
 
 // GetFile Class
 class GetFile : public core::Processor {
@@ -59,7 +59,7 @@ class GetFile : public core::Processor {
   virtual ~GetFile() {
   }
   // Processor Name
-  static const std::string ProcessorName;
+  static constexpr char const* ProcessorName = "GetFile";
   // Supported Properties
   static core::Property Directory;
   static core::Property Recurse;
@@ -76,19 +76,22 @@ class GetFile : public core::Processor {
   static core::Relationship Success;
 
  public:
-  // OnTrigger method, implemented by NiFi GetFile
-  virtual void onTrigger(
-      core::ProcessContext *context,
-      core::ProcessSession *session);
   /**
    * Function that's executed when the processor is scheduled.
    * @param context process context.
    * @param sessionFactory process session factory that is used when creating
    * ProcessSession objects.
    */
-  void onSchedule(
-        core::ProcessContext *context,
-        core::ProcessSessionFactory *sessionFactory);
+  void onSchedule(core::ProcessContext *context,
+                  core::ProcessSessionFactory *sessionFactory);
+  /**
+   * Execution trigger for the GetFile Processor
+   * @param context processor context
+   * @param session processor session reference.
+   */
+  virtual void onTrigger(core::ProcessContext *context,
+                         core::ProcessSession *session);
+
   // Initialize, over write by NiFi GetFile
   virtual void initialize(void);
   /**
@@ -96,7 +99,7 @@ class GetFile : public core::Processor {
    * @param dir directory to list
    * @param request get file request.
    */
-  void performListing(std::string dir,const GetFileRequest &request);
+  void performListing(std::string dir, const GetFileRequest &request);
 
  protected:
 
@@ -114,9 +117,11 @@ class GetFile : public core::Processor {
   // 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,const GetFileRequest &request);
+  void pollListing(std::queue<std::string> &list,
+                   const GetFileRequest &request);
   // Check whether file can be added to the directory listing
-  bool acceptFile(std::string fullName, std::string name, const GetFileRequest &request);
+  bool acceptFile(std::string fullName, std::string name,
+                  const GetFileRequest &request);
   // Get file request object.
   GetFileRequest request_;
   // Mutex for protection of the directory listing

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/processors/ListenHTTP.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/ListenHTTP.h b/libminifi/include/processors/ListenHTTP.h
index adaefb1..69432be 100644
--- a/libminifi/include/processors/ListenHTTP.h
+++ b/libminifi/include/processors/ListenHTTP.h
@@ -28,7 +28,7 @@
 #include "FlowFileRecord.h"
 #include "core/Processor.h"
 #include "core/ProcessSession.h"
-#include "core/core.h"
+#include "core/Core.h"
 
 namespace org {
 namespace apache {
@@ -51,7 +51,7 @@ class ListenHTTP : public core::Processor {
   // Destructor
   virtual ~ListenHTTP();
   // Processor Name
-  static const std::string ProcessorName;
+  static constexpr char const* ProcessorName = "ListenHTTP";
   // Supported Properties
   static core::Property BasePath;
   static core::Property Port;
@@ -64,20 +64,18 @@ class ListenHTTP : public core::Processor {
   // Supported Relationships
   static core::Relationship Success;
 
-  void onTrigger(core::ProcessContext *context,
-                 core::ProcessSession *session);
+  void onTrigger(core::ProcessContext *context, core::ProcessSession *session);
   void initialize();
-  void onSchedule(
-      core::ProcessContext *context,
-      core::ProcessSessionFactory *sessionFactory);
+  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);
+    Handler(core::ProcessContext *context,
+            core::ProcessSessionFactory *sessionFactory,
+            std::string &&authDNPattern,
+            std::string &&headersAsAttributesPattern);
     bool handlePost(CivetServer *server, struct mg_connection *conn);
 
    private:

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/processors/ListenSyslog.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/ListenSyslog.h b/libminifi/include/processors/ListenSyslog.h
index 1e1e11f..cbbdf41 100644
--- a/libminifi/include/processors/ListenSyslog.h
+++ b/libminifi/include/processors/ListenSyslog.h
@@ -35,7 +35,7 @@
 #include "FlowFileRecord.h"
 #include "core/Processor.h"
 #include "core/ProcessSession.h"
-#include "core/core.h"
+#include "core/Core.h"
 
 namespace org {
 namespace apache {
@@ -45,7 +45,7 @@ namespace processors {
 
 // SyslogEvent
 typedef struct {
-  uint8_t *payload;
+  char *payload;
   uint64_t len;
 } SysLogEvent;
 
@@ -95,7 +95,7 @@ class ListenSyslog : public core::Processor {
     }
   }
   // Processor Name
-  static const std::string ProcessorName;
+  static constexpr char const *ProcessorName = "ListenSyslog";
   // Supported Properties
   static core::Property RecvBufSize;
   static core::Property MaxSocketBufSize;
@@ -125,9 +125,8 @@ class ListenSyslog : public core::Processor {
 
  public:
   // OnTrigger method, implemented by NiFi ListenSyslog
-  virtual void onTrigger(
-      core::ProcessContext *context,
-      core::ProcessSession *session);
+  virtual void onTrigger(core::ProcessContext *context,
+                         core::ProcessSession *session);
   // Initialize, over write by NiFi ListenSyslog
   virtual void initialize(void);
 
@@ -163,7 +162,7 @@ class ListenSyslog : public core::Processor {
   void putEvent(uint8_t *payload, uint64_t len) {
     std::lock_guard<std::mutex> lock(mutex_);
     SysLogEvent event;
-    event.payload = payload;
+    event.payload = reinterpret_cast<char*>(payload);
     event.len = len;
     _eventQueue.push(event);
     _eventQueueByteSize += len;
@@ -204,7 +203,7 @@ class ListenSyslog : public core::Processor {
   bool _resetServerSocket;
   bool _serverTheadRunning;
   // buffer for read socket
-  uint8_t _buffer[2048];
+  char _buffer[2048];
 };
 
 } /* namespace processors */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/processors/LogAttribute.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/LogAttribute.h b/libminifi/include/processors/LogAttribute.h
index 37c0ec3..dcc802d 100644
--- a/libminifi/include/processors/LogAttribute.h
+++ b/libminifi/include/processors/LogAttribute.h
@@ -23,7 +23,7 @@
 #include "FlowFileRecord.h"
 #include "core/Processor.h"
 #include "core/ProcessSession.h"
-#include "core/core.h"
+#include "core/Core.h"
 
 namespace org {
 namespace apache {
@@ -46,7 +46,7 @@ class LogAttribute : public core::Processor {
   virtual ~LogAttribute() {
   }
   // Processor Name
-  static const std::string ProcessorName;
+  static constexpr char const* ProcessorName = "LogAttribute";
   // Supported Properties
   static core::Property LogLevel;
   static core::Property AttributesToLog;
@@ -108,9 +108,8 @@ class LogAttribute : public core::Processor {
 
  public:
   // OnTrigger method, implemented by NiFi LogAttribute
-  virtual void onTrigger(
-      core::ProcessContext *context,
-      core::ProcessSession *session);
+  virtual void onTrigger(core::ProcessContext *context,
+                         core::ProcessSession *session);
   // Initialize, over write by NiFi LogAttribute
   virtual void initialize(void);
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/processors/PutFile.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/PutFile.h b/libminifi/include/processors/PutFile.h
index c0effaf..cc5dfca 100644
--- a/libminifi/include/processors/PutFile.h
+++ b/libminifi/include/processors/PutFile.h
@@ -23,7 +23,7 @@
 #include "FlowFileRecord.h"
 #include "core/Processor.h"
 #include "core/ProcessSession.h"
-#include "core/core.h"
+#include "core/Core.h"
 
 namespace org {
 namespace apache {
@@ -35,9 +35,11 @@ namespace processors {
 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;
+  static constexpr char const* CONFLICT_RESOLUTION_STRATEGY_REPLACE = "replace";
+  static constexpr char const* CONFLICT_RESOLUTION_STRATEGY_IGNORE = "ignore";
+  static constexpr char const* CONFLICT_RESOLUTION_STRATEGY_FAIL = "fail";
+
+  static constexpr char const* ProcessorName = "PutFile";
 
   // Constructor
   /*!
@@ -49,8 +51,7 @@ class PutFile : public core::Processor {
   // Destructor
   virtual ~PutFile() {
   }
-  // Processor Name
-  static const std::string ProcessorName;
+
   // Supported Properties
   static core::Property Directory;
   static core::Property ConflictResolution;
@@ -64,8 +65,8 @@ class PutFile : public core::Processor {
    * @param sessionFactory process session factory that is used when creating
    * ProcessSession objects.
    */
-  void onSchedule(core::ProcessContext *context,
-                  core::ProcessSessionFactory *sessionFactory);
+  virtual void onSchedule(core::ProcessContext *context,
+                          core::ProcessSessionFactory *sessionFactory);
 
   // OnTrigger method, implemented by NiFi PutFile
   virtual void onTrigger(core::ProcessContext *context,

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/processors/RealTimeDataCollector.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/RealTimeDataCollector.h b/libminifi/include/processors/RealTimeDataCollector.h
deleted file mode 100644
index 41bd814..0000000
--- a/libminifi/include/processors/RealTimeDataCollector.h
+++ /dev/null
@@ -1,145 +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 "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/be3f2ffe/libminifi/include/processors/TailFile.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/TailFile.h b/libminifi/include/processors/TailFile.h
index c7b7b46..c6349a0 100644
--- a/libminifi/include/processors/TailFile.h
+++ b/libminifi/include/processors/TailFile.h
@@ -23,7 +23,7 @@
 #include "FlowFileRecord.h"
 #include "core/Processor.h"
 #include "core/ProcessSession.h"
-#include "core/core.h"
+#include "core/Core.h"
 
 namespace org {
 namespace apache {
@@ -47,7 +47,7 @@ class TailFile : public core::Processor {
     storeState();
   }
   // Processor Name
-  static const std::string ProcessorName;
+  static constexpr char const* ProcessorName = "TailFile";
   // Supported Properties
   static core::Property FileName;
   static core::Property StateFile;
@@ -76,6 +76,8 @@ class TailFile : public core::Processor {
   uint64_t _currentTailFilePosition;
   bool _stateRecovered;
   uint64_t _currentTailFileCreatedTime;
+  static const int BUFFER_SIZE = 512;
+
   // Utils functions for parse state file
   std::string trimLeft(const std::string& s);
   std::string trimRight(const std::string& s);

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/properties/Configure.h
----------------------------------------------------------------------
diff --git a/libminifi/include/properties/Configure.h b/libminifi/include/properties/Configure.h
index c0d9bd4..d1d045c 100644
--- a/libminifi/include/properties/Configure.h
+++ b/libminifi/include/properties/Configure.h
@@ -27,7 +27,7 @@
 #include <errno.h>
 #include <iostream>
 #include <fstream>
-#include "core/core.h"
+#include "core/Core.h"
 #include "core/logging/Logger.h"
 
 namespace org {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/provenance/Provenance.h
----------------------------------------------------------------------
diff --git a/libminifi/include/provenance/Provenance.h b/libminifi/include/provenance/Provenance.h
index 3d5d19e..82754c4 100644
--- a/libminifi/include/provenance/Provenance.h
+++ b/libminifi/include/provenance/Provenance.h
@@ -30,7 +30,6 @@
 #include <thread>
 #include <vector>
 
-
 #include "core/Repository.h"
 #include "core/Property.h"
 #include "properties/Configure.h"

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/provenance/ProvenanceRepository.h
----------------------------------------------------------------------
diff --git a/libminifi/include/provenance/ProvenanceRepository.h b/libminifi/include/provenance/ProvenanceRepository.h
index 8dc152f..2b71fd9 100644
--- a/libminifi/include/provenance/ProvenanceRepository.h
+++ b/libminifi/include/provenance/ProvenanceRepository.h
@@ -22,7 +22,7 @@
 #include "leveldb/slice.h"
 #include "leveldb/status.h"
 #include "core/Repository.h"
-#include "core/core.h"
+#include "core/Core.h"
 #include "provenance/Provenance.h"
 namespace org {
 namespace apache {
@@ -157,7 +157,7 @@ class ProvenanceRepository : public core::Repository,
     }
   }
   // Run function for the thread
-   void run();
+  void run();
 
   // Prevent default copy constructor and assignment operation
   // Only support pass by reference or pointer

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/utils/FailurePolicy.h
----------------------------------------------------------------------
diff --git a/libminifi/include/utils/FailurePolicy.h b/libminifi/include/utils/FailurePolicy.h
index 98ec18a..38c82f9 100644
--- a/libminifi/include/utils/FailurePolicy.h
+++ b/libminifi/include/utils/FailurePolicy.h
@@ -29,23 +29,23 @@ namespace utils {
  */
 enum FailurePolicy {
 
-	/**
-	 * DO NOTHING
-	 */
-	NOTHING,
-	/**
-	 * Return a response code from the executing function
-	 */
-	RETURN,
-	/**
-	 * Throw an exception for flow control.
-	 */
-	EXCEPT,
-	/**
-	 * Exit the program. This should only be used when something
-	 * precludes us from continuing
-	 */
-	EXIT
+  /**
+   * DO NOTHING
+   */
+  NOTHING,
+  /**
+   * Return a response code from the executing function
+   */
+  RETURN,
+  /**
+   * Throw an exception for flow control.
+   */
+  EXCEPT,
+  /**
+   * Exit the program. This should only be used when something
+   * precludes us from continuing
+   */
+  EXIT
 };
 
 } /* namespace utils */
@@ -54,5 +54,4 @@ enum FailurePolicy {
 } /* namespace apache */
 } /* namespace org */
 
-
 #endif /* LIBMINIFI_INCLUDE_UTILS_FAILUREPOLICY_H_ */


[3/4] nifi-minifi-cpp git commit: MINIFI-254: Incremental update for linter changes

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/utils/StringUtils.h
----------------------------------------------------------------------
diff --git a/libminifi/include/utils/StringUtils.h b/libminifi/include/utils/StringUtils.h
index 82459db..bf802e2 100644
--- a/libminifi/include/utils/StringUtils.h
+++ b/libminifi/include/utils/StringUtils.h
@@ -35,96 +35,97 @@ namespace utils {
  * Purpose: Houses many useful string utilities.
  */
 class StringUtils {
-public:
-	/**
-	 * Converts a string to a boolean
-	 * Better handles mixed case.
-	 * @param input input string
-	 * @param output output string.
-	 */
-	static bool StringToBool(std::string input, bool &output) {
-
-		std::transform(input.begin(), input.end(), input.begin(), ::tolower);
-		std::istringstream(input) >> std::boolalpha >> output;
-		return output;
-	}
-
-	// Trim String utils
-
-	/**
-	 * Trims a string left to right
-	 * @param s incoming string
-	 * @returns modified string
-	 */
-	static std::string trim(std::string s) {
-		return trimRight(trimLeft(s));
-	}
-
-	/**
-	 * Trims left most part of a string
-	 * @param s incoming string
-	 * @returns modified string
-	 */
-	static inline std::string trimLeft(std::string s) {
-		s.erase(s.begin(),
-				std::find_if(s.begin(), s.end(),
-						std::not1(
-								std::pointer_to_unary_function<int, int>(
-										std::isspace))));
-		return s;
-	}
-
-	/**
-	 * Trims a string on the right
-	 * @param s incoming string
-	 * @returns modified string
-	 */
-
-	static inline std::string trimRight(std::string s) {
-		s.erase(
-				std::find_if(s.rbegin(), s.rend(),
-						std::not1(
-								std::pointer_to_unary_function<int, int>(
-										std::isspace))).base(), s.end());
-		return s;
-	}
-
-	/**
-	 * Converts a string to a float
-	 * @param input input string
-	 * @param output output float
-	 * @param cp failure policy
-	 */
-	static bool StringToFloat(std::string input, float &output,
-			FailurePolicy cp = RETURN) {
-		try {
-			output = std::stof(input);
-		} catch (const std::invalid_argument &ie) {
-			switch (cp) {
-			case RETURN:
-			case NOTHING:
-				return false;
-			case EXIT:
-				exit(1);
-			case EXCEPT:
-				throw ie;
-			}
-		} catch (const std::out_of_range &ofr) {
-			switch (cp) {
-			case RETURN:
-			case NOTHING:
-				return false;
-			case EXIT:
-				exit(1);
-			case EXCEPT:
-				throw ofr;
-
-			}
-		}
-
-		return true;
-
-	}
+ public:
+  /**
+   * Converts a string to a boolean
+   * Better handles mixed case.
+   * @param input input string
+   * @param output output string.
+   */
+  static bool StringToBool(std::string input, bool &output) {
+
+    std::transform(input.begin(), input.end(), input.begin(), ::tolower);
+    std::istringstream(input) >> std::boolalpha >> output;
+    return output;
+  }
+
+  // Trim String utils
+
+  /**
+   * Trims a string left to right
+   * @param s incoming string
+   * @returns modified string
+   */
+  static std::string trim(std::string s) {
+    return trimRight(trimLeft(s));
+  }
+
+  /**
+   * Trims left most part of a string
+   * @param s incoming string
+   * @returns modified string
+   */
+  static inline std::string trimLeft(std::string s) {
+    s.erase(
+        s.begin(),
+        std::find_if(
+            s.begin(), s.end(),
+            std::not1(std::pointer_to_unary_function<int, int>(std::isspace))));
+    return s;
+  }
+
+  /**
+   * Trims a string on the right
+   * @param s incoming string
+   * @returns modified string
+   */
+
+  static inline std::string trimRight(std::string s) {
+    s.erase(
+        std::find_if(
+            s.rbegin(), s.rend(),
+            std::not1(std::pointer_to_unary_function<int, int>(std::isspace)))
+            .base(),
+        s.end());
+    return s;
+  }
+
+  /**
+   * Converts a string to a float
+   * @param input input string
+   * @param output output float
+   * @param cp failure policy
+   */
+  static bool StringToFloat(std::string input, float &output, FailurePolicy cp =
+                                RETURN) {
+    try {
+      output = std::stof(input);
+    } catch (const std::invalid_argument &ie) {
+      switch (cp) {
+        case RETURN:
+        case NOTHING:
+          return false;
+        case EXIT:
+          exit(1);
+        case EXCEPT:
+          throw ie;
+      }
+    } catch (const std::out_of_range &ofr) {
+      switch (cp) {
+        case RETURN:
+        case NOTHING:
+          return false;
+        case EXIT:
+          exit(1);
+        case EXCEPT:
+          throw ofr;
+
+      }
+    }
+
+    return true;
+
+  }
 
 };
 
@@ -134,5 +135,4 @@ public:
 } /* namespace apache */
 } /* namespace org */
 
-
 #endif /* LIBMINIFI_INCLUDE_IO_STRINGUTILS_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/include/utils/TimeUtil.h
----------------------------------------------------------------------
diff --git a/libminifi/include/utils/TimeUtil.h b/libminifi/include/utils/TimeUtil.h
index 76b334b..6805419 100644
--- a/libminifi/include/utils/TimeUtil.h
+++ b/libminifi/include/utils/TimeUtil.h
@@ -33,8 +33,8 @@
  * @returns milliseconds since epoch
  */
 inline uint64_t getTimeMillis() {
-	return std::chrono::duration_cast<std::chrono::milliseconds>(
-			std::chrono::system_clock::now().time_since_epoch()).count();
+  return std::chrono::duration_cast<std::chrono::milliseconds>(
+      std::chrono::system_clock::now().time_since_epoch()).count();
 }
 
 /**
@@ -43,8 +43,8 @@ inline uint64_t getTimeMillis() {
  */
 inline uint64_t getTimeNano() {
 
-	return std::chrono::duration_cast<std::chrono::nanoseconds>(
-			std::chrono::system_clock::now().time_since_epoch()).count();
+  return std::chrono::duration_cast<std::chrono::nanoseconds>(
+      std::chrono::system_clock::now().time_since_epoch()).count();
 
 }
 
@@ -54,21 +54,19 @@ inline uint64_t getTimeNano() {
  * @param msec milliseconds since epoch
  * @returns string representing the time
  */
-inline std::string getTimeStr(uint64_t msec, bool enforce_locale = false)
-{
-	char date[120];
-	time_t second = (time_t) (msec/1000);
-	msec = msec % 1000;
-	strftime(date, sizeof(date) / sizeof(*date), TIME_FORMAT,
-	             ( enforce_locale==true ? gmtime(&second) : localtime(&second)));
+inline std::string getTimeStr(uint64_t msec, bool enforce_locale = false) {
+  char date[120];
+  time_t second = (time_t) (msec / 1000);
+  msec = msec % 1000;
+  strftime(date, sizeof(date) / sizeof(*date), TIME_FORMAT,
+           (enforce_locale == true ? gmtime(&second) : localtime(&second)));
 
-	std::string ret = date;
-	date[0] = '\0';
-	sprintf(date, ".%03llu", (unsigned long long) msec);
+  std::string ret = date;
+  date[0] = '\0';
+  sprintf(date, ".%03llu", (unsigned long long) msec);
 
-	ret += date;
-	return ret;
+  ret += date;
+  return ret;
 }
 
-
 #endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/Configure.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/Configure.cpp b/libminifi/src/Configure.cpp
index 96ed7c7..f70686d 100644
--- a/libminifi/src/Configure.cpp
+++ b/libminifi/src/Configure.cpp
@@ -16,8 +16,9 @@
  * limitations under the License.
  */
 #include "properties/Configure.h"
+#include <string>
 #include "utils/StringUtils.h"
-#include "core/core.h"
+#include "core/Core.h"
 
 namespace org {
 namespace apache {
@@ -25,147 +26,149 @@ 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_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.flowcontroller.graceful.shutdown.period";
+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_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_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_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_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_remote_input_secure = "nifi.remote.input.secure";
-const char *Configure::nifi_security_need_ClientAuth = "nifi.security.need.ClientAuth";
-const char *Configure::nifi_security_client_certificate = "nifi.security.client.certificate";
-const char *Configure::nifi_security_client_private_key = "nifi.security.client.private.key";
-const char *Configure::nifi_security_client_pass_phrase = "nifi.security.client.pass.phrase";
-const char *Configure::nifi_security_client_ca_certificate = "nifi.security.client.ca.certificate";
+const char *Configure::nifi_security_need_ClientAuth =
+    "nifi.security.need.ClientAuth";
+const char *Configure::nifi_security_client_certificate =
+    "nifi.security.client.certificate";
+const char *Configure::nifi_security_client_private_key =
+    "nifi.security.client.private.key";
+const char *Configure::nifi_security_client_pass_phrase =
+    "nifi.security.client.pass.phrase";
+const char *Configure::nifi_security_client_ca_certificate =
+    "nifi.security.client.ca.certificate";
+
+#define BUFFER_SIZE 512
 
 // Get the config value
-bool Configure::get(std::string key, std::string &value)
-{
-	std::lock_guard<std::mutex> lock(mutex_);
-	auto it = properties_.find(key);
-
-	if (it != properties_.end())
-	{
-		value = it->second;
-		return true;
-	}
-	else
-	{
-		return false;
-	}
+bool Configure::get(std::string key, std::string &value) {
+  std::lock_guard<std::mutex> lock(mutex_);
+  auto it = properties_.find(key);
+
+  if (it != properties_.end()) {
+    value = it->second;
+    return true;
+  } else {
+    return false;
+  }
 }
 
-
 // Parse one line in configure file like key=value
-void Configure::parseConfigureFileLine(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 = org::apache::nifi::minifi::utils::StringUtils::trimRight(key);
-    value = org::apache::nifi::minifi::utils::StringUtils::trimRight(value);
-    set(key, value);
+void Configure::parseConfigureFileLine(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 = org::apache::nifi::minifi::utils::StringUtils::trimRight(key);
+  value = org::apache::nifi::minifi::utils::StringUtils::trimRight(value);
+  set(key, value);
 }
 
 // Load Configure File
-void Configure::loadConfigureFile(const char *fileName)
-{
-
-    std::string adjustedFilename;
-    if (fileName)
-    {
-        // perform a naive determination if this is a relative path
-        if (fileName[0] != '/')
-        {
-            adjustedFilename = adjustedFilename + configure_->getHome() + "/" + fileName;
-        }
-        else
-        {
-            adjustedFilename += fileName;
-        }
-    }
-    char *path = NULL;
-    char full_path[PATH_MAX];
-    path = realpath(adjustedFilename.c_str(), full_path);
-    logger_->log_info("Using configuration file located at %s", path);
-
-    std::ifstream file(path, std::ifstream::in);
-    if (!file.good())
-    {
-        logger_->log_error("load configure file failed %s", path);
-        return;
-    }
-    this->clear();
-    const unsigned int bufSize = 512;
-    char buf[bufSize];
-    for (file.getline(buf, bufSize); file.good(); file.getline(buf, bufSize))
-    {
-        parseConfigureFileLine(buf);
+void Configure::loadConfigureFile(const char *fileName) {
+  std::string adjustedFilename;
+  if (fileName) {
+    // perform a naive determination if this is a relative path
+    if (fileName[0] != '/') {
+      adjustedFilename = adjustedFilename + configure_->getHome() + "/"
+          + fileName;
+    } else {
+      adjustedFilename += fileName;
     }
+  }
+  char *path = NULL;
+  char full_path[PATH_MAX];
+  path = realpath(adjustedFilename.c_str(), full_path);
+  logger_->log_info("Using configuration file located at %s", path);
+
+  std::ifstream file(path, std::ifstream::in);
+  if (!file.good()) {
+    logger_->log_error("load configure file failed %s", path);
+    return;
+  }
+  this->clear();
+
+  char buf[BUFFER_SIZE];
+  for (file.getline(buf, BUFFER_SIZE); file.good();
+      file.getline(buf, BUFFER_SIZE)) {
+    parseConfigureFileLine(buf);
+  }
 }
 
 // Parse Command Line
-void Configure::parseCommandLine(int argc, char **argv)
-{
-	int i;
-	bool keyFound = false;
-	std::string key, value;
-
-	for (i = 1; i < argc; i++)
-	{
-		if (argv[i][0] == '-' && argv[i][1] != '\0')
-		{
-			keyFound = true;
-			key = &argv[i][1];
-			continue;
-		}
-		if (keyFound)
-		{
-			value = argv[i];
-			set(key,value);
-			keyFound = false;
-		}
-	}
-	return;
+void Configure::parseCommandLine(int argc, char **argv) {
+  int i;
+  bool keyFound = false;
+  std::string key, value;
+
+  for (i = 1; i < argc; i++) {
+    if (argv[i][0] == '-' && argv[i][1] != '\0') {
+      keyFound = true;
+      key = &argv[i][1];
+      continue;
+    }
+    if (keyFound) {
+      value = argv[i];
+      set(key, value);
+      keyFound = false;
+    }
+  }
+  return;
 }
 
 } /* namespace minifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/Connection.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/Connection.cpp b/libminifi/src/Connection.cpp
index 6f64ff3..ab300ba 100644
--- a/libminifi/src/Connection.cpp
+++ b/libminifi/src/Connection.cpp
@@ -17,16 +17,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+#include "Connection.h"
+#include <sys/time.h>
+#include <time.h>
 #include <vector>
 #include <queue>
+#include <memory>
+#include <string>
 #include <map>
 #include <set>
-#include <sys/time.h>
-#include <time.h>
 #include <chrono>
 #include <thread>
 #include <iostream>
-
 #include "core/FlowFile.h"
 #include "Connection.h"
 #include "core/Processor.h"
@@ -95,7 +97,7 @@ void Connection::put(std::shared_ptr<core::FlowFile> flow) {
 
   if (!flow->isStored()) {
     // Save to the flowfile repo
-    FlowFileRecord event(flow_repository_,flow,this->uuidStr_);
+    FlowFileRecord event(flow_repository_, flow, this->uuidStr_);
     if (event.Serialize()) {
       flow->setStoredToRepository(true);
     }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/EventDrivenSchedulingAgent.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/EventDrivenSchedulingAgent.cpp b/libminifi/src/EventDrivenSchedulingAgent.cpp
index 0484139..cbb60ea 100644
--- a/libminifi/src/EventDrivenSchedulingAgent.cpp
+++ b/libminifi/src/EventDrivenSchedulingAgent.cpp
@@ -17,10 +17,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+#include "EventDrivenSchedulingAgent.h"
 #include <chrono>
+#include <memory>
 #include <thread>
 #include <iostream>
-#include "EventDrivenSchedulingAgent.h"
 #include "core/Processor.h"
 #include "core/ProcessContext.h"
 #include "core/ProcessSessionFactory.h"
@@ -31,7 +32,6 @@ namespace apache {
 namespace nifi {
 namespace minifi {
 
-
 void EventDrivenSchedulingAgent::run(
     std::shared_ptr<core::Processor> processor,
     core::ProcessContext *processContext,

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/FlowControlProtocol.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/FlowControlProtocol.cpp b/libminifi/src/FlowControlProtocol.cpp
index 50fc0e2..69f482f 100644
--- a/libminifi/src/FlowControlProtocol.cpp
+++ b/libminifi/src/FlowControlProtocol.cpp
@@ -17,17 +17,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+#include "FlowControlProtocol.h"
 #include <sys/time.h>
 #include <stdio.h>
 #include <time.h>
+#include <netinet/tcp.h>
 #include <chrono>
 #include <thread>
+#include <string>
 #include <random>
-#include <netinet/tcp.h>
 #include <iostream>
 #include "FlowController.h"
-#include "FlowControlProtocol.h"
-#include "core/core.h"
+#include "core/Core.h"
 namespace org {
 namespace apache {
 namespace nifi {
@@ -45,7 +46,7 @@ int FlowControlProtocol::connectServer(const char *host, uint16_t port) {
   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);
+  memcpy(reinterpret_cast<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);
@@ -56,30 +57,26 @@ int FlowControlProtocol::connectServer(const char *host, uint16_t port) {
   int opt = 1;
   bool nagle_off = true;
 
-  if (nagle_off)
-  {
-    if (setsockopt(sock, SOL_TCP, TCP_NODELAY, (void *)&opt, sizeof(opt)) < 0)
-    {
+  if (nagle_off) {
+    if (setsockopt(sock, SOL_TCP, TCP_NODELAY, reinterpret_cast<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;
-    }
-  }
+            reinterpret_cast<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 sndsize = 256*1024;
+      if (setsockopt(sock, SOL_SOCKET, SO_SNDBUF, reinterpret_cast<char*>(&sndsize), sizeof(sndsize)) < 0) {
+        logger_->log_error("setsockopt() SO_SNDBUF failed");
+        close(sock);
+        return 0;
+      }
 #endif
 
   struct sockaddr_in sa;
@@ -123,7 +120,7 @@ int FlowControlProtocol::sendData(uint8_t *buf, int buflen) {
 
   while (bytes < buflen) {
     ret = send(_socket, buf + bytes, buflen - bytes, 0);
-    //check for errors
+    // check for errors
     if (ret == -1) {
       return ret;
     }
@@ -232,8 +229,9 @@ void FlowControlProtocol::run(FlowControlProtocol *protocol) {
     if (!protocol->_registered) {
       // if it is not register yet
       protocol->sendRegisterReq();
-    } else
+    } else {
       protocol->sendReportReq();
+    }
   }
   return;
 }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/FlowController.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/FlowController.cpp b/libminifi/src/FlowController.cpp
index 433387a..1a163ea 100644
--- a/libminifi/src/FlowController.cpp
+++ b/libminifi/src/FlowController.cpp
@@ -17,23 +17,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-#include <vector>
-#include <queue>
-#include <map>
-#include <set>
+#include "FlowController.h"
 #include <sys/time.h>
 #include <time.h>
-#include <chrono>
-#include <thread>
 #include <sys/types.h>
 #include <sys/stat.h>
 #include <unistd.h>
+#include <vector>
+#include <queue>
+#include <map>
+#include <set>
+#include <chrono>
 #include <future>
-#include "FlowController.h"
+#include <thread>
+#include <utility>
+#include <memory>
+#include <string>
 #include "core/ProcessContext.h"
 #include "core/ProcessGroup.h"
 #include "utils/StringUtils.h"
-#include "core/core.h"
+#include "core/Core.h"
 #include "core/repository/FlowFileRepository.h"
 
 namespace org {
@@ -105,7 +108,6 @@ FlowController::FlowController(
 
     initializePaths(adjustedFilename);
   }
-
 }
 
 void FlowController::initializePaths(const std::string &adjustedFilename) {
@@ -125,12 +127,12 @@ void FlowController::initializePaths(const std::string &adjustedFilename) {
   // Create the content repo directory if needed
   struct stat contentDirStat;
 
-  if (stat(ResourceClaim::default_directory_path.c_str(), &contentDirStat)
+  if (stat(ResourceClaim::default_directory_path, &contentDirStat)
       != -1&& S_ISDIR(contentDirStat.st_mode)) {
-    path = realpath(ResourceClaim::default_directory_path.c_str(), full_path);
+    path = realpath(ResourceClaim::default_directory_path, full_path);
     logger_->log_info("FlowController content directory %s", full_path);
   } else {
-    if (mkdir(ResourceClaim::default_directory_path.c_str(), 0777) == -1) {
+    if (mkdir(ResourceClaim::default_directory_path, 0777) == -1) {
       logger_->log_error("FlowController content directory creation failed");
       exit(1);
     }
@@ -144,7 +146,6 @@ void FlowController::initializePaths(const std::string &adjustedFilename) {
         full_path);
     exit(1);
   }
-
 }
 
 FlowController::~FlowController() {
@@ -154,7 +155,6 @@ FlowController::~FlowController() {
     delete protocol_;
   flow_file_repo_ = nullptr;
   provenance_repo_ = nullptr;
-
 }
 
 void FlowController::stop(bool force) {
@@ -173,7 +173,6 @@ void FlowController::stop(bool force) {
     if (this->root_)
       this->root_->stopProcessing(&this->_timerScheduler,
                                   &this->_eventScheduler);
-
   }
 }
 
@@ -200,7 +199,6 @@ void FlowController::waitUnload(const uint64_t timeToWaitMs) {
     if (std::future_status::ready == unload_task.wait_until(wait_time)) {
       running_ = false;
     }
-
   }
 }
 
@@ -278,7 +276,6 @@ bool FlowController::start() {
         "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();

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/FlowFileRecord.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/FlowFileRecord.cpp b/libminifi/src/FlowFileRecord.cpp
index 562a685..de682b0 100644
--- a/libminifi/src/FlowFileRecord.cpp
+++ b/libminifi/src/FlowFileRecord.cpp
@@ -17,16 +17,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+#include "FlowFileRecord.h"
+#include <sys/time.h>
+#include <time.h>
+#include <cstdio>
 #include <vector>
 #include <queue>
 #include <map>
-#include <sys/time.h>
-#include <time.h>
+#include <memory>
+#include <string>
 #include <iostream>
 #include <fstream>
-#include <cstdio>
-
-#include "FlowFileRecord.h"
 #include "core/logging/Logger.h"
 #include "core/Relationship.h"
 #include "core/Repository.h"
@@ -73,18 +74,18 @@ FlowFileRecord::FlowFileRecord(
     : 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();
-	}
+  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();
+  }
 }
 
 FlowFileRecord::FlowFileRecord(
@@ -94,7 +95,6 @@ FlowFileRecord::FlowFileRecord(
       uuid_connection_(""),
       snapshot_(""),
       flow_repository_(flow_repository) {
-
 }
 
 FlowFileRecord::~FlowFileRecord() {
@@ -175,10 +175,10 @@ bool FlowFileRecord::DeSerialize(std::string key) {
     logger_->log_error("NiFi FlowFile Store event %s can not found",
                        key.c_str());
     return false;
-  } else
+  } 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);
@@ -197,14 +197,12 @@ bool FlowFileRecord::DeSerialize(std::string key) {
 }
 
 bool FlowFileRecord::Serialize() {
-
   io::DataStream outStream;
 
   int ret;
 
   ret = write(this->event_time_, &outStream);
   if (ret != 8) {
-
     return false;
   }
 
@@ -215,57 +213,48 @@ bool FlowFileRecord::Serialize() {
 
   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;
   }
 
@@ -281,13 +270,10 @@ bool FlowFileRecord::Serialize() {
     return false;
   }
 
-  // cleanup
-
   return true;
 }
 
 bool FlowFileRecord::DeSerialize(const uint8_t *buffer, const int bufferSize) {
-
   int ret;
 
   io::DataStream outStream(buffer, bufferSize);

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/RemoteProcessorGroupPort.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/RemoteProcessorGroupPort.cpp b/libminifi/src/RemoteProcessorGroupPort.cpp
index 9790256..33f0cb2 100644
--- a/libminifi/src/RemoteProcessorGroupPort.cpp
+++ b/libminifi/src/RemoteProcessorGroupPort.cpp
@@ -17,18 +17,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+#include "RemoteProcessorGroupPort.h"
+#include <sys/time.h>
+#include <string.h>
+#include <time.h>
 #include <vector>
 #include <queue>
 #include <map>
 #include <set>
-#include <sys/time.h>
-#include <time.h>
+#include <string>
+#include <utility>
+#include <memory>
 #include <sstream>
-#include <string.h>
 #include <iostream>
-
-#include "RemoteProcessorGroupPort.h"
-
 #include "../include/io/StreamFactory.h"
 #include "io/ClientSocket.h"
 #include "utils/TimeUtil.h"
@@ -48,13 +49,13 @@ core::Property RemoteProcessorGroupPort::hostName("Host Name",
 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());
+  std::unique_ptr<Site2SiteClientProtocol> return_pointer = std::move(
+      available_protocols_.top());
   available_protocols_.pop();
   return std::move(return_pointer);
 }
@@ -66,7 +67,6 @@ void RemoteProcessorGroupPort::returnProtocol(
 }
 
 void RemoteProcessorGroupPort::initialize() {
-
   // Set the supported properties
   std::set<core::Property> properties;
   properties.insert(hostName);
@@ -76,7 +76,6 @@ void RemoteProcessorGroupPort::initialize() {
   std::set<core::Relationship> relationships;
   relationships.insert(relation);
   setSupportedRelationships(relationships);
-
 }
 
 void RemoteProcessorGroupPort::onTrigger(core::ProcessContext *context,
@@ -90,7 +89,6 @@ void RemoteProcessorGroupPort::onTrigger(core::ProcessContext *context,
 
   // Peer Connection
   if (protocol_ == nullptr) {
-
     protocol_ = std::unique_ptr<Site2SiteClientProtocol>(
         new Site2SiteClientProtocol(0));
     protocol_->setPortId(protocol_uuid_);

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/ResourceClaim.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/ResourceClaim.cpp b/libminifi/src/ResourceClaim.cpp
index 826ca1d..cbe7712 100644
--- a/libminifi/src/ResourceClaim.cpp
+++ b/libminifi/src/ResourceClaim.cpp
@@ -20,6 +20,7 @@
 #include <vector>
 #include <queue>
 #include <map>
+#include <string>
 
 #include "ResourceClaim.h"
 
@@ -30,7 +31,7 @@ namespace minifi {
 
 std::atomic<uint64_t> ResourceClaim::_localResourceClaimNumber(0);
 
-std::string ResourceClaim::default_directory_path = DEFAULT_CONTENT_DIRECTORY;
+char *ResourceClaim::default_directory_path = const_cast<char*>(DEFAULT_CONTENT_DIRECTORY);
 
 ResourceClaim::ResourceClaim(const std::string contentDirectory)
     : _id(_localResourceClaimNumber.load()),

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/SchedulingAgent.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/SchedulingAgent.cpp b/libminifi/src/SchedulingAgent.cpp
index 8cb88e0..d69ba00 100644
--- a/libminifi/src/SchedulingAgent.cpp
+++ b/libminifi/src/SchedulingAgent.cpp
@@ -17,11 +17,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+#include "SchedulingAgent.h"
 #include <chrono>
 #include <thread>
+#include <memory>
 #include <iostream>
 #include "Exception.h"
-#include "SchedulingAgent.h"
 #include "core/Processor.h"
 
 namespace org {
@@ -29,8 +30,7 @@ namespace apache {
 namespace nifi {
 namespace minifi {
 
-bool SchedulingAgent::hasWorkToDo(
-    std::shared_ptr<core::Processor> processor) {
+bool SchedulingAgent::hasWorkToDo(std::shared_ptr<core::Processor> processor) {
   // Whether it has work to do
   if (processor->getTriggerWhenEmpty() || !processor->hasIncomingConnections()
       || processor->flowFilesQueued())
@@ -44,10 +44,9 @@ bool SchedulingAgent::hasTooMuchOutGoing(
   return processor->flowFilesOutGoingFull();
 }
 
-bool SchedulingAgent::onTrigger(
-    std::shared_ptr<core::Processor> processor,
-    core::ProcessContext *processContext,
-    core::ProcessSessionFactory *sessionFactory) {
+bool SchedulingAgent::onTrigger(std::shared_ptr<core::Processor> processor,
+                                core::ProcessContext *processContext,
+                                core::ProcessSessionFactory *sessionFactory) {
   if (processor->isYield())
     return false;
 
@@ -62,8 +61,6 @@ bool SchedulingAgent::onTrigger(
     // need to apply backpressure
     return true;
 
-  //TODO runDuration
-
   processor->incrementActiveTasks();
   try {
     processor->onTrigger(processContext, sessionFactory);
@@ -85,7 +82,6 @@ bool SchedulingAgent::onTrigger(
   return false;
 }
 
-
 } /* namespace minifi */
 } /* namespace nifi */
 } /* namespace apache */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/Site2SiteClientProtocol.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/Site2SiteClientProtocol.cpp b/libminifi/src/Site2SiteClientProtocol.cpp
index e0265bb..52a0a02 100644
--- a/libminifi/src/Site2SiteClientProtocol.cpp
+++ b/libminifi/src/Site2SiteClientProtocol.cpp
@@ -21,9 +21,11 @@
 #include <stdio.h>
 #include <time.h>
 #include <chrono>
+#include <map>
+#include <string>
+#include <memory>
 #include <thread>
 #include <random>
-#include <netinet/tcp.h>
 #include <iostream>
 #include "io/CRCStream.h"
 #include "Site2SitePeer.h"
@@ -356,7 +358,7 @@ int Site2SiteClientProtocol::readRequestType(RequestType &type) {
   if (ret <= 0)
     return ret;
 
-  for (int i = (int) NEGOTIATE_FLOWFILE_CODEC; i <= (int) SHUTDOWN; i++) {
+  for (int i = NEGOTIATE_FLOWFILE_CODEC; i <= SHUTDOWN; i++) {
     if (RequestTypeStr[i] == requestTypeStr) {
       type = (RequestType) i;
       return ret;
@@ -426,12 +428,14 @@ int Site2SiteClientProtocol::writeRespond(RespondCode code,
 
   if (resCode->hasDescription) {
     ret = peer_->writeUTF(message);
-    if (ret > 0)
+    if (ret > 0) {
       return (3 + ret);
-    else
+    } else {
       return ret;
-  } else
+    }
+  } else {
     return 3;
+  }
 }
 
 bool Site2SiteClientProtocol::negotiateCodec() {
@@ -518,7 +522,8 @@ Transaction* Site2SiteClientProtocol::createTransaction(
       return NULL;
     }
 
-    org::apache::nifi::minifi::io::CRCStream<Site2SitePeer> crcstream(peer_.get());
+    org::apache::nifi::minifi::io::CRCStream<Site2SitePeer> crcstream(
+        peer_.get());
     switch (code) {
       case MORE_DATA:
         dataAvailable = true;
@@ -553,7 +558,8 @@ Transaction* Site2SiteClientProtocol::createTransaction(
       // tearDown();
       return NULL;
     } else {
-      org::apache::nifi::minifi::io::CRCStream<Site2SitePeer> crcstream(peer_.get());
+      org::apache::nifi::minifi::io::CRCStream<Site2SitePeer> crcstream(
+          peer_.get());
       transaction = new Transaction(direction, crcstream);
       _transactionMap[transaction->getUUIDStr()] = transaction;
       transactionID = transaction->getUUIDStr();
@@ -680,9 +686,10 @@ bool Site2SiteClientProtocol::receive(std::string transactionID,
   return true;
 }
 
-bool Site2SiteClientProtocol::send(
-    std::string transactionID, DataPacket *packet, std::shared_ptr<FlowFileRecord> flowFile,
-    core::ProcessSession *session) {
+bool Site2SiteClientProtocol::send(std::string transactionID,
+                                   DataPacket *packet,
+                                   std::shared_ptr<FlowFileRecord> flowFile,
+                                   core::ProcessSession *session) {
   int ret;
   Transaction *transaction = NULL;
 
@@ -772,9 +779,8 @@ bool Site2SiteClientProtocol::send(
   return true;
 }
 
-void Site2SiteClientProtocol::receiveFlowFiles(
-    core::ProcessContext *context,
-    core::ProcessSession *session) {
+void Site2SiteClientProtocol::receiveFlowFiles(core::ProcessContext *context,
+                                               core::ProcessSession *session) {
   uint64_t bytes = 0;
   int transfers = 0;
   Transaction *transaction = NULL;
@@ -817,7 +823,9 @@ void Site2SiteClientProtocol::receiveFlowFiles(
         // transaction done
         break;
       }
-      std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast<FlowFileRecord>(session->create());;
+      std::shared_ptr<FlowFileRecord> flowFile = std::static_pointer_cast<
+          FlowFileRecord>(session->create());
+
       if (!flowFile) {
         throw Exception(SITE2SITE_EXCEPTION, "Flow File Creation Failed");
         return;
@@ -930,7 +938,7 @@ bool Site2SiteClientProtocol::confirm(std::string transactionID) {
     // 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();
+    int64_t 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());
@@ -978,7 +986,7 @@ bool Site2SiteClientProtocol::confirm(std::string transactionID) {
           "Site2Site transaction %s peer confirm transaction with CRC %s",
           transactionID.c_str(), message.c_str());
       if (this->_currentVersion > 3) {
-        long crcValue = transaction->getCRC();
+        int64_t crcValue = transaction->getCRC();
         std::string crc = std::to_string(crcValue);
         if (message == crc) {
           logger_->log_info("Site2Site transaction %s CRC matched",
@@ -1113,9 +1121,9 @@ bool Site2SiteClientProtocol::complete(std::string transactionID) {
       logger_->log_info("Site2Site transaction %s send finished",
                         transactionID.c_str());
       ret = this->writeRespond(TRANSACTION_FINISHED, "Finished");
-      if (ret <= 0)
+      if (ret <= 0) {
         return false;
-      else {
+      } else {
         transaction->_state = TRANSACTION_COMPLETED;
         return true;
       }
@@ -1143,10 +1151,11 @@ bool Site2SiteClientProtocol::complete(std::string transactionID) {
   }
 }
 
-void Site2SiteClientProtocol::transferFlowFiles(
-    core::ProcessContext *context,
-    core::ProcessSession *session) {
-  std::shared_ptr<FlowFileRecord> flow = std::static_pointer_cast<FlowFileRecord>(session->get());;
+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)
@@ -1201,7 +1210,8 @@ void Site2SiteClientProtocol::transferFlowFiles(
       if (transferNanos > _batchSendNanos)
         break;
 
-      flow = std::static_pointer_cast<FlowFileRecord>(session->get());;
+      flow = std::static_pointer_cast<FlowFileRecord>(session->get());
+
       if (!flow) {
         continueTransaction = false;
       }
@@ -1240,7 +1250,6 @@ void Site2SiteClientProtocol::transferFlowFiles(
   return;
 }
 
-
 } /* namespace minifi */
 } /* namespace nifi */
 } /* namespace apache */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/Site2SitePeer.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/Site2SitePeer.cpp b/libminifi/src/Site2SitePeer.cpp
index 64732ac..551d466 100644
--- a/libminifi/src/Site2SitePeer.cpp
+++ b/libminifi/src/Site2SitePeer.cpp
@@ -17,6 +17,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+#include "Site2SitePeer.h"
 #include <sys/time.h>
 #include <stdio.h>
 #include <time.h>
@@ -24,43 +25,39 @@
 #include <thread>
 #include <random>
 #include <memory>
-#include <netinet/tcp.h>
 #include <iostream>
 #include "io/ClientSocket.h"
 #include "io/validation.h"
-#include "Site2SitePeer.h"
 #include "FlowController.h"
 
-
 namespace org {
 namespace apache {
 namespace nifi {
 namespace minifi {
 
 bool Site2SitePeer::Open() {
+  if (IsNullOrEmpty(host_))
+    return false;
 
-	if (IsNullOrEmpty (host_))
-		return false;
+  if (stream_->initialize() < 0)
+    return false;
 
-	if (stream_->initialize() < 0)
-		return false;
+  uint16_t data_size = sizeof MAGIC_BYTES;
 
-	uint16_t data_size = sizeof MAGIC_BYTES;
+  if (stream_->writeData(
+      reinterpret_cast<uint8_t *>(const_cast<char*>(MAGIC_BYTES)), data_size)
+      != data_size) {
+    return false;
+  }
 
-	if (stream_->writeData((uint8_t *) MAGIC_BYTES, data_size) != data_size) {
-		return false;
-	}
-
-	return true;
+  return true;
 }
 
 void Site2SitePeer::Close() {
-    if (stream_ != nullptr)
-	stream_->closeStream();
+  if (stream_ != nullptr)
+    stream_->closeStream();
 }
 
-
-
 } /* namespace minifi */
 } /* namespace nifi */
 } /* namespace apache */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/ThreadedSchedulingAgent.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/ThreadedSchedulingAgent.cpp b/libminifi/src/ThreadedSchedulingAgent.cpp
index 6c04281..65e7531 100644
--- a/libminifi/src/ThreadedSchedulingAgent.cpp
+++ b/libminifi/src/ThreadedSchedulingAgent.cpp
@@ -17,11 +17,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+#include "ThreadedSchedulingAgent.h"
+#include <memory>
+#include <string>
+#include <vector>
+#include <map>
 #include <thread>
 #include <iostream>
-
-#include "ThreadedSchedulingAgent.h"
-
 #include "core/Connectable.h"
 #include "core/ProcessorNode.h"
 #include "core/ProcessContext.h"
@@ -35,7 +37,7 @@ namespace minifi {
 
 void ThreadedSchedulingAgent::schedule(
     std::shared_ptr<core::Processor> processor) {
-  std::lock_guard < std::mutex > lock(mutex_);
+  std::lock_guard<std::mutex> lock(mutex_);
 
   _administrativeYieldDuration = 0;
   std::string yieldValue;
@@ -43,10 +45,11 @@ void ThreadedSchedulingAgent::schedule(
   if (configure_->get(Configure::nifi_administrative_yield_duration,
                       yieldValue)) {
     core::TimeUnit unit;
-    if (core::Property::StringToTime(
-        yieldValue, _administrativeYieldDuration, unit)
-        && core::Property::ConvertTimeUnitToMS(
-            _administrativeYieldDuration, unit, _administrativeYieldDuration)) {
+    if (core::Property::StringToTime(yieldValue, _administrativeYieldDuration,
+                                     unit)
+        && core::Property::ConvertTimeUnitToMS(_administrativeYieldDuration,
+                                               unit,
+                                               _administrativeYieldDuration)) {
       logger_->log_debug("nifi_administrative_yield_duration: [%d] ms",
                          _administrativeYieldDuration);
     }
@@ -55,10 +58,9 @@ void ThreadedSchedulingAgent::schedule(
   _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)) {
+    if (core::Property::StringToTime(yieldValue, _boredYieldDuration, unit)
+        && core::Property::ConvertTimeUnitToMS(_boredYieldDuration, unit,
+                                               _boredYieldDuration)) {
       logger_->log_debug("nifi_bored_yield_duration: [%d] ms",
                          _boredYieldDuration);
     }
@@ -80,11 +82,10 @@ void ThreadedSchedulingAgent::schedule(
   }
 
   core::ProcessorNode processor_node(processor);
-  auto processContext = std::make_shared
-      < core::ProcessContext > (processor_node,repo_);
-  auto sessionFactory = std::make_shared
-      < core::ProcessSessionFactory
-      > (processContext.get());
+  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());
 
@@ -105,9 +106,9 @@ void ThreadedSchedulingAgent::schedule(
   return;
 }
 
-void ThreadedSchedulingAgent::unschedule(std::shared_ptr<core::Processor> processor) {
-  std::lock_guard < std::mutex > lock(mutex_);
-
+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());
@@ -139,7 +140,6 @@ void ThreadedSchedulingAgent::unschedule(std::shared_ptr<core::Processor> proces
   processor->clearActiveTask();
 }
 
-
 } /* namespace minifi */
 } /* namespace nifi */
 } /* namespace apache */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/TimerDrivenSchedulingAgent.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/TimerDrivenSchedulingAgent.cpp b/libminifi/src/TimerDrivenSchedulingAgent.cpp
index 3895e81..8d10658 100644
--- a/libminifi/src/TimerDrivenSchedulingAgent.cpp
+++ b/libminifi/src/TimerDrivenSchedulingAgent.cpp
@@ -17,10 +17,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+#include "TimerDrivenSchedulingAgent.h"
 #include <chrono>
 #include <thread>
+#include <memory>
 #include <iostream>
-#include "TimerDrivenSchedulingAgent.h"
 #include "core/Property.h"
 
 namespace org {
@@ -35,7 +36,6 @@ void TimerDrivenSchedulingAgent::run(
   while (this->running_) {
     bool shouldYield = this->onTrigger(processor, processContext,
                                        sessionFactory);
-
     if (processor->isYield()) {
       // Honor the yield
       std::this_thread::sleep_for(

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/core/ConfigurableComponent.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/ConfigurableComponent.cpp b/libminifi/src/core/ConfigurableComponent.cpp
index 67a43dd..fa5ff7d 100644
--- a/libminifi/src/core/ConfigurableComponent.cpp
+++ b/libminifi/src/core/ConfigurableComponent.cpp
@@ -17,7 +17,10 @@
  */
 
 #include "core/ConfigurableComponent.h"
-
+#include <memory>
+#include <utility>
+#include <string>
+#include <set>
 #include "core/Property.h"
 #include "core/logging/Logger.h"
 
@@ -27,19 +30,17 @@ namespace nifi {
 namespace minifi {
 namespace core {
 
-ConfigurableComponent::ConfigurableComponent(std::shared_ptr<logging::Logger> logger)
+ConfigurableComponent::ConfigurableComponent(
+    std::shared_ptr<logging::Logger> logger)
     : my_logger_(logger) {
-
 }
 
 ConfigurableComponent::ConfigurableComponent(
     const ConfigurableComponent &&other)
     : properties_(std::move(other.properties_)),
       my_logger_(std::move(other.my_logger_)) {
-
 }
 ConfigurableComponent::~ConfigurableComponent() {
-
 }
 
 /**
@@ -58,7 +59,7 @@ bool ConfigurableComponent::getProperty(const std::string name,
     Property item = it->second;
     value = item.getValue();
     my_logger_->log_info("Processor %s property name %s value %s", name.c_str(),
-                      item.getName().c_str(), value.c_str());
+                         item.getName().c_str(), value.c_str());
     return true;
   } else {
     return false;
@@ -80,7 +81,7 @@ bool ConfigurableComponent::setProperty(const std::string name,
     item.setValue(value);
     properties_[item.getName()] = item;
     my_logger_->log_info("Component %s property name %s value %s", name.c_str(),
-                      item.getName().c_str(), value.c_str());
+                         item.getName().c_str(), value.c_str());
     return true;
   } else {
     return false;
@@ -102,7 +103,7 @@ bool ConfigurableComponent::setProperty(Property &prop, std::string value) {
     item.setValue(value);
     properties_[item.getName()] = item;
     my_logger_->log_info("property name %s value %s", prop.getName().c_str(),
-                      item.getName().c_str(), value.c_str());
+                         item.getName().c_str(), value.c_str());
     return true;
   } else {
     Property newProp(prop);
@@ -110,7 +111,6 @@ bool ConfigurableComponent::setProperty(Property &prop, std::string value) {
     properties_.insert(
         std::pair<std::string, Property>(prop.getName(), newProp));
     return true;
-
   }
   return false;
 }
@@ -132,11 +132,10 @@ bool ConfigurableComponent::setSupportedProperties(
   for (auto item : properties) {
     properties_[item.getName()] = item;
   }
-
   return true;
 }
 
-} /* namespace components */
+} /* namespace core */
 } /* namespace minifi */
 } /* namespace nifi */
 } /* namespace apache */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/core/ConfigurationFactory.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/ConfigurationFactory.cpp b/libminifi/src/core/ConfigurationFactory.cpp
index 52bde69..e009b1b 100644
--- a/libminifi/src/core/ConfigurationFactory.cpp
+++ b/libminifi/src/core/ConfigurationFactory.cpp
@@ -15,10 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
+#include <type_traits>
+#include <utility>
+#include <string>
+#include <memory>
+#include <algorithm>
+#include <set>
 #include "core/ConfigurationFactory.h"
 #include "core/FlowConfiguration.h"
-#include  <type_traits>
+
 #ifdef YAML_SUPPORT
 #include "core/yaml/YamlConfiguration.h"
 #endif
@@ -29,50 +34,49 @@ namespace nifi {
 namespace minifi {
 namespace core {
 #ifndef YAML_SUPPORT
-  class YamlConfiguration;
+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::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 {
+  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));
 
-      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 (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) {
+    } 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");
       }
     }
-
-    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 */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/core/Connectable.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/Connectable.cpp b/libminifi/src/core/Connectable.cpp
index ac61568..e234a06 100644
--- a/libminifi/src/core/Connectable.cpp
+++ b/libminifi/src/core/Connectable.cpp
@@ -1,13 +1,26 @@
-/*
- * Connectable.cpp
+/**
  *
- *  Created on: Feb 27, 2017
- *      Author: mparisi
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
-
-#include "../../include/core/Connectable.h"
-
+#include "core/Connectable.h"
 #include <uuid/uuid.h>
+#include <utility>
+#include <memory>
+#include <string>
+#include <set>
 #include "core/logging/Logger.h"
 #include "core/Relationship.h"
 
@@ -20,7 +33,6 @@ namespace core {
 Connectable::Connectable(std::string name, uuid_t uuid)
     : CoreComponent(name, uuid),
       max_concurrent_tasks_(1) {
-
 }
 
 Connectable::Connectable(const Connectable &&other)
@@ -31,7 +43,6 @@ Connectable::Connectable(const Connectable &&other)
 }
 
 Connectable::~Connectable() {
-
 }
 
 bool Connectable::setSupportedRelationships(
@@ -51,7 +62,6 @@ bool Connectable::setSupportedRelationships(
     logger_->log_info("Processor %s supported relationship name %s",
                       name_.c_str(), item.getName().c_str());
   }
-
   return true;
 }
 
@@ -89,7 +99,6 @@ bool Connectable::setAutoTerminatedRelationships(
     logger_->log_info("Processor %s auto terminated relationship name %s",
                       name_.c_str(), item.getName().c_str());
   }
-
   return true;
 }
 
@@ -118,7 +127,6 @@ void Connectable::waitForWork(uint64_t timeoutMs) {
     work_condition_.wait_for(lock, std::chrono::milliseconds(timeoutMs),
                              [&] {return has_work_.load();});
   }
-
 }
 
 void Connectable::notifyWork() {
@@ -134,7 +142,6 @@ void Connectable::notifyWork() {
       work_condition_.notify_one();
     }
   }
-
 }
 
 std::set<std::shared_ptr<Connectable>> Connectable::getOutGoingConnections(
@@ -167,7 +174,7 @@ std::shared_ptr<Connectable> Connectable::getNextIncomingConnection() {
   return ret;
 }
 
-} /* namespace components */
+} /* namespace core */
 } /* namespace minifi */
 } /* namespace nifi */
 } /* namespace apache */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/core/Core.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/Core.cpp b/libminifi/src/core/Core.cpp
index 39969f6..cc6445d 100644
--- a/libminifi/src/core/Core.cpp
+++ b/libminifi/src/core/Core.cpp
@@ -1,12 +1,23 @@
-/*
- * Core.cpp
+/**
  *
- *  Created on: Mar 10, 2017
- *      Author: mparisi
+ * 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/Core.h"
+#include <string>
 namespace org {
 namespace apache {
 namespace nifi {
@@ -38,14 +49,13 @@ unsigned const char *CoreComponent::getUUID() {
 // Set Processor Name
 void CoreComponent::setName(const std::string name) {
   name_ = name;
-
 }
 // Get Process Name
 std::string CoreComponent::getName() {
   return name_;
 }
-}
-}
-}
-}
-}
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/core/FlowConfiguration.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/FlowConfiguration.cpp b/libminifi/src/core/FlowConfiguration.cpp
index f2dda0d..68aaf5c 100644
--- a/libminifi/src/core/FlowConfiguration.cpp
+++ b/libminifi/src/core/FlowConfiguration.cpp
@@ -17,6 +17,8 @@
  */
 
 #include "core/FlowConfiguration.h"
+#include <memory>
+#include <string>
 
 namespace org {
 namespace apache {
@@ -25,7 +27,6 @@ namespace minifi {
 namespace core {
 
 FlowConfiguration::~FlowConfiguration() {
-
 }
 
 std::shared_ptr<core::Processor> FlowConfiguration::createProcessor(
@@ -40,11 +41,6 @@ std::shared_ptr<core::Processor> FlowConfiguration::createProcessor(
     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,

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/core/FlowFile.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/FlowFile.cpp b/libminifi/src/core/FlowFile.cpp
new file mode 100644
index 0000000..b3ab741
--- /dev/null
+++ b/libminifi/src/core/FlowFile.cpp
@@ -0,0 +1,223 @@
+/**
+ *
+ * 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/FlowFile.h"
+#include <memory>
+#include <string>
+#include <set>
+#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),
+      event_time_(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_;
+}
+
+} /* namespace core */
+} /* namespace minifi */
+} /* namespace nifi */
+} /* namespace apache */
+} /* namespace org */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/core/ProcessGroup.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/ProcessGroup.cpp b/libminifi/src/core/ProcessGroup.cpp
index baa3ebd..1a6e729 100644
--- a/libminifi/src/core/ProcessGroup.cpp
+++ b/libminifi/src/core/ProcessGroup.cpp
@@ -17,16 +17,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+#include "core/ProcessGroup.h"
+#include <sys/time.h>
+#include <time.h>
 #include <vector>
+#include <memory>
+#include <string>
 #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 {
@@ -63,7 +64,6 @@ ProcessGroup::~ProcessGroup() {
     ProcessGroup *processGroup(*it);
     delete processGroup;
   }
-
 }
 
 bool ProcessGroup::isRootProcessGroup() {
@@ -176,16 +176,12 @@ void ProcessGroup::stopProcessing(TimerDrivenSchedulingAgent *timeScheduler,
 }
 
 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;
@@ -195,37 +191,31 @@ std::shared_ptr<Processor> ProcessGroup::findProcessor(uuid_t uuid) {
         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;
 }
 
@@ -233,18 +223,15 @@ 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;
 }
 
@@ -253,7 +240,6 @@ void ProcessGroup::getConnections(
   for (auto connection : connections_) {
     connectionMap[connection->getUUIDStr()] = connection;
   }
-
   for (auto processGroup : child_process_groups_) {
     processGroup->getConnections(connectionMap);
   }
@@ -305,7 +291,7 @@ void ProcessGroup::removeConnection(std::shared_ptr<Connection> connection) {
   }
 }
 
-} /* namespace processor */
+} /* namespace core */
 } /* namespace minifi */
 } /* namespace nifi */
 } /* namespace apache */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/core/ProcessSession.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/ProcessSession.cpp b/libminifi/src/core/ProcessSession.cpp
index 09c3fa3..70de3f6 100644
--- a/libminifi/src/core/ProcessSession.cpp
+++ b/libminifi/src/core/ProcessSession.cpp
@@ -17,18 +17,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+#include "core/ProcessSession.h"
+#include <sys/time.h>
+#include <time.h>
 #include <vector>
 #include <queue>
 #include <map>
+#include <memory>
+#include <string>
 #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 {
@@ -37,8 +38,8 @@ namespace core {
 
 std::shared_ptr<core::FlowFile> ProcessSession::create() {
   std::map<std::string, std::string> empty;
-  std::shared_ptr<core::FlowFile> record = std::make_shared<FlowFileRecord>(process_context_->getProvenanceRepository(),
-      empty);
+  std::shared_ptr<core::FlowFile> record = std::make_shared<FlowFileRecord>(
+      process_context_->getProvenanceRepository(), empty);
 
   _addedFlowFiles[record->getUUIDStr()] = record;
   logger_->log_debug("Create FlowFile with UUID %s",
@@ -50,10 +51,11 @@ std::shared_ptr<core::FlowFile> ProcessSession::create() {
   return record;
 }
 
-std::shared_ptr<core::FlowFile> ProcessSession::create(std::shared_ptr<core::FlowFile> &&parent) {
+std::shared_ptr<core::FlowFile> ProcessSession::create(
+    std::shared_ptr<core::FlowFile> &&parent) {
   std::map<std::string, std::string> empty;
-  std::shared_ptr<core::FlowFile> record = std::make_shared<FlowFileRecord>(process_context_->getProvenanceRepository(),
-      empty);
+  std::shared_ptr<core::FlowFile> record = std::make_shared<FlowFileRecord>(
+      process_context_->getProvenanceRepository(), empty);
 
   if (record) {
     _addedFlowFiles[record->getUUIDStr()] = record;
@@ -77,7 +79,6 @@ std::shared_ptr<core::FlowFile> ProcessSession::create(std::shared_ptr<core::Flo
     record->setLineageStartDate(parent->getlineageStartDate());
     record->setLineageIdentifiers(parent->getlineageIdentifiers());
     parent->getlineageIdentifiers().insert(parent->getUUIDStr());
-
   }
   return record;
 }
@@ -93,7 +94,6 @@ std::shared_ptr<core::FlowFile> ProcessSession::clone(
       record->setOffset(parent->getOffset());
       record->setSize(parent->getSize());
       record->getResourceClaim()->increaseFlowFileRecordOwnedCount();
-      ;
     }
     provenance_report_->clone(parent, record);
   }
@@ -103,8 +103,8 @@ std::shared_ptr<core::FlowFile> ProcessSession::clone(
 std::shared_ptr<core::FlowFile> ProcessSession::cloneDuringTransfer(
     std::shared_ptr<core::FlowFile> &parent) {
   std::map<std::string, std::string> empty;
-  std::shared_ptr<core::FlowFile> record = std::make_shared<FlowFileRecord>(process_context_->getProvenanceRepository(),
-      empty);
+  std::shared_ptr<core::FlowFile> record = std::make_shared<FlowFileRecord>(
+      process_context_->getProvenanceRepository(), empty);
 
   if (record) {
     this->_clonedFlowFiles[record->getUUIDStr()] = record;
@@ -134,7 +134,6 @@ std::shared_ptr<core::FlowFile> ProcessSession::cloneDuringTransfer(
       record->setOffset(parent->getOffset());
       record->setSize(parent->getSize());
       record->getResourceClaim()->increaseFlowFileRecordOwnedCount();
-      ;
     }
     provenance_report_->clone(parent, record);
   }
@@ -143,12 +142,11 @@ std::shared_ptr<core::FlowFile> ProcessSession::cloneDuringTransfer(
 }
 
 std::shared_ptr<core::FlowFile> ProcessSession::clone(
-    std::shared_ptr<core::FlowFile> &parent, long offset, long size) {
+    std::shared_ptr<core::FlowFile> &parent, int64_t offset, int64_t size) {
   std::shared_ptr<core::FlowFile> record = this->create(parent);
   if (record) {
-
     if (parent->getResourceClaim()) {
-      if ((offset + size) > (long) parent->getSize()) {
+      if ((offset + size) > parent->getSize()) {
         // Set offset and size
         logger_->log_error("clone offset %d and size %d exceed parent size %d",
                            offset, size, parent->getSize());
@@ -165,7 +163,6 @@ std::shared_ptr<core::FlowFile> ProcessSession::clone(
       std::shared_ptr<ResourceClaim> parent_claim = parent->getResourceClaim();
       record->setResourceClaim(parent_claim);
       if (parent_claim != nullptr) {
-
         record->getResourceClaim()->increaseFlowFileRecordOwnedCount();
       }
     }
@@ -531,13 +528,13 @@ void ProcessSession::read(std::shared_ptr<core::FlowFile> &&flow,
 }
 
 /**
-  * Imports a file from the data stream
-  * @param stream incoming data stream that contains the data to store into a file
-  * @param flow flow file
-  *
-  */
+ * Imports a file from the data stream
+ * @param stream incoming data stream that contains the data to store into a file
+ * @param flow flow file
+ *
+ */
 void ProcessSession::importFrom(io::DataStream &stream,
-                            std::shared_ptr<core::FlowFile> &&flow) {
+                                std::shared_ptr<core::FlowFile> &&flow) {
   std::shared_ptr<ResourceClaim> claim = std::make_shared<ResourceClaim>();
 
   int max_read = getpagesize();
@@ -550,27 +547,21 @@ void ProcessSession::importFrom(io::DataStream &stream,
     fs.open(claim->getContentFullPath().c_str(),
             std::fstream::out | std::fstream::binary | std::fstream::trunc);
 
-
-    if (fs.is_open() ) {
-
+    if (fs.is_open()) {
       size_t position = 0;
       const size_t max_size = stream.getSize();
       size_t read_size = max_read;
-      while(position < max_size)
-      {
-        if ((max_size - position) > max_read)
-        {
+      while (position < max_size) {
+        if ((max_size - position) > max_read) {
           read_size = max_read;
-        }
-        else
-        {
+        } else {
           read_size = max_size - position;
         }
         charBuffer.clear();
-        stream.readData(charBuffer,read_size);
+        stream.readData(charBuffer, read_size);
 
-        fs.write((const char*)charBuffer.data(),read_size);
-        position+=read_size;
+        fs.write((const char*) charBuffer.data(), read_size);
+        position += read_size;
       }
       // Open the source file and stream to the flow file
 
@@ -603,7 +594,6 @@ void ProcessSession::importFrom(io::DataStream &stream,
     } else {
       throw Exception(FILE_OPERATION_EXCEPTION, "File Import Error");
     }
-
   } catch (std::exception &exception) {
     if (flow && flow->getResourceClaim() == claim) {
       flow->getResourceClaim()->decreaseFlowFileRecordOwnedCount();
@@ -730,7 +720,6 @@ void ProcessSession::import(std::string source,
           fs.write(buf, input.gcount());
       }
 
-
       if (fs.good() && fs.tellp() >= 0) {
         flow->setSize(fs.tellp());
         flow->setOffset(0);
@@ -786,7 +775,6 @@ void ProcessSession::import(std::string source,
 }
 
 void ProcessSession::commit() {
-
   try {
     // First we clone the flow record based on the transfered relationship for updated flow record
     for (auto && it : _updatedFlowFiles) {
@@ -962,7 +950,6 @@ void ProcessSession::rollback() {
         flowf->setSnapShot(false);
         connection->put(record);
       }
-
     }
     _originalFlowFiles.clear();
 
@@ -1010,7 +997,8 @@ std::shared_ptr<core::FlowFile> ProcessSession::get() {
       _updatedFlowFiles[ret->getUUIDStr()] = ret;
       std::map<std::string, std::string> empty;
       std::shared_ptr<core::FlowFile> snapshot =
-          std::make_shared<FlowFileRecord>(process_context_->getProvenanceRepository(),empty);
+          std::make_shared<FlowFileRecord>(
+              process_context_->getProvenanceRepository(), empty);
       logger_->log_debug("Create Snapshot FlowFile with UUID %s",
                          snapshot->getUUIDStr().c_str());
       snapshot = ret;
@@ -1026,7 +1014,7 @@ std::shared_ptr<core::FlowFile> ProcessSession::get() {
   return NULL;
 }
 
-} /* namespace processor */
+} /* namespace core */
 } /* namespace minifi */
 } /* namespace nifi */
 } /* namespace apache */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/core/ProcessSessionFactory.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/ProcessSessionFactory.cpp b/libminifi/src/core/ProcessSessionFactory.cpp
index 445ca58..31b7481 100644
--- a/libminifi/src/core/ProcessSessionFactory.cpp
+++ b/libminifi/src/core/ProcessSessionFactory.cpp
@@ -19,23 +19,19 @@
  */
 
 #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_));
+std::unique_ptr<ProcessSession> ProcessSessionFactory::createSession() {
+  return std::unique_ptr<ProcessSession>(new ProcessSession(process_context_));
 }
 
-
-} /* namespace processor */
+} /* namespace core */
 } /* namespace minifi */
 } /* namespace nifi */
 } /* namespace apache */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/core/Processor.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/Processor.cpp b/libminifi/src/core/Processor.cpp
index ba52c28..9a0898a 100644
--- a/libminifi/src/core/Processor.cpp
+++ b/libminifi/src/core/Processor.cpp
@@ -17,19 +17,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+#include "core/Processor.h"
+#include <sys/time.h>
+#include <time.h>
 #include <vector>
 #include <queue>
 #include <map>
 #include <set>
-#include <sys/time.h>
-#include <time.h>
 #include <chrono>
+#include <string>
 #include <thread>
 #include <memory>
 #include <functional>
-
-#include "core/Processor.h"
-
 #include "Connection.h"
 #include "core/Connectable.h"
 #include "core/ProcessContext.h"
@@ -45,7 +44,6 @@ 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;
@@ -74,7 +72,6 @@ void Processor::setScheduledState(ScheduledState state) {
 }
 
 bool Processor::addConnection(std::shared_ptr<Connectable> conn) {
-  
   bool ret = false;
 
   if (isRunning()) {
@@ -82,7 +79,8 @@ bool Processor::addConnection(std::shared_ptr<Connectable> conn) {
                       name_.c_str());
     return false;
   }
-  std::shared_ptr<Connection> connection = std::static_pointer_cast<Connection>(conn);
+  std::shared_ptr<Connection> connection = std::static_pointer_cast<Connection>(
+      conn);
   std::lock_guard<std::mutex> lock(mutex_);
 
   uuid_t srcUUID;
@@ -128,7 +126,6 @@ bool Processor::addConnection(std::shared_ptr<Connectable> conn) {
         ret = true;
       }
     } else {
-
       // We do not have any outgoing connection for this relationship yet
       std::set<std::shared_ptr<Connectable>> newConnection;
       newConnection.insert(connection);
@@ -156,8 +153,9 @@ void Processor::removeConnection(std::shared_ptr<Connectable> conn) {
 
   uuid_t srcUUID;
   uuid_t destUUID;
-  
-  std::shared_ptr<Connection> connection = std::static_pointer_cast<Connection>(conn);
+
+  std::shared_ptr<Connection> connection = std::static_pointer_cast<Connection>(
+      conn);
 
   connection->getSourceUUID(srcUUID);
   connection->getDestinationUUID(destUUID);
@@ -193,8 +191,6 @@ void Processor::removeConnection(std::shared_ptr<Connectable> conn) {
   }
 }
 
-
-
 bool Processor::flowFilesQueued() {
   std::lock_guard<std::mutex> lock(mutex_);
 
@@ -202,7 +198,8 @@ bool Processor::flowFilesQueued() {
     return false;
 
   for (auto &&conn : _incomingConnections) {
-    std::shared_ptr<Connection> connection = std::static_pointer_cast<Connection>(conn);
+    std::shared_ptr<Connection> connection =
+        std::static_pointer_cast<Connection>(conn);
     if (connection->getQueueSize() > 0)
       return true;
   }
@@ -217,7 +214,8 @@ bool Processor::flowFilesOutGoingFull() {
     // We already has connection for this relationship
     std::set<std::shared_ptr<Connectable>> existedConnection = connection.second;
     for (const auto conn : existedConnection) {
-      std::shared_ptr<Connection> connection = std::static_pointer_cast<Connection>(conn);
+      std::shared_ptr<Connection> connection = std::static_pointer_cast<
+          Connection>(conn);
       if (connection->isFull())
         return true;
     }
@@ -251,7 +249,8 @@ bool Processor::isWorkAvailable() {
 
   try {
     for (const auto &conn : _incomingConnections) {
-      std::shared_ptr<Connection> connection = std::static_pointer_cast<Connection>(conn);
+      std::shared_ptr<Connection> connection = std::static_pointer_cast<
+          Connection>(conn);
       if (connection->getQueueSize() > 0) {
         hasWork = true;
         break;
@@ -259,13 +258,14 @@ bool Processor::isWorkAvailable() {
     }
   } 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!");
+        "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 core */
 } /* namespace minifi */
 } /* namespace nifi */
 } /* namespace apache */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/core/ProcessorNode.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/ProcessorNode.cpp b/libminifi/src/core/ProcessorNode.cpp
index 44491d3..8979e32 100644
--- a/libminifi/src/core/ProcessorNode.cpp
+++ b/libminifi/src/core/ProcessorNode.cpp
@@ -16,7 +16,7 @@
  */
 
 #include "core/ProcessorNode.h"
-
+#include <memory>
 namespace org {
 namespace apache {
 namespace nifi {
@@ -25,29 +25,23 @@ namespace core {
 
 ProcessorNode::ProcessorNode(const std::shared_ptr<Connectable> processor)
     : processor_(processor),
-      Connectable(processor->getName(),0),
+      Connectable(processor->getName(), 0),
       ConfigurableComponent(logging::Logger::getLogger()) {
-	
-	uuid_t copy;
-	processor->getUUID(copy);
-	setUUID( copy );
-
-
+  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 );
-
+  uuid_t copy;
+  processor_->getUUID(copy);
+  setUUID(copy);
 }
 
 ProcessorNode::~ProcessorNode() {
-
 }
 
 bool ProcessorNode::isWorkAvailable() {
@@ -58,7 +52,7 @@ bool ProcessorNode::isRunning() {
   return processor_->isRunning();
 }
 
-} /* namespace processor */
+} /* namespace core */
 } /* namespace minifi */
 } /* namespace nifi */
 } /* namespace apache */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/be3f2ffe/libminifi/src/core/Property.cpp
----------------------------------------------------------------------
diff --git a/libminifi/src/core/Property.cpp b/libminifi/src/core/Property.cpp
index 287b7ec..aa002af 100644
--- a/libminifi/src/core/Property.cpp
+++ b/libminifi/src/core/Property.cpp
@@ -17,10 +17,11 @@
  */
 
 #include "core/Property.h"
-
+#include <string>
 namespace org {
 namespace apache {
-namespace nifi {namespace minifi {
+namespace nifi {
+namespace minifi {
 namespace core {
 
 // Get Name for the property