You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ab...@apache.org on 2020/03/31 16:31:46 UTC

[nifi-minifi-cpp] branch master updated: MINIFICPP-1096 fix BackTrace, OOB indexing, tests, appveyor reporting

This is an automated email from the ASF dual-hosted git repository.

aboda pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/nifi-minifi-cpp.git


The following commit(s) were added to refs/heads/master by this push:
     new c852d16  MINIFICPP-1096 fix BackTrace, OOB indexing, tests, appveyor reporting
c852d16 is described below

commit c852d16f05a6196912525240a276122c667835bf
Author: Marton Szasz <sz...@gmail.com>
AuthorDate: Mon Feb 10 18:08:03 2020 +0100

    MINIFICPP-1096 fix BackTrace, OOB indexing, tests, appveyor reporting
    
    This closes #731
    
    Squashed commit of the following:
    
    commit b0764d9ad83dee400a0a757578f92eb0d1f37e7e
    Author: Szász Márton <sz...@users.noreply.github.com>
    Date:   Mon Feb 3 11:08:44 2020 +0100
    
        Check unit test failure report on appveyor
    
    commit 8fc8f2a3336111947ab6228c6f05240ad743c91d
    Author: Marton Szasz <sz...@gmail.com>
    Date:   Fri Jan 31 16:15:15 2020 +0000
    
        MINIFICPP-1096 Fix TailFileTests on windows
    
        TailFile only supports single character (line) delimiter, but Windows
        uses CRLF ("\r\n") line endings. Changed the tests to write the test
        file in binary mode to avoid LF -> CRLF conversion that breaks TailFile.
    
        We should consider adapting TailFile to support CRLF as it's standard on
        Windows.
    
    commit c4a1ee6714da534f8fdea96c960a86b2f8dbdcf2
    Author: Marton Szasz <sz...@gmail.com>
    Date:   Wed Jan 29 14:48:40 2020 +0100
    
        MINIFICPP-1096 fix BackTrace and OOB indexing issues in streams
    
    MINIFICPP-1096 fix ProcessorTests on win, fix win build/test error reporting
    
    MINIFICPP-1096 CPPLINT: disable build/c++11 due to <system_error>
    
    and remove duplicate include from DataStream
    
    MINIFICPP-1096 fix PutFile/GenerateFlowFileTests (add binary flag)
    
    Issue: The previously failing test had GenerateFlowFile -> PutFile.
    When GenerateFlowFile happened to generate a payload that contained a
    '\n' character, then during writing it to a file on windows (w/o binary
    flag), it got converted to "\r\n" and when checking the length of a
    string read from the very same file, but this time opened with the
    binary flag, we could read back the "\r\n" and our expectation didn't
    match the result.
    Fix: make PutFile use the binary flag
    
    MINIFICPP-1096 proper windows file hiding impl
    
    i.e. actually fix the previous blind attempt written on linux
    
    Signed-off-by: Arpad Boda <ab...@apache.org>
---
 CPPLINT.cfg                                        |  2 +-
 extensions/http-curl/client/HTTPStream.cpp         | 19 ++++++--
 extensions/http-curl/tests/CivetStream.h           | 10 ++--
 extensions/rocksdb-repos/RocksDbStream.cpp         | 19 ++++++--
 .../standard-processors/processors/ExtractText.cpp | 13 ++---
 .../standard-processors/processors/PutFile.cpp     |  5 +-
 .../standard-processors/processors/TailFile.cpp    |  2 -
 .../tests/unit/GetFileTests.cpp                    |  4 +-
 .../tests/unit/ProcessorTests.cpp                  | 40 ++++++++++++----
 .../tests/unit/TailFileTests.cpp                   | 37 +++++----------
 libminifi/include/core/Repository.h                |  2 +-
 libminifi/include/core/TraceableResource.h         |  2 +-
 libminifi/include/io/AtomicEntryStream.h           | 19 ++++++--
 libminifi/include/io/CRCStream.h                   | 17 ++++---
 libminifi/include/io/DataStream.h                  | 27 +++++------
 libminifi/include/utils/BackTrace.h                | 55 +++++++++-------------
 libminifi/include/utils/Id.h                       |  2 +-
 libminifi/include/utils/file/FileUtils.h           | 52 ++++++++++----------
 libminifi/src/core/ProcessSession.cpp              | 13 ++---
 libminifi/src/io/DataStream.cpp                    | 37 ++++++++++-----
 libminifi/src/io/DescriptorStream.cpp              | 19 ++++++--
 libminifi/src/io/FileStream.cpp                    | 17 +++++--
 libminifi/src/io/Serializable.cpp                  |  2 +-
 libminifi/src/io/posix/ClientSocket.cpp            | 18 +++++--
 libminifi/src/io/tls/SecureDescriptorStream.cpp    | 19 ++++++--
 libminifi/src/io/tls/TLSSocket.cpp                 | 13 +++--
 libminifi/src/io/win/ClientSocket.cpp              | 17 +++++--
 libminifi/src/utils/BackTrace.cpp                  | 24 +++++-----
 libminifi/test/TestBase.h                          |  2 +-
 win_build_vs.bat                                   | 16 +++----
 30 files changed, 303 insertions(+), 221 deletions(-)

diff --git a/CPPLINT.cfg b/CPPLINT.cfg
index 2d4d34d..dd4ad80 100644
--- a/CPPLINT.cfg
+++ b/CPPLINT.cfg
@@ -1,3 +1,3 @@
 set noparent
-filter=-runtime/reference,-runtime/string
+filter=-runtime/reference,-runtime/string,-build/c++11
 
diff --git a/extensions/http-curl/client/HTTPStream.cpp b/extensions/http-curl/client/HTTPStream.cpp
index 8735b61..cbcd3d8 100644
--- a/extensions/http-curl/client/HTTPStream.cpp
+++ b/extensions/http-curl/client/HTTPStream.cpp
@@ -22,6 +22,7 @@
 #include <vector>
 #include <memory>
 #include <string>
+#include <Exception.h>
 
 #include "HTTPCallback.h"
 #include "io/validation.h"
