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 2021/02/01 14:48:20 UTC

[nifi-minifi-cpp] branch main updated (f50d87d -> 8f8136a)

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

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


    from f50d87d  MINIFICPP-1453 Verify Supported TLS Protocols
     new c51df42  MINIFICPP-1462 Fix incorrect usage of move constructors and assignment operators
     new d72a7af  MINIFICPP-1466 Fine tune configuration to save system resources
     new d3a4f84  MINIFICPP-1465 - Add AWS to docker minimal image and 16.04 all tests
     new 8f8136a  MINIFICPP-1375 more robust UCRT search

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .github/workflows/ci.yml                           |  2 +-
 CMakeLists.txt                                     | 22 +++++++++++-----
 conf/minifi.properties                             |  6 +++--
 docker/Dockerfile                                  |  4 +--
 .../mqtt/controllerservice/MQTTControllerService.h | 16 +++++-------
 extensions/standard-processors/processors/GetTCP.h |  5 +++-
 libminifi/include/c2/PayloadSerializer.h           |  2 +-
 libminifi/include/core/ConfigurableComponent.h     |  6 ++++-
 libminifi/include/core/Connectable.h               |  6 ++++-
 libminifi/include/core/ProcessorNode.h             |  6 +++--
 .../core/controller/ControllerServiceProvider.h    | 10 ++++----
 .../controller/StandardControllerServiceProvider.h | 13 ++++------
 libminifi/include/core/state/UpdateController.h    | 30 ++++++++++------------
 libminifi/include/sitetosite/Peer.h                | 12 +++++----
 libminifi/src/core/ConfigurableComponent.cpp       |  7 -----
 libminifi/src/core/Connectable.cpp                 |  9 -------
 libminifi/src/core/ProcessorNode.cpp               | 11 --------
 libminifi/src/core/state/UpdateController.cpp      | 17 ------------
 18 files changed, 78 insertions(+), 106 deletions(-)


[nifi-minifi-cpp] 03/04: MINIFICPP-1465 - Add AWS to docker minimal image and 16.04 all tests

Posted by ab...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit d3a4f843d44392c2f4e1cd9acddd5e84bde70ae8
Author: Gabor Gyimesi <ga...@gmail.com>
AuthorDate: Thu Jan 28 17:00:33 2021 +0100

    MINIFICPP-1465 - Add AWS to docker minimal image and 16.04 all tests
    
    Signed-off-by: Arpad Boda <ab...@apache.org>
    
    This closes #982
---
 .github/workflows/ci.yml | 2 +-
 docker/Dockerfile        | 4 ++--
 2 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml
index ac4b741..211ec49 100644
--- a/.github/workflows/ci.yml
+++ b/.github/workflows/ci.yml
@@ -156,7 +156,7 @@ jobs:
           sudo apt install -y ccache openjdk-8-jdk maven
           echo "PATH=/usr/lib/ccache:$PATH" >> $GITHUB_ENV
       - id: build
-        run: sudo mount tmpfs -t tmpfs /tmp && ./bootstrap.sh -e -t && cd build  && cmake -DUSE_SHARED_LIBS= -DENABLE_LIBRDKAFKA=ON -DENABLE_OPC=ON -DENABLE_SFTP=ON -DENABLE_MQTT=ON -DENABLE_COAP=ON -DENABLE_PYTHON=ON -DENABLE_SQL=ON -DSTRICT_GSL_CHECKS=AUDIT .. &&  cmake --build . --parallel 4  && make test ARGS="--timeout 300 -j8 --output-on-failure"
+        run: sudo mount tmpfs -t tmpfs /tmp && ./bootstrap.sh -e -t && cd build  && cmake -DUSE_SHARED_LIBS= -DENABLE_LIBRDKAFKA=ON -DENABLE_OPC=ON -DENABLE_SFTP=ON -DENABLE_MQTT=ON -DENABLE_COAP=ON -DENABLE_PYTHON=ON -DENABLE_SQL=ON -DENABLE_AWS=ON -DSTRICT_GSL_CHECKS=AUDIT .. &&  cmake --build . --parallel 4  && make test ARGS="--timeout 300 -j8 --output-on-failure"
   debian:
     name: "debian"
     runs-on: ubuntu-18.04
