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 2018/09/20 15:15:02 UTC

[7/9] nifi-minifi-cpp git commit: MINIFICPP-595: Provide basic support for windows. MINIFICPP-32: Add windows event log reader MINIFICPP-596: Build core and libminifi artifacts. Must abstract features that are operating system dependent, such as uuid

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/controllers/ThreadManagementService.h
----------------------------------------------------------------------
diff --git a/libminifi/include/controllers/ThreadManagementService.h b/libminifi/include/controllers/ThreadManagementService.h
index b297834..76bda72 100644
--- a/libminifi/include/controllers/ThreadManagementService.h
+++ b/libminifi/include/controllers/ThreadManagementService.h
@@ -42,13 +42,13 @@ class ThreadManagementService : public core::controller::ControllerService {
         logger_(logging::LoggerFactory<ThreadManagementService>::getLogger()) {
   }
 
-  explicit ThreadManagementService(const std::string &name, uuid_t uuid = 0)
+  explicit ThreadManagementService(const std::string &name, utils::Identifier uuid = utils::Identifier())
       : ControllerService(name, uuid),
         logger_(logging::LoggerFactory<ThreadManagementService>::getLogger()) {
   }
 
   explicit ThreadManagementService(const std::string &name, const std::shared_ptr<Configure> &configuration)
-      : ControllerService(name, nullptr),
+      : ControllerService(name),
         logger_(logging::LoggerFactory<ThreadManagementService>::getLogger()) {
 
   }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/controllers/UpdatePolicyControllerService.h
----------------------------------------------------------------------
diff --git a/libminifi/include/controllers/UpdatePolicyControllerService.h b/libminifi/include/controllers/UpdatePolicyControllerService.h
index 9ca0cf2..dd50a1c 100644
--- a/libminifi/include/controllers/UpdatePolicyControllerService.h
+++ b/libminifi/include/controllers/UpdatePolicyControllerService.h
@@ -47,7 +47,7 @@ class UpdatePolicyControllerService : public core::controller::ControllerService
         logger_(logging::LoggerFactory<UpdatePolicyControllerService>::getLogger()) {
   }
 
-  explicit UpdatePolicyControllerService(const std::string &name, uuid_t uuid = 0)
+  explicit UpdatePolicyControllerService(const std::string &name, utils::Identifier uuid = utils::Identifier())
       : ControllerService(name, uuid),
         persist_updates_(false),
         policy_(new state::UpdatePolicy(false)),
@@ -55,7 +55,7 @@ class UpdatePolicyControllerService : public core::controller::ControllerService
   }
 
   explicit UpdatePolicyControllerService(const std::string &name, const std::shared_ptr<Configure> &configuration)
