You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by sz...@apache.org on 2021/07/09 21:38:40 UTC

[nifi-minifi-cpp] branch main updated: MINIFICPP-1579 Add config file checksums to the heartbeat

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 5a4cb5e  MINIFICPP-1579 Add config file checksums to the heartbeat
5a4cb5e is described below

commit 5a4cb5e80c141a4c9a9e7ecbd681121d4a0a0f9c
Author: Ferenc Gerlits <fg...@gmail.com>
AuthorDate: Fri Jul 9 23:14:31 2021 +0200

    MINIFICPP-1579 Add config file checksums to the heartbeat
    
    Created a new ResponseNode type, ConfigurationChecksums, which can be added to
    nifi.c2.root.classes (not included by default) to produce a new section in
    the heartbeat JSON message which shows the SHA256 checksums of minifi.properties
    and config.yml, except for the agent identifier
    
    This closes #1123
    
    Signed-off-by: Marton Szasz <sz...@apache.org>
---
 C2.md                                              |   7 +-
 libminifi/include/core/FlowConfiguration.h         |   6 +-
 .../core/state/nodes/ConfigurationChecksums.h      |  56 ++++++++++
 libminifi/include/properties/Properties.h          |   7 +-
 libminifi/include/utils/ChecksumCalculator.h       |  57 ++++++++++
 libminifi/src/c2/C2Client.cpp                      |  18 ++-
 libminifi/src/core/FlowConfiguration.cpp           |   4 +-
 .../core/state/nodes/ConfigurationChecksums.cpp    |  51 +++++++++
 libminifi/src/properties/Properties.cpp            |   4 +-
 libminifi/src/utils/ChecksumCalculator.cpp         |  93 ++++++++++++++++
 libminifi/test/unit/ChecksumCalculatorTests.cpp    | 123 +++++++++++++++++++++
 .../test/unit/ConfigurationChecksumsTests.cpp      |  85 ++++++++++++++
 12 files changed, 500 insertions(+), 11 deletions(-)

diff --git a/C2.md b/C2.md
index 864e5202..f55c1c3 100644
--- a/C2.md
+++ b/C2.md
@@ -60,7 +60,12 @@ be requested via C2 DESCRIBE manifest command.
 	# Disable/Enable C2
 	nifi.c2.enable=true
 
-	# specify classes for the AST response
+	# specify which sections the heartbeat message should contain
+	#   DeviceInfoNode: basic info about the system (OS, number of cores etc)
+	#   AgentInformation: info about the MiNiFi agent, may include the manifest
+	#   FlowInformation: information about the current flow, including queue sizes
+	#   ConfigurationChecksums: hashes of the configuration files; can be used to detect unexpected modifications
+	# the default is
 	nifi.c2.root.classes=DeviceInfoNode,AgentInformation,FlowInformation
 	
 	# specify C2 protocol -- default is RESTSender if not specified
diff --git a/libminifi/include/core/FlowConfiguration.h b/libminifi/include/core/FlowConfiguration.h
index 13d366d..b6ad108 100644
--- a/libminifi/include/core/FlowConfiguration.h
+++ b/libminifi/include/core/FlowConfiguration.h
@@ -40,6 +40,7 @@
 #include "io/StreamFactory.h"
 #include "core/state/nodes/FlowInformation.h"
 #include "utils/file/FileSystem.h"
+#include "utils/ChecksumCalculator.h"
 #include "utils/OptionalUtils.h"
 
 namespace org {
@@ -92,6 +93,7 @@ class FlowConfiguration : public CoreComponent {
         logger_->log_error("Couldn't find config file \"%s\".", *path);
         config_path_ = path;
       }
+      checksum_calculator_.setFileLocation(*config_path_);
     }
 
     // it is okay if this has already been called
@@ -162,6 +164,8 @@ class FlowConfiguration : public CoreComponent {
     }
   }
 