diff --git a/docker/Dockerfile b/docker/Dockerfile
index 347745a..e97ec28 100644
--- a/docker/Dockerfile
+++ b/docker/Dockerfile
@@ -77,7 +77,7 @@ FROM build_deps AS build_minimal
 RUN cd ${MINIFI_BASE_DIR} \
   && mkdir build \
   && cd build \
-  && cmake -DDISABLE_LIBARCHIVE=ON -DDISABLE_SCRIPTING=ON -DENABLE_LIBRDKAFKA=ON -DSKIP_TESTS=true -DCMAKE_BUILD_TYPE=MinSizeRel .. \
+  && cmake -DDISABLE_LIBARCHIVE=ON -DDISABLE_SCRIPTING=ON -DENABLE_LIBRDKAFKA=ON -DENABLE_AWS=ON -DSKIP_TESTS=true -DCMAKE_BUILD_TYPE=MinSizeRel .. \
   && make -j$(nproc) package \
   && tar -xzvf ${MINIFI_BASE_DIR}/build/nifi-minifi-cpp-${MINIFI_VERSION}-bin.tar.gz -C ${MINIFI_BASE_DIR}
 
@@ -127,7 +127,7 @@ RUN cd ${MINIFI_BASE_DIR} \
     -DDISABLE_CURL=${DISABLE_CURL} -DDISABLE_JEMALLOC=${DISABLE_JEMALLOC} -DDISABLE_CIVET=${DISABLE_CIVET} \
     -DDISABLE_EXPRESSION_LANGUAGE=${DISABLE_EXPRESSION_LANGUAGE} -DDISABLE_ROCKSDB=${DISABLE_ROCKSDB} \
     -DDISABLE_LIBARCHIVE=${DISABLE_LIBARCHIVE} -DDISABLE_LZMA=${DISABLE_LZMA} -DDISABLE_BZIP2=${DISABLE_BZIP2} \
-    -DDISABLE_SCRIPTING=${DISABLE_SCRIPTING} -DDISABLE_CONTROLLER=${DISABLE_CONTROLLER} .. \
+    -DDISABLE_SCRIPTING=${DISABLE_SCRIPTING} -DDISABLE_CONTROLLER=${DISABLE_CONTROLLER} -DCMAKE_BUILD_TYPE=Release .. \
   && make -j$(nproc) package \
   && tar -xzvf ${MINIFI_BASE_DIR}/build/nifi-minifi-cpp-${MINIFI_VERSION}-bin.tar.gz -C ${MINIFI_BASE_DIR}
 


[nifi-minifi-cpp] 04/04: MINIFICPP-1375 more robust UCRT search

Posted by ab...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 8f8136afb8fc3c8ec21e1ca7a98e63e8461d4437
Author: Marton Szasz <sz...@gmail.com>
AuthorDate: Fri Jan 29 23:11:22 2021 +0100

    MINIFICPP-1375 more robust UCRT search
    
    On some systems, the UCRT redistributable DLLs are not under
    "$ENV{WindowsSdkDir}Redist\\ucrt\\DLLs\\$ENV{Platform}", but
    at "$ENV{WindowsSdkDir}Redist\\$ENV{WindowsSDKVersion}ucrt\\DLLs\\$ENV{Platform}",
    contrary to MS docs. This commit tries the documented place
    first, then falls back to the observed one.
    
    Signed-off-by: Marton Szasz <sz...@gmail.com>
    
    MINIFICPP-1375 not finding ucrt is now a fatal error
    
    Signed-off-by: Arpad Boda <ab...@apache.org>
    
    This closes #985
---
 CMakeLists.txt | 22 +++++++++++++++-------
 1 file changed, 15 insertions(+), 7 deletions(-)

diff --git a/CMakeLists.txt b/CMakeLists.txt
index d6f81d3..ff0024f 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -650,13 +650,21 @@ if(WIN32)
 	if (MSI_REDISTRIBUTE_UCRT_NONASL)
 		set(UCRT_DIR_NAT "$ENV{WindowsSdkDir}Redist\\ucrt\\DLLs\\$ENV{Platform}")
 		file(TO_CMAKE_PATH "${UCRT_DIR_NAT}" UCRT_DIR)