-      : UpdatePolicyControllerService(name, nullptr) {
+      : UpdatePolicyControllerService(name) {
     setConfiguration(configuration);
     initialize();
   }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/ClassLoader.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ClassLoader.h b/libminifi/include/core/ClassLoader.h
index cc8d322..1f5b6d2 100644
--- a/libminifi/include/core/ClassLoader.h
+++ b/libminifi/include/core/ClassLoader.h
@@ -22,7 +22,16 @@
 #include <vector>
 #include <map>
 #include "utils/StringUtils.h"
+#ifndef WIN32
 #include <dlfcn.h>
+#define DLL_EXPORT
+#else
+#define WIN32_LEAN_AND_MEAN 1
+#include <Windows.h>    // Windows specific libraries for collecting software metrics.
+#include <Psapi.h>
+#pragma comment( lib, "psapi.lib" )
+#define DLL_EXPORT __declspec(dllexport)  
+#endif
 #include "core/Core.h"
 #include "io/DataStream.h"
 
@@ -36,6 +45,14 @@ namespace core {
 
 #define RESOURCE_SUCCESS 1
 
+#ifdef WIN32
+#define RTLD_LAZY   0
+#define RTLD_NOW    0
+
+#define RTLD_GLOBAL (1 << 1)
+#define RTLD_LOCAL  (1 << 2)
+#endif
+
 /**
  * Factory that is used as an interface for
  * creating processors from shared objects.
@@ -75,14 +92,14 @@ class ObjectFactory {
   /**
    * Create a shared pointer to a new processor.
    */
-  virtual std::shared_ptr<CoreComponent> create(const std::string &name, uuid_t uuid) {
+  virtual std::shared_ptr<CoreComponent> create(const std::string &name, utils::Identifier & uuid) {
     return nullptr;
   }
 
   /**
    * Create a shared pointer to a new processor.
    */
-  virtual CoreComponent* createRaw(const std::string &name, uuid_t uuid) {
+  virtual CoreComponent* createRaw(const std::string &name, utils::Identifier & uuid) {
     return nullptr;
   }
 
@@ -146,7 +163,7 @@ class DefautObjectFactory : public ObjectFactory {
   /**
    * Create a shared pointer to a new processor.
    */
-  virtual std::shared_ptr<CoreComponent> create(const std::string &name, uuid_t uuid) {
+  virtual std::shared_ptr<CoreComponent> create(const std::string &name, utils::Identifier & uuid) {
     std::shared_ptr<T> ptr = std::make_shared<T>(name, uuid);
     return std::static_pointer_cast<CoreComponent>(ptr);
   }
@@ -162,7 +179,7 @@ class DefautObjectFactory : public ObjectFactory {
   /**
    * Create a shared pointer to a new processor.
    */
-  virtual CoreComponent* createRaw(const std::string &name, uuid_t uuid) {
+  virtual CoreComponent* createRaw(const std::string &name, utils::Identifier &uuid) {
     T *ptr = new T(name, uuid);
     return dynamic_cast<CoreComponent*>(ptr);
   }
@@ -238,12 +255,11 @@ class ClassLoader {
    * Register a class with the give ProcessorFactory
    */
   void registerClass(const std::string &name, std::unique_ptr<ObjectFactory> factory) {
+    std::lock_guard<std::mutex> lock(internal_mutex_);
     if (loaded_factories_.find(name) != loaded_factories_.end()) {
       return;
     }
 
-    std::lock_guard<std::mutex> lock(internal_mutex_);
-
     auto canonical_name = factory->getClassName();
 
     auto group_name = factory->getGroupName();
@@ -309,7 +325,7 @@ class ClassLoader {
    * @return nullptr or object created from class_name definition.
    */
   template<class T = CoreComponent>
-  std::shared_ptr<T> instantiate(const std::string &class_name, uuid_t uuid);
+  std::shared_ptr<T> instantiate(const std::string &class_name, utils::Identifier & uuid);
 
   /**
    * Instantiate object based on class_name
@@ -327,10 +343,147 @@ class ClassLoader {
    * @return nullptr or object created from class_name definition.
    */
   template<class T = CoreComponent>
-  T *instantiateRaw(const std::string &class_name, uuid_t uuid);
+  T *instantiateRaw(const std::string &class_name, utils::Identifier & uuid);
 
  protected:
 
+#ifdef WIN32
+
+  // base_object doesn't have a handle
+  std::map< HMODULE, std::string > resource_mapping_;
+
+  std::string error_str_;
+  std::string current_error_;
+
+  void store_error() {
+    auto error = GetLastError();
+
+    if (error == 0) {
+      error_str_ = "";
+      return;
+    }
+
+    LPSTR messageBuffer = nullptr;
+    size_t size = FormatMessageA(FORMAT_MESSAGE_ALLOCATE_BUFFER | FORMAT_MESSAGE_FROM_SYSTEM | FORMAT_MESSAGE_IGNORE_INSERTS,
+        NULL, error, MAKELANGID(LANG_NEUTRAL, SUBLANG_DEFAULT), (LPSTR)&messageBuffer, 0, NULL);
+
+    current_error_ = std::string(messageBuffer, size);
+
+    //Free the buffer.
+    LocalFree(messageBuffer);
+  }
+
+  void *dlsym(void *handle, const char *name)
+  {
+    FARPROC symbol;
+    HMODULE hModule;
+
+    symbol = GetProcAddress((HMODULE)handle, name);
+
+    if (symbol == nullptr) {
+      store_error();
+
+      for (auto hndl : resource_mapping_)
+      {
+        symbol = GetProcAddress((HMODULE)hndl.first, name);
+        if (symbol != nullptr) {
+          break;
+        }
+      }
+    }
+
+#ifdef _MSC_VER
+#pragma warning( suppress: 4054 )
+#endif
+    return (void*)symbol;
+  }
+
+  const char *dlerror(void)
+  {
+    std::lock_guard<std::mutex> lock(internal_mutex_);
+
+    error_str_ = current_error_;
+
+    current_error_ = "";
+
+    return error_str_.c_str();
+  }
+
+  void *dlopen(const char *file, int mode) {
+    std::lock_guard<std::mutex> lock(internal_mutex_);
+    HMODULE object;
+    char * current_error = NULL;
+    uint32_t uMode = SetErrorMode(SEM_FAILCRITICALERRORS);
+    if (nullptr == file)
+    {
+      HMODULE allModules[1024];
+      HANDLE current_process_id = GetCurrentProcess();
+      DWORD cbNeeded;
+      object = GetModuleHandle(NULL);
+
+      if (!object)
+      store_error();
+      if (EnumProcessModules(current_process_id, allModules,
+              sizeof(allModules), &cbNeeded) != 0)
+      {
+
+        for (uint32_t i = 0; i < cbNeeded / sizeof(HMODULE); i++)
+        {
+          TCHAR szModName[MAX_PATH];
+
+          // Get the full path to the module's file.
+          resource_mapping_.insert(std::make_pair(allModules[i], "minifi-system"));
+        }
+      }
+    }
+    else
+    {
+      char lpFileName[MAX_PATH];
+      int i;
+
+      for (i = 0; i < sizeof(lpFileName) - 1; i++)
+      {
+        if (!file[i])
+        break;
+        else if (file[i] == '/')
+        lpFileName[i] = '\\';
+        else
+        lpFileName[i] = file[i];
+      }
+      lpFileName[i] = '\0';
+      object = LoadLibraryEx(lpFileName, nullptr, LOAD_WITH_ALTERED_SEARCH_PATH);
+      if (!object)
+      store_error();
+      else if ((mode & RTLD_GLOBAL))
+      resource_mapping_.insert(std::make_pair(object, lpFileName));
+    }
+
+    /* Return to previous state of the error-mode bit flags. */
+    SetErrorMode(uMode);
+
+    return (void *)object;
+
+  }
+
+  int dlclose(void *handle)
+  {
+    std::lock_guard<std::mutex> lock(internal_mutex_);
+
+    HMODULE object = (HMODULE)handle;
+    BOOL ret;
+
+    current_error_ = "";
+    ret = FreeLibrary(object);
+
+    resource_mapping_.erase(object);
+
+    ret = !ret;
+
+    return (int)ret;
+  }
+
+#endif
+
   std::map<std::string, std::vector<std::string>> module_mapping_;
 
   std::map<std::string, std::unique_ptr<ObjectFactory>> loaded_factories_;
@@ -355,7 +508,7 @@ std::shared_ptr<T> ClassLoader::instantiate(const std::string &class_name, const
 }
 
 template<class T>
-std::shared_ptr<T> ClassLoader::instantiate(const std::string &class_name, uuid_t uuid) {
+std::shared_ptr<T> ClassLoader::instantiate(const std::string &class_name, utils::Identifier &uuid) {
   std::lock_guard<std::mutex> lock(internal_mutex_);
   auto factory_entry = loaded_factories_.find(class_name);
   if (factory_entry != loaded_factories_.end()) {
@@ -379,7 +532,7 @@ T *ClassLoader::instantiateRaw(const std::string &class_name, const std::string
 }
 
 template<class T>
-T *ClassLoader::instantiateRaw(const std::string &class_name, uuid_t uuid) {
+T *ClassLoader::instantiateRaw(const std::string &class_name, utils::Identifier & uuid) {
   std::lock_guard<std::mutex> lock(internal_mutex_);
   auto factory_entry = loaded_factories_.find(class_name);
   if (factory_entry != loaded_factories_.end()) {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/ConfigurableComponent.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ConfigurableComponent.h b/libminifi/include/core/ConfigurableComponent.h
index 53835b4..9aff609 100644
--- a/libminifi/include/core/ConfigurableComponent.h
+++ b/libminifi/include/core/ConfigurableComponent.h
@@ -19,6 +19,7 @@
 #ifndef LIBMINIFI_INCLUDE_CORE_CONFIGURABLECOMPONENT_H_
 #define LIBMINIFI_INCLUDE_CORE_CONFIGURABLECOMPONENT_H_
 
+#include "Core.h"
 #include <mutex>
 #include <iostream>
 #include <map>
@@ -40,7 +41,7 @@ namespace core {
  * Represents a configurable component
  * Purpose: Extracts configuration items for all components and localized them
  */
-class __attribute__((visibility("default"))) ConfigurableComponent {
+class ConfigurableComponent {
  public:
 
   ConfigurableComponent();

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/Connectable.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/Connectable.h b/libminifi/include/core/Connectable.h
index 1c1fd45..b1a5613 100644
--- a/libminifi/include/core/Connectable.h
+++ b/libminifi/include/core/Connectable.h
@@ -37,10 +37,13 @@ namespace core {
  * Purpose: As in NiFi, this represents a connection point and allows the derived
  * object to be connected to other connectables.
  */
-class __attribute__((visibility("default"))) Connectable : public CoreComponent {
+class Connectable : public CoreComponent {
  public:
 
-  explicit Connectable(std::string name, uuid_t uuid);
+
+  explicit Connectable(std::string name);
+
+  explicit Connectable(std::string name, utils::Identifier &uuid);
 
   explicit Connectable(const Connectable &&other);
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/Core.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/Core.h b/libminifi/include/core/Core.h
index 8109cd1..a379056 100644
--- a/libminifi/include/core/Core.h
+++ b/libminifi/include/core/Core.h
@@ -17,13 +17,48 @@
  */
 #ifndef LIBMINIFI_INCLUDE_CORE_CORE_H_
 #define LIBMINIFI_INCLUDE_CORE_CORE_H_
-
+#define WIN32_LEAN_AND_MEAN 1
 #include <cstdlib>
 #include <iostream>
 #include <memory>
 #include <string>
 #include <uuid/uuid.h>
+
+#ifdef WIN32
+#pragma comment(lib, "shlwapi.lib")
+#endif
+
+#if defined _WIN32 || defined __CYGWIN__
+#ifdef BUILDING_DLL
+#ifdef __GNUC__
+#define DLL_PUBLIC __attribute__ ((dllexport))
+#else
+#define DLL_PUBLIC __declspec(dllexport) // Note: actually gcc seems to also supports this syntax.
+#endif
+#else
+#ifdef __GNUC__
+#define DLL_PUBLIC __attribute__ ((dllimport))
+#else
+#define DLL_PUBLIC __declspec(dllimport) // Note: actually gcc seems to also supports this syntax.
+#endif
+#endif
+#define DLL_LOCAL
+#else
+#if __GNUC__ >= 4
+#define DLL_PUBLIC __attribute__ ((visibility ("default")))
+#define DLL_LOCAL  __attribute__ ((visibility ("hidden")))
+#else
+#define DLL_PUBLIC
+#define DLL_LOCAL
+#endif
+#endif
+
+#ifdef _WIN32
+#define WIN32_LEAN_AND_MEAN
+// can't include cxxabi
+#else
 #include <cxxabi.h>
+#endif
 
 #include "utils/Id.h"
 
@@ -35,7 +70,7 @@ namespace apache {
 namespace nifi {
 namespace minifi {
 namespace utils {
-namespace file{
+namespace file {
 }
 }
 namespace processors {
@@ -47,12 +82,16 @@ namespace core {
 
 template<typename T>
 static inline std::string getClassName() {
+#ifndef WIN32
   char *b = abi::__cxa_demangle(typeid(T).name(), 0, 0, 0);
   if (b == nullptr)
     return std::string();
   std::string name = b;
   std::free(b);
   return name;
+#else
+  return typeid(T).name();
+#endif
 }
 
 template<typename T>
@@ -99,25 +138,52 @@ class CoreComponent {
   /**
    * Constructor that sets the name and uuid.
    */
-  explicit CoreComponent(const std::string name, uuid_t uuid = nullptr)
+
+  explicit CoreComponent(const std::string name, utils::Identifier uuid)
       : name_(name) {
-    if (nullptr == uuid)
+    if (uuid == nullptr) {
       // Generate the global UUID for the flow record
       id_generator_->generate(uuid_);
-    else
-      uuid_copy(uuid_, uuid);
+    } else {
+      uuid_ = uuid;
+    }
 
-    char uuidStr[37] = {0};
-    uuid_unparse_lower(uuid_, uuidStr);
-    uuidStr_ = uuidStr;
+    uuidStr_ = uuid_.to_string();
   }
 
+  explicit CoreComponent(const std::string name)
+      : name_(name) {
+
+    // Generate the global UUID for the flow record
+    id_generator_->generate(uuid_);
+
+    uuidStr_ = uuid_.to_string();
+  }
+  /*
+   #ifdef WIN32
+   #else
+
+   explicit CoreComponent(const std::string name, Identifier uuid = nullptr)
+   : name_(name) {
+   if (nullptr == uuid)
+   // Generate the global UUID for the flow record
+   id_generator_->generate(uuid_);
+   else
+   uuid_copy(uuid_, uuid);
+
+   char uuidStr[37] = { 0 };
+   uuid_unparse_lower(uuid_, uuidStr);
+   uuidStr_ = uuidStr;
+   }
+   #endif
+   */
   /**
    * Move Constructor.
    */
   explicit CoreComponent(const CoreComponent &&other)
       : name_(std::move(other.name_)) {
-    uuid_copy(uuid_, other.uuid_);
+    uuid_ = other.uuid_;
+    //uuid_copy(uuid_, other.uuid_);
   }
 
   virtual ~CoreComponent() {
@@ -137,7 +203,7 @@ class CoreComponent {
    * Set UUID in this instance
    * @param uuid uuid to apply to the internal representation.
    */
-  void setUUID(uuid_t uuid);
+  void setUUID(utils::Identifier &uuid);
 
   void setUUIDStr(const std::string uuidStr);
 
@@ -146,9 +212,10 @@ class CoreComponent {
    * @param uuid uuid struct to which we will copy the memory
    * @return success of request
    */
-  bool getUUID(uuid_t uuid);
+  //bool getUUID(m_uuid uuid);
+  bool getUUID(utils::Identifier &uuid);
 
-  unsigned const char *getUUID();
+  //unsigned const char *getUUID();
   /**
    * Return the UUID string
    * @param constant reference to the UUID str
@@ -162,7 +229,7 @@ class CoreComponent {
 
  protected:
   // A global unique identifier
-  uuid_t uuid_;
+  utils::Identifier uuid_;
   // UUID string
   std::string uuidStr_;
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/FlowConfiguration.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/FlowConfiguration.h b/libminifi/include/core/FlowConfiguration.h
index cf4c7f6..9d2562d 100644
--- a/libminifi/include/core/FlowConfiguration.h
+++ b/libminifi/include/core/FlowConfiguration.h
@@ -41,6 +41,14 @@ namespace nifi {
 namespace minifi {
 namespace core {
 
+class static_initializers {
+ public:
+  std::vector<std::string> statics_sl_funcs_;
+  std::mutex atomic_initialization_;
+};
+
+extern static_initializers &get_static_functions();
+
 /**
  * Purpose: Flow configuration defines the mechanism
  * by which we will configure our flow controller
@@ -70,25 +78,25 @@ class FlowConfiguration : public CoreComponent {
   virtual ~FlowConfiguration();
 
   // Create Processor (Node/Input/Output Port) based on the name
-  std::shared_ptr<core::Processor> createProcessor(std::string name, uuid_t uuid);
+  std::shared_ptr<core::Processor> createProcessor(std::string name, utils::Identifier &uuid);
   // Create Root Processor Group
 
-  std::unique_ptr<core::ProcessGroup> createRootProcessGroup(std::string name, uuid_t uuid, int version);
+  std::unique_ptr<core::ProcessGroup> createRootProcessGroup(std::string name, utils::Identifier & uuid, int version);
 
-  std::shared_ptr<core::controller::ControllerServiceNode> createControllerService(const std::string &class_name, const std::string &name, uuid_t uuid);
+  std::shared_ptr<core::controller::ControllerServiceNode> createControllerService(const std::string &class_name, const std::string &name, utils::Identifier & uuid);
 
   // Create Remote Processor Group
-  std::unique_ptr<core::ProcessGroup> createRemoteProcessGroup(std::string name, uuid_t uuid);
+  std::unique_ptr<core::ProcessGroup> createRemoteProcessGroup(std::string name, utils::Identifier & uuid);
   // Create Connection
-  std::shared_ptr<minifi::Connection> createConnection(std::string name, uuid_t uuid);
+  std::shared_ptr<minifi::Connection> createConnection(std::string name, utils::Identifier & uuid);
   // Create Provenance Report Task
   std::shared_ptr<core::Processor> createProvenanceReportTask(void);
 
-  std::shared_ptr<state::response::FlowVersion> getFlowVersion() const{
+  std::shared_ptr<state::response::FlowVersion> getFlowVersion() const {
     return flow_version_;
   }
 
-  std::shared_ptr<Configure> getConfiguration() { // cannot be const as getters mutate the underlying map
+  std::shared_ptr<Configure> getConfiguration() {  // cannot be const as getters mutate the underlying map
     return configuration_;
   }
 
@@ -124,16 +132,17 @@ class FlowConfiguration : public CoreComponent {
   }
 
   static bool add_static_func(std::string functor) {
-    statics_sl_funcs_.push_back(functor);
+    std::lock_guard<std::mutex> lock(get_static_functions().atomic_initialization_);
+    get_static_functions().statics_sl_funcs_.push_back(functor);
     return true;
   }
 
   static void initialize_static_functions() {
-    std::lock_guard<std::mutex> lock(atomic_initialization_);
-    for (auto sl_func : statics_sl_funcs_) {
+    std::lock_guard<std::mutex> lock(get_static_functions().atomic_initialization_);
+    for (auto sl_func : get_static_functions().statics_sl_funcs_) {
       core::ClassLoader::getDefaultClassLoader().registerResource("", sl_func);
     }
-    statics_sl_funcs_.clear();
+    //get_static_functions().statics_sl_funcs_.clear();
   }
 
  protected:
@@ -162,8 +171,7 @@ class FlowConfiguration : public CoreComponent {
   std::shared_ptr<state::response::FlowVersion> flow_version_;
  private:
   std::shared_ptr<logging::Logger> logger_;
-  static std::mutex atomic_initialization_;
-  static std::vector<std::string> statics_sl_funcs_;
+
 };
 
 } /* namespace core */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/FlowFile.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/FlowFile.h b/libminifi/include/core/FlowFile.h
index 2116e6f..18d5034 100644
--- a/libminifi/include/core/FlowFile.h
+++ b/libminifi/include/core/FlowFile.h
@@ -206,8 +206,8 @@ class FlowFile : public core::Connectable {
    */
   uint64_t getOffset();
 
-  bool getUUID(uuid_t other) {
-    uuid_copy(other, uuid_);
+  bool getUUID(utils::Identifier &other) {
+    other = uuid_;
     return true;
   }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/ProcessContext.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ProcessContext.h b/libminifi/include/core/ProcessContext.h
index e1c42e4..5d2ca55 100644
--- a/libminifi/include/core/ProcessContext.h
+++ b/libminifi/include/core/ProcessContext.h
@@ -69,11 +69,11 @@ class ProcessContext : public controller::ControllerServiceLookup {
   bool getProperty(const std::string &name, std::string &value) {
     return processor_node_->getProperty(name, value);
   }
-  bool getProperty(const std::string &name, std::string &value, const std::shared_ptr<FlowFile> &flow_file);
+  bool getProperty(const Property &property, std::string &value, const std::shared_ptr<FlowFile> &flow_file);
   bool getDynamicProperty(const std::string &name, std::string &value) {
     return processor_node_->getDynamicProperty(name, value);
   }
-  bool getDynamicProperty(const std::string &name, std::string &value, const std::shared_ptr<FlowFile> &flow_file);
+  bool getDynamicProperty(const Property &property, std::string &value, const std::shared_ptr<FlowFile> &flow_file);
   std::vector<std::string> getDynamicPropertyKeys() {
     return processor_node_->getDynamicPropertyKeys();
   }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/ProcessGroup.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ProcessGroup.h b/libminifi/include/core/ProcessGroup.h
index b5d437a..8c2c469 100644
--- a/libminifi/include/core/ProcessGroup.h
+++ b/libminifi/include/core/ProcessGroup.h
@@ -57,7 +57,11 @@ class ProcessGroup {
   /*!
    * Create a new process group
    */
-  ProcessGroup(ProcessGroupType type, std::string name, uuid_t uuid = NULL, int version = 0, ProcessGroup *parent = NULL);
+
+  ProcessGroup(ProcessGroupType type, std::string name, utils::Identifier &uuid, int version, ProcessGroup *parent);
+  ProcessGroup(ProcessGroupType type, std::string name);
+  ProcessGroup(ProcessGroupType type, std::string name, utils::Identifier &uuid);
+  ProcessGroup(ProcessGroupType type, std::string name, utils::Identifier &uuid, int version);
   // Destructor
   virtual ~ProcessGroup();
   // Set Processor Name
@@ -92,8 +96,8 @@ class ProcessGroup {
     return timeOut_;
   }
   // setInterface
-  void setInterface(std::string &interface) {
-    local_network_interface_ = interface;
+  void setInterface(std::string &ifc) {
+    local_network_interface_ = ifc;
   }
   std::string getInterface() {
     return local_network_interface_;
@@ -140,16 +144,16 @@ class ProcessGroup {
     return (yield_period_msec_);
   }
   // Set UUID
-  void setUUID(uuid_t uuid) {
-    uuid_copy(uuid_, uuid);
+  void setUUID(utils::Identifier &uuid) {
+    uuid_ = uuid;
   }
   // Get UUID
-  bool getUUID(uuid_t uuid) {
-    if (uuid) {
-      uuid_copy(uuid, uuid_);
-      return true;
-    } else
+  bool getUUID(utils::Identifier &uuid) {
+    if (uuid_ == nullptr){
       return false;
+    }
+    uuid = uuid_;
+    return true;
   }
   // getVersion
   int getVersion() {
@@ -182,7 +186,7 @@ class ProcessGroup {
   // ! Add connections
   void addConnection(std::shared_ptr<Connection> connection);
   // findProcessor based on UUID
-  std::shared_ptr<Processor> findProcessor(uuid_t uuid);
+  std::shared_ptr<Processor> findProcessor(utils::Identifier &uuid);
   // findProcessor based on name
   std::shared_ptr<Processor> findProcessor(const std::string &processorName);
 
@@ -212,7 +216,7 @@ class ProcessGroup {
 
  protected:
   // A global unique identifier
-  uuid_t uuid_;
+  utils::Identifier uuid_;
   // Processor Group Name
   std::string name_;
   // version

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/Processor.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/Processor.h b/libminifi/include/core/Processor.h
index 459684d..8f09ffb 100644
--- a/libminifi/include/core/Processor.h
+++ b/libminifi/include/core/Processor.h
@@ -31,6 +31,8 @@
 #include <chrono>
 #include <functional>
 
+#include "Core.h"
+#include <utils/Id.h>
 #include "Connectable.h"
 #include "ConfigurableComponent.h"
 #include "io/StreamFactory.h"
@@ -55,15 +57,19 @@ namespace core {
 
 // Default penalization period in second
 
+#define BUILDING_DLL 1
 // Processor Class
-class __attribute__((visibility("default"))) Processor : public Connectable, public ConfigurableComponent, public std::enable_shared_from_this<Processor> {
+class Processor : public Connectable, public ConfigurableComponent, public std::enable_shared_from_this<Processor> {
 
  public:
   // Constructor
   /*!
    * Create a new processor
    */
-  Processor(std::string name, uuid_t uuid = NULL);
+
+  Processor(std::string name, utils::Identifier &uuid);
+
+  Processor(std::string name);
   // Destructor
   virtual ~Processor() {
     notifyStop();
@@ -95,7 +101,9 @@ class __attribute__((visibility("default"))) Processor : public Connectable, pub
   // Set Processor Scheduling Period in Nano Second
   void setSchedulingPeriodNano(uint64_t period) {
     uint64_t minPeriod = MINIMUM_SCHEDULING_NANOS;
-    scheduling_period_nano_ = std::max(period, minPeriod);
+	// std::max has some variances on c++11-c++14 and then c++14 onward.
+    // to avoid macro conditional checks we can use this simple conditional expr. 
+	scheduling_period_nano_ = period > minPeriod ? period : minPeriod; 
   }
   // Get Processor Scheduling Period in Nano Second
   uint64_t getSchedulingPeriodNano(void) {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/ProcessorNode.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/ProcessorNode.h b/libminifi/include/core/ProcessorNode.h
index c64dbb0..f96899b 100644
--- a/libminifi/include/core/ProcessorNode.h
+++ b/libminifi/include/core/ProcessorNode.h
@@ -205,7 +205,7 @@ class ProcessorNode : public ConfigurableComponent, public Connectable {
    * Set UUID in this instance
    * @param uuid uuid to apply to the internal representation.
    */
-  void setUUID(uuid_t uuid) {
+  void setUUID(utils::Identifier &uuid) {
     Connectable::setUUID(uuid);
     processor_->setUUID(uuid);
   }
@@ -243,13 +243,14 @@ class ProcessorNode : public ConfigurableComponent, public Connectable {
    * @param uuid uuid struct to which we will copy the memory
    * @return success of request
    */
-  bool getUUID(uuid_t uuid) {
+  bool getUUID(utils::Identifier &uuid) {
     return processor_->getUUID(uuid);
   }
+  /*
 
   unsigned const char *getUUID() {
     return processor_->getUUID();
-  }
+  }*/
   /**
    * Return the UUID string
    * @param constant reference to the UUID str

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/Property.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/Property.h b/libminifi/include/core/Property.h
index 1b1b587..ff1a536 100644
--- a/libminifi/include/core/Property.h
+++ b/libminifi/include/core/Property.h
@@ -65,7 +65,8 @@ class Property {
         valid_regex_(std::move(valid_regex)),
         dependent_properties_(std::move(dependent_properties)),
         exclusive_of_properties_(std::move(exclusive_of_properties)),
-        is_collection_(false) {
+        is_collection_(false),
+        supports_el_(false) {
     values_.push_back(std::move(value));
   }
 
@@ -73,7 +74,8 @@ class Property {
       : name_(name),
         description_(description),
         is_required_(false),
-        is_collection_(false) {
+        is_collection_(false),
+        supports_el_(false) {
     values_.push_back(std::move(value));
   }
 
@@ -81,7 +83,8 @@ class Property {
       : name_(name),
         description_(description),
         is_required_(false),
-        is_collection_(true) {
+        is_collection_(true),
+        supports_el_(false) {
   }
 
   Property(Property &&other)
@@ -94,7 +97,8 @@ class Property {
         is_collection_(other.is_collection_),
         values_(std::move(other.values_)),
         display_name_(std::move(other.display_name_)),
-        types_(std::move(other.types_)){
+        types_(std::move(other.types_)),
+        supports_el_(other.supports_el_) {
   }
 
   Property(const Property &other)
@@ -107,14 +111,16 @@ class Property {
         is_collection_(other.is_collection_),
         values_(other.values_),
         display_name_(other.display_name_),
-        types_(other.types_){
+        types_(other.types_),
+        supports_el_(other.supports_el_) {
   }
 
   Property()
       : name_(""),
         description_(""),
         is_required_(false),
-        is_collection_(false) {
+        is_collection_(false),
+        supports_el_(false) {
   }
 
   virtual ~Property() = default;
@@ -125,12 +131,14 @@ class Property {
   std::string getDescription() const;
   std::string getValue() const;
   bool getRequired() const;
+  bool supportsExpressionLangauge() const;
   std::string getValidRegex() const;
   std::vector<std::string> getDependentProperties() const;
   std::vector<std::pair<std::string, std::string>> getExclusiveOfProperties() const;
   std::vector<std::string> &getValues();
 
   void setValue(std::string value);
+  void setSupportsExpressionLanguage(bool supportEl);
   /**
    * Add value to the collection of values.
    */
@@ -414,6 +422,7 @@ class Property {
   // types represents the allowable types for this property
   // these types should be the canonical name.
   std::vector<std::string> types_;
+  bool supports_el_;
  private:
 
   friend class PropertyBuilder;
@@ -440,7 +449,12 @@ class PropertyBuilder : public std::enable_shared_from_this<PropertyBuilder> {
   }
 
   std::shared_ptr<PropertyBuilder> isRequired(bool required) {
-    prop.is_required_ = false;
+    prop.is_required_ = required;
+    return shared_from_this();
+  }
+
+  std::shared_ptr<PropertyBuilder> supportsExpressionLanguage(bool sel) {
+    prop.supports_el_ = sel;
     return shared_from_this();
   }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/Repository.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/Repository.h b/libminifi/include/core/Repository.h
index 427a46d..5dd9cc0 100644
--- a/libminifi/include/core/Repository.h
+++ b/libminifi/include/core/Repository.h
@@ -20,7 +20,6 @@
 #ifndef __REPOSITORY_H__
 #define __REPOSITORY_H__
 
-#include <ftw.h>
 #include <uuid/uuid.h>
 #include <atomic>
 #include <cstdint>

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/Resource.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/Resource.h b/libminifi/include/core/Resource.h
index 742d879..f72d1fe 100644
--- a/libminifi/include/core/Resource.h
+++ b/libminifi/include/core/Resource.h
@@ -18,6 +18,8 @@
 #ifndef LIBMINIFI_INCLUDE_CORE_RESOURCE_H_
 #define LIBMINIFI_INCLUDE_CORE_RESOURCE_H_
 
+#define WIN32_LEAN_AND_MEAN 1
+
 #include "ClassLoader.h"
 
 namespace org {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/SerializableComponent.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/SerializableComponent.h b/libminifi/include/core/SerializableComponent.h
index 2cf13b5..847dad3 100644
--- a/libminifi/include/core/SerializableComponent.h
+++ b/libminifi/include/core/SerializableComponent.h
@@ -35,7 +35,12 @@ class SerializableComponent : public core::Connectable, public minifi::io::Seria
 
  public:
 
-  SerializableComponent(const std::string name, uuid_t uuid = nullptr)
+  SerializableComponent(const std::string name)
+        : core::Connectable(name) {
+
+    }
+
+  SerializableComponent(const std::string name, utils::Identifier &uuid )
       : core::Connectable(name, uuid) {
 
   }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/controller/ControllerService.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/controller/ControllerService.h b/libminifi/include/core/controller/ControllerService.h
index b754aa9..7bb5170 100644
--- a/libminifi/include/core/controller/ControllerService.h
+++ b/libminifi/include/core/controller/ControllerService.h
@@ -63,7 +63,7 @@ class ControllerService : public ConfigurableComponent, public Connectable {
    * Controller Service constructor.
    */
   explicit ControllerService()
-      : Connectable(core::getClassName<ControllerService>(), 0),
+      : Connectable(core::getClassName<ControllerService>()),
         configuration_(std::make_shared<Configure>()) {
     current_state_ = DISABLED;
   }
@@ -72,24 +72,31 @@ class ControllerService : public ConfigurableComponent, public Connectable {
    * Controller Service constructor.
    */
   explicit ControllerService(const std::string &name, const std::string &id)
-      : Connectable(name, 0),
+      : Connectable(name),
         configuration_(std::make_shared<Configure>()) {
     current_state_ = DISABLED;
-    uuid_parse(id.c_str(), uuid_);
-    char uuidStr[37];
-    uuid_unparse_lower(uuid_, uuidStr);
-    uuidStr_ = uuidStr;
+    uuid_ = id;
+    uuidStr_ = id;
   }
 
   /**
    * Controller Service constructor.
    */
-  explicit ControllerService(const std::string &name, uuid_t uuid)
+  explicit ControllerService(const std::string &name, utils::Identifier uuid)
       : Connectable(name, uuid),
         configuration_(std::make_shared<Configure>()) {
     current_state_ = DISABLED;
   }
 
+  /**
+     * Controller Service constructor.
+     */
+    explicit ControllerService(const std::string &name)
+        : Connectable(name),
+          configuration_(std::make_shared<Configure>()) {
+      current_state_ = DISABLED;
+    }
+
   virtual void initialize() {
     // set base supported properties
     Property property("Linked Services", "Referenced Controller Services");

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/controller/ControllerServiceMap.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/controller/ControllerServiceMap.h b/libminifi/include/core/controller/ControllerServiceMap.h
index b5e3599..d229364 100644
--- a/libminifi/include/core/controller/ControllerServiceMap.h
+++ b/libminifi/include/core/controller/ControllerServiceMap.h
@@ -66,7 +66,7 @@ class ControllerServiceMap {
    *
    */
   virtual bool removeControllerService(const std::shared_ptr<ControllerServiceNode> &serviceNode) {
-    if (IsNullOrEmpty(serviceNode.get()))
+    if (serviceNode == nullptr || serviceNode.get() == nullptr)
       return false;
     std::lock_guard<std::mutex> lock(mutex_);
     controller_services_[serviceNode->getName()] = nullptr;
@@ -81,7 +81,7 @@ class ControllerServiceMap {
    *
    */
   virtual bool put(const std::string &id, const std::shared_ptr<ControllerServiceNode> &serviceNode) {
-    if (IsNullOrEmpty(id) || IsNullOrEmpty(serviceNode.get()))
+    if (id.empty() || serviceNode == nullptr || serviceNode.get() == nullptr)
       return false;
     std::lock_guard<std::mutex> lock(mutex_);
     controller_services_[id] = serviceNode;

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/controller/ControllerServiceNode.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/controller/ControllerServiceNode.h b/libminifi/include/core/controller/ControllerServiceNode.h
index 6da7537..0fd45e5 100644
--- a/libminifi/include/core/controller/ControllerServiceNode.h
+++ b/libminifi/include/core/controller/ControllerServiceNode.h
@@ -70,7 +70,7 @@ class ControllerServiceNode : public CoreComponent, public ConfigurableComponent
     controller_service_->setName(name);
   }
 
-  void setUUID(uuid_t uuid) {
+  void setUUID(utils::Identifier &uuid) {
     CoreComponent::setUUID(uuid);
     controller_service_->setUUID(uuid);
   }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/reporting/SiteToSiteProvenanceReportingTask.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/reporting/SiteToSiteProvenanceReportingTask.h b/libminifi/include/core/reporting/SiteToSiteProvenanceReportingTask.h
index 041e5ae..bf0fb01 100644
--- a/libminifi/include/core/reporting/SiteToSiteProvenanceReportingTask.h
+++ b/libminifi/include/core/reporting/SiteToSiteProvenanceReportingTask.h
@@ -45,7 +45,7 @@ class SiteToSiteProvenanceReportingTask : public minifi::RemoteProcessorGroupPor
    * Create a new processor
    */
   SiteToSiteProvenanceReportingTask(const std::shared_ptr<io::StreamFactory> &stream_factory, std::shared_ptr<Configure> configure)
-      : minifi::RemoteProcessorGroupPort(stream_factory, ReportTaskName, "", configure, NULL),
+      : minifi::RemoteProcessorGroupPort(stream_factory, ReportTaskName, "", configure),
         logger_(logging::LoggerFactory<SiteToSiteProvenanceReportingTask>::getLogger()) {
     this->setTriggerWhenEmpty(true);
     batch_size_ = 100;
@@ -69,8 +69,8 @@ class SiteToSiteProvenanceReportingTask : public minifi::RemoteProcessorGroupPor
   //! Initialize, over write by NiFi SiteToSiteProvenanceReportingTask
   virtual void initialize(void);
   //! Set Port UUID
-  void setPortUUID(uuid_t port_uuid) {
-    uuid_copy(protocol_uuid_, port_uuid);
+  void setPortUUID(utils::Identifier &port_uuid) {
+    protocol_uuid_ = port_uuid;
   }
 
   //! Set Batch Size
@@ -82,8 +82,8 @@ class SiteToSiteProvenanceReportingTask : public minifi::RemoteProcessorGroupPor
     return (batch_size_);
   }
   //! Get Port UUID
-  void getPortUUID(uuid_t port_uuid) {
-    uuid_copy(port_uuid, protocol_uuid_);
+  void getPortUUID(utils::Identifier & port_uuid) {
+    port_uuid = protocol_uuid_;
   }
 
  protected:

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/repository/VolatileContentRepository.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/repository/VolatileContentRepository.h b/libminifi/include/core/repository/VolatileContentRepository.h
index b26df0b..e203e96 100644
--- a/libminifi/include/core/repository/VolatileContentRepository.h
+++ b/libminifi/include/core/repository/VolatileContentRepository.h
@@ -43,7 +43,7 @@ class VolatileContentRepository : public core::ContentRepository, public virtual
   static const char *minimal_locking;
 
   explicit VolatileContentRepository(std::string name = getClassName<VolatileContentRepository>())
-      : core::SerializableComponent(name, 0),
+      : core::SerializableComponent(name),
         core::repository::VolatileRepository<std::shared_ptr<minifi::ResourceClaim>>(name),
         minimize_locking_(true),
         logger_(logging::LoggerFactory<VolatileContentRepository>::getLogger()) {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/repository/VolatileFlowFileRepository.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/repository/VolatileFlowFileRepository.h b/libminifi/include/core/repository/VolatileFlowFileRepository.h
index 1a6be6b..70affa7 100644
--- a/libminifi/include/core/repository/VolatileFlowFileRepository.h
+++ b/libminifi/include/core/repository/VolatileFlowFileRepository.h
@@ -37,7 +37,7 @@ class VolatileFlowFileRepository : public VolatileRepository<std::string> {
   explicit VolatileFlowFileRepository(std::string repo_name = "", std::string dir = REPOSITORY_DIRECTORY, int64_t maxPartitionMillis = MAX_REPOSITORY_ENTRY_LIFE_TIME, int64_t maxPartitionBytes =
   MAX_REPOSITORY_STORAGE_SIZE,
                                       uint64_t purgePeriod = REPOSITORY_PURGE_PERIOD)
-      : core::SerializableComponent(repo_name, 0),
+      : core::SerializableComponent(repo_name),
         VolatileRepository(repo_name.length() > 0 ? repo_name : core::getClassName<VolatileRepository>(), "", maxPartitionMillis, maxPartitionBytes, purgePeriod)
 
   {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/repository/VolatileProvenanceRepository.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/repository/VolatileProvenanceRepository.h b/libminifi/include/core/repository/VolatileProvenanceRepository.h
index cabb76f..9baff76 100644
--- a/libminifi/include/core/repository/VolatileProvenanceRepository.h
+++ b/libminifi/include/core/repository/VolatileProvenanceRepository.h
@@ -36,7 +36,7 @@ class VolatileProvenanceRepository : public VolatileRepository<std::string> {
   explicit VolatileProvenanceRepository(std::string repo_name = "", std::string dir = REPOSITORY_DIRECTORY, int64_t maxPartitionMillis = MAX_REPOSITORY_ENTRY_LIFE_TIME, int64_t maxPartitionBytes =
   MAX_REPOSITORY_STORAGE_SIZE,
                                         uint64_t purgePeriod = REPOSITORY_PURGE_PERIOD)
-      : core::SerializableComponent(repo_name, 0),VolatileRepository(repo_name.length() > 0 ? repo_name : core::getClassName<VolatileRepository>(), "", maxPartitionMillis, maxPartitionBytes, purgePeriod)
+      : core::SerializableComponent(repo_name),VolatileRepository(repo_name.length() > 0 ? repo_name : core::getClassName<VolatileRepository>(), "", maxPartitionMillis, maxPartitionBytes, purgePeriod)
 
   {
     purge_required_ = false;

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/repository/VolatileRepository.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/repository/VolatileRepository.h b/libminifi/include/core/repository/VolatileRepository.h
index 0f25b7f..4ba6fe7 100644
--- a/libminifi/include/core/repository/VolatileRepository.h
+++ b/libminifi/include/core/repository/VolatileRepository.h
@@ -56,7 +56,7 @@ class VolatileRepository : public core::Repository, public std::enable_shared_fr
   explicit VolatileRepository(std::string repo_name = "", std::string dir = REPOSITORY_DIRECTORY, int64_t maxPartitionMillis = MAX_REPOSITORY_ENTRY_LIFE_TIME, int64_t maxPartitionBytes =
   MAX_REPOSITORY_STORAGE_SIZE,
                               uint64_t purgePeriod = REPOSITORY_PURGE_PERIOD)
-      : core::SerializableComponent(repo_name, 0),
+      : core::SerializableComponent(repo_name),
         Repository(repo_name.length() > 0 ? repo_name : core::getClassName<VolatileRepository>(), "", maxPartitionMillis, maxPartitionBytes, purgePeriod),
         current_size_(0),
         current_index_(0),

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/state/UpdateController.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/state/UpdateController.h b/libminifi/include/core/state/UpdateController.h
index d74143a..fec3cc1 100644
--- a/libminifi/include/core/state/UpdateController.h
+++ b/libminifi/include/core/state/UpdateController.h
@@ -69,6 +69,10 @@ class UpdateStatus {
 class Update {
  public:
 
+	Update()
+		: status_(UpdateStatus(UpdateState::INITIATE, 0)) {
+	}
+
   Update(UpdateStatus status)
       : status_(status) {
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/state/nodes/AgentInformation.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/state/nodes/AgentInformation.h b/libminifi/include/core/state/nodes/AgentInformation.h
index 2f7d5a0..0a93149 100644
--- a/libminifi/include/core/state/nodes/AgentInformation.h
+++ b/libminifi/include/core/state/nodes/AgentInformation.h
@@ -19,10 +19,11 @@
 #define LIBMINIFI_INCLUDE_CORE_STATE_NODES_AGENTINFORMATION_H_
 
 #include "core/Resource.h"
+
+#ifndef WIN32
 #include <sys/socket.h>
 #include <netinet/in.h>
 #include <arpa/inet.h>
-#include <functional>
 #include <sys/ioctl.h>
 #if ( defined(__APPLE__) || defined(__MACH__) || defined(BSD)) 
 #include <net/if_dl.h>
@@ -32,13 +33,17 @@
 #include <net/if.h> 
 #include <unistd.h>
 #include <netinet/in.h>
-#include <string.h>
+
 #include <sys/socket.h>
 #include <netdb.h>
 #include <ifaddrs.h>
+#include <unistd.h>
+#endif
+#include <functional>
+#include <string.h>
 #include <stdio.h>
 #include <stdlib.h>
-#include <unistd.h>
+#include <sstream>
 #include <sstream>
 #include <map>
 #include "core/state/nodes/MetricsBase.h"
@@ -62,12 +67,12 @@ namespace response {
 
 class ComponentManifest : public DeviceInformation {
  public:
-  ComponentManifest(std::string name, uuid_t uuid)
+  ComponentManifest(std::string name, utils::Identifier & uuid)
       : DeviceInformation(name, uuid) {
   }
 
   ComponentManifest(const std::string &name)
-      : DeviceInformation(name, 0) {
+      : DeviceInformation(name) {
   }
 
   std::string getName() const {
@@ -116,6 +121,10 @@ class ComponentManifest : public DeviceInformation {
             descriptorDescription.name = "description";
             descriptorDescription.value = prop.second.getDescription();
 
+            SerializedResponseNode supportsExpressionLanguageScope;
+            supportsExpressionLanguageScope.name = "expressionLanguageScope";
+            supportsExpressionLanguageScope.value = prop.second.supportsExpressionLangauge() ? "FLOWFILE_ATTRIBUTES" : "NONE";
+
             SerializedResponseNode descriptorRequired;
             descriptorRequired.name = "required";
             descriptorRequired.value = prop.second.getRequired();
@@ -155,7 +164,7 @@ class ComponentManifest : public DeviceInformation {
                 SerializedResponseNode typeNode;
                 typeNode.name = "type";
                 std::string typeClazz = type;
-                utils::StringUtils::replaceAll(typeClazz,"::",".");
+                utils::StringUtils::replaceAll(typeClazz, "::", ".");
                 typeNode.value = typeClazz;
                 allowed_type.children.push_back(typeNode);
 
@@ -184,6 +193,7 @@ class ComponentManifest : public DeviceInformation {
             }
             child.children.push_back(descriptorDescription);
             child.children.push_back(descriptorRequired);
+            child.children.push_back(supportsExpressionLanguageScope);
             child.children.push_back(descriptorDefaultValue);
             child.children.push_back(descriptorValidRegex);
             child.children.push_back(descriptorDependentProperties);
@@ -240,13 +250,13 @@ class ComponentManifest : public DeviceInformation {
 
 class Bundles : public DeviceInformation {
  public:
-  Bundles(std::string name, uuid_t uuid)
+  Bundles(std::string name, utils::Identifier & uuid)
       : DeviceInformation(name, uuid) {
     setArray(true);
   }
 
   Bundles(const std::string &name)
-      : DeviceInformation(name, 0) {
+      : DeviceInformation(name) {
     setArray(true);
   }
 
@@ -275,7 +285,7 @@ class Bundles : public DeviceInformation {
       bundle.children.push_back(artifact);
       bundle.children.push_back(version);
 
-      ComponentManifest compMan(group, nullptr);
+      ComponentManifest compMan(group);
       // serialize the component information.
       for (auto component : compMan.serialize()) {
         bundle.children.push_back(component);
@@ -294,13 +304,13 @@ class Bundles : public DeviceInformation {
 class AgentStatus : public StateMonitorNode {
  public:
 
-  AgentStatus(std::string name, uuid_t uuid)
+  AgentStatus(std::string name, utils::Identifier & uuid)
       : StateMonitorNode(name, uuid) {
 
   }
 
   AgentStatus(const std::string &name)
-      : StateMonitorNode(name, 0) {
+      : StateMonitorNode(name) {
   }
 
   std::string getName() const {
@@ -423,13 +433,13 @@ class AgentMonitor {
 class AgentManifest : public DeviceInformation {
  public:
 
-  AgentManifest(std::string name, uuid_t uuid)
+  AgentManifest(std::string name, utils::Identifier & uuid)
       : DeviceInformation(name, uuid) {
     //setArray(true);
   }
 
   AgentManifest(const std::string &name)
-      : DeviceInformation(name, 0) {
+      : DeviceInformation(name) {
     //  setArray(true);
   }
 
@@ -485,7 +495,7 @@ class AgentManifest : public DeviceInformation {
     buildInfo.children.push_back(build_rev);
     buildInfo.children.push_back(build_date);
 
-    Bundles bundles("bundles", nullptr);
+    Bundles bundles("bundles");
 
     serialized.push_back(ident);
     serialized.push_back(type);
@@ -495,7 +505,7 @@ class AgentManifest : public DeviceInformation {
       serialized.push_back(bundle);
     }
 
-    SchedulingDefaults defaults("schedulingDefaults", nullptr);
+    SchedulingDefaults defaults("schedulingDefaults");
 
     for (auto defaultNode : defaults.serialize()) {
       serialized.push_back(defaultNode);
@@ -511,13 +521,13 @@ class AgentManifest : public DeviceInformation {
 class AgentInformation : public DeviceInformation, public AgentMonitor, public AgentIdentifier {
  public:
 
-  AgentInformation(std::string name, uuid_t uuid)
+  AgentInformation(std::string name, utils::Identifier & uuid)
       : DeviceInformation(name, uuid) {
     setArray(false);
   }
 
   AgentInformation(const std::string &name)
-      : DeviceInformation(name, 0) {
+      : DeviceInformation(name) {
     setArray(false);
   }
 
@@ -537,7 +547,7 @@ class AgentInformation : public DeviceInformation, public AgentMonitor, public A
     agentClass.name = "agentClass";
     agentClass.value = agent_class_;
 
-    AgentManifest manifest("manifest", nullptr);
+    AgentManifest manifest("manifest");
 
     SerializedResponseNode agentManifest;
     agentManifest.name = "agentManifest";

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/state/nodes/BuildInformation.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/state/nodes/BuildInformation.h b/libminifi/include/core/state/nodes/BuildInformation.h
index bd7e08c..7290a75 100644
--- a/libminifi/include/core/state/nodes/BuildInformation.h
+++ b/libminifi/include/core/state/nodes/BuildInformation.h
@@ -19,10 +19,11 @@
 #define LIBMINIFI_INCLUDE_CORE_STATE_METRICS_BuildInformation_H_
 
 #include "core/Resource.h"
+
+#ifndef WIN32
 #include <sys/socket.h>
 #include <netinet/in.h>
 #include <arpa/inet.h>
-#include <functional>
 #include <sys/ioctl.h>
 #if ( defined(__APPLE__) || defined(__MACH__) || defined(BSD)) 
 #include <net/if_dl.h>
@@ -32,13 +33,17 @@
 #include <net/if.h> 
 #include <unistd.h>
 #include <netinet/in.h>
-#include <string.h>
+
 #include <sys/socket.h>
 #include <netdb.h>
 #include <ifaddrs.h>
+#include <unistd.h>
+#endif
+#include <functional>
+#include <string.h>
 #include <stdio.h>
 #include <stdlib.h>
-#include <unistd.h>
+
 #include <sstream>
 #include <map>
 #include "../nodes/MetricsBase.h"
@@ -61,12 +66,12 @@ namespace response {
 class BuildInformation : public DeviceInformation {
  public:
 
-  BuildInformation(std::string name, uuid_t uuid)
+  BuildInformation(std::string name, utils::Identifier &uuid)
       : DeviceInformation(name, uuid) {
   }
 
   BuildInformation(const std::string &name)
-      : DeviceInformation(name, 0) {
+      : DeviceInformation(name) {
   }
 
   std::string getName() const {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/state/nodes/DeviceInformation.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/state/nodes/DeviceInformation.h b/libminifi/include/core/state/nodes/DeviceInformation.h
index dfeb265..9859972 100644
--- a/libminifi/include/core/state/nodes/DeviceInformation.h
+++ b/libminifi/include/core/state/nodes/DeviceInformation.h
@@ -19,29 +19,31 @@
 #define LIBMINIFI_INCLUDE_CORE_STATE_NODES_DEVICEINFORMATION_H_
 
 #include "core/Resource.h"
+
+#ifndef WIN32
 #include <sys/socket.h>
 #include <netinet/in.h>
 #include <arpa/inet.h>
-#include <functional>
 #include <sys/ioctl.h>
+#include <sys/utsname.h>
 #if ( defined(__APPLE__) || defined(__MACH__) || defined(BSD)) 
 #include <net/if_dl.h>
 #include <net/if_types.h>
 #endif
-#ifndef _WIN32
-#include <sys/utsname.h>
-#endif
 #include <ifaddrs.h>
 #include <net/if.h> 
 #include <unistd.h>
 #include <netinet/in.h>
-#include <string.h>
+
 #include <sys/socket.h>
 #include <netdb.h>
 #include <ifaddrs.h>
+#include <unistd.h>
+#endif
+#include <functional>
+#include <string.h>
 #include <stdio.h>
 #include <stdlib.h>
-#include <unistd.h>
 #include <sstream>
 #include <map>
 #include "../nodes/MetricsBase.h"
@@ -109,6 +111,7 @@ class Device {
 
   std::vector<std::string> getIpAddresses() {
     std::vector<std::string> ips;
+#ifndef WIN32
     struct ifaddrs *ifaddr, *ifa;
     if (getifaddrs(&ifaddr) == -1) {
       perror("getifaddrs");
@@ -124,6 +127,7 @@ class Device {
     }
 
     freeifaddrs(ifaddr);
+#endif
     return ips;
   }
 
@@ -254,7 +258,7 @@ class Device {
 class DeviceInfoNode : public DeviceInformation {
  public:
 
-  DeviceInfoNode(std::string name, uuid_t uuid)
+  DeviceInfoNode(std::string name, utils::Identifier &  uuid)
       : DeviceInformation(name, uuid) {
     static Device device;
     hostname_ = device.canonical_hostname_;
@@ -263,7 +267,7 @@ class DeviceInfoNode : public DeviceInformation {
   }
 
   DeviceInfoNode(const std::string &name)
-      : DeviceInformation(name, 0) {
+      : DeviceInformation(name) {
     static Device device;
     hostname_ = device.canonical_hostname_;
     ip_ = device.ip_;
@@ -291,16 +295,18 @@ class DeviceInfoNode : public DeviceInformation {
     vcores.value = std::to_string(ncpus);
 
     systemInfo.children.push_back(vcores);
-
+#if defined(_SC_PHYS_PAGES) && defined(_SC_PAGESIZE)
     SerializedResponseNode mem;
     mem.name = "physicalMem";
-#if defined(_SC_PHYS_PAGES) && defined(_SC_PAGESIZE)
+
     size_t mema = (size_t) sysconf( _SC_PHYS_PAGES) * (size_t) sysconf( _SC_PAGESIZE);
-#endif
+
     mem.value = std::to_string(mema);
 
-    systemInfo.children.push_back(mem);
 
+    systemInfo.children.push_back(mem);
+#endif
+#ifndef WIN32
     SerializedResponseNode arch;
     arch.name = "machinearch";
 
@@ -313,7 +319,7 @@ class DeviceInfoNode : public DeviceInformation {
     }
 
     systemInfo.children.push_back(arch);
-
+#endif
     serialized.push_back(identifier);
     serialized.push_back(systemInfo);
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/state/nodes/FlowInformation.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/state/nodes/FlowInformation.h b/libminifi/include/core/state/nodes/FlowInformation.h
index 022b9ce..4560ed3 100644
--- a/libminifi/include/core/state/nodes/FlowInformation.h
+++ b/libminifi/include/core/state/nodes/FlowInformation.h
@@ -19,26 +19,14 @@
 #define LIBMINIFI_INCLUDE_CORE_STATE_NODES_FLOWINFORMATION_H_
 
 #include "core/Resource.h"
-#include <sys/socket.h>
-#include <netinet/in.h>
-#include <arpa/inet.h>
 #include <functional>
-#include <sys/ioctl.h>
 #if ( defined(__APPLE__) || defined(__MACH__) || defined(BSD)) 
 #include <net/if_dl.h>
 #include <net/if_types.h>
 #endif
-#include <ifaddrs.h>
-#include <net/if.h> 
-#include <unistd.h>
-#include <netinet/in.h>
 #include <string.h>
-#include <sys/socket.h>
-#include <netdb.h>
-#include <ifaddrs.h>
 #include <stdio.h>
 #include <stdlib.h>
-#include <unistd.h>
 #include <sstream>
 #include <map>
 #include "../nodes/MetricsBase.h"
@@ -58,17 +46,17 @@ class FlowVersion : public DeviceInformation {
  public:
 
   explicit FlowVersion()
-      : DeviceInformation("FlowVersion", nullptr) {
+      : DeviceInformation("FlowVersion") {
     setFlowVersion("", "", getUUIDStr());
   }
 
   explicit FlowVersion(const std::string &registry_url, const std::string &bucket_id, const std::string &flow_id)
-      : DeviceInformation("FlowVersion", nullptr) {
+      : DeviceInformation("FlowVersion") {
     setFlowVersion(registry_url, bucket_id, flow_id.empty() ? getUUIDStr() : flow_id);
   }
 
   explicit FlowVersion(FlowVersion &&fv)
-      : DeviceInformation("FlowVersion", nullptr),
+      : DeviceInformation("FlowVersion"),
         identifier(std::move(fv.identifier)) {
   }
 
@@ -144,12 +132,12 @@ class FlowVersion : public DeviceInformation {
 class FlowMonitor : public StateMonitorNode {
  public:
 
-  FlowMonitor(std::string name, uuid_t uuid)
+  FlowMonitor(const std::string &name, utils::Identifier &uuid)
       : StateMonitorNode(name, uuid) {
   }
 
   FlowMonitor(const std::string &name)
-      : StateMonitorNode(name, 0) {
+      : StateMonitorNode(name) {
   }
 
   void addConnection(const std::shared_ptr<minifi::Connection> &connection) {
@@ -173,12 +161,12 @@ class FlowMonitor : public StateMonitorNode {
 class FlowInformation : public FlowMonitor {
  public:
 
-  FlowInformation(std::string name, uuid_t uuid)
+  FlowInformation(const std::string &name, utils::Identifier &uuid)
       : FlowMonitor(name, uuid) {
   }
 
   FlowInformation(const std::string &name)
-      : FlowMonitor(name, 0) {
+      : FlowMonitor(name) {
   }
 
   std::string getName() const {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/state/nodes/MetricsBase.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/state/nodes/MetricsBase.h b/libminifi/include/core/state/nodes/MetricsBase.h
index 6e5aaf6..f4d585e 100644
--- a/libminifi/include/core/state/nodes/MetricsBase.h
+++ b/libminifi/include/core/state/nodes/MetricsBase.h
@@ -32,19 +32,22 @@ namespace minifi {
 namespace state {
 namespace response {
 
-
-
 /**
  * Purpose: Defines a metric. Serialization is intended to be thread safe.
  */
 class ResponseNode : public core::Connectable {
  public:
   ResponseNode()
-      : core::Connectable("metric", 0),
+      : core::Connectable("metric"),
+        is_array_(false) {
+  }
+
+  ResponseNode(std::string name)
+      : core::Connectable(name),
         is_array_(false) {
   }
 
-  ResponseNode(std::string name, uuid_t uuid)
+  ResponseNode(std::string name, utils::Identifier & uuid)
       : core::Connectable(name, uuid),
         is_array_(false) {
   }
@@ -67,7 +70,7 @@ class ResponseNode : public core::Connectable {
     return is_array_;
   }
 
-  virtual bool isEmpty(){
+  virtual bool isEmpty() {
     return false;
   }
 
@@ -86,9 +89,12 @@ class ResponseNode : public core::Connectable {
  */
 class DeviceInformation : public ResponseNode {
  public:
-  DeviceInformation(std::string name, uuid_t uuid)
+  DeviceInformation(std::string name, utils::Identifier & uuid)
       : ResponseNode(name, uuid) {
   }
+  DeviceInformation(std::string name)
+      : ResponseNode(name) {
+  }
 };
 
 /**
@@ -96,7 +102,7 @@ class DeviceInformation : public ResponseNode {
  */
 class ObjectNode : public ResponseNode {
  public:
-  ObjectNode(std::string name, uuid_t uuid)
+  ObjectNode(std::string name, utils::Identifier uuid = utils::Identifier())
       : ResponseNode(name, uuid) {
   }
 
@@ -111,7 +117,7 @@ class ObjectNode : public ResponseNode {
   virtual std::vector<SerializedResponseNode> serialize() {
     std::vector<SerializedResponseNode> serialized;
 //    SerializedResponseNode outer_node;
-  //  outer_node.name = getName();
+    //  outer_node.name = getName();
     for (auto &node : nodes_) {
       SerializedResponseNode inner_node;
       inner_node.name = node->getName();
@@ -120,11 +126,11 @@ class ObjectNode : public ResponseNode {
       }
       serialized.push_back(std::move(inner_node));
     }
-   //serialized.push_back(std::move(outer_node));
+    //serialized.push_back(std::move(outer_node));
     return serialized;
   }
 
-  virtual bool isEmpty(){
+  virtual bool isEmpty() {
     return nodes_.empty();
   }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/state/nodes/ProcessMetrics.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/state/nodes/ProcessMetrics.h b/libminifi/include/core/state/nodes/ProcessMetrics.h
index ebef0d1..7d210cc 100644
--- a/libminifi/include/core/state/nodes/ProcessMetrics.h
+++ b/libminifi/include/core/state/nodes/ProcessMetrics.h
@@ -21,8 +21,9 @@
 #include "core/Resource.h"
 #include <sstream>
 #include <map>
-#include <sys/time.h>
+#ifndef WIN32
 #include <sys/resource.h>
+#endif
 
 #include "../nodes/DeviceInformation.h"
 #include "../nodes/MetricsBase.h"
@@ -42,12 +43,12 @@ namespace response {
 class ProcessMetrics : public ResponseNode {
  public:
 
-  ProcessMetrics(const std::string &name, uuid_t uuid)
+  ProcessMetrics(const std::string &name, utils::Identifier &uuid)
       : ResponseNode(name, uuid) {
   }
 
   ProcessMetrics(const std::string &name)
-      : ResponseNode(name, 0) {
+      : ResponseNode(name) {
   }
 
   ProcessMetrics() {
@@ -60,6 +61,7 @@ class ProcessMetrics : public ResponseNode {
   std::vector<SerializedResponseNode> serialize() {
     std::vector<SerializedResponseNode> serialized;
 
+#ifndef WIN32
     struct rusage my_usage;
     getrusage(RUSAGE_SELF, &my_usage);
 
@@ -84,6 +86,7 @@ class ProcessMetrics : public ResponseNode {
     cpu.children.push_back(ics);
     serialized.push_back(cpu);
 
+#endif
     return serialized;
   }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/state/nodes/QueueMetrics.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/state/nodes/QueueMetrics.h b/libminifi/include/core/state/nodes/QueueMetrics.h
index b4c0b64..4b79796 100644
--- a/libminifi/include/core/state/nodes/QueueMetrics.h
+++ b/libminifi/include/core/state/nodes/QueueMetrics.h
@@ -38,16 +38,16 @@ namespace response {
 class QueueMetrics : public ResponseNode {
  public:
 
-  QueueMetrics(const std::string &name, uuid_t uuid)
+  QueueMetrics(const std::string &name, utils::Identifier &  uuid)
       : ResponseNode(name, uuid) {
   }
 
   QueueMetrics(const std::string &name)
-      : ResponseNode(name, 0) {
+      : ResponseNode(name) {
   }
 
   QueueMetrics()
-      : ResponseNode("QueueMetrics", 0) {
+      : ResponseNode("QueueMetrics") {
   }
 
   virtual std::string getName() const{

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/state/nodes/RepositoryMetrics.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/state/nodes/RepositoryMetrics.h b/libminifi/include/core/state/nodes/RepositoryMetrics.h
index 5c2db2a..a375b17 100644
--- a/libminifi/include/core/state/nodes/RepositoryMetrics.h
+++ b/libminifi/include/core/state/nodes/RepositoryMetrics.h
@@ -38,16 +38,16 @@ namespace response {
 class RepositoryMetrics : public ResponseNode {
  public:
 
-  RepositoryMetrics(const std::string &name, uuid_t uuid)
+  RepositoryMetrics(const std::string &name, utils::Identifier &uuid)
       : ResponseNode(name, uuid) {
   }
 
   RepositoryMetrics(const std::string &name)
-      : ResponseNode(name, 0) {
+      : ResponseNode(name) {
   }
 
   RepositoryMetrics()
-      : ResponseNode("RepositoryMetrics", 0) {
+      : ResponseNode("RepositoryMetrics") {
   }
 
   virtual std::string getName() const {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/state/nodes/SchedulingNodes.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/state/nodes/SchedulingNodes.h b/libminifi/include/core/state/nodes/SchedulingNodes.h
index c1d18bc..edad122 100644
--- a/libminifi/include/core/state/nodes/SchedulingNodes.h
+++ b/libminifi/include/core/state/nodes/SchedulingNodes.h
@@ -32,12 +32,12 @@ namespace response {
 
 class SchedulingDefaults : public DeviceInformation {
  public:
-  SchedulingDefaults(std::string name, uuid_t uuid)
+  SchedulingDefaults(std::string name, utils::Identifier &uuid)
       : DeviceInformation(name, uuid) {
   }
 
   SchedulingDefaults(const std::string &name)
-      : DeviceInformation(name, 0) {
+      : DeviceInformation(name) {
   }
 
   std::string getName() const {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/state/nodes/StateMonitor.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/state/nodes/StateMonitor.h b/libminifi/include/core/state/nodes/StateMonitor.h
index 1bf3e23..53cebdd 100644
--- a/libminifi/include/core/state/nodes/StateMonitor.h
+++ b/libminifi/include/core/state/nodes/StateMonitor.h
@@ -19,26 +19,9 @@
 #define LIBMINIFI_INCLUDE_CORE_STATE_NODES_STATEMONITOR_H_
 
 #include "core/Resource.h"
-#include <sys/socket.h>
-#include <netinet/in.h>
-#include <arpa/inet.h>
-#include <functional>
-#include <sys/ioctl.h>
-#if ( defined(__APPLE__) || defined(__MACH__) || defined(BSD)) 
-#include <net/if_dl.h>
-#include <net/if_types.h>
-#endif
-#include <ifaddrs.h>
-#include <net/if.h> 
-#include <unistd.h>
-#include <netinet/in.h>
-#include <string.h>
-#include <sys/socket.h>
-#include <netdb.h>
-#include <ifaddrs.h>
 #include <stdio.h>
 #include <stdlib.h>
-#include <unistd.h>
+
 #include <sstream>
 #include <map>
 #include "../nodes/MetricsBase.h"
@@ -59,14 +42,14 @@ namespace response {
 class StateMonitorNode : public DeviceInformation {
  public:
 
-  StateMonitorNode(std::string name, uuid_t uuid)
+  StateMonitorNode(std::string name, utils::Identifier &uuid)
       : DeviceInformation(name, uuid),
         monitor_(nullptr) {
 
   }
 
   StateMonitorNode(const std::string &name)
-      : DeviceInformation(name, 0),
+      : DeviceInformation(name),
         monitor_(nullptr) {
   }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/state/nodes/SystemMetrics.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/state/nodes/SystemMetrics.h b/libminifi/include/core/state/nodes/SystemMetrics.h
index a7a3c9c..5668e8f 100644
--- a/libminifi/include/core/state/nodes/SystemMetrics.h
+++ b/libminifi/include/core/state/nodes/SystemMetrics.h
@@ -41,16 +41,16 @@ namespace response {
 class SystemInformation : public DeviceInformation {
  public:
 
-  SystemInformation(const std::string &name, uuid_t uuid)
+  SystemInformation(const std::string &name, utils::Identifier &  uuid)
       : DeviceInformation(name, uuid) {
   }
 
   SystemInformation(const std::string &name)
-      : DeviceInformation(name, 0) {
+      : DeviceInformation(name) {
   }
 
   SystemInformation()
-      : DeviceInformation("systeminfo", 0) {
+      : DeviceInformation("systeminfo") {
   }
 
   virtual std::string getName() const {
@@ -62,7 +62,7 @@ class SystemInformation : public DeviceInformation {
     SerializedResponseNode identifier;
     identifier.name = "identifier";
     identifier.value = "identifier";
-
+#ifndef WIN32
     SerializedResponseNode systemInfo;
     systemInfo.name = "systemInfo";
 
@@ -94,10 +94,11 @@ class SystemInformation : public DeviceInformation {
       arch.value = std::string(buf.machine);
     }
 
-    systemInfo.children.push_back(arch);
-
+	    systemInfo.children.push_back(arch);
+		serialized.push_back(systemInfo);
+#endif
     serialized.push_back(identifier);
-    serialized.push_back(systemInfo);
+    
 
     return serialized;
   }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/core/yaml/YamlConfiguration.h
----------------------------------------------------------------------
diff --git a/libminifi/include/core/yaml/YamlConfiguration.h b/libminifi/include/core/yaml/YamlConfiguration.h
index 6e22fe3..b60d384 100644
--- a/libminifi/include/core/yaml/YamlConfiguration.h
+++ b/libminifi/include/core/yaml/YamlConfiguration.h
@@ -34,7 +34,7 @@ namespace nifi {
 namespace minifi {
 namespace core {
 
-#define DEFAULT_FLOW_YAML_FILE_NAME "conf/flow.yml"
+#define DEFAULT_FLOW_YAML_FILE_NAME "conf/config.yml"
 #define CONFIG_YAML_FLOW_CONTROLLER_KEY "Flow Controller"
 #define CONFIG_YAML_PROCESSORS_KEY "Processors"
 #define CONFIG_YAML_CONNECTIONS_KEY "Connections"

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/io/CRCStream.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/CRCStream.h b/libminifi/include/io/CRCStream.h
index 0ca00ba..95e3a11 100644
--- a/libminifi/include/io/CRCStream.h
+++ b/libminifi/include/io/CRCStream.h
@@ -19,9 +19,12 @@
 #define LIBMINIFI_INCLUDE_IO_CRCSTREAM_H_
 
 #include <zlib.h>
-#include <arpa/inet.h>
 #include <memory>
-
+#ifdef WIN32
+#include <winsock2.h>
+#else
+#include <arpa/inet.h>
+#endif
 #include "BaseStream.h"
 #include "Serializable.h"
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/io/ClientSocket.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/ClientSocket.h b/libminifi/include/io/ClientSocket.h
deleted file mode 100644
index e4b7604..0000000
--- a/libminifi/include/io/ClientSocket.h
+++ /dev/null
@@ -1,295 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-#ifndef LIBMINIFI_INCLUDE_IO_CLIENTSOCKET_H_
-#define LIBMINIFI_INCLUDE_IO_CLIENTSOCKET_H_
-
-#include <cstdint>
-#include <sys/types.h>
-#include <sys/socket.h>
-#include <netdb.h>
-#include <unistd.h>
-#include <mutex>
-#include <atomic>
-#include "io/BaseStream.h"
-#include "core/Core.h"
-#include "core/logging/Logger.h"
-#include "io/validation.h"
-#include "properties/Configure.h"
-#include "io/NetworkPrioritizer.h"
-
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
-namespace io {
-
-/**
- * Context class for socket. This is currently only used as a parent class for TLSContext.  It is necessary so the Socket and TLSSocket constructors
- * can be the same.  It also gives us a common place to set timeouts, etc from the Configure object in the future.
- */
-class SocketContext {
- public:
-  SocketContext(const std::shared_ptr<Configure> &configure) {
-  }
-};
-/**
- * Socket class.
- * Purpose: Provides a general purpose socket interface that abstracts
- * connecting information from users
- * Design: Extends DataStream and allows us to perform most streaming
- * operations against a BSD socket
- *
- *
- */
-class Socket : public BaseStream {
- public:
-  /**
-   * Constructor that creates a client socket.
-   * @param context the SocketContext
-   * @param hostname hostname we are connecting to.
-   * @param port port we are connecting to.
-   */
-  explicit Socket(const std::shared_ptr<SocketContext> &context, const std::string &hostname, const uint16_t port);
-
-  /**
-   * Move constructor.
-   */
-  explicit Socket(const Socket &&);
-
-  /**
-   * Static function to return the current machine's host name
-   */
-  static std::string getMyHostName() {
-    static std::string HOSTNAME = init_hostname();
-    return HOSTNAME;
-  }
-
-  /**
-   * Destructor
-   */
-
-  virtual ~Socket();
-
-  virtual void closeStream();
-  /**
-   * Initializes the socket
-   * @return result of the creation operation.
-   */
-  virtual int16_t initialize();
-
-  virtual void setInterface(io::NetworkInterface &&interface) {
-    local_network_interface_ = std::move(interface);
-  }
-
-  /**
-   * Sets the non blocking flag on the file descriptor.
-   */
-  void setNonBlocking();
-
-  std::string getHostname() const;
-
-  /**
-   * Return the port for this socket
-   * @returns port
-   */
-  uint16_t getPort();
-
-  // data stream extensions
-  /**
-   * Reads data and places it into buf
-   * @param buf buffer in which we extract data
-   * @param buflen
-   * @param retrieve_all_bytes determines if we should read all bytes before returning
-   */
-  virtual int readData(std::vector<uint8_t> &buf, int buflen) {
-    return readData(buf, buflen, true);
-  }
-  /**
-   * Reads data and places it into buf
-   * @param buf buffer in which we extract data
-   * @param buflen
-   * @param retrieve_all_bytes determines if we should read all bytes before returning
-   */
-  virtual int readData(uint8_t *buf, int buflen) {
-    return readData(buf, buflen, true);
-  }
-
-  /**
-   * Reads data and places it into buf
-   * @param buf buffer in which we extract data
-   * @param buflen
-   * @param retrieve_all_bytes determines if we should read all bytes before returning
-   */
-  virtual int readData(std::vector<uint8_t> &buf, int buflen, bool retrieve_all_bytes);
-  /**
-   * Reads data and places it into buf
-   * @param buf buffer in which we extract data
-   * @param buflen
-   * @param retrieve_all_bytes determines if we should read all bytes before returning
-   */
-  virtual int readData(uint8_t *buf, int buflen, bool retrieve_all_bytes);
-
-  /**
-   * Write value to the stream using std::vector
-   * @param buf incoming buffer
-   * @param buflen buffer to write
-   *
-   */
-  virtual int writeData(std::vector<uint8_t> &buf, int buflen);
-
-  /**
-   * writes value to stream
-   * @param value value to write
-   * @param size size of value
-   */
-  virtual int writeData(uint8_t *value, int size);
-
-  /**
-   * Writes a system word
-   * @param value value to write
-   */
-  virtual int write(uint64_t value, bool is_little_endian = EndiannessCheck::IS_LITTLE);
-
-  /**
-   * Writes a uint32_t
-   * @param value value to write
-   */
-  virtual int write(uint32_t value, bool is_little_endian = EndiannessCheck::IS_LITTLE);
-
-  /**
-   * Writes a system short
-   * @param value value to write
-   */
-  virtual int write(uint16_t value, bool is_little_endian = EndiannessCheck::IS_LITTLE);
-
-  /**
-   * Reads a system word
-   * @param value value to write
-   */
-  virtual int read(uint64_t &value, bool is_little_endian = EndiannessCheck::IS_LITTLE);
-
-  /**
-   * Reads a uint32_t
-   * @param value value to write
-   */
-  virtual int read(uint32_t &value, bool is_little_endian = EndiannessCheck::IS_LITTLE);
-
-  /**
-   * Reads a system short
-   * @param value value to write
-   */
-  virtual int read(uint16_t &value, bool is_little_endian = EndiannessCheck::IS_LITTLE);
-
-  /**
-   * Returns the underlying buffer
-   * @return vector's array
-   **/
-  const uint8_t *getBuffer() const {
-    return DataStream::getBuffer();
-  }
-
-  /**
-   * Retrieve size of data stream
-   * @return size of data stream
-   **/
-  const uint64_t getSize() const {
-    return DataStream::getSize();
-  }
-
- protected:
-
-  /**
-   * Constructor that accepts host name, port and listeners. With this
-   * contructor we will be creating a server socket
-   * @param context the SocketContext
-   * @param hostname our host name
-   * @param port connecting port
-   * @param listeners number of listeners in the queue
-   */
-  explicit Socket(const std::shared_ptr<SocketContext> &context, const std::string &hostname, const uint16_t port, const uint16_t listeners);
-
-  /**
-   * Creates a vector and returns the vector using the provided
-   * type name.
-   * @param t incoming object
-   * @returns vector.
-   */
-  template<typename T>
-  std::vector<uint8_t> readBuffer(const T&);
-
-  /**
-   * Creates a connection using the address info object.
-   * @param p addrinfo structure.
-   * @returns fd.
-   */
-  virtual int8_t createConnection(const addrinfo *p, in_addr_t &addr);
-
-  /**
-   * Sets socket options depending on the instance.
-   * @param sock socket file descriptor.
-   */
-  virtual int16_t setSocketOptions(const int sock);
-
-  /**
-   * Attempt to select the socket file descriptor
-   * @param msec timeout interval to wait
-   * @returns file descriptor
-   */
-  virtual int16_t select_descriptor(const uint16_t msec);
-
-  addrinfo *addr_info_;
-
-  std::recursive_mutex selection_mutex_;
-
-  std::string requested_hostname_;
-  std::string canonical_hostname_;
-  uint16_t port_;
-
-  bool is_loopback_only_;
-  io::NetworkInterface local_network_interface_;
-
-  // connection information
-  int32_t socket_file_descriptor_;
-
-  fd_set total_list_;
-  fd_set read_fds_;
-  std::atomic<uint16_t> socket_max_;
-  std::atomic<uint64_t> total_written_;
-  std::atomic<uint64_t> total_read_;
-  uint16_t listeners_;
-
-
-  bool nonBlocking_;
- private:
-  std::shared_ptr<logging::Logger> logger_;
-  static std::string init_hostname() {
-    char hostname[1024];
-    gethostname(hostname, 1024);
-    Socket mySock(nullptr, hostname, 0);
-    mySock.initialize();
-    auto resolved_hostname = mySock.getHostname();
-    return !IsNullOrEmpty(resolved_hostname) ? resolved_hostname : hostname;
-  }
-};
-
-} /* namespace io */
-} /* namespace minifi */
-} /* namespace nifi */
-} /* namespace apache */
-} /* namespace org */
-#endif /* LIBMINIFI_INCLUDE_IO_CLIENTSOCKET_H_ */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/io/ServerSocket.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/ServerSocket.h b/libminifi/include/io/ServerSocket.h
index 3af2e4f..fefea0f 100644
--- a/libminifi/include/io/ServerSocket.h
+++ b/libminifi/include/io/ServerSocket.h
@@ -18,7 +18,7 @@
 #ifndef LIBMINIFI_INCLUDE_IO_SERVERSOCKET_H_
 #define LIBMINIFI_INCLUDE_IO_SERVERSOCKET_H_
 
-#include "ClientSocket.h"
+#include "io/ClientSocket.h"
 
 namespace org {
 namespace apache {

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/io/Sockets.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/Sockets.h b/libminifi/include/io/Sockets.h
index 4552fe5..361988c 100644
--- a/libminifi/include/io/Sockets.h
+++ b/libminifi/include/io/Sockets.h
@@ -18,7 +18,7 @@
 #ifndef LIBMINIFI_INCLUDE_IO_SOCKET_H_
 #define LIBMINIFI_INCLUDE_IO_SOCKET_H_
 
-#include "ClientSocket.h"
+#include "io/ClientSocket.h"
 #include "ServerSocket.h"
 
 #ifdef OPENSSL_SUPPORT

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/io/validation.h
----------------------------------------------------------------------
diff --git a/libminifi/include/io/validation.h b/libminifi/include/io/validation.h
index 1cce663..40dd285 100644
--- a/libminifi/include/io/validation.h
+++ b/libminifi/include/io/validation.h
@@ -18,6 +18,7 @@
 
 #ifndef VALIDATION_H
 #define VALIDATION_H
+#include <functional>
 #include <type_traits>
 #include <string>
 #include <cstring>
@@ -41,14 +42,41 @@ class size_function_functor_checker {
   };
 };
 
+
+
+#ifdef WIN32
+
+static auto IsNullOrEmpty(std::string object) {
+	return object.empty();
+}
+
 /**
- * Determines if the variable is null or ::size() == 0
- */
+* Determines if the variable is null or ::size() == 0
+*/
+template<typename T>
+static auto IsNullOrEmpty(T *object) {
+	return (nullptr == object);
+}
+
+
+/**
+* Determines if the variable is null or ::size() == 0
+*/
 template<typename T>
-static auto IsNullOrEmpty(T &object) -> typename std::enable_if<size_function_functor_checker<T>::has_size_function==1, bool>::type {
-  return object.size() == 0;
+static auto IsNullOrEmpty(std::shared_ptr<T> object){
+	return (nullptr == object || nullptr == object.get());
 }
 
+
+#else
+
+/**
+* Determines if the variable is null or ::size() == 0
+*/
+template<typename T>
+static auto IsNullOrEmpty(T &object) -> typename std::enable_if<size_function_functor_checker<T>::has_size_function == 1, bool>::type {
+	return object.size() == 0;
+}
 /**
  * Determines if the variable is null or ::size() == 0
  */
@@ -65,12 +93,15 @@ static auto IsNullOrEmpty(T *object) -> typename std::enable_if<not size_functio
   return (nullptr == object);
 }
 
+
 /**
- * Determines if the variable is null or ::size() == 0
- */
+* Determines if the variable is null or ::size() == 0
+*/
 template<typename T>
-static auto IsNullOrEmpty(std::shared_ptr<T> object) -> typename std::enable_if<not size_function_functor_checker<T>::has_size_function , bool>::type {
-  return (nullptr == object || nullptr == object.get());
+static auto IsNullOrEmpty(std::shared_ptr<T> object) -> typename std::enable_if<not size_function_functor_checker<T>::has_size_function, bool>::type {
+	return (nullptr == object || nullptr == object.get());
 }
 
 #endif
+
+#endif

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/processors/AppendHostInfo.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/AppendHostInfo.h b/libminifi/include/processors/AppendHostInfo.h
index e7c90ac..17f6389 100644
--- a/libminifi/include/processors/AppendHostInfo.h
+++ b/libminifi/include/processors/AppendHostInfo.h
@@ -41,7 +41,7 @@ class AppendHostInfo : public core::Processor {
   /*!
    * Create a new processor
    */
-  AppendHostInfo(std::string name, uuid_t uuid = NULL)
+  AppendHostInfo(std::string name, utils::Identifier uuid = utils::Identifier())
       : core::Processor(name, uuid),
         logger_(logging::LoggerFactory<AppendHostInfo>::getLogger()) {
   }

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/processors/ExecuteProcess.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/ExecuteProcess.h b/libminifi/include/processors/ExecuteProcess.h
index 8cc7a25..cd3a371 100644
--- a/libminifi/include/processors/ExecuteProcess.h
+++ b/libminifi/include/processors/ExecuteProcess.h
@@ -21,16 +21,18 @@
 #define __EXECUTE_PROCESS_H__
 
 #include <stdio.h>
-#include <unistd.h>
 #include <string>
 #include <errno.h>
 #include <chrono>
 #include <thread>
-#include <unistd.h>
-#include <sys/wait.h>
 #include <iostream>
 #include <sys/types.h>
 #include <signal.h>
+#ifndef WIN32
+#include <sys/wait.h>
+#include <sys/types.h>
+#include <signal.h>
+#endif
 #include "io/BaseStream.h"
 #include "FlowFileRecord.h"
 #include "core/Processor.h"
@@ -44,6 +46,7 @@ namespace apache {
 namespace nifi {
 namespace minifi {
 namespace processors {
+#ifndef WIN32
 
 // ExecuteProcess Class
 class ExecuteProcess : public core::Processor {
@@ -52,7 +55,7 @@ class ExecuteProcess : public core::Processor {
   /*!
    * Create a new processor
    */
-  ExecuteProcess(std::string name, uuid_t uuid = NULL)
+  ExecuteProcess(std::string name,  utils::Identifier uuid = utils::Identifier())
       : Processor(name, uuid),
         logger_(logging::LoggerFactory<ExecuteProcess>::getLogger()) {
     _redirectErrorStream = false;
@@ -120,7 +123,7 @@ class ExecuteProcess : public core::Processor {
 };
 
 REGISTER_RESOURCE(ExecuteProcess);
-
+#endif
 } /* namespace processors */
 } /* namespace minifi */
 } /* namespace nifi */

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/processors/ExtractText.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/ExtractText.h b/libminifi/include/processors/ExtractText.h
index 2d7ba56..b01665e 100644
--- a/libminifi/include/processors/ExtractText.h
+++ b/libminifi/include/processors/ExtractText.h
@@ -40,7 +40,7 @@ public:
     /*!
      * Create a new processor
      */
-    explicit ExtractText(std::string name, uuid_t uuid = nullptr)
+    explicit ExtractText(std::string name,  utils::Identifier uuid = utils::Identifier())
     : Processor(name, uuid)
     {
         logger_ = logging::LoggerFactory<ExtractText>::getLogger();

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/processors/GenerateFlowFile.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/GenerateFlowFile.h b/libminifi/include/processors/GenerateFlowFile.h
index 660ff05..d3c4c44 100644
--- a/libminifi/include/processors/GenerateFlowFile.h
+++ b/libminifi/include/processors/GenerateFlowFile.h
@@ -38,7 +38,7 @@ class GenerateFlowFile : public core::Processor {
   /*!
    * Create a new processor
    */
-  GenerateFlowFile(std::string name, uuid_t uuid = NULL)
+  GenerateFlowFile(std::string name, utils::Identifier uuid = utils::Identifier())
       : Processor(name, uuid) {
     _data = NULL;
     _dataSize = 0;

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/processors/GetFile.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/GetFile.h b/libminifi/include/processors/GetFile.h
index 1c4d0db..fb1191e 100644
--- a/libminifi/include/processors/GetFile.h
+++ b/libminifi/include/processors/GetFile.h
@@ -50,13 +50,13 @@ struct GetFileRequest {
 class GetFileMetrics : public state::response::ResponseNode {
  public:
   GetFileMetrics()
-      : state::response::ResponseNode("GetFileMetrics", 0) {
+      : state::response::ResponseNode("GetFileMetrics") {
     iterations_ = 0;
     accepted_files_ = 0;
     input_bytes_ = 0;
   }
 
-  GetFileMetrics(std::string name, uuid_t uuid)
+  GetFileMetrics(std::string name, utils::Identifier &uuid)
       : state::response::ResponseNode(name, uuid) {
     iterations_ = 0;
     accepted_files_ = 0;
@@ -109,7 +109,7 @@ class GetFile : public core::Processor, public state::response::MetricsNodeSourc
   /*!
    * Create a new processor
    */
-  explicit GetFile(std::string name, uuid_t uuid = NULL)
+  explicit GetFile(std::string name, utils::Identifier uuid = utils::Identifier())
       : Processor(name, uuid),
         logger_(logging::LoggerFactory<GetFile>::getLogger()) {
     metrics_ = std::make_shared<GetFileMetrics>();

http://git-wip-us.apache.org/repos/asf/nifi-minifi-cpp/blob/2b0a55e4/libminifi/include/processors/GetTCP.h
----------------------------------------------------------------------
diff --git a/libminifi/include/processors/GetTCP.h b/libminifi/include/processors/GetTCP.h
index 741d190..85b449f 100644
--- a/libminifi/include/processors/GetTCP.h
+++ b/libminifi/include/processors/GetTCP.h
@@ -119,13 +119,13 @@ class DataHandler {
 class GetTCPMetrics : public state::response::ResponseNode {
  public:
   GetTCPMetrics()
-      : state::response::ResponseNode("GetTCPMetrics", 0) {
+      : state::response::ResponseNode("GetTCPMetrics") {
     iterations_ = 0;
     accepted_files_ = 0;
     input_bytes_ = 0;
   }
 
-  GetTCPMetrics(std::string name, uuid_t uuid)
+  GetTCPMetrics(std::string name, utils::Identifier &uuid)
       : state::response::ResponseNode(name, uuid) {
     iterations_ = 0;
     accepted_files_ = 0;
@@ -178,7 +178,7 @@ class GetTCP : public core::Processor, public state::response::MetricsNodeSource
   /*!
    * Create a new processor
    */
-  explicit GetTCP(std::string name, uuid_t uuid = NULL)
+  explicit GetTCP(std::string name, utils::Identifier uuid = utils::Identifier())
       : Processor(name, uuid),
         running_(false),
         stay_connected_(true),