+  utils::ChecksumCalculator& getChecksumCalculator() { return checksum_calculator_; }
+
  protected:
   void registerResource(const std::string &resource_function) {
     core::ClassLoader::getDefaultClassLoader().registerResource("", resource_function);
@@ -185,8 +189,8 @@ class FlowConfiguration : public CoreComponent {
   std::shared_ptr<io::StreamFactory> stream_factory_;
   std::shared_ptr<Configure> configuration_;
   std::shared_ptr<state::response::FlowVersion> flow_version_;
-
   std::shared_ptr<utils::file::FileSystem> filesystem_;
+  utils::ChecksumCalculator checksum_calculator_;
 
  private:
   std::shared_ptr<logging::Logger> logger_;
diff --git a/libminifi/include/core/state/nodes/ConfigurationChecksums.h b/libminifi/include/core/state/nodes/ConfigurationChecksums.h
new file mode 100644
index 0000000..5b60af6
--- /dev/null
+++ b/libminifi/include/core/state/nodes/ConfigurationChecksums.h
@@ -0,0 +1,56 @@
+/**
+ * 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.
+ */
+
+#pragma once
+
+#include <string>
+#include <vector>
+
+#include "core/Resource.h"
+#include "core/state/nodes/MetricsBase.h"
+#include "utils/ChecksumCalculator.h"
+#include "utils/gsl.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace state {
+namespace response {
+
+class ConfigurationChecksums : public ResponseNode {
+ public:
+  ConfigurationChecksums() = default;
+  explicit ConfigurationChecksums(const std::string& name, const utils::Identifier& uuid = {}) : ResponseNode(name, uuid) {}
+
+  void addChecksumCalculator(utils::ChecksumCalculator& checksum_calculator);
+
+  std::string getName() const override { return "configurationChecksums"; }
+  std::vector<SerializedResponseNode> serialize() override;
+
+ private:
+  std::vector<gsl::not_null<utils::ChecksumCalculator*>> checksum_calculators_;
+};
+
+REGISTER_RESOURCE(ConfigurationChecksums, "Node part of an AST that defines checksums of configuration files in the C2 protocol");
+
+}  // namespace response
+}  // namespace state
+}  // namespace minifi
+}  // namespace nifi
+}  // namespace apache
+}  // namespace org
diff --git a/libminifi/include/properties/Properties.h b/libminifi/include/properties/Properties.h
index 1b3cc94..f11fe05 100644
--- a/libminifi/include/properties/Properties.h
+++ b/libminifi/include/properties/Properties.h
@@ -27,6 +27,7 @@
 #include <utility>
 
 #include "core/logging/Logger.h"
+#include "utils/ChecksumCalculator.h"
 #include "utils/OptionalUtils.h"
 
 namespace org {
@@ -41,7 +42,7 @@ class Properties {
   };
 
  public:
-  Properties(const std::string& name = ""); // NOLINT
+  explicit Properties(const std::string& name = "");
 
   virtual ~Properties() = default;
 
@@ -113,6 +114,8 @@ class Properties {
 
   bool persistProperties();
 
+  utils::ChecksumCalculator& getChecksumCalculator() { return checksum_calculator_; }
+
  protected:
   std::map<std::string, std::string> getProperties() const;
 
@@ -123,6 +126,8 @@ class Properties {
 
   std::string properties_file_;
 
+  utils::ChecksumCalculator checksum_calculator_;
+
   // Mutex for protection
   mutable std::mutex mutex_;
   // Logger
diff --git a/libminifi/include/utils/ChecksumCalculator.h b/libminifi/include/utils/ChecksumCalculator.h
new file mode 100644
index 0000000..a8230bb
--- /dev/null
+++ b/libminifi/include/utils/ChecksumCalculator.h
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string>
+#include <utility>
+
+#include "utils/OptionalUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+class ChecksumCalculator {
+ public:
+  static constexpr const char* CHECKSUM_TYPE = "SHA256";
+  static constexpr size_t LENGTH_OF_HASH_IN_BYTES = 32;
+
+  void setFileLocation(const std::string& file_location);
+  std::string getFileName() const;
+  std::string getChecksum();
+  void invalidateChecksum();
+
+ private:
+  static std::string computeChecksum(const std::string& file_location);
+
+  utils::optional<std::string> file_location_;
+  utils::optional<std::string> file_name_;
+  utils::optional<std::string> checksum_;
+};
+
+inline void ChecksumCalculator::invalidateChecksum() {
+  checksum_.reset();
+}
+
+}  // namespace utils
+}  // namespace minifi
+}  // namespace nifi
+}  // namespace apache
+}  // namespace org
diff --git a/libminifi/src/c2/C2Client.cpp b/libminifi/src/c2/C2Client.cpp
index 7547ba8..5502376 100644
--- a/libminifi/src/c2/C2Client.cpp
+++ b/libminifi/src/c2/C2Client.cpp
@@ -22,6 +22,7 @@
 #include "core/state/nodes/MetricsBase.h"
 #include "core/state/nodes/QueueMetrics.h"
 #include "core/state/nodes/AgentInformation.h"