-		message("Using UCRT from ${UCRT_DIR}")
-		file(GLOB UCRT_DLLS "${UCRT_DIR}/*.dll")
-		file(COPY ${UCRT_DLLS} DESTINATION "${CMAKE_CURRENT_BINARY_DIR}/ucrt")
-		install(DIRECTORY "${CMAKE_CURRENT_BINARY_DIR}/ucrt/"
-			DESTINATION bin
-			COMPONENT bin
-		)
+		if (NOT EXISTS "${UCRT_DIR}")
+			set(UCRT_DIR_NAT "$ENV{WindowsSdkDir}Redist\\$ENV{WindowsSDKVersion}ucrt\\DLLs\\$ENV{Platform}")
+			file(TO_CMAKE_PATH "${UCRT_DIR_NAT}" UCRT_DIR)
+		endif()
+		if (NOT EXISTS "${UCRT_DIR}")
+			message(FATAL_ERROR "Couldn't find UCRT")
+		else()
+			message("Using UCRT from ${UCRT_DIR}")
+			file(GLOB UCRT_DLLS "${UCRT_DIR}/*.dll")
+			file(COPY ${UCRT_DLLS} DESTINATION "${CMAKE_CURRENT_BINARY_DIR}/ucrt")
+			install(DIRECTORY "${CMAKE_CURRENT_BINARY_DIR}/ucrt/"
+					DESTINATION bin
+					COMPONENT bin
+					)
+		endif()
 	endif()
 
 	if (INSTALLER_MERGE_MODULES)


[nifi-minifi-cpp] 02/04: MINIFICPP-1466 Fine tune configuration to save system resources

Posted by ab...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit d72a7afbf51c014de4cc289f06bc04b27b2d56ce
Author: Gabor Gyimesi <ga...@gmail.com>
AuthorDate: Mon Feb 1 12:44:19 2021 +0100

    MINIFICPP-1466 Fine tune configuration to save system resources
    
    Signed-off-by: Arpad Boda <ab...@apache.org>
    
    This closes #986
---
 conf/minifi.properties | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)

diff --git a/conf/minifi.properties b/conf/minifi.properties
index 56113de..e5f2a1a 100644
--- a/conf/minifi.properties
+++ b/conf/minifi.properties
@@ -18,7 +18,7 @@ nifi.version=0.7.0
 nifi.flow.configuration.file=./conf/config.yml
 nifi.administrative.yield.duration=30 sec
 # If a component has no work to do (is "bored"), how long should we wait before checking again for work?
-nifi.bored.yield.duration=10 millis
+nifi.bored.yield.duration=100 millis
 
 # Provenance Repository #
 nifi.provenance.repository.directory.default=${MINIFI_HOME}/provenance_repository
@@ -26,6 +26,8 @@ nifi.provenance.repository.max.storage.time=1 MIN
 nifi.provenance.repository.max.storage.size=1 MB
 nifi.flowfile.repository.directory.default=${MINIFI_HOME}/flowfile_repository
 nifi.database.content.repository.directory.default=${MINIFI_HOME}/content_repository
+nifi.provenance.repository.class.name=NoOpRepository
+nifi.content.repository.class.name=DatabaseContentRepository
 
 #nifi.remote.input.secure=true
 #nifi.security.need.ClientAuth=
@@ -75,7 +77,7 @@ nifi.database.content.repository.directory.default=${MINIFI_HOME}/content_reposi
 #nifi.c2.rest.url.ack=
 nifi.c2.root.classes=DeviceInfoNode,AgentInformation,FlowInformation
 ## Minimize heartbeat payload size by excluding agent manifest from the heartbeat
-#nifi.c2.full.heartbeat=false
+nifi.c2.full.heartbeat=false
 ## heartbeat 4 times a second
 #nifi.c2.agent.heartbeat.period=250
 ## define parameters about your agent 


[nifi-minifi-cpp] 01/04: MINIFICPP-1462 Fix incorrect usage of move constructors and assignment operators

Posted by ab...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit c51df42bcfc6d576e8fb57be4385fc6f6a16393c
Author: Robert Kalmar <rk...@cloudera.com>
AuthorDate: Thu Jan 28 09:58:54 2021 +0100

    MINIFICPP-1462 Fix incorrect usage of move constructors and assignment operators
    
    Signed-off-by: Arpad Boda <ab...@apache.org>
    
    This closes #984