@@ -54,10 +55,14 @@ void HttpStream::seek(uint64_t offset) {
 }
 
 int HttpStream::writeData(std::vector<uint8_t> &buf, int buflen) {
-  if ((int) buf.capacity() < buflen) {
+  if (buflen < 0) {
+    throw minifi::Exception{ExceptionType::GENERAL_EXCEPTION, "negative buflen"};
+  }
+
+  if (buf.size() < static_cast<size_t>(buflen)) {
     return -1;
   }
-  return writeData(reinterpret_cast<uint8_t *>(&buf[0]), buflen);
+  return writeData(buf.data(), buflen);
 }
 
 // data stream overrides
@@ -90,13 +95,17 @@ inline std::vector<uint8_t> HttpStream::readBuffer(const T& t) {
 }
 
 int HttpStream::readData(std::vector<uint8_t> &buf, int buflen) {
-  if ((int) buf.capacity() < buflen) {
+  if (buflen < 0) {
+    throw minifi::Exception{ExceptionType::GENERAL_EXCEPTION, "negative buflen"};
+  }
+
+  if (buf.size() < static_cast<size_t>(buflen)) {
     buf.resize(buflen);
   }
-  int ret = readData(reinterpret_cast<uint8_t*>(&buf[0]), buflen);
+  int ret = readData(buf.data(), buflen);
 
   if (ret < buflen) {
-    buf.resize(ret);
+    buf.resize((std::max)(ret, 0));
   }
   return ret;
 }
diff --git a/extensions/http-curl/tests/CivetStream.h b/extensions/http-curl/tests/CivetStream.h
index 571b0ca..4ce54ec 100644
--- a/extensions/http-curl/tests/CivetStream.h
+++ b/extensions/http-curl/tests/CivetStream.h
@@ -65,13 +65,17 @@ class CivetStream : public io::BaseStream {
    * @param buflen
    */
   virtual int readData(std::vector<uint8_t> &buf, int buflen) {
-    if (buf.capacity() < buflen) {
+    if (buflen < 0) {
+      throw minifi::Exception{ExceptionType::GENERAL_EXCEPTION, "negative buflen"};
+    }
+
+    if (buf.size() < buflen) {
       buf.resize(buflen);
     }
-    int ret = readData(reinterpret_cast<uint8_t*>(&buf[0]), buflen);
+    int ret = readData(buf.data(), buflen);
 
     if (ret < buflen) {
-      buf.resize(ret);
+      buf.resize((std::max)(ret, 0));
     }
     return ret;
   }
diff --git a/extensions/rocksdb-repos/RocksDbStream.cpp b/extensions/rocksdb-repos/RocksDbStream.cpp
index 469b537..c55eedb 100644
--- a/extensions/rocksdb-repos/RocksDbStream.cpp
+++ b/extensions/rocksdb-repos/RocksDbStream.cpp
@@ -21,6 +21,7 @@
 #include <vector>
 #include <memory>
 #include <string>
+#include <Exception.h>
 #include "io/validation.h"
 namespace org {
 namespace apache {
@@ -53,10 +54,14 @@ void RocksDbStream::seek(uint64_t offset) {
 }
 
 int RocksDbStream::writeData(std::vector<uint8_t> &buf, int buflen) {
-  if (buf.capacity() < buflen) {
+  if (buflen < 0) {
+    throw minifi::Exception{ExceptionType::GENERAL_EXCEPTION, "negative buflen"};
+  }
+
+  if (buf.size() < static_cast<size_t>(buflen)) {
     return -1;
   }
-  return writeData(reinterpret_cast<uint8_t *>(&buf[0]), buflen);
+  return writeData(buf.data(), buflen);
 }
 
 // data stream overrides
@@ -88,13 +93,17 @@ inline std::vector<uint8_t> RocksDbStream::readBuffer(const T& t) {
 }
 
 int RocksDbStream::readData(std::vector<uint8_t> &buf, int buflen) {
-  if (buf.capacity() < buflen) {
+  if (buflen < 0) {
+    throw minifi::Exception{ExceptionType::GENERAL_EXCEPTION, "negative buflen"};
+  }
+
+  if (buf.size() < buflen) {
     buf.resize(buflen);
   }
-  int ret = readData(reinterpret_cast<uint8_t*>(&buf[0]), buflen);
+  int ret = readData(buf.data(), buflen);
 
   if (ret < buflen) {
-    buf.resize(ret);
+    buf.resize((std::max)(ret, 0));
   }
   return ret;
 }
diff --git a/extensions/standard-processors/processors/ExtractText.cpp b/extensions/standard-processors/processors/ExtractText.cpp
index b218221..669e2ab 100644
--- a/extensions/standard-processors/processors/ExtractText.cpp
+++ b/extensions/standard-processors/processors/ExtractText.cpp
@@ -25,6 +25,7 @@
 
 #include <iostream>
 #include <sstream>
+#include <utility>
 
 #include "ExtractText.h"
 #include "core/ProcessContext.h"
@@ -121,7 +122,7 @@ int64_t ExtractText::ReadCallback::process(std::shared_ptr<io::BaseStream> strea
   ctx_->getProperty(SizeLimit.getName(), sizeLimitStr);
   ctx_->getProperty(RegexMode.getName(), regex_mode);
 
-  if (sizeLimitStr == "")
+  if (sizeLimitStr.empty())
     size_limit = DEFAULT_SIZE_LIMIT;
   else if (sizeLimitStr != "0")
     size_limit = std::stoi(sizeLimitStr);
@@ -130,7 +131,7 @@ int64_t ExtractText::ReadCallback::process(std::shared_ptr<io::BaseStream> strea
 
   while (read_size < size_limit) {
     // Don't read more than config limit or the size of the buffer
-    ret = stream->readData(buffer_, std::min<uint64_t>((size_limit - read_size), buffer_.capacity()));
+    ret = stream->readData(buffer_, std::min<uint64_t>(size_limit - read_size, buffer_.size()));
 
     if (ret < 0) {
       return -1;  // Stream error
@@ -138,7 +139,7 @@ int64_t ExtractText::ReadCallback::process(std::shared_ptr<io::BaseStream> strea
       break;  // End of stream, no more data
     }
 
-    contentStream.write(reinterpret_cast<const char*>(&buffer_[0]), ret);
+    contentStream.write(reinterpret_cast<const char*>(buffer_.data()), ret);
     read_size += ret;
     if (contentStream.fail()) {
       return -1;
@@ -212,10 +213,10 @@ int64_t ExtractText::ReadCallback::process(std::shared_ptr<io::BaseStream> strea
 }
 
 ExtractText::ReadCallback::ReadCallback(std::shared_ptr<core::FlowFile> flowFile, core::ProcessContext *ctx,  std::shared_ptr<logging::Logger> lgr)
-    : flowFile_(flowFile),
+    : flowFile_(std::move(flowFile)),
       ctx_(ctx),
-      logger_(lgr) {
-  buffer_.reserve(std::min<uint64_t>(flowFile->getSize(), MAX_BUFFER_SIZE));
+      logger_(std::move(lgr)) {
+  buffer_.resize(std::min<uint64_t>(flowFile_->getSize(), MAX_BUFFER_SIZE));
 }
 
 } /* namespace processors */
diff --git a/extensions/standard-processors/processors/PutFile.cpp b/extensions/standard-processors/processors/PutFile.cpp
index 1496feb..e7ec1eb 100644
--- a/extensions/standard-processors/processors/PutFile.cpp
+++ b/extensions/standard-processors/processors/PutFile.cpp
@@ -225,13 +225,12 @@ bool PutFile::putFile(core::ProcessSession *session, std::shared_ptr<FlowFileRec
     logger_->log_debug("Committing %s", destFile);
     success = cb.commit();
   } else {
-    std::ofstream outfile(destFile);
+    std::ofstream outfile(destFile, std::ios::out | std::ios::binary);
     if (!outfile.good()) {
       logger_->log_error("Failed to create empty file: %s", destFile);
     } else {
       success = true;
     }
-    outfile.close();
   }
 
   if (success) {
@@ -256,7 +255,7 @@ int64_t PutFile::ReadCallback::process(std::shared_ptr<io::BaseStream> stream) {
   size_t size = 0;
   uint8_t buffer[1024];
 
-  std::ofstream tmp_file_os(tmp_file_);
+  std::ofstream tmp_file_os(tmp_file_, std::ios::out | std::ios::binary);
 
   do {
     int read = stream->read(buffer, 1024);
diff --git a/extensions/standard-processors/processors/TailFile.cpp b/extensions/standard-processors/processors/TailFile.cpp
index f74417f..9e7bbf1 100644
--- a/extensions/standard-processors/processors/TailFile.cpp
+++ b/extensions/standard-processors/processors/TailFile.cpp
@@ -340,8 +340,6 @@ void TailFile::checkRollOver(TailState &file, const std::string &base_file_name)
     file.current_file_name_ = item.fileName;
 
     storeState();
-  } else {
-    return;
   }
 }
 
diff --git a/extensions/standard-processors/tests/unit/GetFileTests.cpp b/extensions/standard-processors/tests/unit/GetFileTests.cpp
index 272b87b..4e62c4d 100644
--- a/extensions/standard-processors/tests/unit/GetFileTests.cpp
+++ b/extensions/standard-processors/tests/unit/GetFileTests.cpp
@@ -25,6 +25,7 @@
 #include "TestBase.h"
 #include "LogAttribute.h"
 #include "GetFile.h"
+#include "utils/file/FileUtils.h"
 
 #ifdef WIN32
 #include <fileapi.h>
@@ -77,7 +78,8 @@ TEST_CASE("GetFile: MaxSize", "[getFileFifo]") {  // NOLINT
   hidden_in_file_stream << "But noone has ever seen it" << std::endl;
   hidden_in_file_stream.close();
 #ifdef WIN32
-  REQUIRE(SetFileAttributesA(hidden_in_file.c_str(), FILE_ATTRIBUTE_HIDDEN));
+  const auto hide_file_err = utils::file::FileUtils::hide_file(hidden_in_file.c_str());
+  REQUIRE(!hide_file_err);
 #endif
   plan->runNextProcessor();  // Get
   plan->runNextProcessor();  // Log
diff --git a/extensions/standard-processors/tests/unit/ProcessorTests.cpp b/extensions/standard-processors/tests/unit/ProcessorTests.cpp
index 99b2ae8..19eb414 100644
--- a/extensions/standard-processors/tests/unit/ProcessorTests.cpp
+++ b/extensions/standard-processors/tests/unit/ProcessorTests.cpp
@@ -22,11 +22,16 @@
 #include <set>
 #include <fstream>
 #include <GenerateFlowFile.h>
+#ifdef WIN32
+#include <fileapi.h>
+#include <system_error>
+#endif /* WIN32 */
 
 #include "TestBase.h"
 #include "LogAttribute.h"
 #include "GetFile.h"
 #include "unit/ProvenanceTestHelper.h"
+#include "utils/file/FileUtils.h"
 #include "core/Core.h"
 #include "core/FlowFile.h"
 #include "core/Processor.h"
@@ -136,10 +141,10 @@ TEST_CASE("Test GetFile Ignore", "[getfileCreate3]") {
   std::shared_ptr<TestRepository> repo = std::static_pointer_cast<TestRepository>(test_repo);
 
   char format[] = "/tmp/gt.XXXXXX";
-  auto dir = testController.createTempDirectory(format);
+  const auto dir = testController.createTempDirectory(format);
 
   utils::Identifier processoruuid;
-  REQUIRE(true == processor->getUUID(processoruuid));
+  REQUIRE(processor->getUUID(processoruuid));
 
   std::shared_ptr<minifi::Connection> connection = std::make_shared<minifi::Connection>(test_repo, content_repo, "getfileCreate2Connection");
 
@@ -180,19 +185,34 @@ TEST_CASE("Test GetFile Ignore", "[getfileCreate3]") {
   auto records = reporter->getEvents();
   record = session->get();
   REQUIRE(record == nullptr);
-  REQUIRE(records.size() == 0);
+  REQUIRE(records.empty());
 
-  std::fstream file;
-  std::stringstream ss;
-  ss << dir << utils::file::FileUtils::get_separator() << ".filewithoutanext";
-  file.open(ss.str(), std::ios::out);
-  file << "tempFile";
-  file.close();
+  const std::string hidden_file_name = [&] {
+    std::stringstream ss;
+    ss << dir << utils::file::FileUtils::get_separator() << ".filewithoutanext";
+    return ss.str();
+  }();
+  {
+    std::ofstream file{ hidden_file_name };
+    file << "tempFile";
+  }
+
+#ifdef WIN32
+  {
+    // hide file on windows, because a . prefix in the filename doesn't imply a hidden file
+    const auto hide_file_error = utils::file::FileUtils::hide_file(hidden_file_name.c_str());
+    REQUIRE(!hide_file_error);
+  }
+#endif /* WIN32 */
 
   processor->incrementActiveTasks();
   processor->setScheduledState(core::ScheduledState::RUNNING);
   processor->onTrigger(context, session);
-  unlink(ss.str().c_str());
+#ifndef WIN32
+  unlink(hidden_file_name.c_str());
+#else
+  _unlink(hidden_file_name.c_str());
+#endif /* !WIN32 */
   reporter = session->getProvenanceReporter();
 
   REQUIRE(processor->getName() == "getfileCreate2");
diff --git a/extensions/standard-processors/tests/unit/TailFileTests.cpp b/extensions/standard-processors/tests/unit/TailFileTests.cpp
index fda9df2..b715fde 100644
--- a/extensions/standard-processors/tests/unit/TailFileTests.cpp
+++ b/extensions/standard-processors/tests/unit/TailFileTests.cpp
@@ -64,7 +64,7 @@ TEST_CASE("TailFileWithDelimiter", "[tailfiletest2]") {
   temp_file << dir << utils::file::FileUtils::get_separator() << TMP_FILE;
 
   std::ofstream tmpfile;
-  tmpfile.open(temp_file.str());
+  tmpfile.open(temp_file.str(), std::ios::out | std::ios::binary);
   tmpfile << NEWLINE_FILE;
   tmpfile.close();
 
@@ -82,11 +82,7 @@ TEST_CASE("TailFileWithDelimiter", "[tailfiletest2]") {
   testController.runSession(plan, false);
 
   REQUIRE(LogTestController::getInstance().contains("Logged 1 flow files"));
-#ifdef WIN32
-  REQUIRE(LogTestController::getInstance().contains("Size:" + std::to_string(NEWLINE_FILE.find_first_of('\n')+1) + " Offset:0"));
-#else
   REQUIRE(LogTestController::getInstance().contains("Size:" + std::to_string(NEWLINE_FILE.find_first_of('\n')) + " Offset:0"));
-#endif
 
   LogTestController::getInstance().reset();
 
@@ -116,7 +112,7 @@ TEST_CASE("TestNewContent", "[tailFileWithDelimiterState]") {
   temp_file << dir << utils::file::FileUtils::get_separator() << TMP_FILE;
 
   std::ofstream tmpfile;
-  tmpfile.open(temp_file.str());
+  tmpfile.open(temp_file.str(), std::ios::out | std::ios::binary);
   tmpfile << NEWLINE_FILE;
   tmpfile.close();
 
@@ -134,12 +130,11 @@ TEST_CASE("TestNewContent", "[tailFileWithDelimiterState]") {
   plan->reset(true);  // start a new but with state file
 
   std::ofstream appendStream;
-  appendStream.open(temp_file.str(), std::ios_base::app);
+  appendStream.open(temp_file.str(), std::ios_base::app | std::ios_base::binary);
   appendStream << std::endl;
   testController.runSession(plan, true);
 
   REQUIRE(LogTestController::getInstance().contains("position 14"));
-
   REQUIRE(LogTestController::getInstance().contains("minifi-tmpfile.14-34.txt"));
 
   LogTestController::getInstance().reset();
@@ -171,7 +166,7 @@ TEST_CASE("TestDeleteState", "[tailFileWithDelimiterState]") {
   temp_file << dir << utils::file::FileUtils::get_separator() << TMP_FILE;
 
   std::ofstream tmpfile;
-  tmpfile.open(temp_file.str());
+  tmpfile.open(temp_file.str(), std::ios::out | std::ios::binary);
   tmpfile << NEWLINE_FILE;
   tmpfile.close();
 
@@ -183,11 +178,7 @@ TEST_CASE("TestDeleteState", "[tailFileWithDelimiterState]") {
   plan->setProperty(tailfile, org::apache::nifi::minifi::processors::TailFile::Delimiter.getName(), "\n");
   testController.runSession(plan, true);
 
-#ifdef WIN32
-  REQUIRE(LogTestController::getInstance().contains("minifi-tmpfile.0-14.txt"));
-#else
   REQUIRE(LogTestController::getInstance().contains("minifi-tmpfile.0-13.txt"));
-#endif
 
   plan->reset(true);  // start a new but with state file
   remove(std::string(state_file.str() + "." + id).c_str());
@@ -197,11 +188,7 @@ TEST_CASE("TestDeleteState", "[tailFileWithDelimiterState]") {
   REQUIRE(LogTestController::getInstance().contains("position 0"));
 
   // if we lose state we restart
-#ifdef WIN32
-  REQUIRE(LogTestController::getInstance().contains("minifi-tmpfile.0-14.txt"));
-#else
   REQUIRE(LogTestController::getInstance().contains("minifi-tmpfile.0-13.txt"));
-#endif
 
   // Delete the test and state file.
 }
@@ -226,12 +213,12 @@ TEST_CASE("TestChangeState", "[tailFileWithDelimiterState]") {
   temp_file << dir << utils::file::FileUtils::get_separator() << TMP_FILE;
 
   std::ofstream tmpfile;
-  tmpfile.open(temp_file.str());
+  tmpfile.open(temp_file.str(), std::ios::out | std::ios::binary);
   tmpfile << NEWLINE_FILE;
   tmpfile.close();
 
   std::ofstream appendStream;
-  appendStream.open(temp_file.str(), std::ios_base::app);
+  appendStream.open(temp_file.str(), std::ios_base::app | std::ios_base::binary);
   appendStream.write("\n", 1);
   appendStream.close();
 
@@ -243,11 +230,9 @@ TEST_CASE("TestChangeState", "[tailFileWithDelimiterState]") {
   plan->setProperty(tailfile, org::apache::nifi::minifi::processors::TailFile::Delimiter.getName(), "\n");
 
   testController.runSession(plan, true);
-#ifdef WIN32
-  REQUIRE(LogTestController::getInstance().contains("minifi-tmpfile.0-14.txt"));
-#else
+
   REQUIRE(LogTestController::getInstance().contains("minifi-tmpfile.0-13.txt"));
-#endif
+
 
   // should stay the same
   for (int i = 0; i < 5; i++) {
@@ -396,7 +381,7 @@ TEST_CASE("TailFileWithOutDelimiter", "[tailfiletest2]") {
   std::stringstream temp_file;
   temp_file << dir << utils::file::FileUtils::get_separator() << TMP_FILE;
   std::ofstream tmpfile;
-  tmpfile.open(temp_file.str());
+  tmpfile.open(temp_file.str(), std::ios::out | std::ios::binary);
   tmpfile << NEWLINE_FILE;
   tmpfile.close();
 
@@ -443,7 +428,7 @@ TEST_CASE("TailFileLongWithDelimiter", "[tailfiletest2]") {
   std::stringstream temp_file;
   temp_file << dir << utils::file::FileUtils::get_separator() << TMP_FILE;
   std::ofstream tmpfile;
-  tmpfile.open(temp_file.str());
+  tmpfile.open(temp_file.str(), std::ios::out | std::ios::binary);
   tmpfile << line1 << "\n" << line2 << "\n" << line3 << "\n" << line4;
   tmpfile.close();
 
@@ -525,7 +510,7 @@ TEST_CASE("TailFileWithDelimiterMultipleDelimiters", "[tailfiletest2]") {
   std::stringstream temp_file;
   temp_file << dir << utils::file::FileUtils::get_separator() << TMP_FILE;
   std::ofstream tmpfile;
-  tmpfile.open(temp_file.str());
+  tmpfile.open(temp_file.str(), std::ios::out | std::ios::binary);
   tmpfile << line1 << "\n" << line2 << "\n" << line3 << "\n" << line4;
   tmpfile.close();
 
diff --git a/libminifi/include/core/Repository.h b/libminifi/include/core/Repository.h
index f71bf1a..424f313 100644
--- a/libminifi/include/core/Repository.h
+++ b/libminifi/include/core/Repository.h
@@ -127,7 +127,7 @@ class Repository : public virtual core::SerializableComponent, public core::Trac
   /**
    * Since SerializableComponents represent a runnable object, we should return traces
    */
-  virtual BackTrace &&getTraces() {
+  virtual BackTrace getTraces() {
     return TraceResolver::getResolver().getBackTrace(getName(), thread_.native_handle());
   }
 
diff --git a/libminifi/include/core/TraceableResource.h b/libminifi/include/core/TraceableResource.h
index e22ce28..afdcc05 100644
--- a/libminifi/include/core/TraceableResource.h
+++ b/libminifi/include/core/TraceableResource.h
@@ -41,7 +41,7 @@ class TraceableResource {
   /**
    * Since SerializableComponents represent a runnable object, we should return traces
    */
-  virtual BackTrace &&getTraces() = 0;
+  virtual BackTrace getTraces() = 0;
 };
 
 } /* namespace core */
diff --git a/libminifi/include/io/AtomicEntryStream.h b/libminifi/include/io/AtomicEntryStream.h
index d383125..9f9043e 100644
--- a/libminifi/include/io/AtomicEntryStream.h
+++ b/libminifi/include/io/AtomicEntryStream.h
@@ -20,6 +20,7 @@
 
 #include <mutex>
 #include <cstring>
+#include <algorithm>
 #include "BaseStream.h"
 #include "core/repository/AtomicRepoEntries.h"
 #include "Exception.h"
@@ -129,9 +130,13 @@ void AtomicEntryStream<T>::seek(uint64_t offset) {
 
 template<typename T>
 int AtomicEntryStream<T>::writeData(std::vector<uint8_t> &buf, int buflen) {
-  if ((int)buf.capacity() < buflen || invalid_stream_)
+  if (buflen < 0) {
+    throw minifi::Exception{ExceptionType::GENERAL_EXCEPTION, "negative buflen"};
+  }
+
+  if (buf.size() < static_cast<size_t>(buflen) || invalid_stream_)
     return -1;
-  return writeData(reinterpret_cast<uint8_t *>(&buf[0]), buflen);
+  return writeData(buf.data(), buflen);
 }
 
 // data stream overrides
@@ -154,16 +159,20 @@ int AtomicEntryStream<T>::writeData(uint8_t *value, int size) {
 
 template<typename T>
 int AtomicEntryStream<T>::readData(std::vector<uint8_t> &buf, int buflen) {
+  if (buflen < 0) {
+    throw minifi::Exception{ExceptionType::GENERAL_EXCEPTION, "negative buflen"};
+  }
+
   if (invalid_stream_) {
     return -1;
   }
-  if ((int)buf.capacity() < buflen) {
+  if (buf.size() < static_cast<size_t>(buflen)) {
     buf.resize(buflen);
   }
-  int ret = readData(reinterpret_cast<uint8_t*>(&buf[0]), buflen);
+  int ret = readData(buf.data(), buflen);
 
   if (ret < buflen) {
-    buf.resize(ret);
+    buf.resize((std::max)(ret, 0));
   }
   return ret;
 }
diff --git a/libminifi/include/io/CRCStream.h b/libminifi/include/io/CRCStream.h
index 2c7f3d5..cd789e1 100644
--- a/libminifi/include/io/CRCStream.h
+++ b/libminifi/include/io/CRCStream.h
@@ -27,6 +27,7 @@
 #endif
 #include "BaseStream.h"
 #include "Serializable.h"
+#include "Exception.h"
 
 namespace org {
 namespace apache {
@@ -180,10 +181,13 @@ CRCStream<T>::CRCStream(CRCStream<T> &&move)
 
 template<typename T>
 int CRCStream<T>::readData(std::vector<uint8_t> &buf, int buflen) {
+  if (buflen < 0) {
+    throw minifi::Exception{ExceptionType::GENERAL_EXCEPTION, "negative buflen"};
+  }
 
-  if ((int)buf.capacity() < buflen)
+  if (buf.size() < static_cast<size_t>(buflen))
     buf.resize(buflen);
-  return readData((uint8_t*) &buf[0], buflen);
+  return readData(buf.data(), buflen);
 }
 
 template<typename T>
@@ -197,21 +201,22 @@ int CRCStream<T>::readData(uint8_t *buf, int buflen) {
 
 template<typename T>
 int CRCStream<T>::writeData(std::vector<uint8_t> &buf, int buflen) {
+  if (buflen < 0) {
+    throw minifi::Exception{ExceptionType::GENERAL_EXCEPTION, "negative buflen"};
+  }
 
-  if ((int)buf.capacity() < buflen)
+  if (buf.size() < static_cast<size_t>(buflen))
     buf.resize(buflen);
-  return writeData((uint8_t*) &buf[0], buflen);
+  return writeData(buf.data(), buflen);
 }
 
 template<typename T>
 int CRCStream<T>::writeData(uint8_t *value, int size) {
-
   int ret = child_stream_->write(value, size);
   if (ret > 0) {
     crc_ = crc32(crc_, value, ret);
   }
   return ret;
-
 }
 template<typename T>
 void CRCStream<T>::reset() {
diff --git a/libminifi/include/io/DataStream.h b/libminifi/include/io/DataStream.h
index 825cd89..a5700b3 100644
--- a/libminifi/include/io/DataStream.h
+++ b/libminifi/include/io/DataStream.h
@@ -37,22 +37,15 @@ namespace io {
 class DataStream {
  public:
 
-  DataStream()
-      : readBuffer(0) {
+  DataStream() = default;
 
-  }
-
-  virtual ~DataStream() {
-
-  }
+  virtual ~DataStream() = default;
 
   /**
    * Constructor
    **/
-  explicit DataStream(const uint8_t *buf, const uint32_t buflen)
-      : DataStream() {
+  explicit DataStream(const uint8_t *buf, const uint32_t buflen) {
     writeData((uint8_t*) buf, buflen);
-
   }
 
   virtual short initialize() {
@@ -65,9 +58,7 @@ class DataStream {
     readBuffer += offset;
   }
 
-  virtual void closeStream() {
-
-  }
+  virtual void closeStream() { }
 
   /**
    * Reads data and places it into buf
@@ -112,7 +103,7 @@ class DataStream {
    * @return vector's array
    **/
   const uint8_t *getBuffer() const {
-    return &buffer[0];
+    return buffer.data();
   }
 
   /**
@@ -126,7 +117,13 @@ class DataStream {
  protected:
   // All serialization related method and internal buf
   std::vector<uint8_t> buffer;
-  uint32_t readBuffer;
+
+  // read offset to buffer
+  uint32_t readBuffer = 0;
+
+ private:
+
+   int doReadData(uint8_t *buf, int buflen) noexcept;
 };
 
 } /* namespace io */
diff --git a/libminifi/include/utils/BackTrace.h b/libminifi/include/utils/BackTrace.h
index 5c7bb80..8614500 100644
--- a/libminifi/include/utils/BackTrace.h
+++ b/libminifi/include/utils/BackTrace.h
@@ -19,9 +19,10 @@
 
 #ifdef HAS_EXECINFO
 #include <execinfo.h>
-#include <signal.h>
+#include <csignal>
 #endif
 #include <thread>
+#include <utility>
 #include <vector>
 #include <mutex>
 #include <iostream>
@@ -36,25 +37,21 @@
 class TraceResolver;
 
 /**
- * Purpose: Backtrace is a movable vector of trace lines.
+ * Purpose: Backtrace is a vector of trace lines.
  *
  */
 class BackTrace {
  public:
-  BackTrace() {
-  }
-  BackTrace(const std::string &name)
-      : name_(name) {
+  BackTrace() = default;
+
+  BackTrace(std::string name)
+      : name_(std::move(name)) {
   }
-  BackTrace(BackTrace &&) = default;
-  BackTrace(BackTrace &) = delete;
 
   std::vector<std::string> getTraces() const {
     return trace_;
   }
 
-  BackTrace &operator=(BackTrace &&other) = default;
-
   /**
    * Return thread name of f this caller
    * @returns name ;
@@ -64,8 +61,8 @@ class BackTrace {
   }
 
  protected:
-  void addLine(const std::string &symbol_line) {
-    trace_.emplace_back(symbol_line);
+  void addLine(std::string symbol_line) {
+    trace_.emplace_back(std::move(symbol_line));
   }
 
  private:
@@ -77,14 +74,8 @@ class BackTrace {
 /**
  * Pulls the trace and places it onto the TraceResolver instance.
  */
-void pull_trace(const uint8_t frames_to_skip = 1);
+void pull_trace(uint8_t frames_to_skip = 1);
 
-#ifdef HAS_EXECINFO
-/**
- * Signal handler that will run via TraceResolver
- */
-void handler(int signr, siginfo_t *info, void *secret);
-#endif
 /**
  * Emplaces a signal handler for SIGUSR2
  */
@@ -101,18 +92,18 @@ class TraceResolver {
    * Retrieves the backtrace for the provided thread reference
    * @return BackTrace instance
    */
-  BackTrace &&getBackTrace(const std::string &thread_name, std::thread::native_handle_type thread);
+  BackTrace getBackTrace(std::string thread_name, std::thread::native_handle_type thread);
 
   /**
    * Retrieves the backtrace for the calling thread
    * @returns BackTrace instance
    */
-  BackTrace &&getBackTrace(const std::string &thread_name) {
+  BackTrace getBackTrace(std::string thread_name) {
 #ifdef WIN32
-	  // currrently not supported in windows
-	  return BackTrace(thread_name);
+    // currrently not supported in windows
+    return BackTrace(std::move(thread_name));
 #else
-    return std::move(getBackTrace(thread_name, pthread_self()));
+    return getBackTrace(std::move(thread_name), pthread_self());
 #endif
   }
 
@@ -141,27 +132,23 @@ class TraceResolver {
   /**
    * Returns the thread handle reference in the native format.
    */
-  const std::thread::native_handle_type getThreadHandle() {
+  std::thread::native_handle_type getThreadHandle() {
     return thread_handle_;
   }
 
   /**
-   * Returns the caller handle refernce in the native format.
+   * Returns the caller handle reference in the native format.
    */
-  const std::thread::native_handle_type getCallerHandle() {
+  std::thread::native_handle_type getCallerHandle() {
     return caller_handle_;
   }
 
  private:
-  TraceResolver()  // can't use = default due to handle_types not defaulting.
-      : thread_handle_(0),
-        caller_handle_(0) {
-    ;
-  }
+  TraceResolver() = default;
 
   BackTrace trace_;
-  std::thread::native_handle_type thread_handle_;
-  std::thread::native_handle_type caller_handle_;
+  std::thread::native_handle_type thread_handle_{0};
+  std::thread::native_handle_type caller_handle_{0};
   std::mutex mutex_;
 };
 
diff --git a/libminifi/include/utils/Id.h b/libminifi/include/utils/Id.h
index 175cb5e..4d8de41 100644
--- a/libminifi/include/utils/Id.h
+++ b/libminifi/include/utils/Id.h
@@ -166,8 +166,8 @@ class IdGenerator {
   unsigned char deterministic_prefix_[8];
   std::atomic<uint64_t> incrementor_;
 
-#ifndef WIN32
   std::mutex uuid_mutex_;
+#ifndef WIN32
   std::unique_ptr<uuid> uuid_impl_;
   bool generateWithUuidImpl(unsigned int mode, UUID_FIELD output);
 #endif
diff --git a/libminifi/include/utils/file/FileUtils.h b/libminifi/include/utils/file/FileUtils.h
index 7b90978..23dc084 100644
--- a/libminifi/include/utils/file/FileUtils.h
+++ b/libminifi/include/utils/file/FileUtils.h
@@ -112,31 +112,21 @@ class FileUtils {
 #endif
   }
 
-  static std::string create_temp_directory(char *format) {
+  static std::string create_temp_directory(char* format) {
 #ifdef WIN32
-	  std::string tempDirectory;
-	  char tempBuffer[MAX_PATH];
-	  auto ret = GetTempPath(MAX_PATH, tempBuffer); 
-	  if (ret <= MAX_PATH && ret != 0)
-	  {
-		  static std::shared_ptr<minifi::utils::IdGenerator> generator;
-		  if (!generator) {
-			  generator = minifi::utils::IdGenerator::getIdGenerator();
-			  generator->initialize(std::make_shared<minifi::Properties>());
-		  }
-		  tempDirectory = tempBuffer;
-		  minifi::utils::Identifier id;
-		  generator->generate(id);
-		  tempDirectory += id.to_string();
-		  create_dir(tempDirectory);
-	  }
-	  return tempDirectory;
+    char tempBuffer[MAX_PATH];
+    const auto ret = GetTempPath(MAX_PATH, tempBuffer);
+    if (ret <= MAX_PATH && ret != 0)
+    {
+      const std::string tempDirectory = tempBuffer
+          + minifi::utils::IdGenerator::getIdGenerator()->generate().to_string();
+      create_dir(tempDirectory);
+      return tempDirectory;
+    }
+    return {};
 #else
-	  auto dir = mkdtemp(format);
-	  if (nullptr == dir) {
-		  return "";
-	  }
-	  else return dir;
+    if (mkdtemp(format) == nullptr) { return ""; }
+    return format;
 #endif
   }
 
@@ -640,10 +630,10 @@ class FileUtils {
     std::vector<char> buf(1024U);
     while (true) {
       ssize_t ret = readlink("/proc/self/exe", buf.data(), buf.size());
-      if (ret == -1) {
+      if (ret < 0) {
         return "";
       }
-      if (ret == buf.size()) {
+      if (static_cast<size_t>(ret) == buf.size()) {
         /* It may have been truncated */
         buf.resize(buf.size() * 2);
         continue;
@@ -694,6 +684,18 @@ class FileUtils {
     }
     return get_parent_path(executable_path);
   }
+
+#ifdef WIN32
+  static std::error_code hide_file(const char* const file_name) {
+    const bool success = SetFileAttributesA(file_name, FILE_ATTRIBUTE_HIDDEN);
+    if (!success) {
+      // note: All possible documented error codes from GetLastError are in [0;15999] at the time of writing.
+      // The below casting is safe in [0;std::numeric_limits<int>::max()], int max is guaranteed to be at least 32767
+      return { static_cast<int>(GetLastError()), std::system_category() };
+    }
+    return {};
+  }
+#endif /* WIN32 */
 };
 
 } /* namespace file */
diff --git a/libminifi/src/core/ProcessSession.cpp b/libminifi/src/core/ProcessSession.cpp
index 333b7a5..9d5a13d 100644
--- a/libminifi/src/core/ProcessSession.cpp
+++ b/libminifi/src/core/ProcessSession.cpp
@@ -19,17 +19,16 @@
  */
 #include "core/ProcessSession.h"
 #include "core/ProcessSessionReadCallback.h"
-#include <time.h>
+#include <ctime>
 #include <vector>
-#include <queue>
 #include <map>
 #include <memory>
 #include <string>
 #include <set>
 #include <chrono>
-#include <thread>
 #include <iostream>
 #include <cinttypes>
+#include <algorithm>
 /* This implementation is only for native Windows systems.  */
 #if (defined _WIN32 || defined __WIN32__) && !defined __CYGWIN__
 #define _WINSOCKAPI_
@@ -399,14 +398,8 @@ void ProcessSession::importFrom(io::DataStream &stream, const std::shared_ptr<co
     }
     size_t position = 0;
     const size_t max_size = stream.getSize();
-    size_t read_size = max_read;
     while (position < max_size) {
-      if ((max_size - position) > max_read) {
-        read_size = max_read;
-      } else {
-        read_size = max_size - position;
-      }
-      charBuffer.clear();
+      const size_t read_size = (std::min)(max_read, max_size - position);
       stream.readData(charBuffer, read_size);
 
       content_stream->write(charBuffer.data(), read_size);
diff --git a/libminifi/src/io/DataStream.cpp b/libminifi/src/io/DataStream.cpp
index d558b69..3a53489 100644
--- a/libminifi/src/io/DataStream.cpp
+++ b/libminifi/src/io/DataStream.cpp
@@ -17,13 +17,12 @@
  */
 #include "io/DataStream.h"
 #include <vector>
-#include <iostream>
 #include <cstdint>
-#include <cstdio>
-#include <cstring>
 #include <string>
 #include <algorithm>
 #include <iterator>
+#include <cassert>
+#include <Exception.h>
 
 namespace org {
 namespace apache {
@@ -89,28 +88,44 @@ int DataStream::read(uint16_t &value, bool is_little_endian) {
 }
 
 int DataStream::readData(std::vector<uint8_t> &buf, int buflen) {
+  if (buflen < 0) {
+    throw minifi::Exception{ExceptionType::GENERAL_EXCEPTION, "negative buflen"};
+  }
+
   if ((buflen + readBuffer) > buffer.size()) {
     // if read exceed
     return -1;
   }
 
-  if (static_cast<int>(buf.capacity()) < buflen)
-    buf.resize(buflen+1);
+  if (buf.size() < static_cast<size_t>(buflen))
+    buf.resize(buflen);
 
-  // back inserter works differently on win32 versions
-  buf.insert(buf.begin(), &buffer[readBuffer], &buffer[(readBuffer + buflen)]);
-
-  readBuffer += buflen;
-  return buflen;
+  return this->doReadData(buf.data(), buflen);
 }
 
 int DataStream::readData(uint8_t *buf, int buflen) {
+  if (buflen < 0) {
+    throw minifi::Exception{ ExceptionType::GENERAL_EXCEPTION, "negative buflen" };
+  }
+
   if ((buflen + readBuffer) > buffer.size()) {
     // if read exceed
     return -1;
   }
-  std::copy(&buffer[readBuffer], &buffer[(readBuffer + buflen)], buf);
+
+  return this->doReadData(buf, buflen);
+}
+
+int DataStream::doReadData(uint8_t *buf, int buflen) noexcept {
+  const auto read_start = std::next(std::begin(buffer), readBuffer);
+  const auto read_end = std::next(read_start, buflen);
+  const auto write_end = std::copy(read_start, read_end, buf);
+
+  assert(std::distance(buf, write_end) == buflen && "read buflen bytes");
+
+  // increase offset for the next read
   readBuffer += buflen;
+
   return buflen;
 }
 
diff --git a/libminifi/src/io/DescriptorStream.cpp b/libminifi/src/io/DescriptorStream.cpp
index 6431ef5..707d282 100644
--- a/libminifi/src/io/DescriptorStream.cpp
+++ b/libminifi/src/io/DescriptorStream.cpp
@@ -21,6 +21,7 @@
 #include <vector>
 #include <memory>
 #include <string>
+#include <Exception.h>
 #include "io/validation.h"
 namespace org {
 namespace apache {
@@ -39,10 +40,14 @@ void DescriptorStream::seek(uint64_t offset) {
 }
 
 int DescriptorStream::writeData(std::vector<uint8_t> &buf, int buflen) {
-  if (static_cast<int>(buf.capacity()) < buflen) {
+  if (buflen < 0) {
+    throw minifi::Exception{ExceptionType::GENERAL_EXCEPTION, "negative buflen"};
+  }
+
+  if (buf.size() < static_cast<size_t>(buflen)) {
     return -1;
   }
-  return writeData(reinterpret_cast<uint8_t *>(&buf[0]), buflen);
+  return writeData(buf.data(), buflen);
 }
 
 // data stream overrides
@@ -69,13 +74,17 @@ inline std::vector<uint8_t> DescriptorStream::readBuffer(const T& t) {
 }
 
 int DescriptorStream::readData(std::vector<uint8_t> &buf, int buflen) {
-  if (static_cast<int>(buf.capacity()) < buflen) {
+  if (buflen < 0) {
+    throw minifi::Exception{ExceptionType::GENERAL_EXCEPTION, "negative buflen"};
+  }
+
+  if (buf.size() < static_cast<size_t>(buflen)) {
     buf.resize(buflen);
   }
-  int ret = readData(reinterpret_cast<uint8_t*>(&buf[0]), buflen);
+  int ret = readData(buf.data(), buflen);
 
   if (ret < buflen) {
-    buf.resize(ret);
+    buf.resize((std::max)(ret, 0));
   }
   return ret;
 }
diff --git a/libminifi/src/io/FileStream.cpp b/libminifi/src/io/FileStream.cpp
index 6ee0e95..b0463e7 100644
--- a/libminifi/src/io/FileStream.cpp
+++ b/libminifi/src/io/FileStream.cpp
@@ -21,6 +21,7 @@
 #include <vector>
 #include <memory>
 #include <string>
+#include <Exception.h>
 #include "io/validation.h"
 namespace org {
 namespace apache {
@@ -86,10 +87,14 @@ void FileStream::seek(uint64_t offset) {
 }
 
 int FileStream::writeData(std::vector<uint8_t> &buf, int buflen) {
-  if (static_cast<int>(buf.capacity()) < buflen) {
+  if (buflen < 0) {
+    throw minifi::Exception{ExceptionType::GENERAL_EXCEPTION, "negative buflen"};
+  }
+
+  if (buf.size() < static_cast<size_t>(buflen)) {
     return -1;
   }
-  return writeData(reinterpret_cast<uint8_t *>(&buf[0]), buflen);
+  return writeData(buf.data(), buflen);
 }
 
 // data stream overrides
@@ -122,10 +127,14 @@ inline std::vector<uint8_t> FileStream::readBuffer(const T& t) {
 }
 
 int FileStream::readData(std::vector<uint8_t> &buf, int buflen) {
-  if (static_cast<int>(buf.capacity()) < buflen) {
+  if (buflen < 0) {
+    throw minifi::Exception{ExceptionType::GENERAL_EXCEPTION, "negative buflen"};
+  }
+
+  if (buf.size() < static_cast<size_t>(buflen)) {
     buf.resize(buflen);
   }
-  int ret = readData(reinterpret_cast<uint8_t*>(&buf[0]), buflen);
+  int ret = readData(buf.data(), buflen);
 
   if (ret < buflen) {
     buf.resize(ret);
diff --git a/libminifi/src/io/Serializable.cpp b/libminifi/src/io/Serializable.cpp
index a12b410..a9e8e78 100644
--- a/libminifi/src/io/Serializable.cpp
+++ b/libminifi/src/io/Serializable.cpp
@@ -89,7 +89,7 @@ int Serializable::readUTF(std::string &str, DataStream *stream, bool widen) {
   }
 
   std::vector<uint8_t> buf;
-  ret = stream->readData(buf, utflen);
+  stream->readData(buf, utflen);
 
   // The number of chars produced may be less than utflen
   str = std::string((const char*) &buf[0], utflen);
diff --git a/libminifi/src/io/posix/ClientSocket.cpp b/libminifi/src/io/posix/ClientSocket.cpp
index fd69bb5..d65d68d 100644
--- a/libminifi/src/io/posix/ClientSocket.cpp
+++ b/libminifi/src/io/posix/ClientSocket.cpp
@@ -33,6 +33,7 @@
 #include <cerrno>
 #include <iostream>
 #include <string>
+#include "Exception.h"
 #include "io/validation.h"
 #include "core/logging/LoggerConfiguration.h"
 
@@ -352,9 +353,14 @@ std::string Socket::getHostname() const {
 }
 
 int Socket::writeData(std::vector<uint8_t> &buf, int buflen) {
-  if (static_cast<int>(buf.capacity()) < buflen)
+  if (buflen < 0) {
+    throw minifi::Exception{ExceptionType::GENERAL_EXCEPTION, "negative buflen"};
+  }
+
+  if (buf.size() < static_cast<size_t>(buflen))
     return -1;
-  return writeData(reinterpret_cast<uint8_t *>(&buf[0]), buflen);
+
+  return writeData(buf.data(), buflen);
 }
 
 // data stream overrides
@@ -436,10 +442,14 @@ int Socket::read(uint16_t &value, bool is_little_endian) {
 }
 
 int Socket::readData(std::vector<uint8_t> &buf, int buflen, bool retrieve_all_bytes) {
-  if (static_cast<int>(buf.capacity()) < buflen) {
+  if (buflen < 0) {
+    throw minifi::Exception{ExceptionType::GENERAL_EXCEPTION, "negative buflen"};
+  }
+
+  if (buf.size() < static_cast<size_t>(buflen)) {
     buf.resize(buflen);
   }
-  return readData(reinterpret_cast<uint8_t*>(&buf[0]), buflen, retrieve_all_bytes);
+  return readData(buf.data(), buflen, retrieve_all_bytes);
 }
 
 int Socket::readData(uint8_t *buf, int buflen, bool retrieve_all_bytes) {
diff --git a/libminifi/src/io/tls/SecureDescriptorStream.cpp b/libminifi/src/io/tls/SecureDescriptorStream.cpp
index 9e1334f..de3bf41 100644
--- a/libminifi/src/io/tls/SecureDescriptorStream.cpp
+++ b/libminifi/src/io/tls/SecureDescriptorStream.cpp
@@ -21,6 +21,7 @@
 #include <vector>
 #include <memory>
 #include <string>
+#include <Exception.h>
 #include "io/validation.h"
 namespace org {
 namespace apache {
@@ -39,10 +40,14 @@ void SecureDescriptorStream::seek(uint64_t offset) {
 }
 
 int SecureDescriptorStream::writeData(std::vector<uint8_t> &buf, int buflen) {
-  if (static_cast<int>(buf.capacity()) < buflen) {
+  if (buflen < 0) {
+    throw minifi::Exception{ExceptionType::GENERAL_EXCEPTION, "negative buflen"};
+  }
+
+  if (buf.size() < static_cast<size_t>(buflen)) {
     return -1;
   }
-  return writeData(reinterpret_cast<uint8_t *>(&buf[0]), buflen);
+  return writeData(buf.data(), buflen);
 }
 
 // data stream overrides
@@ -78,13 +83,17 @@ inline std::vector<uint8_t> SecureDescriptorStream::readBuffer(const T& t) {
 }
 
 int SecureDescriptorStream::readData(std::vector<uint8_t> &buf, int buflen) {
-  if (static_cast<int>(buf.capacity()) < buflen) {
+  if (buflen < 0) {
+    throw minifi::Exception{ExceptionType::GENERAL_EXCEPTION, "negative buflen"};
+  }
+
+  if (buf.size() < static_cast<size_t>(buflen)) {
     buf.resize(buflen);
   }
-  int ret = readData(reinterpret_cast<uint8_t*>(&buf[0]), buflen);
+  int ret = readData(buf.data(), buflen);
 
   if (ret < buflen) {
-    buf.resize(ret);
+    buf.resize((std::max)(ret, 0));
   }
   return ret;
 }
diff --git a/libminifi/src/io/tls/TLSSocket.cpp b/libminifi/src/io/tls/TLSSocket.cpp
index a1a1926..f251e73 100644
--- a/libminifi/src/io/tls/TLSSocket.cpp
+++ b/libminifi/src/io/tls/TLSSocket.cpp
@@ -22,6 +22,7 @@
 #include <utility>
 #include <string>
 #include <vector>
+#include <Exception.h>
 #include "io/tls/TLSSocket.h"
 #include "io/tls/TLSUtils.h"
 #include "properties/Configure.h"
@@ -351,10 +352,14 @@ int TLSSocket::writeData(std::vector<uint8_t>& buf, int buflen) {
 }
 
 int TLSSocket::readData(std::vector<uint8_t> &buf, int buflen, bool retrieve_all_bytes) {
-  if (static_cast<int>(buf.capacity()) < buflen) {
+  if (buflen < 0) {
+    throw minifi::Exception{ExceptionType::GENERAL_EXCEPTION, "negative buflen"};
+  }
+
+  if (buf.size() < static_cast<size_t>(buflen)) {
     buf.resize(buflen);
   }
-  return readData(reinterpret_cast<uint8_t*>(&buf[0]), buflen, retrieve_all_bytes);
+  return readData(buf.data(), buflen, retrieve_all_bytes);
 }
 
 int TLSSocket::readData(uint8_t *buf, int buflen, bool retrieve_all_bytes) {
@@ -394,8 +399,8 @@ int TLSSocket::readData(uint8_t *buf, int buflen, bool retrieve_all_bytes) {
 int TLSSocket::readData(std::vector<uint8_t> &buf, int buflen) {
   if (buflen < 0)
     return -1;
-  if (buf.capacity() < static_cast<size_t>(buflen)) {
-    buf.reserve(buflen);
+  if (buf.size() < static_cast<size_t>(buflen)) {
+    buf.resize(buflen);
   }
   int total_read = 0;
   int status = 0;
diff --git a/libminifi/src/io/win/ClientSocket.cpp b/libminifi/src/io/win/ClientSocket.cpp
index 3c8b1af..5e82bfe 100644
--- a/libminifi/src/io/win/ClientSocket.cpp
+++ b/libminifi/src/io/win/ClientSocket.cpp
@@ -34,6 +34,7 @@
 #include <vector>
 #include <cerrno>
 #include <string>
+#include "Exception.h"
 #include "io/validation.h"
 #include "core/logging/LoggerConfiguration.h"
 
@@ -393,9 +394,13 @@ std::string Socket::getHostname() const {
 }
 
 int Socket::writeData(std::vector<uint8_t> &buf, int buflen) {
-  if (static_cast<int>(buf.capacity()) < buflen)
+  if (buflen < 0) {
+    throw minifi::Exception{ExceptionType::GENERAL_EXCEPTION, "negative buflen"};
+  }
+
+  if (buf.size() < static_cast<size_t>(buflen))
     return -1;
-  return writeData(reinterpret_cast<uint8_t *>(&buf[0]), buflen);
+  return writeData(buf.data(), buflen);
 }
 
 // data stream overrides
@@ -493,10 +498,14 @@ int Socket::read(uint16_t &value, bool is_little_endian) {
 }
 
 int Socket::readData(std::vector<uint8_t> &buf, int buflen, bool retrieve_all_bytes) {
-  if (static_cast<int>(buf.capacity()) < buflen) {
+  if (buflen < 0) {
+    throw minifi::Exception{ExceptionType::GENERAL_EXCEPTION, "negative buflen"};
+  }
+
+  if (buf.size() < static_cast<size_t>(buflen)) {
     buf.resize(buflen);
   }
-  return readData(reinterpret_cast<uint8_t*>(&buf[0]), buflen, retrieve_all_bytes);
+  return readData(buf.data(), buflen, retrieve_all_bytes);
 }
 
 int Socket::readData(uint8_t *buf, int buflen, bool retrieve_all_bytes) {
diff --git a/libminifi/src/utils/BackTrace.cpp b/libminifi/src/utils/BackTrace.cpp
index 160a070..0e6c856 100644
--- a/libminifi/src/utils/BackTrace.cpp
+++ b/libminifi/src/utils/BackTrace.cpp
@@ -17,9 +17,9 @@
 #ifdef HAS_EXECINFO
 #include <execinfo.h>
 #include <iostream>
+#include <utility>
 #include <cxxabi.h>
 #endif
-#define NAME_SIZE 256
 
 void pull_trace(const uint8_t frames_to_skip) {
 #ifdef HAS_EXECINFO
@@ -33,9 +33,9 @@ void pull_trace(const uint8_t frames_to_skip) {
    * we can skip the signal handler, call to pull_trace, and the first entry for backtrace_symbols
    */
   for (int i = frames_to_skip; i < trace_size; i++) {
-    char *start_parenthetical = 0;
-    char *functor = 0;
-    char *stop_parenthetical = 0;
+    char *start_parenthetical = nullptr;
+    char *functor = nullptr;
+    char *stop_parenthetical = nullptr;
 
     for (char *p = symboltable[i]; *p; ++p) {
       if (*p == '(') {
@@ -76,14 +76,14 @@ void pull_trace(const uint8_t frames_to_skip) {
 #endif
 }
 
-BackTrace &&TraceResolver::getBackTrace(const std::string &thread_name, std::thread::native_handle_type thread_handle) {
+BackTrace TraceResolver::getBackTrace(std::string thread_name, std::thread::native_handle_type thread_handle) {
   // lock so that we only perform one backtrace at a time.
 #ifdef HAS_EXECINFO
   std::lock_guard<std::mutex> lock(mutex_);
 
   caller_handle_ = pthread_self();
   thread_handle_ = thread_handle;
-  trace_ = BackTrace(thread_name);
+  trace_ = BackTrace(std::move(thread_name));
 
   if (0 == thread_handle_ || pthread_equal(caller_handle_, thread_handle)) {
     pull_trace();
@@ -102,16 +102,14 @@ BackTrace &&TraceResolver::getBackTrace(const std::string &thread_name, std::thr
   }
 #else
   // even if tracing is disabled, include thread name into the trace object
-  trace_ = BackTrace(thread_name);
+  trace_ = BackTrace(std::move(thread_name));
 #endif
   return std::move(trace_);
 }
 #ifdef HAS_EXECINFO
-void handler(int signr, siginfo_t *info, void *secret) {
-  auto curThread = pthread_self();
-
+static void handler(int, siginfo_t*, void*) {
   // not the intended thread
-  if (!pthread_equal(curThread, TraceResolver::getResolver().getThreadHandle())) {
+  if (!pthread_equal(pthread_self(), TraceResolver::getResolver().getThreadHandle())) {
     return;
   }
 
@@ -123,10 +121,10 @@ void handler(int signr, siginfo_t *info, void *secret) {
 
 void emplace_handler() {
 #ifdef HAS_EXECINFO
-  struct sigaction sa;
+  struct sigaction sa{};
   sigfillset(&sa.sa_mask);
   sa.sa_flags = SA_SIGINFO;
   sa.sa_sigaction = handler;
-  sigaction(SIGUSR2, &sa, NULL);
+  sigaction(SIGUSR2, &sa, nullptr);
 #endif
 }
diff --git a/libminifi/test/TestBase.h b/libminifi/test/TestBase.h
index 7e5416b..a51b573 100644
--- a/libminifi/test/TestBase.h
+++ b/libminifi/test/TestBase.h
@@ -365,7 +365,7 @@ class TestController {
    * format will be changed by mkdtemp, so don't rely on a shared variable.
    */
   std::string createTempDirectory(char *format) {
-    auto dir = utils::file::FileUtils::create_temp_directory(format);
+    const auto dir = utils::file::FileUtils::create_temp_directory(format);
     directories.push_back(dir);
     return dir;
   }
diff --git a/win_build_vs.bat b/win_build_vs.bat
index c7df9d4..69129a2 100644
--- a/win_build_vs.bat
+++ b/win_build_vs.bat
@@ -61,22 +61,20 @@ for %%x in (%*) do (
     )
 )
 
-
 mkdir %builddir%
-
-cd %builddir%\
-
-
+pushd %builddir%\
 
 cmake -G %generator% -DENABLE_SQL=%build_SQL% -DCMAKE_BUILD_TYPE_INIT=%cmake_build_type% -DCMAKE_BUILD_TYPE=%cmake_build_type% -DWIN32=WIN32 -DENABLE_LIBRDKAFKA=%build_kafka% -DENABLE_JNI=%build_jni% -DOPENSSL_OFF=OFF -DENABLE_COAP=%build_coap% -DUSE_SHARED_LIBS=OFF -DDISABLE_CONTROLLER=ON  -DBUILD_ROCKSDB=ON -DFORCE_WINDOWS=ON -DUSE_SYSTEM_UUID=OFF -DDISABLE_LIBARCHIVE=ON -DDISABLE_SCRIPTING=ON -DEXCLUDE_BOOST=ON -DENABLE_WEL=TRUE -DFAIL_ON_WARNINGS=OFF -DSKIP_TESTS=%skiptests% .. && ms [...]
+IF %ERRORLEVEL% NEQ 0 EXIT /b %ERRORLEVEL%
 if [%cpack%] EQU [ON] ( 
 	cpack
-    )
+	IF !ERRORLEVEL! NEQ 0 ( popd & exit /b !ERRORLEVEL! )
+)
 if [%skiptests%] NEQ [ON] ( 
 	ctest -C %cmake_build_type%
-	IF %ERRORLEVEL% NEQ 0 EXIT %ERRORLEVEL%
-	)
-cd ..
+	IF !ERRORLEVEL! NEQ 0 ( popd & exit /b !ERRORLEVEL! )
+)
+popd
 goto :eof
 
 :usage