+#include "core/state/nodes/ConfigurationChecksums.h"
 #include "core/state/nodes/RepositoryMetrics.h"
 #include "properties/Configure.h"
 #include "core/state/UpdateController.h"
@@ -85,22 +86,27 @@ void C2Client::initialize(core::controller::ControllerServiceProvider *controlle
     std::vector<std::string> classes = utils::StringUtils::split(class_csv, ",");
 
     for (const std::string& clazz : classes) {
-      auto processor = std::dynamic_pointer_cast<state::response::ResponseNode>(core::ClassLoader::getDefaultClassLoader().instantiate(clazz, clazz));
-      if (nullptr == processor) {
+      auto response_node = std::dynamic_pointer_cast<state::response::ResponseNode>(core::ClassLoader::getDefaultClassLoader().instantiate(clazz, clazz));
+      if (nullptr == response_node) {
         logger_->log_error("No metric defined for %s", clazz);
         continue;
       }
-      auto identifier = std::dynamic_pointer_cast<state::response::AgentIdentifier>(processor);
+      auto identifier = std::dynamic_pointer_cast<state::response::AgentIdentifier>(response_node);
       if (identifier != nullptr) {
         identifier->setAgentIdentificationProvider(configuration_);
       }
-      auto monitor = std::dynamic_pointer_cast<state::response::AgentMonitor>(processor);
+      auto monitor = std::dynamic_pointer_cast<state::response::AgentMonitor>(response_node);
       if (monitor != nullptr) {
         monitor->addRepository(provenance_repo_);
         monitor->addRepository(flow_file_repo_);
         monitor->setStateMonitor(update_sink);
       }
-      auto flowMonitor = std::dynamic_pointer_cast<state::response::FlowMonitor>(processor);
+      auto configuration_checksums = std::dynamic_pointer_cast<state::response::ConfigurationChecksums>(response_node);
+      if (configuration_checksums) {
+        configuration_checksums->addChecksumCalculator(configuration_->getChecksumCalculator());
+        configuration_checksums->addChecksumCalculator(flow_configuration_->getChecksumCalculator());
+      }
+      auto flowMonitor = std::dynamic_pointer_cast<state::response::FlowMonitor>(response_node);
       if (flowMonitor != nullptr) {
         for (auto &con : connections) {
           flowMonitor->addConnection(con.second);
@@ -109,7 +115,7 @@ void C2Client::initialize(core::controller::ControllerServiceProvider *controlle
         flowMonitor->setFlowVersion(flow_configuration_->getFlowVersion());
       }
       std::lock_guard<std::mutex> guard(metrics_mutex_);
-      root_response_nodes_[processor->getName()] = processor;
+      root_response_nodes_[response_node->getName()] = response_node;
     }
   }
 
diff --git a/libminifi/src/core/FlowConfiguration.cpp b/libminifi/src/core/FlowConfiguration.cpp
index 53104c9..d8aad17 100644
--- a/libminifi/src/core/FlowConfiguration.cpp
+++ b/libminifi/src/core/FlowConfiguration.cpp
@@ -109,7 +109,9 @@ bool FlowConfiguration::persist(const std::string &configuration) {
     logger_->log_debug("Copy %s to %s", *config_path_, config_file_backup);
   }
 
-  return filesystem_->write(*config_path_, configuration);
+  const bool status = filesystem_->write(*config_path_, configuration);
+  checksum_calculator_.invalidateChecksum();
+  return status;
 }
 
 std::unique_ptr<core::ProcessGroup> FlowConfiguration::createRootProcessGroup(const std::string &name, const utils::Identifier &uuid, int version) {
diff --git a/libminifi/src/core/state/nodes/ConfigurationChecksums.cpp b/libminifi/src/core/state/nodes/ConfigurationChecksums.cpp
new file mode 100644
index 0000000..50a40e7
--- /dev/null
+++ b/libminifi/src/core/state/nodes/ConfigurationChecksums.cpp
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "core/state/nodes/ConfigurationChecksums.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace state {
+namespace response {
+
+void ConfigurationChecksums::addChecksumCalculator(utils::ChecksumCalculator& checksum_calculator) {
+  checksum_calculators_.push_back(gsl::make_not_null(&checksum_calculator));
+}
+
+std::vector<SerializedResponseNode> ConfigurationChecksums::serialize() {
+  SerializedResponseNode checksums_node;
+  checksums_node.name = utils::ChecksumCalculator::CHECKSUM_TYPE;
+  checksums_node.children.reserve(checksum_calculators_.size());
+
+  for (auto checksum_calculator : checksum_calculators_) {
+    SerializedResponseNode file_checksum_node;
+    file_checksum_node.name = checksum_calculator->getFileName();
+    file_checksum_node.value = checksum_calculator->getChecksum();
+    checksums_node.children.push_back(file_checksum_node);
+  }
+
+  return std::vector<SerializedResponseNode>{checksums_node};
+}
+
+}  // namespace response
+}  // namespace state
+}  // namespace minifi
+}  // namespace nifi
+}  // namespace apache
+}  // namespace org
diff --git a/libminifi/src/properties/Properties.cpp b/libminifi/src/properties/Properties.cpp
index a062fd0..34d265d 100644
--- a/libminifi/src/properties/Properties.cpp
+++ b/libminifi/src/properties/Properties.cpp
@@ -88,6 +88,7 @@ void Properties::loadConfigureFile(const char *fileName) {
   for (const auto& line : PropertiesFile{file}) {
     properties_[line.getKey()] = {utils::StringUtils::replaceEnvironmentVariables(line.getValue()), false};
   }
+  checksum_calculator_.setFileLocation(properties_file_);
   dirty_ = false;
 }
 