---
 .../mqtt/controllerservice/MQTTControllerService.h | 16 +++++-------
 extensions/standard-processors/processors/GetTCP.h |  5 +++-
 libminifi/include/c2/PayloadSerializer.h           |  2 +-
 libminifi/include/core/ConfigurableComponent.h     |  6 ++++-
 libminifi/include/core/Connectable.h               |  6 ++++-
 libminifi/include/core/ProcessorNode.h             |  6 +++--
 .../core/controller/ControllerServiceProvider.h    | 10 ++++----
 .../controller/StandardControllerServiceProvider.h | 13 ++++------
 libminifi/include/core/state/UpdateController.h    | 30 ++++++++++------------
 libminifi/include/sitetosite/Peer.h                | 12 +++++----
 libminifi/src/core/ConfigurableComponent.cpp       |  7 -----
 libminifi/src/core/Connectable.cpp                 |  9 -------
 libminifi/src/core/ProcessorNode.cpp               | 11 --------
 libminifi/src/core/state/UpdateController.cpp      | 17 ------------
 14 files changed, 56 insertions(+), 94 deletions(-)

diff --git a/extensions/mqtt/controllerservice/MQTTControllerService.h b/extensions/mqtt/controllerservice/MQTTControllerService.h
index 956475a..7aa2489 100644
--- a/extensions/mqtt/controllerservice/MQTTControllerService.h
+++ b/extensions/mqtt/controllerservice/MQTTControllerService.h
@@ -49,17 +49,15 @@ class Message {
       : topic_(topic),
         data_((uint8_t*) data, ((uint8_t*)data + dataLen)) {
   }
-  explicit Message(const Message &&other)
-      : topic_(std::move(other.topic_)),
-        data_(std::move(other.data_)) {
-  }
+
+  Message(const Message &other) = default;
+  Message(Message &&other) = default;
+
   ~Message() = default;
 
-  Message &operator=(const Message &&other) {
-    topic_ = std::move(other.topic_);
-    data_ = std::move(other.data_);
-    return *this;
-  }
+  Message &operator=(const Message &other) = default;
+  Message &operator=(Message &&other) = default;
+  
   std::string topic_;
   std::vector<uint8_t> data_;
 };
diff --git a/extensions/standard-processors/processors/GetTCP.h b/extensions/standard-processors/processors/GetTCP.h
index 48883ed..f0cb78c 100644
--- a/extensions/standard-processors/processors/GetTCP.h
+++ b/extensions/standard-processors/processors/GetTCP.h
@@ -52,8 +52,11 @@ class SocketAfterExecute : public utils::AfterExecute<int> {
         list_(list) {
   }
 
-  SocketAfterExecute(const SocketAfterExecute&&) = delete;
+  SocketAfterExecute(const SocketAfterExecute&) = delete;
+  SocketAfterExecute(SocketAfterExecute&&) = delete;
+
   SocketAfterExecute& operator=(const SocketAfterExecute&) = delete;
+  SocketAfterExecute& operator=(SocketAfterExecute&&) = delete;
 
   ~SocketAfterExecute() override = default;
 
diff --git a/libminifi/include/c2/PayloadSerializer.h b/libminifi/include/c2/PayloadSerializer.h
index 3c71d09..6c72299 100644
--- a/libminifi/include/c2/PayloadSerializer.h
+++ b/libminifi/include/c2/PayloadSerializer.h
@@ -78,7 +78,7 @@ class PayloadSerializer {
     uint8_t st;
     uint32_t size = gsl::narrow<uint32_t>(payload.getNestedPayloads().size());
     stream->write(size);
-    for (auto nested_payload : payload.getNestedPayloads()) {
+    for (const auto &nested_payload : payload.getNestedPayloads()) {
       op = opToInt(nested_payload.getOperation());
       stream->write(op);
       stream->write(&st, 1);
diff --git a/libminifi/include/core/ConfigurableComponent.h b/libminifi/include/core/ConfigurableComponent.h
index e4af935..0cd8723 100644
--- a/libminifi/include/core/ConfigurableComponent.h
+++ b/libminifi/include/core/ConfigurableComponent.h
@@ -50,7 +50,11 @@ class ConfigurableComponent {
  public:
   ConfigurableComponent();
 
-  explicit ConfigurableComponent(const ConfigurableComponent &&other);
+  ConfigurableComponent(const ConfigurableComponent &other) = delete;
+  ConfigurableComponent(ConfigurableComponent &&other) = delete;
+
+  ConfigurableComponent& operator=(const ConfigurableComponent &other) = delete;
+  ConfigurableComponent& operator=(ConfigurableComponent &&other) = delete;
 
   /**
    * Get property using the provided name.
diff --git a/libminifi/include/core/Connectable.h b/libminifi/include/core/Connectable.h
index 2d54211..f9b6e5e 100644
--- a/libminifi/include/core/Connectable.h
+++ b/libminifi/include/core/Connectable.h
@@ -51,7 +51,11 @@ class Connectable : public CoreComponent {
 
   explicit Connectable(const std::string &name, const utils::Identifier &uuid);
 
-  explicit Connectable(const Connectable &&other);
+  Connectable(const Connectable &other) = delete;
+  Connectable(Connectable &&other) = delete;
+
+  Connectable& operator=(const Connectable &other) = delete;
+  Connectable& operator=(Connectable&& other) = delete;
 
   bool setSupportedRelationships(const std::set<Relationship> &relationships);
 
diff --git a/libminifi/include/core/ProcessorNode.h b/libminifi/include/core/ProcessorNode.h
index 0844a09..786456c 100644
--- a/libminifi/include/core/ProcessorNode.h
+++ b/libminifi/include/core/ProcessorNode.h
@@ -40,9 +40,11 @@ class ProcessorNode : public ConfigurableComponent, public Connectable {
  public:
   explicit ProcessorNode(const std::shared_ptr<Connectable> &processor);
 
-  explicit ProcessorNode(const ProcessorNode &other);
+  ProcessorNode(const ProcessorNode &other) = delete;
+  ProcessorNode(ProcessorNode &&other) = delete;
 
-  explicit ProcessorNode(const ProcessorNode &&other);
+  ProcessorNode& operator=(const ProcessorNode &other) = delete;
+  ProcessorNode& operator=(ProcessorNode &&other) = delete;
 
   /**
    * Get property using the provided name.
diff --git a/libminifi/include/core/controller/ControllerServiceProvider.h b/libminifi/include/core/controller/ControllerServiceProvider.h
index 24861eb..0009ffd 100644
--- a/libminifi/include/core/controller/ControllerServiceProvider.h
+++ b/libminifi/include/core/controller/ControllerServiceProvider.h
@@ -58,11 +58,11 @@ class ControllerServiceProvider : public CoreComponent, public ConfigurableCompo
         controller_map_(services) {
   }
 
-  explicit ControllerServiceProvider(const ControllerServiceProvider &&other)
-      : CoreComponent(std::move(other)),
-        ConfigurableComponent(std::move(other)),
-        controller_map_(std::move(other.controller_map_)) {
-  }
+  ControllerServiceProvider(const ControllerServiceProvider &other) = delete;
+  ControllerServiceProvider(ControllerServiceProvider &&other) = delete;
+
+  ControllerServiceProvider& operator=(const ControllerServiceProvider &other) = delete;
+  ControllerServiceProvider& operator=(ControllerServiceProvider &&other) = delete;
 
   virtual ~ControllerServiceProvider() = default;
 
diff --git a/libminifi/include/core/controller/StandardControllerServiceProvider.h b/libminifi/include/core/controller/StandardControllerServiceProvider.h
index 638c1da..faa2106 100644
--- a/libminifi/include/core/controller/StandardControllerServiceProvider.h
+++ b/libminifi/include/core/controller/StandardControllerServiceProvider.h
@@ -62,14 +62,11 @@ class StandardControllerServiceProvider : public ControllerServiceProvider, publ
         logger_(logging::LoggerFactory<StandardControllerServiceProvider>::getLogger()) {
   }
 
-  explicit StandardControllerServiceProvider(const StandardControllerServiceProvider && other)
-      : ControllerServiceProvider(std::move(other)),
-        agent_(std::move(other.agent_)),
-        extension_loader_(other.extension_loader_),
-        root_group_(std::move(other.root_group_)),
-        configuration_(other.configuration_),
-        logger_(logging::LoggerFactory<StandardControllerServiceProvider>::getLogger()) {
-  }
+  StandardControllerServiceProvider(const StandardControllerServiceProvider &other) = delete;
+  StandardControllerServiceProvider(StandardControllerServiceProvider &&other) = delete;
+
+  StandardControllerServiceProvider& operator=(const StandardControllerServiceProvider &other) = delete;
+  StandardControllerServiceProvider& operator=(StandardControllerServiceProvider &&other) = delete;
 
   void setRootGroup(std::shared_ptr<ProcessGroup> rg) {
     root_group_ = rg;
diff --git a/libminifi/include/core/state/UpdateController.h b/libminifi/include/core/state/UpdateController.h
index 1d4c96e..3bfe9a5 100644
--- a/libminifi/include/core/state/UpdateController.h
+++ b/libminifi/include/core/state/UpdateController.h
@@ -50,9 +50,9 @@ class UpdateStatus {
  public:
   UpdateStatus(UpdateState state, int16_t reason = 0); // NOLINT
 
-  UpdateStatus(const UpdateStatus &other);
+  UpdateStatus(const UpdateStatus &other) = default;
 
-  UpdateStatus(const UpdateStatus &&other);
+  UpdateStatus(UpdateStatus &&other) = default;
 
   UpdateState getState() const;
 
@@ -60,9 +60,9 @@ class UpdateStatus {
 
   int16_t getReadonCode() const;
 
-  UpdateStatus &operator=(const UpdateStatus &&other);
+  UpdateStatus &operator=(UpdateStatus &&other) = default;
 
-  UpdateStatus &operator=(const UpdateStatus &other);
+  UpdateStatus &operator=(const UpdateStatus &other) = default;
  private:
   UpdateState state_;
   std::string error_;
@@ -72,18 +72,16 @@ class UpdateStatus {
 class Update {
  public:
   Update()
-      : status_(UpdateStatus(UpdateState::INITIATE, 0)) {
+      : status_(UpdateState::INITIATE, 0) {
   }
 
   Update(UpdateStatus status) // NOLINT
-      : status_(status) {
+      : status_(std::move(status)) {
   }
 
   Update(const Update &other) = default;
 
-  Update(const Update &&other)
-      : status_(std::move(other.status_)) {
-  }
+  Update(Update &&other) = default;
 
   virtual ~Update() = default;
 
@@ -95,10 +93,7 @@ class Update {
     return status_;
   }
 
-  Update &operator=(const Update &&other) {
-    status_ = std::move(other.status_);
-    return *this;
-  }
+  Update &operator=(Update &&other) = default;
 
   Update &operator=(const Update &other) = default;
 
@@ -120,13 +115,14 @@ class UpdateRunner : public utils::AfterExecute<Update> {
         delay_(delay) {
   }
 
-  explicit UpdateRunner(UpdateRunner && other)
-      : running_(std::move(other.running_)),
-        delay_(std::move(other.delay_)) {
-  }
+  UpdateRunner(const UpdateRunner &other) = delete;
+  UpdateRunner(UpdateRunner &&other) = delete;
 
   ~UpdateRunner() = default;
 
+  UpdateRunner& operator=(const UpdateRunner &other) = delete;
+  UpdateRunner& operator=(UpdateRunner &&other) = delete;
+
   virtual bool isFinished(const Update &result) {
     if ((result.getStatus().getState() == UpdateState::FULLY_APPLIED || result.getStatus().getState() == UpdateState::READ_COMPLETE) && *running_) {
       return false;
diff --git a/libminifi/include/sitetosite/Peer.h b/libminifi/include/sitetosite/Peer.h
index 4217be3..19d0a03 100644
--- a/libminifi/include/sitetosite/Peer.h
+++ b/libminifi/include/sitetosite/Peer.h
@@ -108,11 +108,13 @@ class PeerStatus {
         flow_file_count_(flow_file_count),
         query_for_peers_(query_for_peers) {
   }
-  PeerStatus(const PeerStatus &&other)
-      : peer_(std::move(other.peer_)),
-        flow_file_count_(std::move(other.flow_file_count_)),
-        query_for_peers_(std::move(other.query_for_peers_)) {
-  }
+
+  PeerStatus(const PeerStatus &other) = default;
+  PeerStatus(PeerStatus &&other) = default;
+
+  PeerStatus& operator=(const PeerStatus &other) = default;
+  PeerStatus& operator=(PeerStatus &&other) = default;
+
   const std::shared_ptr<Peer> &getPeer() const {
     return peer_;
   }
diff --git a/libminifi/src/core/ConfigurableComponent.cpp b/libminifi/src/core/ConfigurableComponent.cpp
index 20dbaa2..f97b78f 100644
--- a/libminifi/src/core/ConfigurableComponent.cpp
+++ b/libminifi/src/core/ConfigurableComponent.cpp
@@ -37,13 +37,6 @@ ConfigurableComponent::ConfigurableComponent()
       logger_(logging::LoggerFactory<ConfigurableComponent>::getLogger()) {
 }
 
-ConfigurableComponent::ConfigurableComponent(const ConfigurableComponent &&other)
-    : accept_all_properties_(false),
-      properties_(std::move(other.properties_)),
-      dynamic_properties_(std::move(other.dynamic_properties_)),
-      logger_(logging::LoggerFactory<ConfigurableComponent>::getLogger()) {
-}
-
 ConfigurableComponent::~ConfigurableComponent() = default;
 
 bool ConfigurableComponent::getProperty(const std::string &name, Property &prop) const {
diff --git a/libminifi/src/core/Connectable.cpp b/libminifi/src/core/Connectable.cpp
index 101a7c4..310d2de 100644
--- a/libminifi/src/core/Connectable.cpp
+++ b/libminifi/src/core/Connectable.cpp
@@ -43,15 +43,6 @@ Connectable::Connectable(const std::string &name)
       logger_(logging::LoggerFactory<Connectable>::getLogger()) {
 }
 
-Connectable::Connectable(const Connectable &&other)
-    : CoreComponent(std::move(other)),
-      max_concurrent_tasks_(std::move(other.max_concurrent_tasks_)),
-      connectable_version_(std::move(other.connectable_version_)),
-      logger_(std::move(other.logger_)) {
-  has_work_ = other.has_work_.load();
-  strategy_ = other.strategy_.load();
-}
-
 Connectable::~Connectable() = default;
 
 bool Connectable::setSupportedRelationships(const std::set<core::Relationship> &relationships) {
diff --git a/libminifi/src/core/ProcessorNode.cpp b/libminifi/src/core/ProcessorNode.cpp
index ba7bc7b..7db71e2 100644
--- a/libminifi/src/core/ProcessorNode.cpp
+++ b/libminifi/src/core/ProcessorNode.cpp
@@ -31,17 +31,6 @@ ProcessorNode::ProcessorNode(const std::shared_ptr<Connectable> &processor)
   setUUID(processor->getUUID());
 }
 
-ProcessorNode::ProcessorNode(const ProcessorNode &other)
-    : processor_(other.processor_),
-      Connectable(other.getName()) {
-  setUUID(processor_->getUUID());
-}
-
-ProcessorNode::ProcessorNode(const ProcessorNode &&other)
-    : Connectable(std::move(other)),
-      processor_(std::move(other.processor_)) {
-}
-
 ProcessorNode::~ProcessorNode() = default;
 
 bool ProcessorNode::isWorkAvailable() {
diff --git a/libminifi/src/core/state/UpdateController.cpp b/libminifi/src/core/state/UpdateController.cpp
index 11f01f0..6978100 100644
--- a/libminifi/src/core/state/UpdateController.cpp
+++ b/libminifi/src/core/state/UpdateController.cpp
@@ -31,14 +31,6 @@ UpdateStatus::UpdateStatus(UpdateState state, int16_t reason)
       reason_(reason) {
 }
 
-UpdateStatus::UpdateStatus(const UpdateStatus &other) = default;
-
-UpdateStatus::UpdateStatus(const UpdateStatus &&other)
-    : error_(std::move(other.error_)),
-      reason_(std::move(other.reason_)),
-      state_(std::move(other.state_)) {
-}
-
 UpdateState UpdateStatus::getState() const {
   return state_;
 }
@@ -51,15 +43,6 @@ int16_t UpdateStatus::getReadonCode() const {
   return reason_;
 }
 
-UpdateStatus &UpdateStatus::operator=(const UpdateStatus &&other) {
-  error_ = std::move(other.error_);
-  reason_ = std::move(other.reason_);
-  state_ = std::move(other.state_);
-  return *this;
-}
-
-UpdateStatus &UpdateStatus::operator=(const UpdateStatus &other) = default;
-
 } /* namespace state */
 } /* namespace minifi */
 } /* namespace nifi */