@@ -125,8 +126,9 @@ bool Properties::persistProperties() {
   }
 
   const std::string backup = properties_file_ + ".bak";
-  if (!utils::file::FileUtils::copy_file(properties_file_, backup) && !utils::file::FileUtils::copy_file(new_file, properties_file_)) {
+  if (utils::file::FileUtils::copy_file(properties_file_, backup) == 0 && utils::file::FileUtils::copy_file(new_file, properties_file_) == 0) {
     logger_->log_info("Persisted %s", properties_file_);
+    checksum_calculator_.invalidateChecksum();
     dirty_ = false;
     return true;
   }
diff --git a/libminifi/src/utils/ChecksumCalculator.cpp b/libminifi/src/utils/ChecksumCalculator.cpp
new file mode 100644
index 0000000..b509f3b
--- /dev/null
+++ b/libminifi/src/utils/ChecksumCalculator.cpp
@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "utils/ChecksumCalculator.h"
+
+#include <fstream>
+
+#include "sodium/crypto_hash_sha256.h"
+#include "utils/file/FileUtils.h"
+#include "utils/StringUtils.h"
+
+namespace {
+
+const std::string AGENT_IDENTIFIER_KEY = "nifi.c2.agent.identifier=";
+
+}
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+void ChecksumCalculator::setFileLocation(const std::string& file_location) {
+  file_location_ = file_location;
+  file_name_ = utils::file::get_child_path(file_location);
+  invalidateChecksum();
+}
+
+std::string ChecksumCalculator::getFileName() const {
+  gsl_Expects(file_name_);
+  return *file_name_;
+}
+
+std::string ChecksumCalculator::getChecksum() {
+  gsl_Expects(file_location_);
+  if (!checksum_) {
+    checksum_ = computeChecksum(*file_location_);
+  }
+  return *checksum_;
+}
+
+std::string ChecksumCalculator::computeChecksum(const std::string& file_location) {
+  using org::apache::nifi::minifi::utils::StringUtils;
+
+  std::ifstream input_file{file_location, std::ios::in | std::ios::binary};
+  if (!input_file.is_open()) {
+    throw std::runtime_error(StringUtils::join_pack("Could not open config file '", file_location, "' to compute the checksum: ", std::strerror(errno)));
+  }
+
+  crypto_hash_sha256_state state;
+  crypto_hash_sha256_init(&state);
+
+  std::string line;
+  while (std::getline(input_file, line)) {
+    // skip lines containing the agent identifier, so agents in the same class will have the same checksum
+    if (StringUtils::startsWith(line, AGENT_IDENTIFIER_KEY)) {
+      continue;
+    }
+    if (!input_file.eof()) {  // eof() means we have just read the last line, which was not terminated by a newline
+      line.append("\n");
+    }
+    crypto_hash_sha256_update(&state, reinterpret_cast<const unsigned char*>(line.data()), line.size());
+  }
+  if (input_file.bad()) {
+    throw std::runtime_error(StringUtils::join_pack("Error reading config file '", file_location, "' while computing the checksum: ", std::strerror(errno)));
+  }
+
+  unsigned char hash[LENGTH_OF_HASH_IN_BYTES] = {0};
+  crypto_hash_sha256_final(&state, hash);
+
+  return StringUtils::to_hex(reinterpret_cast<const uint8_t*>(hash), LENGTH_OF_HASH_IN_BYTES);
+}
+
+}  // namespace utils
+}  // namespace minifi
+}  // namespace nifi
+}  // namespace apache
+}  // namespace org
diff --git a/libminifi/test/unit/ChecksumCalculatorTests.cpp b/libminifi/test/unit/ChecksumCalculatorTests.cpp
new file mode 100644
index 0000000..389005e
--- /dev/null
+++ b/libminifi/test/unit/ChecksumCalculatorTests.cpp
@@ -0,0 +1,123 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <numeric>
+
+#include "../TestBase.h"
+#include "utils/TestUtils.h"
+#include "utils/ChecksumCalculator.h"
+
+namespace {
+  constexpr const char* CHECKSUM_FOR_ONE_LINE_OF_TEXT = "e26d1a9f3c3cb9f55e797ce1c9b15e89b4a23d4a301d92da23e684c8a25bf641";
+  constexpr const char* CHECKSUM_FOR_TWO_LINES_OF_TEXT = "7614d0e3b10a3ae41fd50130aa8b83c1bd1248fb6dcfc25cb135665ea3c4f8c8";
+}
+
+TEST_CASE("ChecksumCalculator can calculate the checksum, which is equal to sha256sum", "[ChecksumCalculator]") {
+  TestController test_controller;
+  std::string test_dir = utils::createTempDir(&test_controller);
+  std::string file_location = utils::putFileToDir(test_dir, "simple.txt", "one line of text\n");
+
+  REQUIRE(std::string{utils::ChecksumCalculator::CHECKSUM_TYPE} == std::string{"SHA256"});
+  // the first size_t{} is required by Catch2; it can't use a constexpr expression directly
+  REQUIRE(size_t{utils::ChecksumCalculator::LENGTH_OF_HASH_IN_BYTES} == size_t{32});
+
+  utils::ChecksumCalculator checksum_calculator;
+  checksum_calculator.setFileLocation(file_location);
+  REQUIRE(checksum_calculator.getChecksum() == CHECKSUM_FOR_ONE_LINE_OF_TEXT);
+}
+
+TEST_CASE("On Windows text files, the checksum calculated is also the same as sha256sum", "[ChecksumCalculator]") {
+  TestController test_controller;
+  std::string test_dir = utils::createTempDir(&test_controller);
+  std::string file_location = utils::putFileToDir(test_dir, "simple.txt", "one line of text\r\n");
+
+  utils::ChecksumCalculator checksum_calculator;
+  checksum_calculator.setFileLocation(file_location);
+  REQUIRE(checksum_calculator.getChecksum() == "94fc46c62ef6cc5b45cbad9fd53116cfb15a80960a9b311c1c27e5b5265ad4b4");
+}
+
+TEST_CASE("The checksum can be reset and recomputed", "[ChecksumCalculator]") {
+  TestController test_controller;
+  std::string test_dir = utils::createTempDir(&test_controller);
+  std::string file_location = utils::putFileToDir(test_dir, "simple.txt", "one line of text\n");
+
+  utils::ChecksumCalculator checksum_calculator;
+  checksum_calculator.setFileLocation(file_location);
+  REQUIRE(checksum_calculator.getChecksum() == CHECKSUM_FOR_ONE_LINE_OF_TEXT);
+
+  std::ofstream append_to_file(file_location, std::ios::binary | std::ios::app);
+  append_to_file << "another line of text\n";
+  append_to_file.close();
+
+  REQUIRE(checksum_calculator.getChecksum() == CHECKSUM_FOR_ONE_LINE_OF_TEXT);  // not updated, needs to be notified
+
+  checksum_calculator.invalidateChecksum();
+  REQUIRE(checksum_calculator.getChecksum() == CHECKSUM_FOR_TWO_LINES_OF_TEXT);  // now it is updated
+}
+
+TEST_CASE("If the file location is updated, the checksum will be recomputed", "[ChecksumCalculator]") {
+  TestController test_controller;
+  std::string test_dir = utils::createTempDir(&test_controller);
+  std::string file_location = utils::putFileToDir(test_dir, "simple.txt", "one line of text\n");
+
+  utils::ChecksumCalculator checksum_calculator;
+  checksum_calculator.setFileLocation(file_location);
+  REQUIRE(checksum_calculator.getChecksum() == CHECKSUM_FOR_ONE_LINE_OF_TEXT);
+
+  std::string other_file_location = utils::putFileToDir(test_dir, "long.txt", "one line of text\nanother line of text\n");
+  checksum_calculator.setFileLocation(other_file_location);
+  REQUIRE(checksum_calculator.getChecksum() == CHECKSUM_FOR_TWO_LINES_OF_TEXT);
+}
+
+TEST_CASE("Checksums can be computed for binary (eg. encrypted) files, too", "[ChecksumCalculator]") {
+  TestController test_controller;
+  std::string test_dir = utils::createTempDir(&test_controller);
+  std::string binary_data(size_t{256}, '\0');
+  std::iota(binary_data.begin(), binary_data.end(), 'x');
+  std::string file_location = utils::putFileToDir(test_dir, "simple.txt", binary_data);
+
+  utils::ChecksumCalculator checksum_calculator;
+  checksum_calculator.setFileLocation(file_location);
+  REQUIRE(checksum_calculator.getChecksum() == "bdec77160c394c067419735de757e4daa1c4679ea45e82a33fa8f706eed87709");
+}
+
+TEST_CASE("The agent identifier is excluded from the checksum", "[ChecksumCalculator]") {
+  TestController test_controller;
+  std::string test_dir = utils::createTempDir(&test_controller);
+  std::string file_location_1 = utils::putFileToDir(test_dir, "agent_one.txt",
+      "nifi.c2.agent.class=Test\n"
+      "nifi.c2.agent.identifier=Test-111\n"
+      "nifi.c2.agent.heartbeat.period=10000\n");
+  std::string file_location_2 = utils::putFileToDir(test_dir, "agent_two.txt",
+      "nifi.c2.agent.class=Test\n"
+      "nifi.c2.agent.identifier=Test-222\n"
+      "nifi.c2.agent.heartbeat.period=10000\n");
+
+  utils::ChecksumCalculator checksum_calculator_1;
+  checksum_calculator_1.setFileLocation(file_location_1);
+  utils::ChecksumCalculator checksum_calculator_2;
+  checksum_calculator_2.setFileLocation(file_location_2);
+
+  REQUIRE(checksum_calculator_1.getChecksum() == checksum_calculator_2.getChecksum());
+}
+
+TEST_CASE("ChecksumCalculator::getChecksum will throw if the file does not exist", "[ChecksumCalculator]") {
+  utils::ChecksumCalculator checksum_calculator;
+  checksum_calculator.setFileLocation("/this/file/does/not/exist/84a77fd9-16b3-49d2-aead-a1f9e58e530d");
+
+  REQUIRE_THROWS(checksum_calculator.getChecksum());
+}
diff --git a/libminifi/test/unit/ConfigurationChecksumsTests.cpp b/libminifi/test/unit/ConfigurationChecksumsTests.cpp
new file mode 100644
index 0000000..a18ff47
--- /dev/null
+++ b/libminifi/test/unit/ConfigurationChecksumsTests.cpp
@@ -0,0 +1,85 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "../TestBase.h"
+#include "utils/TestUtils.h"
+#include "core/state/nodes/ConfigurationChecksums.h"
+#include "utils/ChecksumCalculator.h"
+
+using org::apache::nifi::minifi::state::response::ConfigurationChecksums;
+
+TEST_CASE("If no checksum calculators are added, we get an empty node", "[ConfigurationChecksums]") {
+  ConfigurationChecksums configuration_checksums;
+
+  const auto serialized_response_nodes = configuration_checksums.serialize();
+
+  REQUIRE(serialized_response_nodes.size() == 1);
+  const auto& checksum_node = serialized_response_nodes[0];
+  REQUIRE(checksum_node.children.size() == 0);
+}
+
+TEST_CASE("If one checksum calculator is added, we get a node with one child", "[ConfigurationChecksums]") {
+  TestController test_controller;
+  std::string test_dir = utils::createTempDir(&test_controller);
+  std::string file_location = utils::putFileToDir(test_dir, "simple.txt", "one line of text\n");
+
+  utils::ChecksumCalculator checksum_calculator;
+  checksum_calculator.setFileLocation(file_location);
+
+  ConfigurationChecksums configuration_checksums;
+  configuration_checksums.addChecksumCalculator(checksum_calculator);
+
+  const auto serialized_response_nodes = configuration_checksums.serialize();
+
+  REQUIRE(serialized_response_nodes.size() == 1);
+  const auto& checksum_node = serialized_response_nodes[0];
+  REQUIRE(checksum_node.children.size() == 1);
+
+  const auto& file_checksum_node = checksum_node.children[0];
+  REQUIRE(file_checksum_node.name == "simple.txt");
+  REQUIRE(file_checksum_node.value == "e26d1a9f3c3cb9f55e797ce1c9b15e89b4a23d4a301d92da23e684c8a25bf641");
+}
+
+TEST_CASE("If two checksum calculators are added, we get a node with two children", "[ConfigurationChecksums]") {
+  TestController test_controller;
+  std::string test_dir = utils::createTempDir(&test_controller);
+  std::string file_location_1 = utils::putFileToDir(test_dir, "first.txt", "this is the first file\n");
+  std::string file_location_2 = utils::putFileToDir(test_dir, "second.txt", "this is the second file\n");
+
+  utils::ChecksumCalculator checksum_calculator_1;
+  checksum_calculator_1.setFileLocation(file_location_1);
+  utils::ChecksumCalculator checksum_calculator_2;
+  checksum_calculator_2.setFileLocation(file_location_2);
+
+  ConfigurationChecksums configuration_checksums;
+  configuration_checksums.addChecksumCalculator(checksum_calculator_1);
+  configuration_checksums.addChecksumCalculator(checksum_calculator_2);
+
+  const auto serialized_response_nodes = configuration_checksums.serialize();
+
+  REQUIRE(serialized_response_nodes.size() == 1);
+  const auto& checksum_node = serialized_response_nodes[0];
+  REQUIRE(checksum_node.children.size() == 2);
+
+  const auto& file_checksum_node_1 = checksum_node.children[0];
+  REQUIRE(file_checksum_node_1.name == "first.txt");
+  REQUIRE(file_checksum_node_1.value == "413a91c71ae3cc76e641a7fdfe8a18ab043f9eb9838edcaadfb71194385653c2");
+
+  const auto& file_checksum_node_2 = checksum_node.children[1];
+  REQUIRE(file_checksum_node_2.name == "second.txt");
+  REQUIRE(file_checksum_node_2.value == "d2931f19097c7a8382567a45f617d567c26d10b547c7243f04e77f83af620d42");
+}