From 92b46fb9d762e90f23ab564f1169d991ab00e651 Mon Sep 17 00:00:00 2001 From: Marc Parisi Date: Wed, 24 May 2017 14:39:08 -0400 Subject: [PATCH 1/6] MINIFI-249: Update prov repo to better abstract deser. Deserialization and serialization are better abstracted into SerializableComponent allowing us to use all repos with the same [de]serialization interfaces. MINIFI-249: Update Test resources to use local http server when possible MINIFI-330: convert const char* to std::string --- libminifi/include/Connection.h | 6 +- .../include/EventDrivenSchedulingAgent.h | 5 +- libminifi/include/FlowController.h | 23 +- libminifi/include/FlowFileRecord.h | 31 +- libminifi/include/ResourceClaim.h | 26 +- libminifi/include/SchedulingAgent.h | 7 +- libminifi/include/Site2SiteClientProtocol.h | 36 +- libminifi/include/ThreadedSchedulingAgent.h | 6 +- .../include/TimerDrivenSchedulingAgent.h | 6 +- libminifi/include/core/ConfigurationFactory.h | 11 +- libminifi/include/core/ContentRepository.h | 60 ++ libminifi/include/core/Core.h | 3 + libminifi/include/core/FlowConfiguration.h | 16 +- libminifi/include/core/FlowFile.h | 4 + libminifi/include/core/ProcessContext.h | 21 +- libminifi/include/core/ProcessGroup.h | 3 +- libminifi/include/core/ProcessSession.h | 2 +- libminifi/include/core/Repository.h | 92 ++- libminifi/include/core/RepositoryFactory.h | 16 +- .../include/core/SerializableComponent.h | 88 +++ libminifi/include/core/StreamManager.h | 81 +++ .../core/logging/LoggerConfiguration.h | 3 +- .../SiteToSiteProvenanceReportingTask.h | 4 +- .../core/repository/AtomicRepoEntries.h | 430 +++++++++++++ .../core/repository/FileSystemRepository.h | 75 +++ .../core/repository/FlowFileRepository.h | 16 +- .../repository/VolatileContentRepository.h | 124 ++++ .../repository/VolatileFlowFileRepository.h | 82 +++ .../repository/VolatileProvenanceRepository.h | 60 ++ .../core/repository/VolatileRepository.h | 519 ++++++++-------- .../include/core/yaml/YamlConfiguration.h | 39 +- libminifi/include/io/AtomicEntryStream.h | 192 ++++++ libminifi/include/io/BaseStream.h | 13 + libminifi/include/io/ClientSocket.h | 6 +- libminifi/include/io/DataStream.h | 8 +- libminifi/include/io/FileStream.h | 136 +++++ libminifi/include/processors/ExecuteProcess.h | 8 +- .../include/processors/GenerateFlowFile.h | 6 +- libminifi/include/processors/InvokeHTTP.h | 6 + libminifi/include/processors/ListenHTTP.h | 2 +- libminifi/include/processors/ListenSyslog.h | 10 +- libminifi/include/processors/LogAttribute.h | 28 +- libminifi/include/processors/PutFile.h | 3 +- libminifi/include/properties/Configure.h | 1 + libminifi/include/provenance/Provenance.h | 34 +- .../include/provenance/ProvenanceRepository.h | 79 ++- libminifi/include/utils/ByteInputCallBack.h | 10 +- libminifi/src/ConfigurationListener.cpp | 12 +- libminifi/src/Configure.cpp | 52 +- libminifi/src/Connection.cpp | 20 +- libminifi/src/FlowControlProtocol.cpp | 5 +- libminifi/src/FlowController.cpp | 47 +- libminifi/src/FlowFileRecord.cpp | 13 +- libminifi/src/HttpConfigurationListener.cpp | 16 +- libminifi/src/Properties.cpp | 4 +- libminifi/src/RemoteProcessorGroupPort.cpp | 66 +- libminifi/src/ResourceClaim.cpp | 23 +- libminifi/src/SchedulingAgent.cpp | 4 +- libminifi/src/Site2SiteClientProtocol.cpp | 19 +- libminifi/src/ThreadedSchedulingAgent.cpp | 8 +- .../src/controllers/SSLContextService.cpp | 14 +- libminifi/src/core/ClassLoader.cpp | 6 +- libminifi/src/core/ConfigurableComponent.cpp | 13 +- libminifi/src/core/ConfigurationFactory.cpp | 12 +- libminifi/src/core/Connectable.cpp | 12 +- libminifi/src/core/Core.cpp | 5 + libminifi/src/core/FlowConfiguration.cpp | 12 +- libminifi/src/core/ProcessGroup.cpp | 29 +- libminifi/src/core/ProcessSession.cpp | 564 +++++++++--------- libminifi/src/core/ProcessSessionFactory.cpp | 2 +- libminifi/src/core/Processor.cpp | 20 +- libminifi/src/core/Repository.cpp | 3 +- libminifi/src/core/RepositoryFactory.cpp | 47 +- .../StandardControllerServiceNode.cpp | 6 +- .../src/core/logging/LoggerConfiguration.cpp | 16 +- .../SiteToSiteProvenanceReportingTask.cpp | 31 +- .../core/repository/FileSystemRepository.cpp | 49 ++ .../core/repository/FlowFileRepository.cpp | 23 +- .../repository/VolatileContentRepository.cpp | 129 ++++ .../core/repository/VolatileRepository.cpp | 29 +- libminifi/src/core/yaml/YamlConfiguration.cpp | 62 +- libminifi/src/io/AtomicEntryStream.cpp | 34 ++ libminifi/src/io/ClientSocket.cpp | 14 +- libminifi/src/io/FileStream.cpp | 124 ++++ libminifi/src/io/StreamFactory.cpp | 6 +- libminifi/src/processors/ExecuteProcess.cpp | 6 +- libminifi/src/processors/GenerateFlowFile.cpp | 4 +- libminifi/src/processors/GetFile.cpp | 8 +- libminifi/src/processors/InvokeHTTP.cpp | 28 +- libminifi/src/processors/ListenHTTP.cpp | 10 +- libminifi/src/processors/ListenSyslog.cpp | 2 +- libminifi/src/processors/LogAttribute.cpp | 6 +- libminifi/src/processors/PutFile.cpp | 45 +- libminifi/src/processors/TailFile.cpp | 26 +- libminifi/src/provenance/Provenance.cpp | 50 +- .../src/provenance/ProvenanceRepository.cpp | 1 - libminifi/test/TestServer.h | 137 +++++ .../ControllerServiceIntegrationTests.cpp | 11 +- .../HttpConfigurationListenerTest.cpp | 67 +-- .../integration/HttpGetIntegrationTest.cpp | 67 ++- .../integration/HttpPostIntegrationTest.cpp | 34 +- .../integration/ProvenanceReportingTest.cpp | 11 +- .../test/integration/Site2SiteRestTest.cpp | 61 +- .../test/integration/TestExecuteProcess.cpp | 16 +- libminifi/test/resources/TestHTTPGet.yml | 2 +- .../test/resources/TestHTTPGetSecure.yml | 2 +- libminifi/test/resources/TestHTTPPost.yml | 2 +- libminifi/test/resources/cn.ckey.pem | 1 - libminifi/test/resources/cn.crt.pem | 1 - libminifi/test/resources/nifi-cert.pem | 27 + libminifi/test/unit/FileStreamTests.cpp | 210 +++++++ libminifi/test/unit/InvokeHTTPTests.cpp | 38 +- libminifi/test/unit/ProcessorTests.cpp | 22 +- libminifi/test/unit/ProvenanceTestHelper.h | 70 ++- libminifi/test/unit/ProvenanceTests.cpp | 33 +- libminifi/test/unit/RepoTests.cpp | 14 +- libminifi/test/unit/TailFileTests.cpp | 8 +- .../test/unit/YamlConfigurationTests.cpp | 6 +- main/MiNiFiMain.cpp | 59 +- 119 files changed, 3820 insertions(+), 1253 deletions(-) create mode 100644 libminifi/include/core/ContentRepository.h create mode 100644 libminifi/include/core/SerializableComponent.h create mode 100644 libminifi/include/core/StreamManager.h create mode 100644 libminifi/include/core/repository/AtomicRepoEntries.h create mode 100644 libminifi/include/core/repository/FileSystemRepository.h create mode 100644 libminifi/include/core/repository/VolatileContentRepository.h create mode 100644 libminifi/include/core/repository/VolatileFlowFileRepository.h create mode 100644 libminifi/include/core/repository/VolatileProvenanceRepository.h create mode 100644 libminifi/include/io/AtomicEntryStream.h create mode 100644 libminifi/include/io/FileStream.h create mode 100644 libminifi/src/core/repository/FileSystemRepository.cpp create mode 100644 libminifi/src/core/repository/VolatileContentRepository.cpp create mode 100644 libminifi/src/io/AtomicEntryStream.cpp create mode 100644 libminifi/src/io/FileStream.cpp create mode 100644 libminifi/test/TestServer.h create mode 100644 libminifi/test/unit/FileStreamTests.cpp diff --git a/libminifi/include/Connection.h b/libminifi/include/Connection.h index be51fceb91..e40fe75a15 100644 --- a/libminifi/include/Connection.h +++ b/libminifi/include/Connection.h @@ -47,7 +47,7 @@ class Connection : public core::Connectable, public std::enable_shared_from_this /* * Create a new processor */ - explicit Connection(std::shared_ptr flow_repository, std::string name, uuid_t uuid = NULL, uuid_t srcUUID = NULL, uuid_t destUUID = NULL); + explicit Connection(const std::shared_ptr &flow_repository, const std::shared_ptr &content_repo, std::string name, uuid_t uuid = NULL, uuid_t srcUUID = NULL,uuid_t destUUID = NULL); // Destructor virtual ~Connection() { } @@ -123,7 +123,7 @@ class Connection : public core::Connectable, public std::enable_shared_from_this bool isFull(); // Get queue size uint64_t getQueueSize() { - std::lock_guard lock(mutex_); + std::lock_guard < std::mutex > lock(mutex_); return queue_.size(); } // Get queue data size @@ -168,6 +168,8 @@ class Connection : public core::Connectable, public std::enable_shared_from_this std::atomic expired_duration_; // flow file repository std::shared_ptr flow_repository_; + // content repository reference. + std::shared_ptr content_repo_; private: // Mutex for protection diff --git a/libminifi/include/EventDrivenSchedulingAgent.h b/libminifi/include/EventDrivenSchedulingAgent.h index 6a63dc54a7..b7c21c7f73 100644 --- a/libminifi/include/EventDrivenSchedulingAgent.h +++ b/libminifi/include/EventDrivenSchedulingAgent.h @@ -38,8 +38,9 @@ class EventDrivenSchedulingAgent : public ThreadedSchedulingAgent { /*! * Create a new event driven scheduling agent. */ - EventDrivenSchedulingAgent(std::shared_ptr controller_service_provider, std::shared_ptr repo, std::shared_ptr configuration) - : ThreadedSchedulingAgent(controller_service_provider, repo, configuration) { + EventDrivenSchedulingAgent(std::shared_ptr controller_service_provider, std::shared_ptr repo, + std::shared_ptr content_repo, std::shared_ptr configuration) + : ThreadedSchedulingAgent(controller_service_provider, repo, content_repo, configuration) { } // Destructor virtual ~EventDrivenSchedulingAgent() { diff --git a/libminifi/include/FlowController.h b/libminifi/include/FlowController.h index c2fef2a215..d9a0452a37 100644 --- a/libminifi/include/FlowController.h +++ b/libminifi/include/FlowController.h @@ -70,10 +70,23 @@ class FlowController : public core::controller::ControllerServiceProvider, publi /** * Flow controller constructor */ - FlowController(std::shared_ptr provenance_repo, std::shared_ptr flow_file_repo, std::shared_ptr configure, - std::unique_ptr flow_configuration, - const std::string name = DEFAULT_ROOT_GROUP_NAME, - bool headless_mode = false); + explicit FlowController(std::shared_ptr provenance_repo, std::shared_ptr flow_file_repo, std::shared_ptr configure, + std::unique_ptr flow_configuration, + std::shared_ptr content_repo, const std::string name, bool headless_mode); + + explicit FlowController(std::shared_ptr provenance_repo, std::shared_ptr flow_file_repo, std::shared_ptr configure, + std::unique_ptr flow_configuration, + std::shared_ptr content_repo) + : FlowController(provenance_repo, flow_file_repo, configure, std::move(flow_configuration), content_repo, DEFAULT_ROOT_GROUP_NAME, false) + { + } + + explicit FlowController(std::shared_ptr provenance_repo, std::shared_ptr flow_file_repo, std::shared_ptr configure, + std::unique_ptr flow_configuration) + : FlowController(provenance_repo, flow_file_repo, configure, std::move(flow_configuration), std::make_shared(), DEFAULT_ROOT_GROUP_NAME, false) + { + content_repo_->initialize(configure); + } // Destructor virtual ~FlowController(); @@ -301,6 +314,8 @@ class FlowController : public core::controller::ControllerServiceProvider, publi // FlowFile Repo std::shared_ptr flow_file_repo_; + std::shared_ptr content_repo_; + // Flow Engines // Flow Timer Scheduler std::shared_ptr timer_scheduler_; diff --git a/libminifi/include/FlowFileRecord.h b/libminifi/include/FlowFileRecord.h index 3d6057d47c..d6e5f2e6db 100644 --- a/libminifi/include/FlowFileRecord.h +++ b/libminifi/include/FlowFileRecord.h @@ -30,7 +30,8 @@ #include #include #include - +#include "core/ContentRepository.h" +#include "io/BaseStream.h" #include "io/Serializable.h" #include "core/FlowFile.h" #include "utils/TimeUtil.h" @@ -81,11 +82,21 @@ inline const char *FlowAttributeKey(FlowAttribute attribute) { // throw exception for error class InputStreamCallback { public: - virtual void process(std::ifstream *stream) = 0; + virtual ~InputStreamCallback() { + + } + //virtual void process(std::ifstream *stream) = 0; + + virtual int64_t process(std::shared_ptr stream) = 0; }; class OutputStreamCallback { public: - virtual void process(std::ofstream *stream) = 0; + virtual ~OutputStreamCallback() { + + } + //virtual void process(std::ofstream *stream) = 0; + virtual int64_t process(std::shared_ptr stream) = 0; + }; class FlowFileRecord : public core::FlowFile, public io::Serializable { @@ -94,14 +105,17 @@ class FlowFileRecord : public core::FlowFile, public io::Serializable { /* * Create a new flow record */ - explicit FlowFileRecord(std::shared_ptr flow_repository, std::map attributes, std::shared_ptr claim = nullptr); + explicit FlowFileRecord(std::shared_ptr flow_repository, const std::shared_ptr &content_repo, std::map attributes, + std::shared_ptr claim = nullptr); - explicit FlowFileRecord(std::shared_ptr flow_repository, std::shared_ptr &event); + explicit FlowFileRecord(std::shared_ptr flow_repository, const std::shared_ptr &content_repo, std::shared_ptr &event); - explicit FlowFileRecord(std::shared_ptr flow_repository, std::shared_ptr &event, const std::string &uuidConnection); + explicit FlowFileRecord(std::shared_ptr flow_repository, const std::shared_ptr &content_repo, std::shared_ptr &event, + const std::string &uuidConnection); - explicit FlowFileRecord(std::shared_ptr flow_repository) + explicit FlowFileRecord(std::shared_ptr flow_repository, const std::shared_ptr &content_repo) : FlowFile(), + content_repo_(content_repo), flow_repository_(flow_repository), snapshot_("") { @@ -168,6 +182,9 @@ class FlowFileRecord : public core::FlowFile, public io::Serializable { // repository reference. std::shared_ptr flow_repository_; + // content repo reference. + std::shared_ptr content_repo_; + // Snapshot flow record for session rollback bool snapshot_; // Prevent default copy constructor and assignment operation diff --git a/libminifi/include/ResourceClaim.h b/libminifi/include/ResourceClaim.h index 49faed1299..b235ba225d 100644 --- a/libminifi/include/ResourceClaim.h +++ b/libminifi/include/ResourceClaim.h @@ -25,9 +25,11 @@ #include #include #include +#include #include #include #include "core/Core.h" +#include "core/StreamManager.h" #include "properties/Configure.h" #include "utils/Id.h" @@ -40,7 +42,7 @@ namespace minifi { #define DEFAULT_CONTENT_DIRECTORY "./content_repository" // ResourceClaim Class -class ResourceClaim { +class ResourceClaim : public std::enable_shared_from_this { public: @@ -49,7 +51,9 @@ class ResourceClaim { /*! * Create a new resource claim */ - ResourceClaim(const std::string contentDirectory = default_directory_path); + ResourceClaim(std::shared_ptr> claim_manager, const std::string contentDirectory = default_directory_path); + + ResourceClaim(const std::string path, std::shared_ptr> claim_manager , bool deleted = false); // Destructor virtual ~ResourceClaim() { } @@ -59,7 +63,11 @@ class ResourceClaim { } // decreaseFlowFileRecordOwenedCount void decreaseFlowFileRecordOwnedCount() { - --_flowFileRecordOwnedCount; + + if (_flowFileRecordOwnedCount > 0) { + _flowFileRecordOwnedCount--; + } + } // getFlowFileRecordOwenedCount uint64_t getFlowFileRecordOwnedCount() { @@ -74,14 +82,26 @@ class ResourceClaim { _contentFullPath = path; } + void deleteClaim() { + if (!deleted_) + { + deleted_ = true; + } + + } + protected: + std::atomic deleted_; // Full path to the content std::string _contentFullPath; // How many FlowFileRecord Own this cliam std::atomic _flowFileRecordOwnedCount; + std::shared_ptr> claim_manager_; + private: + // Logger std::shared_ptr logger_; // Prevent default copy constructor and assignment operation diff --git a/libminifi/include/SchedulingAgent.h b/libminifi/include/SchedulingAgent.h index 22f79db5c6..7333cc57c3 100644 --- a/libminifi/include/SchedulingAgent.h +++ b/libminifi/include/SchedulingAgent.h @@ -52,10 +52,13 @@ class SchedulingAgent { /*! * Create a new scheduling agent. */ - SchedulingAgent(std::shared_ptr controller_service_provider, std::shared_ptr repo, std::shared_ptr configuration) + SchedulingAgent(std::shared_ptr controller_service_provider, std::shared_ptr repo, + std::shared_ptr content_repo, + std::shared_ptr configuration) : configure_(configuration), admin_yield_duration_(0), bored_yield_duration_(0), + content_repo_(content_repo), controller_service_provider_(controller_service_provider), logger_(logging::LoggerFactory::getLogger()) { running_ = false; @@ -108,6 +111,8 @@ class SchedulingAgent { std::shared_ptr configure_; std::shared_ptr repo_; + + std::shared_ptr content_repo_; // thread pool for components. utils::ThreadPool component_lifecycle_thread_pool_; // controller service provider reference diff --git a/libminifi/include/Site2SiteClientProtocol.h b/libminifi/include/Site2SiteClientProtocol.h index 8d89004adb..c47254e1a5 100644 --- a/libminifi/include/Site2SiteClientProtocol.h +++ b/libminifi/include/Site2SiteClientProtocol.h @@ -549,7 +549,8 @@ class Site2SiteClientProtocol { : _packet(packet) { } DataPacket *_packet; - void process(std::ofstream *stream) { + //void process(std::ofstream *stream) { + int64_t process(std::shared_ptr stream) { uint8_t buffer[8192]; int len = _packet->_size; while (len > 0) { @@ -557,11 +558,12 @@ class Site2SiteClientProtocol { int ret = _packet->_transaction->getStream().readData(buffer, size); if (ret != size) { _packet->_protocol->logger_->log_error("Site2Site Receive Flow Size %d Failed %d", size, ret); - break; + return -1; } - stream->write((const char *) buffer, size); + stream->write(buffer, size); len -= size; } + return len; } }; // Nest Callback Class for read stream @@ -571,22 +573,30 @@ class Site2SiteClientProtocol { : _packet(packet) { } DataPacket *_packet; - void process(std::ifstream *stream) { + int64_t process(std::shared_ptr stream) { _packet->_size = 0; - uint8_t buffer[8192]; + uint8_t buffer[8192] = { 0 }; int readSize; - while (stream->good()) { - if (!stream->read((char *) buffer, 8192)) - readSize = stream->gcount(); - else - readSize = 8192; + size_t size = 0; + do { + readSize = stream->read(buffer, 8192); + + if (readSize == 0) { + break; + } + if (readSize < 0) + { + return -1; + } int ret = _packet->_transaction->getStream().writeData(buffer, readSize); if (ret != readSize) { _packet->_protocol->logger_->log_error("Site2Site Send Flow Size %d Failed %d", readSize, ret); - break; + return -1; } - _packet->_size += readSize; - } + size += readSize; + } while (size < stream->getSize()); + _packet->_size = size; + return size; } }; diff --git a/libminifi/include/ThreadedSchedulingAgent.h b/libminifi/include/ThreadedSchedulingAgent.h index 50ab6c97ee..ea6696f1ab 100644 --- a/libminifi/include/ThreadedSchedulingAgent.h +++ b/libminifi/include/ThreadedSchedulingAgent.h @@ -42,8 +42,10 @@ class ThreadedSchedulingAgent : public SchedulingAgent { /*! * Create a new threaded scheduling agent. */ - ThreadedSchedulingAgent(std::shared_ptr controller_service_provider, std::shared_ptr repo, std::shared_ptr configuration) - : SchedulingAgent(controller_service_provider, repo, configuration), + ThreadedSchedulingAgent(std::shared_ptr controller_service_provider, std::shared_ptr repo, + std::shared_ptr content_repo, + std::shared_ptr configuration) + : SchedulingAgent(controller_service_provider, repo, content_repo, configuration), logger_(logging::LoggerFactory::getLogger()) { } // Destructor diff --git a/libminifi/include/TimerDrivenSchedulingAgent.h b/libminifi/include/TimerDrivenSchedulingAgent.h index 597dc76b80..602b2e4a7c 100644 --- a/libminifi/include/TimerDrivenSchedulingAgent.h +++ b/libminifi/include/TimerDrivenSchedulingAgent.h @@ -37,8 +37,10 @@ class TimerDrivenSchedulingAgent : public ThreadedSchedulingAgent { /*! * Create a new processor */ - TimerDrivenSchedulingAgent(std::shared_ptr controller_service_provider, std::shared_ptr repo, std::shared_ptr configure) - : ThreadedSchedulingAgent(controller_service_provider, repo, configure) { + TimerDrivenSchedulingAgent(std::shared_ptr controller_service_provider, std::shared_ptr repo, + std::shared_ptr content_repo, + std::shared_ptr configure) + : ThreadedSchedulingAgent(controller_service_provider, repo, content_repo, configure) { } // Destructor virtual ~TimerDrivenSchedulingAgent() { diff --git a/libminifi/include/core/ConfigurationFactory.h b/libminifi/include/core/ConfigurationFactory.h index b58c1704b1..61af8cd804 100644 --- a/libminifi/include/core/ConfigurationFactory.h +++ b/libminifi/include/core/ConfigurationFactory.h @@ -30,6 +30,7 @@ namespace core { template typename std::enable_if::value, T*>::type instantiate(const std::shared_ptr &repo, const std::shared_ptr &flow_file_repo, + const std::shared_ptr &content_repo, std::shared_ptr configuration, const std::string path) { throw std::runtime_error("Cannot instantiate class"); @@ -37,16 +38,20 @@ typename std::enable_if::value, T*>::type instantiate(const template typename std::enable_if::value, T*>::type instantiate(const std::shared_ptr &repo, const std::shared_ptr &flow_file_repo, + const std::shared_ptr &content_repo, const std::shared_ptr &stream_factory, - std::shared_ptr configuration, const std::string path) { - return new T(repo, flow_file_repo, stream_factory, configuration, path); + std::shared_ptr configuration, + const std::string path) { + return new T(repo, flow_file_repo, content_repo, stream_factory, configuration, path); } /** * Configuration factory is used to create a new FlowConfiguration * object. */ -std::unique_ptr createFlowConfiguration(std::shared_ptr repo, std::shared_ptr flow_file_repo, std::shared_ptr configure, +std::unique_ptr createFlowConfiguration(std::shared_ptr repo, std::shared_ptr flow_file_repo, + std::shared_ptr content_repo, + std::shared_ptr configure, std::shared_ptr stream_factory, const std::string configuration_class_name, const std::string path = "", bool fail_safe = false); diff --git a/libminifi/include/core/ContentRepository.h b/libminifi/include/core/ContentRepository.h new file mode 100644 index 0000000000..b544ca03ce --- /dev/null +++ b/libminifi/include/core/ContentRepository.h @@ -0,0 +1,60 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#ifndef LIBMINIFI_INCLUDE_CORE_CONTENTREPOSITORY_H_ +#define LIBMINIFI_INCLUDE_CORE_CONTENTREPOSITORY_H_ + +#include "properties/Configure.h" +#include "ResourceClaim.h" +#include "io/DataStream.h" +#include "io/BaseStream.h" +#include "StreamManager.h" + +namespace org { +namespace apache { +namespace nifi { +namespace minifi { +namespace core { + +/** + * Content repository definition that extends StreamManager. + */ +class ContentRepository : public StreamManager { + public: + virtual ~ContentRepository() { + + } + + /** + * initialize this content repository using the provided configuration. + */ + virtual bool initialize(const std::shared_ptr &configure) = 0; + + /** + * Stops this repository. + */ + virtual void stop() = 0; + +}; + +} /* namespace core */ +} /* namespace minifi */ +} /* namespace nifi */ +} /* namespace apache */ +} /* namespace org */ + +#endif /* LIBMINIFI_INCLUDE_CORE_CONTENTREPOSITORY_H_ */ diff --git a/libminifi/include/core/Core.h b/libminifi/include/core/Core.h index c32eb59e56..1dc79e743a 100644 --- a/libminifi/include/core/Core.h +++ b/libminifi/include/core/Core.h @@ -19,6 +19,7 @@ #define LIBMINIFI_INCLUDE_CORE_CORE_H_ #include +#include #include #include #include @@ -132,6 +133,8 @@ class CoreComponent { */ void setUUID(uuid_t uuid); + void setUUIDStr(const std::string uuidStr); + /** * Returns the UUID through the provided object. * @param uuid uuid struct to which we will copy the memory diff --git a/libminifi/include/core/FlowConfiguration.h b/libminifi/include/core/FlowConfiguration.h index 3429166caa..43d2bc04a3 100644 --- a/libminifi/include/core/FlowConfiguration.h +++ b/libminifi/include/core/FlowConfiguration.h @@ -58,11 +58,12 @@ class FlowConfiguration : public CoreComponent { * Constructor that will be used for configuring * the flow controller. */ - explicit FlowConfiguration(std::shared_ptr repo, std::shared_ptr flow_file_repo, std::shared_ptr stream_factory, - std::shared_ptr configuration, - const std::string path) + explicit FlowConfiguration(std::shared_ptr repo, std::shared_ptr flow_file_repo, std::shared_ptr content_repo, + std::shared_ptr stream_factory, + std::shared_ptr configuration, const std::string path) : CoreComponent(core::getClassName()), flow_file_repo_(flow_file_repo), + content_repo_(content_repo), config_path_(path), stream_factory_(stream_factory), configuration_(configuration), @@ -76,8 +77,9 @@ class FlowConfiguration : public CoreComponent { // Create Processor (Node/Input/Output Port) based on the name std::shared_ptr createProcessor(std::string name, uuid_t uuid); // Create Root Processor Group - std::unique_ptr createRootProcessGroup(std::string name, - uuid_t uuid, int version); + + std::unique_ptr createRootProcessGroup(std::string name, uuid_t uuid, int version); + std::shared_ptr createControllerService(const std::string &class_name, const std::string &name, uuid_t uuid); // Create Remote Processor Group @@ -100,7 +102,7 @@ class FlowConfiguration : public CoreComponent { } virtual std::unique_ptr getRootFromPayload( - std::string &yamlConfigPayload) { + std::string &yamlConfigPayload) { return nullptr; } @@ -127,6 +129,8 @@ class FlowConfiguration : public CoreComponent { std::string config_path_; // flow file repo std::shared_ptr flow_file_repo_; + // content repository. + std::shared_ptr content_repo_; // stream factory std::shared_ptr stream_factory_; std::shared_ptr configuration_; diff --git a/libminifi/include/core/FlowFile.h b/libminifi/include/core/FlowFile.h index 95b901b216..ff8e8cb6de 100644 --- a/libminifi/include/core/FlowFile.h +++ b/libminifi/include/core/FlowFile.h @@ -224,6 +224,10 @@ class FlowFile { void setStoredToRepository(bool storedInRepository) { stored = storedInRepository; + if (!stored && nullptr != claim_) + { + claim_->decreaseFlowFileRecordOwnedCount(); + } } bool isStored() { diff --git a/libminifi/include/core/ProcessContext.h b/libminifi/include/core/ProcessContext.h index 48e0108539..7b269ca34b 100644 --- a/libminifi/include/core/ProcessContext.h +++ b/libminifi/include/core/ProcessContext.h @@ -25,8 +25,10 @@ #include #include #include - +#include #include "Property.h" +#include "core/ContentRepository.h" +#include "core/repository/FileSystemRepository.h" #include "core/controller/ControllerServiceProvider.h" #include "core/controller/ControllerServiceLookup.h" #include "core/logging/LoggerConfiguration.h" @@ -46,10 +48,12 @@ class ProcessContext : public controller::ControllerServiceLookup { /*! * Create a new process context associated with the processor/controller service/state manager */ - ProcessContext(ProcessorNode &processor, std::shared_ptr &controller_service_provider, std::shared_ptr repo) + ProcessContext(ProcessorNode &processor, std::shared_ptr &controller_service_provider, std::shared_ptr repo, + std::shared_ptr content_repo = std::make_shared()) : processor_node_(processor), controller_service_provider_(controller_service_provider), - logger_(logging::LoggerFactory::getLogger()) { + logger_(logging::LoggerFactory::getLogger()), + content_repo_(content_repo) { repo_ = repo; } // Destructor @@ -92,6 +96,14 @@ class ProcessContext : public controller::ControllerServiceLookup { return repo_; } + /** + * Returns a reference to the content repository for the running instance. + * @return content repository shared pointer. + */ + std::shared_ptr getContentRepository() { + return content_repo_; + } + // Prevent default copy constructor and assignment operation // Only support pass by reference or pointer ProcessContext(const ProcessContext &parent) = delete; @@ -145,6 +157,9 @@ class ProcessContext : public controller::ControllerServiceLookup { std::shared_ptr controller_service_provider_; // repository shared pointer. std::shared_ptr repo_; + + // repository shared pointer. + std::shared_ptr content_repo_; // Processor ProcessorNode processor_node_; // Logger diff --git a/libminifi/include/core/ProcessGroup.h b/libminifi/include/core/ProcessGroup.h index 410480a33f..a0e51e3c8e 100644 --- a/libminifi/include/core/ProcessGroup.h +++ b/libminifi/include/core/ProcessGroup.h @@ -56,8 +56,7 @@ class ProcessGroup { /*! * Create a new process group */ - ProcessGroup(ProcessGroupType type, std::string name, uuid_t uuid = NULL, int version = 0, - ProcessGroup *parent = NULL); + ProcessGroup(ProcessGroupType type, std::string name, uuid_t uuid = NULL, int version = 0, ProcessGroup *parent = NULL); // Destructor virtual ~ProcessGroup(); // Set Processor Name diff --git a/libminifi/include/core/ProcessSession.h b/libminifi/include/core/ProcessSession.h index ad79d125c5..d853e9bf31 100644 --- a/libminifi/include/core/ProcessSession.h +++ b/libminifi/include/core/ProcessSession.h @@ -50,7 +50,7 @@ class ProcessSession { ProcessSession(ProcessContext *processContext = NULL) : process_context_(processContext), logger_(logging::LoggerFactory::getLogger()) { - logger_->log_trace("ProcessSession created for %s", process_context_->getProcessorNode().getName().c_str()); + logger_->log_trace("ProcessSession created for %s", process_context_->getProcessorNode().getName()); auto repo = processContext->getProvenanceRepository(); provenance_report_ = new provenance::ProvenanceReporter(repo, process_context_->getProcessorNode().getUUIDStr(), process_context_->getProcessorNode().getName()); } diff --git a/libminifi/include/core/Repository.h b/libminifi/include/core/Repository.h index 5f7e6c2117..f1b47ae6c1 100644 --- a/libminifi/include/core/Repository.h +++ b/libminifi/include/core/Repository.h @@ -31,7 +31,8 @@ #include #include #include - +#include "core/ContentRepository.h" +#include "core/SerializableComponent.h" #include "properties/Configure.h" #include "core/logging/LoggerConfiguration.h" #include "core/Property.h" @@ -52,15 +53,15 @@ namespace core { #define MAX_REPOSITORY_ENTRY_LIFE_TIME (600000) // 10 minute #define REPOSITORY_PURGE_PERIOD (2500) // 2500 msec -class Repository : public CoreComponent { +class Repository : public core::SerializableComponent { public: /* * Constructor for the repository */ Repository(std::string repo_name = "Repository", std::string directory = REPOSITORY_DIRECTORY, int64_t maxPartitionMillis = MAX_REPOSITORY_ENTRY_LIFE_TIME, int64_t maxPartitionBytes = - MAX_REPOSITORY_STORAGE_SIZE, + MAX_REPOSITORY_STORAGE_SIZE, uint64_t purgePeriod = REPOSITORY_PURGE_PERIOD) - : CoreComponent(repo_name), + : core::SerializableComponent(repo_name), thread_(), logger_(logging::LoggerFactory::getLogger()) { directory_ = directory; @@ -81,7 +82,7 @@ class Repository : public CoreComponent { return true; } // Put - virtual bool Put(std::string key, uint8_t *buf, int bufLen) { + virtual bool Put(std::string key, const uint8_t *buf, size_t bufLen) { return true; } // Delete @@ -89,7 +90,14 @@ class Repository : public CoreComponent { return true; } - virtual bool Get(std::string key, std::string &value) { + virtual bool Delete(std::vector> &storedValues) { + bool found = true; + for (auto storedValue : storedValues) { + found &= Delete(storedValue->getName()); + } + return found; + } + virtual bool Get(const std::string &key, std::string &value) { return false; } @@ -109,10 +117,82 @@ class Repository : public CoreComponent { virtual bool isRunning() { return running_; } + + /** + * Specialization that allows us to serialize max_size objects into store. + * the lambdaConstructor will create objects to put into store + * @param store vector in which we can store serialized object + * @param max_size reference that stores the max number of objects to retrieve and serialize. + * upon return max_size will represent the number of serialized objects. + * @return status of this operation + * + * Base implementation returns true; + */ + virtual bool Serialize(std::vector> &store, size_t max_size) { + return true; + } + + /** + * Specialization that allows us to deserialize max_size objects into store. + * @param store vector in which we can store deserialized object + * @param max_size reference that stores the max number of objects to retrieve and deserialize. + * upon return max_size will represent the number of deserialized objects. + * @return status of this operation + * + * Base implementation returns true; + */ + virtual bool DeSerialize(std::vector> &store, size_t &max_size) { + return true; + } + + /** + * Specialization that allows us to deserialize max_size objects into store. + * the lambdaConstructor will create objects to put into store + * @param store vector in which we can store deserialized object + * @param max_size reference that stores the max number of objects to retrieve and deserialize. + * upon return max_size will represent the number of deserialized objects. + * @param lambdaConstructor reference that will create the objects for store + * @return status of this operation + * + * Base implementation returns true; + */ + virtual bool DeSerialize(std::vector> &store, size_t &max_size, std::function()> lambdaConstructor) { + return true; + } + + /** + * Base implementation returns true; + */ + virtual bool Serialize(const std::shared_ptr &store) { + return true; + } + + /** + * Base implementation returns true; + */ + virtual bool DeSerialize(const std::shared_ptr &store) { + return true; + } + + /** + * Base implementation returns true; + */ + virtual bool DeSerialize(const uint8_t *buffer, const size_t bufferSize) { + return true; + } + + virtual bool Serialize(const std::string &key, const uint8_t *buffer, const size_t bufferSize) { + return Put(key, buffer, bufferSize); + } + uint64_t incrementSize(const char *fpath, const struct stat *sb, int typeflag) { return (repo_size_ += sb->st_size); } + virtual void loadComponent(const std::shared_ptr &content_repo) { + + } + // Prevent default copy constructor and assignment operation // Only support pass by reference or pointer Repository(const Repository &parent) = delete; diff --git a/libminifi/include/core/RepositoryFactory.h b/libminifi/include/core/RepositoryFactory.h index 9fafb5728e..b123a6d07f 100644 --- a/libminifi/include/core/RepositoryFactory.h +++ b/libminifi/include/core/RepositoryFactory.h @@ -19,8 +19,8 @@ #ifndef LIBMINIFI_INCLUDE_CORE_REPOSITORYFACTORY_H_ #define LIBMINIFI_INCLUDE_CORE_REPOSITORYFACTORY_H_ +#include "core/ContentRepository.h" #include "core/Repository.h" -#include "core/repository/VolatileRepository.h" #include "Core.h" namespace org { @@ -30,8 +30,22 @@ namespace minifi { namespace core { +/** + * Create a repository represented by the configuration class name + * @param configuration_class_name configuration class name + * @param fail_safe determines whether or not to make the default class if configuration_class_name is invalid + * @param repo_name name of the repository + */ std::shared_ptr createRepository(const std::string configuration_class_name, bool fail_safe = false, const std::string repo_name = ""); +/** + * Create a context repository + * @param configuration_class_name configuration class name + * @param fail_safe determines whether or not to make the default class if configuration_class_name is invalid + * @param repo_name name of the repository + */ +std::shared_ptr createContentRepository(const std::string configuration_class_name, bool fail_safe = false, const std::string repo_name = ""); + } /* namespace core */ } /* namespace minifi */ } /* namespace nifi */ diff --git a/libminifi/include/core/SerializableComponent.h b/libminifi/include/core/SerializableComponent.h new file mode 100644 index 0000000000..f7f9feb0a6 --- /dev/null +++ b/libminifi/include/core/SerializableComponent.h @@ -0,0 +1,88 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#ifndef LIBMINIFI_INCLUDE_CORE_SERIALIZABLECOMPONENT_H_ +#define LIBMINIFI_INCLUDE_CORE_SERIALIZABLECOMPONENT_H_ + +#include "io/Serializable.h" +#include "core/Core.h" + +namespace org { +namespace apache { +namespace nifi { +namespace minifi { +namespace core { + +/** + * Represents a component that is serializable and an extension point of core Component + */ +class SerializableComponent : public core::CoreComponent, public minifi::io::Serializable { + + public: + + SerializableComponent(const std::string name, uuid_t uuid = nullptr) + : core::CoreComponent(name, uuid) { + + } + + virtual ~SerializableComponent() { + + } + + /** + * Serialize this object into the the store + * @param store object in which we are serializing data into + * @return status of this serialization. + */ + virtual bool Serialize(const std::shared_ptr &store) = 0; + + /** + * Deserialization from the parameter store into the current object + * @param store from which we are deserializing the current object + * @return status of this deserialization. + */ + virtual bool DeSerialize(const std::shared_ptr &store) = 0; + + /** + * Deserializes the current object using buffer + * @param buffer buffer from which we can deserialize the currenet object + * @param bufferSize length of buffer from which we can deserialize the current object. + * @return status of the deserialization. + */ + virtual bool DeSerialize(const uint8_t *buffer, const size_t bufferSize) = 0; + + /** + * Serialization of this object into buffer + * @param key string that represents this objects identifier + * @param buffer buffer that contains the serialized object + * @param bufferSize length of buffer + * @return status of serialization + */ + virtual bool Serialize(const std::string &key, const uint8_t *buffer, const size_t bufferSize) { + return false; + } + +}; + +} /* namespace core */ +} /* namespace minifi */ +} /* namespace nifi */ +} /* namespace apache */ +} /* namespace org */ + +#endif /* LIBMINIFI_INCLUDE_CORE_SERIALIZABLECOMPONENT_H_ */ + diff --git a/libminifi/include/core/StreamManager.h b/libminifi/include/core/StreamManager.h new file mode 100644 index 0000000000..468526d08e --- /dev/null +++ b/libminifi/include/core/StreamManager.h @@ -0,0 +1,81 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#ifndef LIBMINIFI_INCLUDE_CORE_STREAMMANAGER_H_ +#define LIBMINIFI_INCLUDE_CORE_STREAMMANAGER_H_ + +#include "properties/Configure.h" +#include "ResourceClaim.h" +#include "io/DataStream.h" +#include "io/BaseStream.h" + +namespace org { +namespace apache { +namespace nifi { +namespace minifi { +namespace core { + +/** + * Purpose: Provides a base for all stream based managers. The goal here is to provide + * a small set of interfaces that provide a small set of operations to provide state + * management for streams. + */ +template +class StreamManager { + public: + virtual ~StreamManager() { + + } + + /** + * Create a write stream using the streamId as a reference. + * @param streamId stream identifier + * @return stream pointer. + */ + virtual std::shared_ptr write(const std::shared_ptr &streamId) = 0; + + /** + * Create a read stream using the streamId as a reference. + * @param streamId stream identifier + * @return stream pointer. + */ + virtual std::shared_ptr read(const std::shared_ptr &streamId) = 0; + + /** + * Closes the stream + * @param streamId stream identifier + * @return result of operation. + */ + virtual bool close(const std::shared_ptr &streamId) = 0; + + /** + * Removes the stream from this stream manager. The end result + * is dependent on the stream manager implementation. + * @param streamId stream identifier + * @return result of operation. + */ + virtual bool remove(const std::shared_ptr &streamId) = 0; + +}; + +} /* namespace core */ +} /* namespace minifi */ +} /* namespace nifi */ +} /* namespace apache */ +} /* namespace org */ + +#endif /* LIBMINIFI_INCLUDE_CORE_STREAMMANAGER_H_ */ diff --git a/libminifi/include/core/logging/LoggerConfiguration.h b/libminifi/include/core/logging/LoggerConfiguration.h index aa4a1d0858..787fec55ee 100644 --- a/libminifi/include/core/logging/LoggerConfiguration.h +++ b/libminifi/include/core/logging/LoggerConfiguration.h @@ -105,8 +105,7 @@ class LoggerConfiguration { protected: static std::shared_ptr initialize_namespaces(const std::shared_ptr &logger_properties); static std::shared_ptr get_logger(std::shared_ptr logger, const std::shared_ptr &root_namespace, const std::string &name, - std::shared_ptr formatter, - bool remove_if_present = false); + std::shared_ptr formatter, bool remove_if_present = false); private: static std::shared_ptr create_default_root(); diff --git a/libminifi/include/core/reporting/SiteToSiteProvenanceReportingTask.h b/libminifi/include/core/reporting/SiteToSiteProvenanceReportingTask.h index e1d80e817a..2bd4099960 100644 --- a/libminifi/include/core/reporting/SiteToSiteProvenanceReportingTask.h +++ b/libminifi/include/core/reporting/SiteToSiteProvenanceReportingTask.h @@ -54,7 +54,6 @@ class SiteToSiteProvenanceReportingTask : public minifi::RemoteProcessorGroupPor } //! Destructor virtual ~SiteToSiteProvenanceReportingTask() { - } //! Report Task Name static constexpr char const* ReportTaskName = "SiteToSiteProvenanceReportingTask"; @@ -62,7 +61,8 @@ class SiteToSiteProvenanceReportingTask : public minifi::RemoteProcessorGroupPor public: //! Get provenance json report - void getJsonReport(core::ProcessContext *context, core::ProcessSession *session, std::vector> &records, std::string &report); + void getJsonReport(core::ProcessContext *context, core::ProcessSession *session, std::vector> &records, std::string &report); + void onSchedule(core::ProcessContext *context, core::ProcessSessionFactory *sessionFactory); //! OnTrigger method, implemented by NiFi SiteToSiteProvenanceReportingTask virtual void onTrigger(core::ProcessContext *context, core::ProcessSession *session); diff --git a/libminifi/include/core/repository/AtomicRepoEntries.h b/libminifi/include/core/repository/AtomicRepoEntries.h new file mode 100644 index 0000000000..a6bf39c00b --- /dev/null +++ b/libminifi/include/core/repository/AtomicRepoEntries.h @@ -0,0 +1,430 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#ifndef LIBMINIFI_INCLUDE_CORE_REPOSITORY_ATOMICREPOENTRIES_H_ +#define LIBMINIFI_INCLUDE_CORE_REPOSITORY_ATOMICREPOENTRIES_H_ + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +static uint16_t accounting_size = sizeof(std::vector) + sizeof(std::string) + sizeof(size_t); + +namespace org { +namespace apache { +namespace nifi { +namespace minifi { +namespace core { +namespace repository { + +/** + * Purpose: Repo value represents an item that will support a move operation within an AtomicEntry + * + * Justification: Since AtomicEntry is a static entry that does not move or change, the underlying + * RepoValue can be changed to support atomic operations. + */ +template +class RepoValue { + public: + + explicit RepoValue() { + } + + /** + * Constructor that populates the item allowing for a custom key comparator. + * @param key key for this repo value. + * @param ptr buffer + * @param size size buffer + * @param comparator custom comparator. + */ + explicit RepoValue(T key, const uint8_t *ptr, size_t size, std::function comparator = nullptr) + : key_(key), + comparator_(comparator) { + if (nullptr == ptr) { + size = 0; + } + buffer_.resize(size); + if (size > 0) { + std::memcpy(buffer_.data(), ptr, size); + } + } + + /** + * RepoValue that moves the other object into this. + */ + explicit RepoValue(RepoValue &&other) +noexcept : key_(std::move(other.key_)), + buffer_(std::move(other.buffer_)), + comparator_(std::move(other.comparator_)) { + } + + ~RepoValue() + { + } + + T &getKey() { + return key_; + } + + /** + * Sets the key, relacing the custom comparator if needed. + */ + void setKey(const T key, std::function comparator = nullptr) { + key_ = key; + comparator_ = comparator; + } + + /** + * Determines if the key is the same using the custom comparator + * @param other object to compare against + * @return result of the comparison + */ + inline bool isEqual(RepoValue *other) + { + return comparator_ == nullptr ? key_ == other->key_ : comparator_(key_,other->key_); + } + + /** + * Determines if the key is the same using the custom comparator + * @param other object to compare against + * @return result of the comparison + */ + inline bool isKey(T other) + { + return comparator_ == nullptr ? key_ == other : comparator_(key_,other); + } + + /** + * Clears the buffer. + */ + void clearBuffer() { + buffer_.resize(0); + buffer_.clear(); + } + + /** + * Return the size of the memory within the key + * buffer, the size of timestamp, and the general + * system word size + */ + uint64_t size() { + return buffer_.size(); + } + + size_t getBufferSize() { + return buffer_.size(); + } + + const uint8_t *getBuffer() + { + return buffer_.data(); + } + + /** + * Places the contents of buffer into str + * @param strnig into which we are placing the memory contained in buffer. + */ + void emplace(std::string &str) { + str.insert(0, reinterpret_cast(buffer_.data()), buffer_.size()); + } + + /** + * Appends ptr to the end of buffer. + * @param ptr pointer containing data to add to buffer_ + */ + void append(uint8_t *ptr, size_t size) + { + buffer_.insert(buffer_.end(), ptr, ptr + size); + } + + RepoValue &operator=(RepoValue &&other) noexcept { + key_ = std::move(other.key_); + buffer_ = std::move(other.buffer_); + return *this; + } + + private: + T key_; + std::function comparator_; + std::vector buffer_; + }; + + /** + * Purpose: Atomic Entry allows us to create a statically + * sized ring buffer, with the ability to create + * + **/ +template +class AtomicEntry { + + public: + /** + * Constructor that accepts a max size and an atomic counter for the total + * size allowd by this and other atomic entries. + */ + explicit AtomicEntry(std::atomic *total_size, size_t *max_size) + : write_pending_(false), + has_value_(false), + total_size_(total_size), + max_size_(max_size) { + + } + + /** + * Sets the repo value, moving the old value into old_value. + * @param new_value new value to move into value_. + * @param old_value the previous value of value_ will be moved into old_value + * @param prev_size size reclaimed. + * @return result of this set. If true old_value will be populated. + */ + bool setRepoValue(RepoValue &new_value, RepoValue &old_value, size_t &prev_size) { + // delete the underlying pointer + bool lock = false; + if (!write_pending_.compare_exchange_weak(lock, true) && !lock) + { + return false; + } + if (has_value_) { + prev_size = value_.size(); + } + old_value = std::move(value_); + value_ = std::move(new_value); + has_value_ = true; + try_unlock(); + return true; + } + + /** + * A test and set operation, which is used to allow a function to test + * if an item can be released and a function used for reclaiming memory associated + * with said object. + * A custom comparator can be provided to augment the key being added into value_ + */ + bool testAndSetKey(const T str, std::function releaseTest = nullptr, std::function reclaimer = nullptr, std::function comparator = nullptr) { + // delete the underlying pointer + bool lock = false; + + if (!write_pending_.compare_exchange_weak(lock, true) && !lock) + return false; + + if (has_value_) { + // we either don't have a release test or we cannot release this + // entity + if (releaseTest != nullptr && reclaimer != nullptr && releaseTest(value_.getKey())) + { + reclaimer(value_.getKey()); + } + else { + try_unlock(); + return false; + } + + } + value_.setKey(str, comparator); + has_value_ = true; + try_unlock(); + return true; + } + + /** + * Moved the value into the argument + * @param value the previous value will be moved into this parameter + * @return success of get operation based on whether or not this atomic entry has a value. + */ + bool getValue(RepoValue &value) { + try_lock(); + if (!has_value_) { + try_unlock(); + return false; + } + value = std::move(value_); + has_value_ = false; + try_unlock(); + return true; + } + + /** + * Moved the value into the argument + * @param value the previous value will be moved into this parameter + * @return success of get operation based on whether or not this atomic entry has a value. + */ + bool getValue(const T &key, RepoValue &value) { + try_lock(); + if (!has_value_) { + try_unlock(); + return false; + } + if (!value_.isKey(key)) { + try_unlock(); + return false; + } + value = std::move(value_); + has_value_ = false; + try_unlock(); + return true; + } + + /** + * Moved the value into the argument + * @param value the previous value will be moved into this parameter + * @return success of get operation based on whether or not this atomic entry has a value. + */ + bool getValue(const T &key, RepoValue **value) { + try_lock(); + if (!has_value_) { + try_unlock(); + return false; + } + if (!value_.isKey(key)) { + try_unlock(); + return false; + } + *value = &value_; + try_unlock(); + return true; + } + + /** + * Operation that will be used to test and free if a release is required without + * setting a new object. + * @param releaseTest function that will be used to free the RepoValue key from + * this atomic entry. + * @param freedValue informs the caller if an item was freed. + */ + T testAndFree(std::function releaseTest, bool &freedValue) { + try_lock(); + T ref; + if (!has_value_) { + try_unlock(); + return ref; + } + + if (releaseTest(value_.getKey())) { + size_t bufferSize = value_.getBufferSize(); + value_.clearBuffer(); + ref = value_.getKey(); + has_value_ = false; + if (total_size_ != nullptr) { + *total_size_ -= bufferSize; + } + + } + try_unlock(); + return ref; + } + + /** + * sets has_value to false; however, does not call + * any external entity to further free RepoValue + */ + bool freeValue(const T &key) { + try_lock(); + if (!has_value_) { + try_unlock(); + return false; + } + if (!value_.isKey(key)) { + try_unlock(); + return false; + } + size_t bufferSize = value_.getBufferSize(); + value_.clearBuffer(); + has_value_ = false; + if (total_size_ != nullptr) { + *total_size_ -= bufferSize; + } + try_unlock(); + return true; + } + + /** + * Appends buffer onto this atomic entry if key matches + * the current RepoValue's key. + */ + bool insert(const T key, uint8_t *buffer, size_t size) { + try_lock(); + + if (!has_value_) { + try_unlock(); + return false; + } + + if (!value_.isKey(key)) { + try_unlock(); + return false; + } + + if ((total_size_ != nullptr && max_size_ != nullptr) && (*total_size_ + size > *max_size_)) { + // can't support this write + try_unlock(); + return false; + } + + value_.append(buffer, size); + (*total_size_) += size; + try_unlock(); + return true; + } + + private: + + /** + * Spin lock to unlock the current atomic entry. + */ + inline void try_lock() { + bool lock = false; + while (!write_pending_.compare_exchange_weak(lock, true) && !lock) { + // attempt again + } + } + + /** + * Spin lock to unlock the current atomic entry. + */ + inline void try_unlock() { + bool lock = true; + while (!write_pending_.compare_exchange_weak(lock, false) && lock) { + // attempt again + } + } + + // atomic size pointer. + std::atomic *total_size_; + // max size + size_t *max_size_; + // determines if a write is pending. + std::atomic write_pending_; + // used to determine if a value is present in this atomic entry. + std::atomic has_value_; + // repo value. + RepoValue value_; +}; + +} /* namespace repository */ +} /* namespace core */ +} /* namespace minifi */ +} /* namespace nifi */ +} /* namespace apache */ +} /* namespace org */ + +#endif /* LIBMINIFI_INCLUDE_CORE_REPOSITORY_ATOMICREPOENTRIES_H_ */ diff --git a/libminifi/include/core/repository/FileSystemRepository.h b/libminifi/include/core/repository/FileSystemRepository.h new file mode 100644 index 0000000000..3792ebe7aa --- /dev/null +++ b/libminifi/include/core/repository/FileSystemRepository.h @@ -0,0 +1,75 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#ifndef LIBMINIFI_INCLUDE_CORE_REPOSITORY_FileSystemRepository_H_ +#define LIBMINIFI_INCLUDE_CORE_REPOSITORY_FileSystemRepository_H_ + +#include "core/Core.h" +#include "../ContentRepository.h" +#include "properties/Configure.h" +#include "core/logging/LoggerConfiguration.h" +namespace org { +namespace apache { +namespace nifi { +namespace minifi { +namespace core { +namespace repository { + +/** + * FileSystemRepository is a content repository that stores data onto the local file system. + */ +class FileSystemRepository : public core::ContentRepository, public core::CoreComponent { + public: + FileSystemRepository(std::string name = getClassName()) + : core::CoreComponent(name), + logger_(logging::LoggerFactory::getLogger()) { + + } + virtual ~FileSystemRepository() { + + } + + + virtual bool initialize(const std::shared_ptr &configuration); + + virtual void stop(); + + virtual std::shared_ptr write(const std::shared_ptr &claim); + + virtual std::shared_ptr read(const std::shared_ptr &claim); + + virtual bool close(const std::shared_ptr &claim) { + return true; + } + + virtual bool remove(const std::shared_ptr &claim) { + return true; + } + + private: + + std::shared_ptr logger_; +}; + +} /* namespace repository */ +} /* namespace core */ +} /* namespace minifi */ +} /* namespace nifi */ +} /* namespace apache */ +} /* namespace org */ + +#endif /* LIBMINIFI_INCLUDE_CORE_REPOSITORY_FileSystemRepository_H_ */ diff --git a/libminifi/include/core/repository/FlowFileRepository.h b/libminifi/include/core/repository/FlowFileRepository.h index 2e19286d84..6022e654b7 100644 --- a/libminifi/include/core/repository/FlowFileRepository.h +++ b/libminifi/include/core/repository/FlowFileRepository.h @@ -48,12 +48,10 @@ class FlowFileRepository : public core::Repository, public std::enable_shared_fr // Constructor FlowFileRepository(const std::string repo_name = "", std::string directory = FLOWFILE_REPOSITORY_DIRECTORY, int64_t maxPartitionMillis = MAX_FLOWFILE_REPOSITORY_ENTRY_LIFE_TIME, - int64_t maxPartitionBytes = MAX_FLOWFILE_REPOSITORY_STORAGE_SIZE, - uint64_t purgePeriod = FLOWFILE_REPOSITORY_PURGE_PERIOD) + int64_t maxPartitionBytes = MAX_FLOWFILE_REPOSITORY_STORAGE_SIZE, uint64_t purgePeriod = FLOWFILE_REPOSITORY_PURGE_PERIOD) : Repository(repo_name.length() > 0 ? repo_name : core::getClassName(), directory, maxPartitionMillis, maxPartitionBytes, purgePeriod), - logger_(logging::LoggerFactory::getLogger()) - - { + logger_(logging::LoggerFactory::getLogger()), + content_repo_(nullptr) { db_ = NULL; } @@ -66,6 +64,7 @@ class FlowFileRepository : public core::Repository, public std::enable_shared_fr // initialize virtual bool initialize(const std::shared_ptr &configure) { std::string value; + std::cout << "initialize " << std::endl; if (configure->get(Configure::nifi_flowfile_repository_directory_default, value)) { directory_ = value; @@ -95,7 +94,7 @@ class FlowFileRepository : public core::Repository, public std::enable_shared_fr virtual void run(); - virtual bool Put(std::string key, uint8_t *buf, int bufLen) { + virtual bool Put(std::string key, const uint8_t *buf, size_t bufLen) { // persistent to the DB leveldb::Slice value((const char *) buf, bufLen); @@ -123,7 +122,7 @@ class FlowFileRepository : public core::Repository, public std::enable_shared_fr * Sets the value from the provided key * @return status of the get operation. */ - virtual bool Get(std::string key, std::string &value) { + virtual bool Get(const std::string &key, std::string &value) { leveldb::Status status; status = db_->Get(leveldb::ReadOptions(), key, &value); if (status.ok()) @@ -135,7 +134,7 @@ class FlowFileRepository : public core::Repository, public std::enable_shared_fr void setConnectionMap(std::map> &connectionMap) { this->connectionMap = connectionMap; } - void loadComponent(); + virtual void loadComponent(const std::shared_ptr &content_repo); void start() { if (this->purge_period_ <= 0) @@ -150,6 +149,7 @@ class FlowFileRepository : public core::Repository, public std::enable_shared_fr private: std::map> connectionMap; + std::shared_ptr content_repo_; leveldb::DB* db_; std::shared_ptr logger_; }; diff --git a/libminifi/include/core/repository/VolatileContentRepository.h b/libminifi/include/core/repository/VolatileContentRepository.h new file mode 100644 index 0000000000..9ea6e52acb --- /dev/null +++ b/libminifi/include/core/repository/VolatileContentRepository.h @@ -0,0 +1,124 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#ifndef LIBMINIFI_INCLUDE_CORE_REPOSITORY_VolatileContentRepository_H_ +#define LIBMINIFI_INCLUDE_CORE_REPOSITORY_VolatileContentRepository_H_ + +#include "core/Core.h" +#include "AtomicRepoEntries.h" +#include "io/AtomicEntryStream.h" +#include "../ContentRepository.h" +#include "core/repository/VolatileRepository.h" +#include "properties/Configure.h" +#include "core/logging/LoggerConfiguration.h" +namespace org { +namespace apache { +namespace nifi { +namespace minifi { +namespace core { +namespace repository { + +/** + * Purpose: Stages content into a volatile area of memory. Note that when the maximum number + * of entries is consumed we will rollback a session to wait for others to be freed. + */ +class VolatileContentRepository : public core::ContentRepository, public core::repository::VolatileRepository> { + public: + VolatileContentRepository(std::string name = getClassName()) + : core::repository::VolatileRepository>(name), + logger_(logging::LoggerFactory::getLogger()) { + max_count_ = 15000; + } + virtual ~VolatileContentRepository() { + + } + + /** + * Initialize the volatile content repo + * @param configure configuration + */ + virtual bool initialize(const std::shared_ptr &configure); + + /** + * Stop any thread associated with the volatile content repository. + */ + virtual void stop(); + + /** + * Creates writable stream. + * @param claim resource claim + * @return BaseStream shared pointer that represents the stream the consumer will write to. + */ + virtual std::shared_ptr write(const std::shared_ptr &claim); + + /** + * Creates readable stream. + * @param claim resource claim + * @return BaseStream shared pointer that represents the stream from which the consumer will read.. + */ + virtual std::shared_ptr read(const std::shared_ptr &claim); + + /** + * Closes the claim. + * @return whether or not the claim is associated with content stored in volatile memory. + */ + virtual bool close(const std::shared_ptr &claim) { + return remove(claim); + } + + /** + * Closes the claim. + * @return whether or not the claim is associated with content stored in volatile memory. + */ + virtual bool remove(const std::shared_ptr &claim); + + protected: + virtual void start(); + + virtual void run(); + + template + std::shared_ptr shared_from_parent() + { + return std::static_pointer_cast(shared_from_this()); + } + + private: + + // function pointers that are associated with the claims. + std::function, std::shared_ptr)> resource_claim_comparator_; + std::function)> resource_claim_check_; + std::function)> claim_reclaimer_; + + // logger + std::shared_ptr logger_; + + // mutex and master list that represent a cache of Atomic entries. this exists so that we don't have to walk the atomic entry list. + // The idea is to reduce the computational complexity while keeping access as maximally lock free as we can. + std::mutex map_mutex_; + + std::map>*> master_list_; +}; + +} /* namespace repository */ +} /* namespace core */ +} /* namespace minifi */ +} /* namespace nifi */ +} /* namespace apache */ +} /* namespace org */ + +#endif /* LIBMINIFI_INCLUDE_CORE_REPOSITORY_VolatileContentRepository_H_ */ diff --git a/libminifi/include/core/repository/VolatileFlowFileRepository.h b/libminifi/include/core/repository/VolatileFlowFileRepository.h new file mode 100644 index 0000000000..059c1de6b6 --- /dev/null +++ b/libminifi/include/core/repository/VolatileFlowFileRepository.h @@ -0,0 +1,82 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#ifndef LIBMINIFI_INCLUDE_CORE_REPOSITORY_VOLATILEFLOWFILEREPOSITORY_H_ +#define LIBMINIFI_INCLUDE_CORE_REPOSITORY_VOLATILEFLOWFILEREPOSITORY_H_ + +#include "VolatileRepository.h" + +namespace org { +namespace apache { +namespace nifi { +namespace minifi { +namespace core { +namespace repository { + +/** + * Volatile flow file repository. keeps a running counter of the current location, freeing + * those which we no longer hold. + */ +class VolatileFlowFileRepository : public VolatileRepository +{ + public: + explicit VolatileFlowFileRepository(std::string repo_name = "", std::string dir = REPOSITORY_DIRECTORY, int64_t maxPartitionMillis = MAX_REPOSITORY_ENTRY_LIFE_TIME, int64_t maxPartitionBytes = + MAX_REPOSITORY_STORAGE_SIZE, + uint64_t purgePeriod = REPOSITORY_PURGE_PERIOD) + : VolatileRepository(repo_name.length() > 0 ? repo_name : core::getClassName(), "", maxPartitionMillis, maxPartitionBytes, purgePeriod) + + { + purge_required_ = true; + content_repo_ = nullptr; + } + + virtual void run() { + repo_full_ = false; + while (running_) { + std::this_thread::sleep_for(std::chrono::milliseconds(purge_period_)); + if (purge_required_ && nullptr != content_repo_) + { + std::lock_guard lock(purge_mutex_); + for (auto purgeItem : purge_list_) + { + std::shared_ptr newClaim = std::make_shared(purgeItem, content_repo_, true); + content_repo_->remove(newClaim); + } + purge_list_.resize(0); + purge_list_.clear(); + } + } + } + + void loadComponent(const std::shared_ptr &content_repo) { + content_repo_ = content_repo; + + } + + protected: + + std::shared_ptr content_repo_; + +}; +} /* namespace repository */ +} /* namespace core */ +} /* namespace minifi */ +} /* namespace nifi */ +} /* namespace apache */ +} /* namespace org */ + +#endif /* LIBMINIFI_INCLUDE_CORE_REPOSITORY_VOLATILEFLOWFILEREPOSITORY_H_ */ diff --git a/libminifi/include/core/repository/VolatileProvenanceRepository.h b/libminifi/include/core/repository/VolatileProvenanceRepository.h new file mode 100644 index 0000000000..7397751f10 --- /dev/null +++ b/libminifi/include/core/repository/VolatileProvenanceRepository.h @@ -0,0 +1,60 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#ifndef LIBMINIFI_INCLUDE_CORE_REPOSITORY_VOLATILEPROVENANCEREPOSITORY_H_ +#define LIBMINIFI_INCLUDE_CORE_REPOSITORY_VOLATILEPROVENANCEREPOSITORY_H_ + +#include "VolatileRepository.h" + +namespace org { +namespace apache { +namespace nifi { +namespace minifi { +namespace core { +namespace repository { + +/** + * Volatile provenance repository. + */ +class VolatileProvenanceRepository : public VolatileRepository +{ + + public: + explicit VolatileProvenanceRepository(std::string repo_name = "", std::string dir = REPOSITORY_DIRECTORY, int64_t maxPartitionMillis = MAX_REPOSITORY_ENTRY_LIFE_TIME, int64_t maxPartitionBytes = + MAX_REPOSITORY_STORAGE_SIZE, + uint64_t purgePeriod = REPOSITORY_PURGE_PERIOD) + : VolatileRepository(repo_name.length() > 0 ? repo_name : core::getClassName(), "", maxPartitionMillis, maxPartitionBytes, purgePeriod) + + { + purge_required_ = false; + } + + virtual void run() { + repo_full_ = false; + } + private: + +}; + +} /* namespace repository */ +} /* namespace core */ +} /* namespace minifi */ +} /* namespace nifi */ +} /* namespace apache */ +} /* namespace org */ + +#endif /* LIBMINIFI_INCLUDE_CORE_REPOSITORY_VOLATILEPROVENANCEREPOSITORY_H_ */ diff --git a/libminifi/include/core/repository/VolatileRepository.h b/libminifi/include/core/repository/VolatileRepository.h index 870a1f5b79..07d9cc46a1 100644 --- a/libminifi/include/core/repository/VolatileRepository.h +++ b/libminifi/include/core/repository/VolatileRepository.h @@ -22,9 +22,11 @@ #include #include #include +#include "core/SerializableComponent.h" #include "core/Core.h" #include "Connection.h" #include "utils/StringUtils.h" +#include "AtomicRepoEntries.h" namespace org { namespace apache { @@ -33,290 +35,89 @@ namespace minifi { namespace core { namespace repository { -static uint16_t accounting_size = sizeof(std::vector) + sizeof(std::string) + sizeof(size_t); - -class RepoValue { - public: - - explicit RepoValue() { - } - - explicit RepoValue(std::string key, uint8_t *ptr, size_t size) - : key_(key) { - buffer_.resize(size); - std::memcpy(buffer_.data(), ptr, size); - fast_size_ = key.size() + size; - } - - explicit RepoValue(RepoValue &&other) -noexcept : key_(std::move(other.key_)), - buffer_(std::move(other.buffer_)), - fast_size_(other.fast_size_) { - } - - ~RepoValue() - { - } - - std::string &getKey() { - return key_; - } - - /** - * Return the size of the memory within the key - * buffer, the size of timestamp, and the general - * system word size - */ - uint64_t size() { - return fast_size_; - } - - size_t bufferSize() { - return buffer_.size(); - } - - void emplace(std::string &str) { - str.insert(0, reinterpret_cast(buffer_.data()), buffer_.size()); - } - - RepoValue &operator=(RepoValue &&other) noexcept { - key_ = std::move(other.key_); - buffer_ = std::move(other.buffer_); - other.buffer_.clear(); - return *this; - } - - private: - size_t fast_size_; - std::string key_; - std::vector buffer_; - }; - - /** - * Purpose: Atomic Entry allows us to create a statically - * sized ring buffer, with the ability to create - **/ -class AtomicEntry { - - public: - AtomicEntry() - : write_pending_(false), - has_value_(false) { - - } - - bool setRepoValue(RepoValue &new_value, size_t &prev_size) { - // delete the underlying pointer - bool lock = false; - if (!write_pending_.compare_exchange_weak(lock, true) && !lock) - return false; - if (has_value_) { - prev_size = value_.size(); - } - value_ = std::move(new_value); - has_value_ = true; - try_unlock(); - return true; - } - - bool getValue(RepoValue &value) { - try_lock(); - if (!has_value_) { - try_unlock(); - return false; - } - value = std::move(value_); - has_value_ = false; - try_unlock(); - return true; - } - - bool getValue(const std::string &key, RepoValue &value) { - try_lock(); - if (!has_value_) { - try_unlock(); - return false; - } - if (value_.getKey() != key) { - try_unlock(); - return false; - } - value = std::move(value_); - has_value_ = false; - try_unlock(); - return true; - } - - private: - - inline void try_lock() { - bool lock = false; - while (!write_pending_.compare_exchange_weak(lock, true) && !lock) { - // attempt again - } - } - - inline void try_unlock() { - bool lock = true; - while (!write_pending_.compare_exchange_weak(lock, false) && lock) { - // attempt again - } - } - - std::atomic write_pending_; - std::atomic has_value_; - RepoValue value_; -}; - /** * Flow File repository * Design: Extends Repository and implements the run function, using LevelDB as the primary substrate. */ -class VolatileRepository : public core::Repository, public std::enable_shared_from_this { +template +class VolatileRepository : public core::Repository, public std::enable_shared_from_this> { public: static const char *volatile_repo_max_count; + static const char *volatile_repo_max_bytes; // Constructor - VolatileRepository(std::string repo_name = "", std::string dir = REPOSITORY_DIRECTORY, int64_t maxPartitionMillis = MAX_REPOSITORY_ENTRY_LIFE_TIME, int64_t maxPartitionBytes = + explicit VolatileRepository(std::string repo_name = "", std::string dir = REPOSITORY_DIRECTORY, int64_t maxPartitionMillis = MAX_REPOSITORY_ENTRY_LIFE_TIME, int64_t maxPartitionBytes = MAX_REPOSITORY_STORAGE_SIZE, - uint64_t purgePeriod = REPOSITORY_PURGE_PERIOD) + uint64_t purgePeriod = REPOSITORY_PURGE_PERIOD) : Repository(repo_name.length() > 0 ? repo_name : core::getClassName(), "", maxPartitionMillis, maxPartitionBytes, purgePeriod), max_size_(maxPartitionBytes * 0.75), current_index_(0), max_count_(10000), + current_size_(0), logger_(logging::LoggerFactory::getLogger()) { - + purge_required_ = false; } // Destructor - ~VolatileRepository() { - for (auto ent : value_vector_) { - delete ent; - } - } + virtual ~VolatileRepository(); /** * Initialize thevolatile repsitory **/ - virtual bool initialize(const std::shared_ptr &configure) { - std::string value = ""; - - if (configure != nullptr) { - int64_t max_cnt = 0; - std::stringstream strstream; - strstream << Configure::nifi_volatile_repository_options << getName() << "." << volatile_repo_max_count; - if (configure->get(strstream.str(), value)) { - if (core::Property::StringToInt(value, max_cnt)) { - max_count_ = max_cnt; - } - } - } - - logger_->log_debug("Resizing value_vector_ for %s count is %d", getName(), max_count_); - value_vector_.reserve(max_count_); - for (int i = 0; i < max_count_; i++) { - value_vector_.emplace_back(new AtomicEntry()); - } - return true; - } + virtual bool initialize(const std::shared_ptr &configure); - virtual void run(); + virtual void run() = 0; /** * Places a new object into the volatile memory area * @param key key to add to the repository * @param buf buffer **/ - virtual bool Put(std::string key, uint8_t *buf, int bufLen) { - RepoValue new_value(key, buf, bufLen); - - const size_t size = new_value.size(); - bool updated = false; - size_t reclaimed_size = 0; - do { - - int private_index = current_index_.fetch_add(1); - // round robin through the beginning - if (private_index >= max_count_) { - uint16_t new_index = 0; - if (current_index_.compare_exchange_weak(new_index, 0)) { - private_index = 0; - } else { - continue; - } - } - logger_->log_info("Set repo value at %d out of %d", private_index, max_count_); - updated = value_vector_.at(private_index)->setRepoValue(new_value, reclaimed_size); + virtual bool Put(T key, const uint8_t *buf, size_t bufLen); - if (reclaimed_size > 0) { - current_size_ -= reclaimed_size; - } - - } while (!updated); - current_size_ += size; - - logger_->log_info("VolatileRepository -- put %s %d %d", key, current_size_.load(), current_index_.load()); - return true; - } /** - *c * Deletes the key * @return status of the delete operation */ - virtual bool Delete(std::string key) { - - logger_->log_info("VolatileRepository -- delete %s", key); - for (auto ent : value_vector_) { - // let the destructor do the cleanup - RepoValue value; - if (ent->getValue(key, value)) { - current_size_ -= value.size(); - return true; - } + virtual bool Delete(T key); - } - return false; - } /** * Sets the value from the provided key. Once the item is retrieved * it may not be retrieved again. * @return status of the get operation. */ - virtual bool Get(std::string key, std::string &value) { - for (auto ent : value_vector_) { - // let the destructor do the cleanup - RepoValue repo_value; - - if (ent->getValue(key, repo_value)) { - current_size_ -= value.size(); - repo_value.emplace(value); - logger_->log_info("VolatileRepository -- get %s %d", key, current_size_.load()); - return true; - } + virtual bool Get(const T &key, std::string &value); + /** + * Deserializes objects into store + * @param store vector in which we will store newly created objects. + * @param max_size size of objects deserialized + */ + virtual bool DeSerialize(std::vector> &store, size_t &max_size, std::function()> lambda); - } - return false; - } + /** + * Deserializes objects into a store that contains a fixed number of objects in which + * we will deserialize from this repo + * @param store precreated object vector + * @param max_size size of objects deserialized + */ + virtual bool DeSerialize(std::vector> &store, size_t &max_size); - void setConnectionMap(std::map> &connectionMap) { - this->connectionMap = connectionMap; - } - void loadComponent(); - - void start() { - if (this->purge_period_ <= 0) - return; - if (running_) - return; - thread_ = std::thread(&VolatileRepository::run, shared_from_this()); - thread_.detach(); - running_ = true; - logger_->log_info("%s Repository Monitor Thread Start", name_.c_str()); - } + /** + * Set the connection map + * @param connectionMap map of all connections through this repo. + */ + void setConnectionMap(std::map> &connectionMap); + + /** + * Function to load this component. + */ + virtual void loadComponent(const std::shared_ptr &content_repo); + + virtual void start(); protected: @@ -331,22 +132,242 @@ class VolatileRepository : public core::Repository, public std::enable_shared_fr else return false; } - /** - * Purges the volatile repository. - */ - void purge(); - private: std::map> connectionMap; - - std::atomic current_size_; + // current size of the volatile repo. + std::atomic current_size_; + // current index. std::atomic current_index_; - std::vector value_vector_; + // value vector. + std::vector*> value_vector_; + // max count we are allowed to store. uint32_t max_count_; - uint32_t max_size_; + // maximum estimated size + size_t max_size_; + + bool purge_required_; + + std::mutex purge_mutex_; + // purge list + std::vector purge_list_; + + private: std::shared_ptr logger_; + +}; + +template +const char *VolatileRepository::volatile_repo_max_count = "max.count"; +template +const char *VolatileRepository::volatile_repo_max_bytes = "max.bytes"; +/* + template + void VolatileRepository::run() { + repo_full_ = false; + } + */ +template +void VolatileRepository::loadComponent(const std::shared_ptr &content_repo) { +} + +// Destructor +template +VolatileRepository::~VolatileRepository() { + for (auto ent : value_vector_) { + delete ent; + } +} + +/** + * Initialize the volatile repsitory + **/ +template +bool VolatileRepository::initialize(const std::shared_ptr &configure) { + std::string value = ""; + + if (configure != nullptr) { + int64_t max_cnt = 0; + std::stringstream strstream; + strstream << Configure::nifi_volatile_repository_options << getName() << "." << volatile_repo_max_count; + if (configure->get(strstream.str(), value)) { + if (core::Property::StringToInt(value, max_cnt)) { + max_count_ = max_cnt; + } + } + + strstream.str(""); + strstream.clear(); + int64_t max_bytes = 0; + strstream << Configure::nifi_volatile_repository_options << getName() << "." << volatile_repo_max_bytes; + if (configure->get(strstream.str(), value)) { + if (core::Property::StringToInt(value, max_bytes)) { + if (max_bytes <= 0) { + max_size_ = std::numeric_limits::max(); + } else { + max_size_ = max_bytes; + } + } + } + } + + logger_->log_info("Resizing value_vector_ for %s count is %d", getName(), max_count_); + logger_->log_info("Using a maximum size of %u", max_size_); + value_vector_.reserve(max_count_); + for (int i = 0; i < max_count_; i++) { + value_vector_.emplace_back(new AtomicEntry(¤t_size_, &max_size_)); + } + return true; +} + +/** + * Places a new object into the volatile memory area + * @param key key to add to the repository + * @param buf buffer + **/ +template +bool VolatileRepository::Put(T key, const uint8_t *buf, size_t bufLen) { + RepoValue new_value(key, buf, bufLen); + + const size_t size = new_value.size(); + bool updated = false; + size_t reclaimed_size = 0; + RepoValue old_value; + do { + int private_index = current_index_.fetch_add(1); + // round robin through the beginning + if (private_index >= max_count_) { + uint16_t new_index = 0; + if (current_index_.compare_exchange_weak(new_index, 0)) { + private_index = 0; + } else { + continue; + } + } + logger_->log_debug("Set repo value at %d out of %d", private_index, max_count_); + updated = value_vector_.at(private_index)->setRepoValue(new_value, old_value, reclaimed_size); + if (updated) + { + purge_list_.push_back(old_value.getKey()); + } + if (reclaimed_size > 0) { + /** + * this is okay since current_size_ is really an estimate. + * we don't need precise counts. + */ + if (current_size_ < reclaimed_size) { + current_size_ = 0; + } else { + current_size_ -= reclaimed_size; + } + } + } while (!updated); + current_size_ += size; + + logger_->log_debug("VolatileRepository -- put %d %d", current_size_.load(), current_index_.load()); + return true; +} +/** + * Deletes the key + * @return status of the delete operation + */ +template +bool VolatileRepository::Delete(T key) { + for (auto ent : value_vector_) { + // let the destructor do the cleanup + RepoValue value; + if (ent->getValue(key, value)) { + current_size_ -= value.size(); + return true; + } + } + return false; +} +/** + * Sets the value from the provided key. Once the item is retrieved + * it may not be retrieved again. + * @return status of the get operation. + */ +template +bool VolatileRepository::Get(const T &key, std::string &value) { + + for (auto ent : value_vector_) { + // let the destructor do the cleanup + RepoValue repo_value; + if (ent->getValue(key, repo_value)) { + current_size_ -= value.size(); + repo_value.emplace(value); + return true; + } + } + return false; +} + +template +bool VolatileRepository::DeSerialize(std::vector> &store, size_t &max_size, std::function()> lambda) { + size_t requested_batch = max_size; + max_size = 0; + for (auto ent : value_vector_) { + // let the destructor do the cleanup + RepoValue repo_value; + + if (ent->getValue(repo_value)) { + std::shared_ptr newComponent = lambda(); + // we've taken ownership of this repo value + newComponent->DeSerialize(repo_value.getBuffer(), repo_value.getBufferSize()); + + store.push_back(newComponent); + + if (max_size++ >= requested_batch) { + break; + } + } + } + if (max_size > 0) { + return true; + } else { + return false; + } +} + +template +bool VolatileRepository::DeSerialize(std::vector> &store, size_t &max_size) { + logger_->log_debug("VolatileRepository -- DeSerialize %d", current_size_.load()); + max_size = 0; + for (auto ent : value_vector_) { + // let the destructor do the cleanup + RepoValue repo_value; + + if (ent->getValue(repo_value)) { + // we've taken ownership of this repo value + store.at(max_size)->DeSerialize(repo_value.getBuffer(), repo_value.getBufferSize()); + if (max_size++ >= store.size()) { + break; + } + } + } + if (max_size > 0) { + return true; + } else { + return false; + } +} + +template +void VolatileRepository::setConnectionMap(std::map> &connectionMap) { + this->connectionMap = connectionMap; +} + +template +void VolatileRepository::start() { + if (this->purge_period_ <= 0) + return; + if (running_) + return; + thread_ = std::thread(&VolatileRepository::run, std::enable_shared_from_this>::shared_from_this()); + thread_.detach(); + running_ = true; + logger_->log_info("%s Repository Monitor Thread Start", name_); } -; } /* namespace repository */ } /* namespace core */ diff --git a/libminifi/include/core/yaml/YamlConfiguration.h b/libminifi/include/core/yaml/YamlConfiguration.h index e03c794bf4..17b060fea7 100644 --- a/libminifi/include/core/yaml/YamlConfiguration.h +++ b/libminifi/include/core/yaml/YamlConfiguration.h @@ -21,7 +21,7 @@ #include "core/ProcessorConfig.h" #include "yaml-cpp/yaml.h" #include "processors/LoadProcessors.h" -#include "../FlowConfiguration.h" +#include "core/FlowConfiguration.h" #include "Site2SiteClientProtocol.h" #include #include "io/validation.h" @@ -46,13 +46,12 @@ namespace core { class YamlConfiguration : public FlowConfiguration { public: - explicit YamlConfiguration(std::shared_ptr repo, std::shared_ptr flow_file_repo, std::shared_ptr stream_factory, - std::shared_ptr configuration, - const std::string path = DEFAULT_FLOW_YAML_FILE_NAME) - : FlowConfiguration(repo, flow_file_repo, stream_factory, configuration, path), + explicit YamlConfiguration(std::shared_ptr repo, std::shared_ptr flow_file_repo, std::shared_ptr content_repo, + std::shared_ptr stream_factory, std::shared_ptr configuration, const std::string path = DEFAULT_FLOW_YAML_FILE_NAME) + : FlowConfiguration(repo, flow_file_repo, content_repo, stream_factory, configuration, path), logger_(logging::LoggerFactory::getLogger()) { stream_factory_ = stream_factory; - if (IsNullOrEmpty(config_path_)) { + if (IsNullOrEmpty (config_path_)) { config_path_ = DEFAULT_FLOW_YAML_FILE_NAME; } } @@ -93,20 +92,20 @@ class YamlConfiguration : public FlowConfiguration { } /** - * Returns a shared pointer to a ProcessGroup object containing the - * flow configuration. The yamlConfigPayload argument must be - * a payload for the raw YAML configuration. - * - * @param yamlConfigPayload an input payload for the raw YAML configuration - * to be parsed and loaded into the flow - * configuration tree - * @return the root ProcessGroup node of the flow - * configuration tree - */ - std::unique_ptr getRootFromPayload(std::string &yamlConfigPayload) { - YAML::Node rootYamlNode = YAML::Load(yamlConfigPayload); - return getRoot(&rootYamlNode); - } + * Returns a shared pointer to a ProcessGroup object containing the + * flow configuration. The yamlConfigPayload argument must be + * a payload for the raw YAML configuration. + * + * @param yamlConfigPayload an input payload for the raw YAML configuration + * to be parsed and loaded into the flow + * configuration tree + * @return the root ProcessGroup node of the flow + * configuration tree + */ + std::unique_ptr getRootFromPayload(std::string &yamlConfigPayload) { + YAML::Node rootYamlNode = YAML::Load(yamlConfigPayload); + return getRoot(&rootYamlNode); + } protected: diff --git a/libminifi/include/io/AtomicEntryStream.h b/libminifi/include/io/AtomicEntryStream.h new file mode 100644 index 0000000000..48322a42fb --- /dev/null +++ b/libminifi/include/io/AtomicEntryStream.h @@ -0,0 +1,192 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#ifndef LIBMINIFI_INCLUDE_IO_ATOMICENTRYSTREAM_H_ +#define LIBMINIFI_INCLUDE_IO_ATOMICENTRYSTREAM_H_ + +#include +#include +#include "BaseStream.h" +#include "core/repository/AtomicRepoEntries.h" +#include "Exception.h" +#include "core/logging/LoggerConfiguration.h" +namespace org { +namespace apache { +namespace nifi { +namespace minifi { +namespace io { + +template +class AtomicEntryStream : public BaseStream { + public: + AtomicEntryStream(const T key, core::repository::AtomicEntry *entry) + : key_(key), + entry_(entry), + offset_(0), + length_(0), + logger_(logging::LoggerFactory::getLogger()) { + core::repository::RepoValue *value; + if (entry_->getValue(key, &value)) { + length_ = value->getBufferSize(); + std::cout << "Length is " << length_ << std::endl; + } else { + throw Exception(FILE_OPERATION_EXCEPTION, "Could not create valid entry"); + } + } + + virtual void closeStream() { + + } + + /** + * Skip to the specified offset. + * @param offset offset to which we will skip + */ + void seek(uint64_t offset); + + virtual const uint32_t getSize() const { + std::cout << "Length is " << length_ << std::endl; + return length_; + } + + // data stream extensions + /** + * Reads data and places it into buf + * @param buf buffer in which we extract data + * @param buflen + */ + virtual int readData(std::vector &buf, int buflen); + /** + * Reads data and places it into buf + * @param buf buffer in which we extract data + * @param buflen + */ + virtual int readData(uint8_t *buf, int buflen); + + /** + * Write value to the stream using std::vector + * @param buf incoming buffer + * @param buflen buffer to write + * + */ + virtual int writeData(std::vector &buf, int buflen); + + /** + * writes value to stream + * @param value value to write + * @param size size of value + */ + virtual int writeData(uint8_t *value, int size); + + /** + * Returns the underlying buffer + * @return vector's array + **/ + const uint8_t *getBuffer() const { + throw std::runtime_error("Stream does not support this operation"); + } + + protected: + size_t length_; + size_t offset_; + T key_; + core::repository::AtomicEntry *entry_; + std::recursive_mutex entry_lock_; + + // Logger + std::shared_ptr logger_; + +}; + +template +void AtomicEntryStream::seek(uint64_t offset) { + std::lock_guard lock(entry_lock_); + offset_ = offset; +} + +template +int AtomicEntryStream::writeData(std::vector &buf, int buflen) { + if (buf.capacity() < buflen) + return -1; + return writeData(reinterpret_cast(&buf[0]), buflen); +} + +// data stream overrides +template +int AtomicEntryStream::writeData(uint8_t *value, int size) { + if (nullptr != value) { + std::lock_guard lock(entry_lock_); + if (entry_->insert(key_, value, size)) { + offset_ += size; + if (offset_ > length_) + { + length_ = offset_; + } + return size; + } + else { + logger_->log_debug("Cannot insert %d bytes due to insufficient space in atomic entry", size); + } + + } + return -1; + +} + +template +int AtomicEntryStream::readData(std::vector &buf, int buflen) { + if (buf.capacity() < buflen) { + buf.resize(buflen); + } + int ret = readData(reinterpret_cast(&buf[0]), buflen); + + if (ret < buflen) { + buf.resize(ret); + } + return ret; +} + +template +int AtomicEntryStream::readData(uint8_t *buf, int buflen) { + if (nullptr != buf) { + std::lock_guard lock(entry_lock_); + int len = buflen; + core::repository::RepoValue *value; + if (entry_->getValue(key_, &value)) { + if (offset_ + len > value->getBufferSize()) { + len = value->getBufferSize() - offset_; + if (len <= 0) { + return 0; + } + } + std::memcpy(buf, reinterpret_cast(const_cast(value->getBuffer()) + offset_), len); + offset_ += len; + + return len; + } + + } + return 0; +} + +} /* namespace io */ +} /* namespace minifi */ +} /* namespace nifi */ +} /* namespace apache */ +} /* namespace org */ + +#endif /* LIBMINIFI_INCLUDE_IO_ATOMICENTRYSTREAM_H_ */ diff --git a/libminifi/include/io/BaseStream.h b/libminifi/include/io/BaseStream.h index cae8a43571..cd982bb6c2 100644 --- a/libminifi/include/io/BaseStream.h +++ b/libminifi/include/io/BaseStream.h @@ -30,6 +30,11 @@ namespace nifi { namespace minifi { namespace io { +/** + * Base Stream. Not intended to be thread safe as it is not intended to be shared + * + * Extensions may be thread safe and thus shareable, but that is up to the implementation. + */ class BaseStream : public DataStream, public Serializable { public: @@ -55,6 +60,14 @@ class BaseStream : public DataStream, public Serializable { int writeData(uint8_t *value, int size); + virtual void seek(uint32_t offset) { + if (composable_stream_ != this) { + composable_stream_->seek(offset); + } else { + DataStream::seek(offset); + } + } + /** * write 2 bytes to stream * @param base_value non encoded value diff --git a/libminifi/include/io/ClientSocket.h b/libminifi/include/io/ClientSocket.h index c7db7f14fb..cd8a4fc135 100644 --- a/libminifi/include/io/ClientSocket.h +++ b/libminifi/include/io/ClientSocket.h @@ -84,7 +84,7 @@ class Socket : public BaseStream { * Static function to return the current machine's host name */ static std::string getMyHostName() { - static char *HOSTNAME = init_hostname(); + static std::string HOSTNAME = init_hostname(); return HOSTNAME; } @@ -239,12 +239,12 @@ class Socket : public BaseStream { private: std::shared_ptr logger_; - static char* init_hostname() { + static std::string init_hostname() { char hostname[1024]; gethostname(hostname, 1024); Socket mySock(nullptr, hostname, 0); mySock.initialize(); - return const_cast(mySock.getHostname().c_str()); + return mySock.getHostname(); } }; diff --git a/libminifi/include/io/DataStream.h b/libminifi/include/io/DataStream.h index 460930d0ff..2ebc9a43b5 100644 --- a/libminifi/include/io/DataStream.h +++ b/libminifi/include/io/DataStream.h @@ -30,6 +30,8 @@ namespace io { /** * DataStream defines the mechanism through which * binary data will be written to a sink + * + * This object is not intended to be thread safe. */ class DataStream { public: @@ -58,6 +60,10 @@ class DataStream { return 0; } + virtual void seek(uint32_t offset) { + readBuffer += offset; + } + virtual void closeStream() { } @@ -111,7 +117,7 @@ class DataStream { * Retrieve size of data stream * @return size of data stream **/ - const uint32_t getSize() const { + virtual const uint32_t getSize() const { return buffer.size(); } diff --git a/libminifi/include/io/FileStream.h b/libminifi/include/io/FileStream.h new file mode 100644 index 0000000000..06cb58ffb7 --- /dev/null +++ b/libminifi/include/io/FileStream.h @@ -0,0 +1,136 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#ifndef LIBMINIFI_INCLUDE_IO_TLS_FILESTREAM_H_ +#define LIBMINIFI_INCLUDE_IO_TLS_FILESTREAM_H_ + +#include +#include +#include +#include "EndianCheck.h" +#include "BaseStream.h" +#include "Serializable.h" +#include "core/logging/LoggerConfiguration.h" + +namespace org { +namespace apache { +namespace nifi { +namespace minifi { +namespace io { + +/** + * Purpose: File Stream Base stream extension. This is intended to be a thread safe access to + * read/write to the local file system. + * + * Design: Simply extends BaseStream and overrides readData/writeData to allow a sink to the + * fstream object. + */ +class FileStream : public io::BaseStream { + public: + /** + * File Stream constructor that accepts an fstream shared pointer. + * It must already be initialized for read and write. + */ + explicit FileStream(const std::string &path, uint32_t offset = 0); + + virtual ~FileStream() { + closeStream(); + } + + virtual void closeStream() { + std::lock_guard lock(file_lock_); + if (file_stream_ != nullptr) { + file_stream_->close(); + file_stream_ = nullptr; + } + } + + /** + * Skip to the specified offset. + * @param offset offset to which we will skip + */ + void seek(uint64_t offset); + + const uint32_t getSize() const { + return length_; + } + + // data stream extensions + /** + * Reads data and places it into buf + * @param buf buffer in which we extract data + * @param buflen + */ + virtual int readData(std::vector &buf, int buflen); + /** + * Reads data and places it into buf + * @param buf buffer in which we extract data + * @param buflen + */ + virtual int readData(uint8_t *buf, int buflen); + + /** + * Write value to the stream using std::vector + * @param buf incoming buffer + * @param buflen buffer to write + * + */ + virtual int writeData(std::vector &buf, int buflen); + + /** + * writes value to stream + * @param value value to write + * @param size size of value + */ + virtual int writeData(uint8_t *value, int size); + + /** + * Returns the underlying buffer + * @return vector's array + **/ + const uint8_t *getBuffer() const { + throw std::runtime_error("Stream does not support this operation"); + } + + protected: + + /** + * Creates a vector and returns the vector using the provided + * type name. + * @param t incoming object + * @returns vector. + */ + template + std::vector readBuffer(const T&); + std::recursive_mutex file_lock_; + std::unique_ptr file_stream_; + size_t offset_; + size_t length_; + + private: + + std::shared_ptr logger_; + +}; + +} /* namespace io */ +} /* namespace minifi */ +} /* namespace nifi */ +} /* namespace apache */ +} /* namespace org */ + +#endif /* LIBMINIFI_INCLUDE_IO_TLS_FILESTREAM_H_ */ diff --git a/libminifi/include/processors/ExecuteProcess.h b/libminifi/include/processors/ExecuteProcess.h index 28dcf76d4e..8cc7a251fd 100644 --- a/libminifi/include/processors/ExecuteProcess.h +++ b/libminifi/include/processors/ExecuteProcess.h @@ -31,6 +31,7 @@ #include #include #include +#include "io/BaseStream.h" #include "FlowFileRecord.h" #include "core/Processor.h" #include "core/ProcessSession.h" @@ -85,9 +86,12 @@ class ExecuteProcess : public core::Processor { } char *_data; uint64_t _dataSize; - void process(std::ofstream *stream) { + //void process(std::ofstream *stream) { + int64_t process(std::shared_ptr stream) { + int64_t ret = 0; if (_data && _dataSize > 0) - stream->write(_data, _dataSize); + ret = stream->write(reinterpret_cast(_data), _dataSize); + return ret; } }; diff --git a/libminifi/include/processors/GenerateFlowFile.h b/libminifi/include/processors/GenerateFlowFile.h index abb57400a2..7551e88d07 100644 --- a/libminifi/include/processors/GenerateFlowFile.h +++ b/libminifi/include/processors/GenerateFlowFile.h @@ -68,9 +68,11 @@ class GenerateFlowFile : public core::Processor { } char *_data; uint64_t _dataSize; - void process(std::ofstream *stream) { + int64_t process(std::shared_ptr stream) { + int64_t ret = 0; if (_data && _dataSize > 0) - stream->write(_data, _dataSize); + ret = stream->write(reinterpret_cast(_data), _dataSize); + return ret; } }; diff --git a/libminifi/include/processors/InvokeHTTP.h b/libminifi/include/processors/InvokeHTTP.h index d55a5be1d3..03a1611c14 100644 --- a/libminifi/include/processors/InvokeHTTP.h +++ b/libminifi/include/processors/InvokeHTTP.h @@ -104,6 +104,12 @@ class InvokeHTTP : public core::Processor { void onTrigger(core::ProcessContext *context, core::ProcessSession *session); void initialize(); void onSchedule(core::ProcessContext *context, core::ProcessSessionFactory *sessionFactory); + /** + * Provides a reference to the URL. + */ + const std::string &getUrl() { + return url_; + } protected: diff --git a/libminifi/include/processors/ListenHTTP.h b/libminifi/include/processors/ListenHTTP.h index c9e42bc6ad..1b58dcd83f 100644 --- a/libminifi/include/processors/ListenHTTP.h +++ b/libminifi/include/processors/ListenHTTP.h @@ -92,7 +92,7 @@ class ListenHTTP : public core::Processor { class WriteCallback : public OutputStreamCallback { public: WriteCallback(struct mg_connection *conn, const struct mg_request_info *reqInfo); - void process(std::ofstream *stream); + int64_t process(std::shared_ptr stream); private: // Logger diff --git a/libminifi/include/processors/ListenSyslog.h b/libminifi/include/processors/ListenSyslog.h index ed54b44315..25acac9f1b 100644 --- a/libminifi/include/processors/ListenSyslog.h +++ b/libminifi/include/processors/ListenSyslog.h @@ -114,14 +114,16 @@ class ListenSyslog : public core::Processor { class WriteCallback : public OutputStreamCallback { public: WriteCallback(char *data, uint64_t size) - : _data(data), + : _data(reinterpret_cast(data)), _dataSize(size) { } - char *_data; + uint8_t *_data; uint64_t _dataSize; - void process(std::ofstream *stream) { + int64_t process(std::shared_ptr stream) { + int64_t ret = 0; if (_data && _dataSize > 0) - stream->write(_data, _dataSize); + ret = stream->write(_data, _dataSize); + return ret; } }; diff --git a/libminifi/include/processors/LogAttribute.h b/libminifi/include/processors/LogAttribute.h index 88230f73f9..b9e333f0e2 100644 --- a/libminifi/include/processors/LogAttribute.h +++ b/libminifi/include/processors/LogAttribute.h @@ -87,25 +87,27 @@ class LogAttribute : public core::Processor { // Nest Callback Class for read stream class ReadCallback : public InputStreamCallback { public: - ReadCallback(uint64_t size) { - _bufferSize = size; - _buffer = new char[_bufferSize]; + ReadCallback(uint64_t size) + : read_size_(0) { + buffer_size_ = size; + buffer_ = new uint8_t[buffer_size_]; } ~ReadCallback() { - if (_buffer) - delete[] _buffer; + if (buffer_) + delete[] buffer_; } - void process(std::ifstream *stream) { - - stream->read(_buffer, _bufferSize); + int64_t process(std::shared_ptr stream) { + int64_t ret = 0; + ret = stream->read(buffer_, buffer_size_); if (!stream) - _readSize = stream->gcount(); + read_size_ = stream->getSize(); else - _readSize = _bufferSize; + read_size_ = buffer_size_; + return ret; } - char *_buffer; - uint64_t _bufferSize; - uint64_t _readSize; + uint8_t *buffer_; + uint64_t buffer_size_; + uint64_t read_size_; }; public: diff --git a/libminifi/include/processors/PutFile.h b/libminifi/include/processors/PutFile.h index f67e5126f4..c7f2823407 100644 --- a/libminifi/include/processors/PutFile.h +++ b/libminifi/include/processors/PutFile.h @@ -80,7 +80,8 @@ class PutFile : public core::Processor { public: ReadCallback(const std::string &tmpFile, const std::string &destFile); ~ReadCallback(); - virtual void process(std::ifstream *stream);bool commit(); + virtual int64_t process(std::shared_ptr stream); + bool commit(); private: std::shared_ptr logger_; diff --git a/libminifi/include/properties/Configure.h b/libminifi/include/properties/Configure.h index 13da55a646..341b89c15e 100644 --- a/libminifi/include/properties/Configure.h +++ b/libminifi/include/properties/Configure.h @@ -40,6 +40,7 @@ class Configure : public Properties { static const char *nifi_server_name; static const char *nifi_configuration_class_name; static const char *nifi_flow_repository_class_name; + static const char *nifi_content_repository_class_name; static const char *nifi_volatile_repository_options; static const char *nifi_provenance_repository_class_name; static const char *nifi_server_port; diff --git a/libminifi/include/provenance/Provenance.h b/libminifi/include/provenance/Provenance.h index 1479514899..d7c30e092e 100644 --- a/libminifi/include/provenance/Provenance.h +++ b/libminifi/include/provenance/Provenance.h @@ -29,7 +29,8 @@ #include #include #include - +#include "core/Core.h" +#include "core/SerializableComponent.h" #include "core/Repository.h" #include "core/Property.h" #include "properties/Configure.h" @@ -50,7 +51,7 @@ namespace provenance { #define PROVENANCE_EVENT_RECORD_SEG_SIZE 2048 // Provenance Event Record -class ProvenanceEventRecord : protected org::apache::nifi::minifi::io::Serializable { +class ProvenanceEventRecord : public core::SerializableComponent { public: enum ProvenanceEventType { @@ -163,7 +164,9 @@ class ProvenanceEventRecord : protected org::apache::nifi::minifi::io::Serializa */ ProvenanceEventRecord(ProvenanceEventType event, std::string componentId, std::string componentType); - ProvenanceEventRecord() { + + ProvenanceEventRecord() + : core::SerializableComponent(core::getClassName()){ _eventTime = getTimeMillis(); } @@ -172,7 +175,11 @@ class ProvenanceEventRecord : protected org::apache::nifi::minifi::io::Serializa } // Get the Event ID std::string getEventId() { - return _eventIdStr; + return uuidStr_; + } + + void setEventId(const std::string &id) { + setUUIDStr(id); } // Get Attributes std::map getAttributes() { @@ -220,7 +227,7 @@ class ProvenanceEventRecord : protected org::apache::nifi::minifi::io::Serializa } // Get FlowFileUuid std::string getFlowFileUuid() { - return uuid_; + return flow_uuid_; } // Get content full path std::string getContentFullPath() { @@ -333,7 +340,7 @@ class ProvenanceEventRecord : protected org::apache::nifi::minifi::io::Serializa _entryDate = flow->getEntryDate(); _lineageStartDate = flow->getlineageStartDate(); _lineageIdentifiers = flow->getlineageIdentifiers(); - uuid_ = flow->getUUIDStr(); + flow_uuid_ = flow->getUUIDStr(); _attributes = flow->getAttributes(); _size = flow->getSize(); _offset = flow->getOffset(); @@ -344,15 +351,15 @@ class ProvenanceEventRecord : protected org::apache::nifi::minifi::io::Serializa } } // Serialize and Persistent to the repository - bool Serialize(const std::shared_ptr &repo); + bool Serialize(const std::shared_ptr &repo); // DeSerialize - bool DeSerialize(const uint8_t *buffer, const int bufferSize); + bool DeSerialize(const uint8_t *buffer, const size_t bufferSize); // DeSerialize bool DeSerialize(org::apache::nifi::minifi::io::DataStream &stream) { return DeSerialize(stream.getBuffer(), stream.getSize()); } // DeSerialize - bool DeSerialize(const std::shared_ptr &repo, std::string key); + bool DeSerialize(const std::shared_ptr &repo); protected: @@ -373,15 +380,13 @@ class ProvenanceEventRecord : protected org::apache::nifi::minifi::io::Serializa // Size in bytes of the data corresponding to this flow file uint64_t _size; // flow uuid - std::string uuid_; + std::string flow_uuid_; // Offset to the content uint64_t _offset; // Full path to the content std::string _contentFullPath; // Attributes key/values pairs for the flow record std::map _attributes; - // provenance ID - uuid_t _eventId; // UUID string for all parents std::set _lineageIdentifiers; // transitUri @@ -396,8 +401,6 @@ class ProvenanceEventRecord : protected org::apache::nifi::minifi::io::Serializa std::string _details; // sourceQueueIdentifier std::string _sourceQueueIdentifier; - // event ID Str - std::string _eventIdStr; // relationship std::string _relationship; // alternateIdentifierUri; @@ -496,10 +499,9 @@ class ProvenanceReporter { private: + std::shared_ptr logger_; // Incoming connection Iterator std::set _events; - // Logger - std::shared_ptr logger_; // provenance repository. std::shared_ptr repo_; diff --git a/libminifi/include/provenance/ProvenanceRepository.h b/libminifi/include/provenance/ProvenanceRepository.h index dd2c5ecf55..8f669678e3 100644 --- a/libminifi/include/provenance/ProvenanceRepository.h +++ b/libminifi/include/provenance/ProvenanceRepository.h @@ -42,12 +42,11 @@ class ProvenanceRepository : public core::Repository, public std::enable_shared_ /*! * Create a new provenance repository */ - ProvenanceRepository(const std::string repo_name = "", std::string directory = PROVENANCE_DIRECTORY, int64_t maxPartitionMillis = - MAX_PROVENANCE_ENTRY_LIFE_TIME, - int64_t maxPartitionBytes = MAX_PROVENANCE_STORAGE_SIZE, uint64_t purgePeriod = PROVENANCE_PURGE_PERIOD) + ProvenanceRepository(const std::string repo_name = "", std::string directory = PROVENANCE_DIRECTORY, int64_t maxPartitionMillis = MAX_PROVENANCE_ENTRY_LIFE_TIME, int64_t maxPartitionBytes = + MAX_PROVENANCE_STORAGE_SIZE, + uint64_t purgePeriod = PROVENANCE_PURGE_PERIOD) : Repository(repo_name.length() > 0 ? repo_name : core::getClassName(), directory, maxPartitionMillis, maxPartitionBytes, purgePeriod), logger_(logging::LoggerFactory::getLogger()) { - db_ = NULL; } @@ -98,7 +97,7 @@ class ProvenanceRepository : public core::Repository, public std::enable_shared_ return true; } // Put - virtual bool Put(std::string key, uint8_t *buf, int bufLen) { + virtual bool Put(std::string key, const uint8_t *buf, size_t bufLen) { if (repo_full_) return false; @@ -122,7 +121,7 @@ class ProvenanceRepository : public core::Repository, public std::enable_shared_ return false; } // Get - virtual bool Get(std::string key, std::string &value) { + virtual bool Get(const std::string &key, std::string &value) { leveldb::Status status; status = db_->Get(leveldb::ReadOptions(), key, &value); if (status.ok()) @@ -130,17 +129,53 @@ class ProvenanceRepository : public core::Repository, public std::enable_shared_ else return false; } - // Persistent event - void registerEvent(std::shared_ptr &event) { - event->Serialize(std::static_pointer_cast(shared_from_this())); - } + // Remove event void removeEvent(ProvenanceEventRecord *event) { Delete(event->getEventId()); } + + virtual bool get(std::vector> &store, size_t max_size) { + leveldb::Iterator* it = db_->NewIterator(leveldb::ReadOptions()); + for (it->SeekToFirst(); it->Valid(); it->Next()) { + std::shared_ptr eventRead = std::make_shared(); + std::string key = it->key().ToString(); + if (store.size() >= max_size) + break; + if (eventRead->DeSerialize((uint8_t *) it->value().data(), (int) it->value().size())) { + store.push_back(std::dynamic_pointer_cast(eventRead)); + } + } + delete it; + return true; + } + + virtual bool DeSerialize(std::vector> &records, size_t &max_size, std::function()> lambda) { + leveldb::Iterator* it = db_->NewIterator(leveldb::ReadOptions()); + size_t requested_batch = max_size; + max_size = 0; + for (it->SeekToFirst(); it->Valid(); it->Next()) { + + if (max_size >= requested_batch) + break; + std::shared_ptr eventRead = lambda(); + std::string key = it->key().ToString(); + if (eventRead->DeSerialize((uint8_t *) it->value().data(), (int) it->value().size())) { + max_size++; + records.push_back(eventRead); + } + + } + delete it; + + if (max_size > 0) { + return true; + } else { + return false; + } + } //! get record void getProvenanceRecord(std::vector> &records, int maxSize) { - std::lock_guard lock(mutex_); leveldb::Iterator* it = db_->NewIterator(leveldb::ReadOptions()); for (it->SeekToFirst(); it->Valid(); it->Next()) { std::shared_ptr eventRead = std::make_shared(); @@ -153,9 +188,29 @@ class ProvenanceRepository : public core::Repository, public std::enable_shared_ } delete it; } + + virtual bool DeSerialize(std::vector> &store, size_t &max_size) { + leveldb::Iterator* it = db_->NewIterator(leveldb::ReadOptions()); + max_size = 0; + for (it->SeekToFirst(); it->Valid(); it->Next()) { + std::shared_ptr eventRead = std::make_shared(); + std::string key = it->key().ToString(); + + if (store.at(max_size)->DeSerialize((uint8_t *) it->value().data(), (int) it->value().size())) { + max_size++; + } + if (store.size() >= max_size) + break; + } + delete it; + if (max_size > 0) { + return true; + } else { + return false; + } + } //! purge record void purgeProvenanceRecord(std::vector> &records) { - std::lock_guard lock(mutex_); for (auto record : records) { Delete(record->getEventId()); } diff --git a/libminifi/include/utils/ByteInputCallBack.h b/libminifi/include/utils/ByteInputCallBack.h index a2b7838ce6..059de99e1e 100644 --- a/libminifi/include/utils/ByteInputCallBack.h +++ b/libminifi/include/utils/ByteInputCallBack.h @@ -32,20 +32,24 @@ namespace utils { */ class ByteInputCallBack : public InputStreamCallback { public: - ByteInputCallBack() { + ByteInputCallBack() + : ptr(nullptr) { } virtual ~ByteInputCallBack() { } - virtual void process(std::ifstream *stream) { + int64_t process(std::shared_ptr stream) { - std::vector nv = std::vector(std::istreambuf_iterator(*stream), std::istreambuf_iterator()); + std::vector nv = std::vector(reinterpret_cast(const_cast(stream->getBuffer())), + reinterpret_cast(const_cast(stream->getBuffer())) + stream->getSize()); vec = std::move(nv); ptr = &vec[0]; + return vec.size(); + } char *getBuffer() { diff --git a/libminifi/src/ConfigurationListener.cpp b/libminifi/src/ConfigurationListener.cpp index aaf50ce89a..858e455bf9 100644 --- a/libminifi/src/ConfigurationListener.cpp +++ b/libminifi/src/ConfigurationListener.cpp @@ -35,14 +35,10 @@ void ConfigurationListener::start() { pull_interval_ = 60 * 1000; std::string value; // grab the value for configuration - if (configure_->get(Configure::nifi_configuration_listener_pull_interval, - value)) { + if (configure_->get(Configure::nifi_configuration_listener_pull_interval, value)) { core::TimeUnit unit; - if (core::Property::StringToTime(value, pull_interval_, unit) - && core::Property::ConvertTimeUnitToMS(pull_interval_, unit, - pull_interval_)) { - logger_->log_info("Configuration Listener pull interval: [%d] ms", - pull_interval_); + if (core::Property::StringToTime(value, pull_interval_, unit) && core::Property::ConvertTimeUnitToMS(pull_interval_, unit, pull_interval_)) { + logger_->log_info("Configuration Listener pull interval: [%d] ms", pull_interval_); } } @@ -62,7 +58,7 @@ void ConfigurationListener::stop() { } void ConfigurationListener::run() { - std::unique_lock lk(mutex_); + std::unique_lock < std::mutex > lk(mutex_); std::condition_variable cv; int64_t interval = 0; while (!cv.wait_for(lk, std::chrono::milliseconds(100), [this] {return (running_ == false);})) { diff --git a/libminifi/src/Configure.cpp b/libminifi/src/Configure.cpp index 8bbc5fc820..acad1fdecd 100644 --- a/libminifi/src/Configure.cpp +++ b/libminifi/src/Configure.cpp @@ -31,7 +31,8 @@ const char *Configure::nifi_graceful_shutdown_seconds = "nifi.flowcontroller.gra const char *Configure::nifi_log_level = "nifi.log.level"; const char *Configure::nifi_server_name = "nifi.server.name"; const char *Configure::nifi_configuration_class_name = "nifi.flow.configuration.class.name"; -const char *Configure::nifi_flow_repository_class_name = "nifi.flow.repository.class.name"; +const char *Configure::nifi_flow_repository_class_name = "nifi.flowfile.repository.class.name"; +const char *Configure::nifi_content_repository_class_name = "nifi.content.repository.class.name"; const char *Configure::nifi_volatile_repository_options = "nifi.volatile.repository.options."; const char *Configure::nifi_provenance_repository_class_name = "nifi.provenance.repository.class.name"; const char *Configure::nifi_server_port = "nifi.server.port"; @@ -43,39 +44,22 @@ const char *Configure::nifi_flowfile_repository_max_storage_size = "nifi.flowfil const char *Configure::nifi_flowfile_repository_max_storage_time = "nifi.flowfile.repository.max.storage.time"; const char *Configure::nifi_flowfile_repository_directory_default = "nifi.flowfile.repository.directory.default"; const char *Configure::nifi_remote_input_secure = "nifi.remote.input.secure"; -const char *Configure::nifi_security_need_ClientAuth = - "nifi.security.need.ClientAuth"; -const char *Configure::nifi_security_client_certificate = - "nifi.security.client.certificate"; -const char *Configure::nifi_security_client_private_key = - "nifi.security.client.private.key"; -const char *Configure::nifi_security_client_pass_phrase = - "nifi.security.client.pass.phrase"; -const char *Configure::nifi_security_client_ca_certificate = - "nifi.security.client.ca.certificate"; -const char *Configure::nifi_configuration_listener_pull_interval = - "nifi.configuration.listener.pull.interval"; -const char *Configure::nifi_configuration_listener_http_url = - "nifi.configuration.listener.http.url"; -const char *Configure::nifi_configuration_listener_rest_url = - "nifi.configuration.listener.rest.url"; -const char *Configure::nifi_configuration_listener_type = - "nifi.configuration.listener.type"; -const char *Configure::nifi_https_need_ClientAuth = - "nifi.https.need.ClientAuth"; -const char *Configure::nifi_https_client_certificate = - "nifi.https.client.certificate"; -const char *Configure::nifi_https_client_private_key = - "nifi.https.client.private.key"; -const char *Configure::nifi_https_client_pass_phrase = - "nifi.https.client.pass.phrase"; -const char *Configure::nifi_https_client_ca_certificate = - "nifi.https.client.ca.certificate"; -const char *Configure::nifi_rest_api_user_name = - "nifi.rest.api.user.name"; -const char *Configure::nifi_rest_api_password = - "nifi.rest.api.password"; - +const char *Configure::nifi_security_need_ClientAuth = "nifi.security.need.ClientAuth"; +const char *Configure::nifi_security_client_certificate = "nifi.security.client.certificate"; +const char *Configure::nifi_security_client_private_key = "nifi.security.client.private.key"; +const char *Configure::nifi_security_client_pass_phrase = "nifi.security.client.pass.phrase"; +const char *Configure::nifi_security_client_ca_certificate = "nifi.security.client.ca.certificate"; +const char *Configure::nifi_configuration_listener_pull_interval = "nifi.configuration.listener.pull.interval"; +const char *Configure::nifi_configuration_listener_http_url = "nifi.configuration.listener.http.url"; +const char *Configure::nifi_configuration_listener_rest_url = "nifi.configuration.listener.rest.url"; +const char *Configure::nifi_configuration_listener_type = "nifi.configuration.listener.type"; +const char *Configure::nifi_https_need_ClientAuth = "nifi.https.need.ClientAuth"; +const char *Configure::nifi_https_client_certificate = "nifi.https.client.certificate"; +const char *Configure::nifi_https_client_private_key = "nifi.https.client.private.key"; +const char *Configure::nifi_https_client_pass_phrase = "nifi.https.client.pass.phrase"; +const char *Configure::nifi_https_client_ca_certificate = "nifi.https.client.ca.certificate"; +const char *Configure::nifi_rest_api_user_name = "nifi.rest.api.user.name"; +const char *Configure::nifi_rest_api_password = "nifi.rest.api.password"; } /* namespace minifi */ } /* namespace nifi */ diff --git a/libminifi/src/Connection.cpp b/libminifi/src/Connection.cpp index 0901a30ab1..841d941a7f 100644 --- a/libminifi/src/Connection.cpp +++ b/libminifi/src/Connection.cpp @@ -39,9 +39,11 @@ namespace apache { namespace nifi { namespace minifi { -Connection::Connection(std::shared_ptr flow_repository, std::string name, uuid_t uuid, uuid_t srcUUID, uuid_t destUUID) +Connection::Connection(const std::shared_ptr &flow_repository, const std::shared_ptr &content_repo, std::string name, uuid_t uuid, uuid_t srcUUID, + uuid_t destUUID) : core::Connectable(name, uuid), flow_repository_(flow_repository), + content_repo_(content_repo), logger_(logging::LoggerFactory::getLogger()) { if (srcUUID) @@ -60,13 +62,13 @@ Connection::Connection(std::shared_ptr flow_repository, std::s } bool Connection::isEmpty() { - std::lock_guard lock(mutex_); + std::lock_guard < std::mutex > lock(mutex_); return queue_.empty(); } bool Connection::isFull() { - std::lock_guard lock(mutex_); + std::lock_guard < std::mutex > lock(mutex_); if (max_queue_size_ <= 0 && max_data_queue_size_ <= 0) // No back pressure setting @@ -83,7 +85,7 @@ bool Connection::isFull() { void Connection::put(std::shared_ptr flow) { { - std::lock_guard lock(mutex_); + std::lock_guard < std::mutex > lock(mutex_); queue_.push(flow); @@ -94,7 +96,7 @@ void Connection::put(std::shared_ptr flow) { if (!flow->isStored()) { // Save to the flowfile repo - FlowFileRecord event(flow_repository_, flow, this->uuidStr_); + FlowFileRecord event(flow_repository_, content_repo_, flow, this->uuidStr_); if (event.Serialize()) { flow->setStoredToRepository(true); } @@ -107,7 +109,7 @@ void Connection::put(std::shared_ptr flow) { } std::shared_ptr Connection::poll(std::set> &expiredFlowRecords) { - std::lock_guard lock(mutex_); + std::lock_guard < std::mutex > lock(mutex_); while (!queue_.empty()) { std::shared_ptr item = queue_.front(); @@ -130,7 +132,7 @@ std::shared_ptr Connection::poll(std::setgetSize(); break; } - std::shared_ptr connectable = std::static_pointer_cast(shared_from_this()); + std::shared_ptr connectable = std::static_pointer_cast < Connectable > (shared_from_this()); item->setOriginalConnection(connectable); logger_->log_debug("Dequeue flow file UUID %s from connection %s", item->getUUIDStr().c_str(), name_.c_str()); @@ -149,7 +151,7 @@ std::shared_ptr Connection::poll(std::setgetSize(); break; } - std::shared_ptr connectable = std::static_pointer_cast(shared_from_this()); + std::shared_ptr connectable = std::static_pointer_cast < Connectable > (shared_from_this()); item->setOriginalConnection(connectable); logger_->log_debug("Dequeue flow file UUID %s from connection %s", item->getUUIDStr().c_str(), name_.c_str()); // delete from the flowfile repo @@ -165,7 +167,7 @@ std::shared_ptr Connection::poll(std::set lock(mutex_); + std::lock_guard < std::mutex > lock(mutex_); while (!queue_.empty()) { auto &&item = queue_.front(); diff --git a/libminifi/src/FlowControlProtocol.cpp b/libminifi/src/FlowControlProtocol.cpp index dbe27e8ad0..74a1573263 100644 --- a/libminifi/src/FlowControlProtocol.cpp +++ b/libminifi/src/FlowControlProtocol.cpp @@ -63,15 +63,14 @@ int FlowControlProtocol::connectServer(const char *host, uint16_t port) { close(sock); return 0; } - if (setsockopt(sock, SOL_SOCKET, SO_REUSEADDR, - reinterpret_cast(&opt), sizeof(opt)) < 0) { + if (setsockopt(sock, SOL_SOCKET, SO_REUSEADDR, reinterpret_cast(&opt), sizeof(opt)) < 0) { logger_->log_error("setsockopt() SO_REUSEADDR failed"); close(sock); return 0; } } - int sndsize = 256*1024; + int sndsize = 256 * 1024; if (setsockopt(sock, SOL_SOCKET, SO_SNDBUF, reinterpret_cast(&sndsize), sizeof(sndsize)) < 0) { logger_->log_error("setsockopt() SO_SNDBUF failed"); close(sock); diff --git a/libminifi/src/FlowController.cpp b/libminifi/src/FlowController.cpp index 2c84811a8e..798db715b1 100644 --- a/libminifi/src/FlowController.cpp +++ b/libminifi/src/FlowController.cpp @@ -39,8 +39,8 @@ #include "utils/StringUtils.h" #include "core/Core.h" #include "core/controller/ControllerServiceProvider.h" -#include "core/repository/FlowFileRepository.h" #include "core/logging/LoggerConfiguration.h" +#include "core/repository/FlowFileRepository.h" namespace org { namespace apache { @@ -52,7 +52,7 @@ std::shared_ptr FlowController::id_generator_ = utils::IdGen #define DEFAULT_CONFIG_NAME "conf/flow.yml" FlowController::FlowController(std::shared_ptr provenance_repo, std::shared_ptr flow_file_repo, std::shared_ptr configure, - std::unique_ptr flow_configuration, const std::string name, bool headless_mode) + std::unique_ptr flow_configuration, std::shared_ptr content_repo, const std::string name, bool headless_mode) : core::controller::ControllerServiceProvider(core::getClassName()), root_(nullptr), max_timer_driven_threads_(0), @@ -68,6 +68,7 @@ FlowController::FlowController(std::shared_ptr provenance_repo controller_service_provider_(nullptr), flow_configuration_(std::move(flow_configuration)), configuration_(configure), + content_repo_(content_repo), logger_(logging::LoggerFactory::getLogger()) { if (provenance_repo == nullptr) throw std::runtime_error("Provenance Repo should not be null"); @@ -159,8 +160,7 @@ bool FlowController::applyConfiguration(std::string &configurePayload) { std::unique_ptr newRoot; try { newRoot = std::move(flow_configuration_->getRootFromPayload(configurePayload)); - } - catch (const YAML::Exception& e) { + } catch (const YAML::Exception& e) { logger_->log_error("Invalid configuration payload"); return false; } @@ -168,10 +168,9 @@ bool FlowController::applyConfiguration(std::string &configurePayload) { if (newRoot == nullptr) return false; - logger_->log_info("Starting to reload Flow Controller with flow control name %s, version %d", - newRoot->getName().c_str(), newRoot->getVersion()); + logger_->log_info("Starting to reload Flow Controller with flow control name %s, version %d", newRoot->getName().c_str(), newRoot->getVersion()); - std::lock_guard flow_lock(mutex_); + std::lock_guard < std::recursive_mutex > flow_lock(mutex_); stop(true); waitUnload(30000); this->root_ = std::move(newRoot); @@ -181,7 +180,7 @@ bool FlowController::applyConfiguration(std::string &configurePayload) { } void FlowController::stop(bool force) { - std::lock_guard flow_lock(mutex_); + std::lock_guard < std::recursive_mutex > flow_lock(mutex_); if (running_) { // immediately indicate that we are not running running_ = false; @@ -222,7 +221,7 @@ void FlowController::waitUnload(const uint64_t timeToWaitMs) { } void FlowController::unload() { - std::lock_guard flow_lock(mutex_); + std::lock_guard < std::recursive_mutex > flow_lock(mutex_); if (running_) { stop(true); } @@ -237,7 +236,7 @@ void FlowController::unload() { } void FlowController::load() { - std::lock_guard flow_lock(mutex_); + std::lock_guard < std::recursive_mutex > flow_lock(mutex_); if (running_) { stop(true); } @@ -246,29 +245,30 @@ void FlowController::load() { // grab the value for configuration if (this->http_configuration_listener_ == nullptr && configuration_->get(Configure::nifi_configuration_listener_type, listenerType)) { if (listenerType == "http") { - this->http_configuration_listener_ = - std::unique_ptr(new minifi::HttpConfigurationListener(shared_from_this(), configuration_)); + this->http_configuration_listener_ = std::unique_ptr < minifi::HttpConfigurationListener > (new minifi::HttpConfigurationListener(shared_from_this(), configuration_)); } } logger_->log_info("Initializing timers"); if (nullptr == timer_scheduler_) { - timer_scheduler_ = std::make_shared(std::static_pointer_cast(shared_from_this()), provenance_repo_, configuration_); + timer_scheduler_ = std::make_shared < TimerDrivenSchedulingAgent + > (std::static_pointer_cast < core::controller::ControllerServiceProvider > (shared_from_this()), provenance_repo_, content_repo_, configuration_); } if (nullptr == event_scheduler_) { - event_scheduler_ = std::make_shared(std::static_pointer_cast(shared_from_this()), provenance_repo_, configuration_); + event_scheduler_ = std::make_shared < EventDrivenSchedulingAgent + > (std::static_pointer_cast < core::controller::ControllerServiceProvider > (shared_from_this()), provenance_repo_, content_repo_, configuration_); } logger_->log_info("Load Flow Controller from file %s", configuration_filename_.c_str()); - this->root_ = std::shared_ptr(flow_configuration_->getRoot(configuration_filename_)); + this->root_ = std::shared_ptr < core::ProcessGroup > (flow_configuration_->getRoot(configuration_filename_)); logger_->log_info("Loaded root processor Group"); controller_service_provider_ = flow_configuration_->getControllerServiceProvider(); - std::static_pointer_cast(controller_service_provider_)->setRootGroup(root_); - std::static_pointer_cast(controller_service_provider_)->setSchedulingAgent( - std::static_pointer_cast(event_scheduler_)); + std::static_pointer_cast < core::controller::StandardControllerServiceProvider > (controller_service_provider_)->setRootGroup(root_); + std::static_pointer_cast < core::controller::StandardControllerServiceProvider + > (controller_service_provider_)->setSchedulingAgent(std::static_pointer_cast < minifi::SchedulingAgent > (event_scheduler_)); logger_->log_info("Loaded controller service provider"); // Load Flow File from Repo @@ -279,7 +279,7 @@ void FlowController::load() { } void FlowController::reload(std::string yamlFile) { - std::lock_guard flow_lock(mutex_); + std::lock_guard < std::recursive_mutex > flow_lock(mutex_); logger_->log_info("Starting to reload Flow Controller with yaml %s", yamlFile.c_str()); stop(true); unload(); @@ -305,18 +305,18 @@ void FlowController::loadFlowRepo() { this->root_->getConnections(connectionMap); } logger_->log_debug("Number of connections from connectionMap %d", connectionMap.size()); - auto rep = std::dynamic_pointer_cast(flow_file_repo_); + auto rep = std::dynamic_pointer_cast < core::repository::FlowFileRepository > (flow_file_repo_); if (nullptr != rep) { rep->setConnectionMap(connectionMap); } - flow_file_repo_->loadComponent(); + flow_file_repo_->loadComponent(content_repo_); } else { logger_->log_debug("Flow file repository is not set"); } } bool FlowController::start() { - std::lock_guard flow_lock(mutex_); + std::lock_guard < std::recursive_mutex > flow_lock(mutex_); if (!initialized_) { logger_->log_error("Can not start Flow Controller because it has not been initialized"); return false; @@ -349,8 +349,7 @@ bool FlowController::start() { * @param id service identifier * @param firstTimeAdded first time this CS was added */ -std::shared_ptr FlowController::createControllerService(const std::string &type, const std::string &id, -bool firstTimeAdded) { +std::shared_ptr FlowController::createControllerService(const std::string &type, const std::string &id, bool firstTimeAdded) { return controller_service_provider_->createControllerService(type, id, firstTimeAdded); } diff --git a/libminifi/src/FlowFileRecord.cpp b/libminifi/src/FlowFileRecord.cpp index 12711a9286..dfe0d07811 100644 --- a/libminifi/src/FlowFileRecord.cpp +++ b/libminifi/src/FlowFileRecord.cpp @@ -40,8 +40,10 @@ namespace minifi { std::shared_ptr FlowFileRecord::logger_ = logging::LoggerFactory::getLogger(); std::atomic FlowFileRecord::local_flow_seq_number_(0); -FlowFileRecord::FlowFileRecord(std::shared_ptr flow_repository, std::map attributes, std::shared_ptr claim) +FlowFileRecord::FlowFileRecord(std::shared_ptr flow_repository, const std::shared_ptr &content_repo, std::map attributes, + std::shared_ptr claim) : FlowFile(), + content_repo_(content_repo), flow_repository_(flow_repository) { id_ = local_flow_seq_number_.load(); claim_ = claim; @@ -64,9 +66,11 @@ FlowFileRecord::FlowFileRecord(std::shared_ptr flow_repository claim_->increaseFlowFileRecordOwnedCount(); } -FlowFileRecord::FlowFileRecord(std::shared_ptr flow_repository, std::shared_ptr &event, const std::string &uuidConnection) +FlowFileRecord::FlowFileRecord(std::shared_ptr flow_repository, const std::shared_ptr &content_repo, std::shared_ptr &event, + const std::string &uuidConnection) : FlowFile(), snapshot_(""), + content_repo_(content_repo), flow_repository_(flow_repository) { entry_date_ = event->getEntryDate(); lineage_start_date_ = event->getlineageStartDate(); @@ -82,10 +86,11 @@ FlowFileRecord::FlowFileRecord(std::shared_ptr flow_repository } } -FlowFileRecord::FlowFileRecord(std::shared_ptr flow_repository, std::shared_ptr &event) +FlowFileRecord::FlowFileRecord(std::shared_ptr flow_repository, const std::shared_ptr &content_repo, std::shared_ptr &event) : FlowFile(), uuid_connection_(""), snapshot_(""), + content_repo_(content_repo), flow_repository_(flow_repository) { } @@ -101,7 +106,7 @@ FlowFileRecord::~FlowFileRecord() { if (claim_->getFlowFileRecordOwnedCount() <= 0) { logger_->log_debug("Delete Resource Claim %s", claim_->getContentFullPath().c_str()); if (!this->stored || !flow_repository_->Get(uuid_str_, value)) { - std::remove(claim_->getContentFullPath().c_str()); + content_repo_->remove(claim_); } } } diff --git a/libminifi/src/HttpConfigurationListener.cpp b/libminifi/src/HttpConfigurationListener.cpp index 39da67bb13..c16ca75eca 100644 --- a/libminifi/src/HttpConfigurationListener.cpp +++ b/libminifi/src/HttpConfigurationListener.cpp @@ -63,17 +63,14 @@ bool HttpConfigurationListener::pullConfiguration(std::string &configuration) { } utils::HTTPRequestResponse content; - curl_easy_setopt(http_session, CURLOPT_WRITEFUNCTION, - &utils::HTTPRequestResponse::recieve_write); + curl_easy_setopt(http_session, CURLOPT_WRITEFUNCTION, &utils::HTTPRequestResponse::recieve_write); - curl_easy_setopt(http_session, CURLOPT_WRITEDATA, - static_cast(&content)); + curl_easy_setopt(http_session, CURLOPT_WRITEDATA, static_cast(&content)); CURLcode res = curl_easy_perform(http_session); if (res == CURLE_OK) { - logger_->log_debug("HttpConfigurationListener -- curl successful to %s", - fullUrl.c_str()); + logger_->log_debug("HttpConfigurationListener -- curl successful to %s", fullUrl.c_str()); std::string response_body(content.data.begin(), content.data.end()); int64_t http_code = 0; @@ -82,8 +79,7 @@ bool HttpConfigurationListener::pullConfiguration(std::string &configuration) { /* ask for the content-type */ curl_easy_getinfo(http_session, CURLINFO_CONTENT_TYPE, &content_type); - bool isSuccess = ((int32_t) (http_code / 100)) == 2 - && res != CURLE_ABORTED_BY_CALLBACK; + bool isSuccess = ((int32_t) (http_code / 100)) == 2 && res != CURLE_ABORTED_BY_CALLBACK; bool body_empty = IsNullOrEmpty(content.data); if (isSuccess && !body_empty) { @@ -94,9 +90,7 @@ bool HttpConfigurationListener::pullConfiguration(std::string &configuration) { logger_->log_error("Cannot output body to content"); } } else { - logger_->log_error( - "HttpConfigurationListener -- curl_easy_perform() failed %s\n", - curl_easy_strerror(res)); + logger_->log_error("HttpConfigurationListener -- curl_easy_perform() failed %s\n", curl_easy_strerror(res)); } curl_easy_cleanup(http_session); diff --git a/libminifi/src/Properties.cpp b/libminifi/src/Properties.cpp index abebfbba59..076cefcbf3 100644 --- a/libminifi/src/Properties.cpp +++ b/libminifi/src/Properties.cpp @@ -34,7 +34,7 @@ Properties::Properties() // Get the config value bool Properties::get(std::string key, std::string &value) { - std::lock_guard lock(mutex_); + std::lock_guard < std::mutex > lock(mutex_); auto it = properties_.find(key); if (it != properties_.end()) { @@ -46,7 +46,7 @@ bool Properties::get(std::string key, std::string &value) { } int Properties::getInt(const std::string &key, int default_value) { - std::lock_guard lock(mutex_); + std::lock_guard < std::mutex > lock(mutex_); auto it = properties_.find(key); if (it != properties_.end()) { diff --git a/libminifi/src/RemoteProcessorGroupPort.cpp b/libminifi/src/RemoteProcessorGroupPort.cpp index d1862cdddd..86f8360e8b 100644 --- a/libminifi/src/RemoteProcessorGroupPort.cpp +++ b/libminifi/src/RemoteProcessorGroupPort.cpp @@ -18,7 +18,7 @@ * limitations under the License. */ -#include "../include/RemoteProcessorGroupPort.h" +#include "RemoteProcessorGroupPort.h" #include #include @@ -30,18 +30,20 @@ #include #include #include + #include #include #include #include "json/json.h" #include "json/writer.h" -#include "../include/core/logging/Logger.h" -#include "../include/core/ProcessContext.h" -#include "../include/core/ProcessorNode.h" -#include "../include/core/Property.h" -#include "../include/core/Relationship.h" -#include "../include/Site2SitePeer.h" +#include "Exception.h" +#include "core/logging/Logger.h" +#include "core/ProcessContext.h" +#include "core/ProcessorNode.h" +#include "core/Property.h" +#include "core/Relationship.h" +#include "Site2SitePeer.h" namespace org { namespace apache { @@ -54,8 +56,7 @@ core::Property RemoteProcessorGroupPort::port("Port", "Remote Port", ""); core::Property RemoteProcessorGroupPort::portUUID("Port UUID", "Specifies remote NiFi Port UUID.", ""); core::Relationship RemoteProcessorGroupPort::relation; -std::unique_ptr RemoteProcessorGroupPort::getNextProtocol( -bool create = true) { +std::unique_ptr RemoteProcessorGroupPort::getNextProtocol(bool create = true) { std::unique_ptr nextProtocol = nullptr; if (!available_protocols_.try_dequeue(nextProtocol)) { if (create) { @@ -170,31 +171,40 @@ void RemoteProcessorGroupPort::onTrigger(core::ProcessContext *context, core::Pr uuid_parse(value.c_str(), protocol_uuid_); } - std::unique_ptr protocol_ = getNextProtocol(); + std::unique_ptr protocol_ = nullptr; + try { + protocol_ = getNextProtocol(); - if (!protocol_) { - context->yield(); - return; - } + if (!protocol_) { + context->yield(); + return; + } + if (!protocol_->bootstrap()) { + // bootstrap the client protocol if needeed + context->yield(); + std::shared_ptr processor = std::static_pointer_cast < Processor > (context->getProcessorNode().getProcessor()); + logger_->log_error("Site2Site bootstrap failed yield period %d peer ", processor->getYieldPeriodMsec()); + + return; + } + + if (direction_ == RECEIVE) { + protocol_->receiveFlowFiles(context, session); + } else { + protocol_->transferFlowFiles(context, session); + } - if (!protocol_->bootstrap()) { - // bootstrap the client protocol if needeed - context->yield(); - std::shared_ptr processor = std::static_pointer_cast(context->getProcessorNode().getProcessor()); - logger_->log_error("Site2Site bootstrap failed yield period %d peer ", processor->getYieldPeriodMsec()); returnProtocol(std::move(protocol_)); return; + } catch (const minifi::Exception &ex2) { + context->yield(); + session->rollback(); + } catch (...) { + context->yield(); + session->rollback(); } - if (direction_ == RECEIVE) { - protocol_->receiveFlowFiles(context, session); - } else { - protocol_->transferFlowFiles(context, session); - } - - returnProtocol(std::move(protocol_)); - - return; + throw std::exception(); } void RemoteProcessorGroupPort::refreshRemoteSite2SiteInfo() { diff --git a/libminifi/src/ResourceClaim.cpp b/libminifi/src/ResourceClaim.cpp index 1a9f2fefa2..0986ca7741 100644 --- a/libminifi/src/ResourceClaim.cpp +++ b/libminifi/src/ResourceClaim.cpp @@ -17,14 +17,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +#include "ResourceClaim.h" #include - #include #include #include #include - -#include "ResourceClaim.h" +#include +#include "core/StreamManager.h" +#include "utils/Id.h" #include "core/logging/LoggerConfiguration.h" namespace org { @@ -36,14 +37,26 @@ utils::NonRepeatingStringGenerator ResourceClaim::non_repeating_string_generator char *ResourceClaim::default_directory_path = const_cast(DEFAULT_CONTENT_DIRECTORY); -ResourceClaim::ResourceClaim(const std::string contentDirectory) +ResourceClaim::ResourceClaim(std::shared_ptr> claim_manager, const std::string contentDirectory) : _flowFileRecordOwnedCount(0), + claim_manager_(claim_manager), + deleted_(false), logger_(logging::LoggerFactory::getLogger()) { - // Create the full content path for the content + + // Create the full content path for the content _contentFullPath = contentDirectory + "/" + non_repeating_string_generator_.generate(); logger_->log_debug("Resource Claim created %s", _contentFullPath); } +ResourceClaim::ResourceClaim(const std::string path, std::shared_ptr> claim_manager, bool deleted) + : claim_manager_(claim_manager), + deleted_(deleted) +{ + _contentFullPath = path; + + +} + } /* namespace minifi */ } /* namespace nifi */ } /* namespace apache */ diff --git a/libminifi/src/SchedulingAgent.cpp b/libminifi/src/SchedulingAgent.cpp index 24ba14629c..26dd165375 100644 --- a/libminifi/src/SchedulingAgent.cpp +++ b/libminifi/src/SchedulingAgent.cpp @@ -42,7 +42,7 @@ bool SchedulingAgent::hasWorkToDo(std::shared_ptr processor) { void SchedulingAgent::enableControllerService(std::shared_ptr &serviceNode) { logger_->log_trace("Enabling CSN in SchedulingAgent %s", serviceNode->getName()); // reference the enable function from serviceNode - std::function f_ex = [serviceNode] { + std::function < bool() > f_ex = [serviceNode] { return serviceNode->enable(); }; // create a functor that will be submitted to the thread pool. @@ -55,7 +55,7 @@ void SchedulingAgent::enableControllerService(std::shared_ptr &serviceNode) { // reference the disable function from serviceNode - std::function f_ex = [serviceNode] { + std::function < bool() > f_ex = [serviceNode] { return serviceNode->disable(); }; // create a functor that will be submitted to the thread pool. diff --git a/libminifi/src/Site2SiteClientProtocol.cpp b/libminifi/src/Site2SiteClientProtocol.cpp index 7d6e3f3115..024bd35ef0 100644 --- a/libminifi/src/Site2SiteClientProtocol.cpp +++ b/libminifi/src/Site2SiteClientProtocol.cpp @@ -726,13 +726,19 @@ bool Site2SiteClientProtocol::send(std::string transactionID, DataPacket *packet if (ret != 8) { return false; } - if (flowFile->getSize()) { + if (flowFile->getSize() > 0) { Site2SiteClientProtocol::ReadCallback callback(packet); session->read(flowFile, &callback); if (flowFile->getSize() != packet->_size) { return false; } } + if (packet->payload_.length() == 0 && len == 0) { + if (flowFile->getResourceClaim() == nullptr) + logger_->log_debug("no claim"); + else + logger_->log_debug("Flowfile empty %s", flowFile->getResourceClaim()->getContentFullPath()); + } } else if (packet->payload_.length() > 0) { len = packet->payload_.length(); @@ -1101,8 +1107,9 @@ void Site2SiteClientProtocol::transferFlowFiles(core::ProcessContext *context, c Transaction *transaction = NULL; - if (!flow) + if (!flow) { return; + } if (_peerState != READY) { bootstrap(); @@ -1158,11 +1165,15 @@ void Site2SiteClientProtocol::transferFlowFiles(core::ProcessContext *context, c } // while true if (!confirm(transactionID)) { - throw Exception(SITE2SITE_EXCEPTION, "Confirm Failed"); + std::stringstream ss; + ss << "Confirm Failed for " << transactionID; + throw Exception(SITE2SITE_EXCEPTION, ss.str().c_str()); return; } if (!complete(transactionID)) { - throw Exception(SITE2SITE_EXCEPTION, "Complete Failed"); + std::stringstream ss; + ss << "Complete Failed for " << transactionID; + throw Exception(SITE2SITE_EXCEPTION, ss.str().c_str()); return; } logger_->log_info("Site2Site transaction %s successfully send flow record %d, content bytes %d", transactionID.c_str(), transaction->_transfers, transaction->_bytes); diff --git a/libminifi/src/ThreadedSchedulingAgent.cpp b/libminifi/src/ThreadedSchedulingAgent.cpp index 46a4710f21..2bd7f13439 100644 --- a/libminifi/src/ThreadedSchedulingAgent.cpp +++ b/libminifi/src/ThreadedSchedulingAgent.cpp @@ -36,7 +36,7 @@ namespace nifi { namespace minifi { void ThreadedSchedulingAgent::schedule(std::shared_ptr processor) { - std::lock_guard lock(mutex_); + std::lock_guard < std::mutex > lock(mutex_); admin_yield_duration_ = 0; std::string yieldValue; @@ -68,8 +68,8 @@ void ThreadedSchedulingAgent::schedule(std::shared_ptr processo } core::ProcessorNode processor_node(processor); - auto processContext = std::make_shared(processor_node, controller_service_provider_, repo_); - auto sessionFactory = std::make_shared(processContext.get()); + auto processContext = std::make_shared < core::ProcessContext > (processor_node, controller_service_provider_, repo_, content_repo_); + auto sessionFactory = std::make_shared < core::ProcessSessionFactory > (processContext.get()); processor->onSchedule(processContext.get(), sessionFactory.get()); @@ -89,7 +89,7 @@ void ThreadedSchedulingAgent::schedule(std::shared_ptr processo } void ThreadedSchedulingAgent::unschedule(std::shared_ptr processor) { - std::lock_guard lock(mutex_); + std::lock_guard < std::mutex > lock(mutex_); logger_->log_info("Shutting down threads for processor %s/%s", processor->getName().c_str(), processor->getUUIDStr().c_str()); if (processor->getScheduledState() != core::RUNNING) { diff --git a/libminifi/src/controllers/SSLContextService.cpp b/libminifi/src/controllers/SSLContextService.cpp index a9450f664c..73c9e351e3 100644 --- a/libminifi/src/controllers/SSLContextService.cpp +++ b/libminifi/src/controllers/SSLContextService.cpp @@ -35,7 +35,7 @@ void SSLContextService::initialize() { if (initialized_) return; - std::lock_guard lock(initialization_mutex_); + std::lock_guard < std::mutex > lock(initialization_mutex_); ControllerService::initialize(); @@ -75,31 +75,31 @@ std::unique_ptr SSLContextService::createSSLContext() { if (retp == 0) { logger_->log_error("Can not load CA certificate, Exiting, error : %s", std::strerror(errno)); } - return std::unique_ptr(new SSLContext(ctx)); + return std::unique_ptr < SSLContext > (new SSLContext(ctx)); } const std::string &SSLContextService::getCertificateFile() { - std::lock_guard lock(initialization_mutex_); + std::lock_guard < std::mutex > lock(initialization_mutex_); return certificate; } const std::string &SSLContextService::getPassphrase() { - std::lock_guard lock(initialization_mutex_); + std::lock_guard < std::mutex > lock(initialization_mutex_); return passphrase_; } const std::string &SSLContextService::getPassphraseFile() { - std::lock_guard lock(initialization_mutex_); + std::lock_guard < std::mutex > lock(initialization_mutex_); return passphrase_file_; } const std::string &SSLContextService::getPrivateKeyFile() { - std::lock_guard lock(initialization_mutex_); + std::lock_guard < std::mutex > lock(initialization_mutex_); return private_key_; } const std::string &SSLContextService::getCACertificate() { - std::lock_guard lock(initialization_mutex_); + std::lock_guard < std::mutex > lock(initialization_mutex_); return ca_certificate_; } diff --git a/libminifi/src/core/ClassLoader.cpp b/libminifi/src/core/ClassLoader.cpp index 9bead0eca7..fbd46f6cac 100644 --- a/libminifi/src/core/ClassLoader.cpp +++ b/libminifi/src/core/ClassLoader.cpp @@ -43,7 +43,7 @@ uint16_t ClassLoader::registerResource(const std::string &resource) { logger_->log_error("Cannot load library: %s", dlerror()); return RESOURCE_FAILURE; } else { - std::lock_guard lock(internal_mutex_); + std::lock_guard < std::mutex > lock(internal_mutex_); dl_handles_.push_back(resource_ptr); } @@ -60,9 +60,9 @@ uint16_t ClassLoader::registerResource(const std::string &resource) { ObjectFactory *factory = create_factory_func(); - std::lock_guard lock(internal_mutex_); + std::lock_guard < std::mutex > lock(internal_mutex_); - loaded_factories_[factory->getClassName()] = std::unique_ptr(factory); + loaded_factories_[factory->getClassName()] = std::unique_ptr < ObjectFactory > (factory); return RESOURCE_SUCCESS; } diff --git a/libminifi/src/core/ConfigurableComponent.cpp b/libminifi/src/core/ConfigurableComponent.cpp index f5247ac74c..62a08db511 100644 --- a/libminifi/src/core/ConfigurableComponent.cpp +++ b/libminifi/src/core/ConfigurableComponent.cpp @@ -29,6 +29,7 @@ namespace apache { namespace nifi { namespace minifi { namespace core { + ConfigurableComponent::ConfigurableComponent() : logger_(logging::LoggerFactory::getLogger()) { } @@ -42,7 +43,7 @@ ConfigurableComponent::~ConfigurableComponent() { } bool ConfigurableComponent::getProperty(const std::string &name, Property &prop) { - std::lock_guard lock(configuration_mutex_); + std::lock_guard < std::mutex > lock(configuration_mutex_); auto &&it = properties_.find(name); @@ -61,7 +62,7 @@ bool ConfigurableComponent::getProperty(const std::string &name, Property &prop) * @return result of getting property. */ bool ConfigurableComponent::getProperty(const std::string name, std::string &value) { - std::lock_guard lock(configuration_mutex_); + std::lock_guard < std::mutex > lock(configuration_mutex_); auto &&it = properties_.find(name); if (it != properties_.end()) { @@ -80,7 +81,7 @@ bool ConfigurableComponent::getProperty(const std::string name, std::string &val * @return result of setting property. */ bool ConfigurableComponent::setProperty(const std::string name, std::string value) { - std::lock_guard lock(configuration_mutex_); + std::lock_guard < std::mutex > lock(configuration_mutex_); auto &&it = properties_.find(name); if (it != properties_.end()) { @@ -101,7 +102,7 @@ bool ConfigurableComponent::setProperty(const std::string name, std::string valu * @return result of setting property. */ bool ConfigurableComponent::updateProperty(const std::string &name, const std::string &value) { - std::lock_guard lock(configuration_mutex_); + std::lock_guard < std::mutex > lock(configuration_mutex_); auto &&it = properties_.find(name); if (it != properties_.end()) { @@ -122,7 +123,7 @@ bool ConfigurableComponent::updateProperty(const std::string &name, const std::s * @return whether property was set or not */ bool ConfigurableComponent::setProperty(Property &prop, std::string value) { - std::lock_guard lock(configuration_mutex_); + std::lock_guard < std::mutex > lock(configuration_mutex_); auto it = properties_.find(prop.getName()); if (it != properties_.end()) { @@ -150,7 +151,7 @@ bool ConfigurableComponent::setSupportedProperties(std::set properties return false; } - std::lock_guard lock(configuration_mutex_); + std::lock_guard < std::mutex > lock(configuration_mutex_); properties_.clear(); for (auto item : properties) { diff --git a/libminifi/src/core/ConfigurationFactory.cpp b/libminifi/src/core/ConfigurationFactory.cpp index ea2ed5c317..0a0e911431 100644 --- a/libminifi/src/core/ConfigurationFactory.cpp +++ b/libminifi/src/core/ConfigurationFactory.cpp @@ -39,7 +39,8 @@ namespace core { class YamlConfiguration; #endif -std::unique_ptr createFlowConfiguration(std::shared_ptr repo, std::shared_ptr flow_file_repo, std::shared_ptr configure, +std::unique_ptr createFlowConfiguration(std::shared_ptr repo, std::shared_ptr flow_file_repo, + std::shared_ptr content_repo, std::shared_ptr configure, std::shared_ptr stream_factory, const std::string configuration_class_name, const std::string path, bool fail_safe) { std::string class_name_lc = configuration_class_name; @@ -47,22 +48,23 @@ std::unique_ptr createFlowConfiguration(std::shared_ptr try { if (class_name_lc == "flowconfiguration") { // load the base configuration. - return std::unique_ptr(new core::FlowConfiguration(repo, flow_file_repo, stream_factory, configure, path)); + + return std::unique_ptr < core::FlowConfiguration > (new core::FlowConfiguration(repo, flow_file_repo, content_repo, stream_factory, configure, path)); } else if (class_name_lc == "yamlconfiguration") { // only load if the class is defined. - return std::unique_ptr(instantiate(repo, flow_file_repo, stream_factory, configure, path)); + return std::unique_ptr < core::FlowConfiguration > (instantiate(repo, flow_file_repo, content_repo, stream_factory, configure, path)); } else { if (fail_safe) { - return std::unique_ptr(new core::FlowConfiguration(repo, flow_file_repo, stream_factory, configure, path)); + return std::unique_ptr < core::FlowConfiguration > (new core::FlowConfiguration(repo, flow_file_repo, content_repo, stream_factory, configure, path)); } else { throw std::runtime_error("Support for the provided configuration class could not be found"); } } } catch (const std::runtime_error &r) { if (fail_safe) { - return std::unique_ptr(new core::FlowConfiguration(repo, flow_file_repo, stream_factory, configure, path)); + return std::unique_ptr < core::FlowConfiguration > (new core::FlowConfiguration(repo, flow_file_repo, content_repo, stream_factory, configure, path)); } } diff --git a/libminifi/src/core/Connectable.cpp b/libminifi/src/core/Connectable.cpp index cf01f0cf24..9c3b26aaa1 100644 --- a/libminifi/src/core/Connectable.cpp +++ b/libminifi/src/core/Connectable.cpp @@ -53,7 +53,7 @@ bool Connectable::setSupportedRelationships(std::set relatio return false; } - std::lock_guard lock(relationship_mutex_); + std::lock_guard < std::mutex > lock(relationship_mutex_); relationships_.clear(); for (auto item : relationships) { @@ -67,7 +67,7 @@ bool Connectable::setSupportedRelationships(std::set relatio bool Connectable::isSupportedRelationship(core::Relationship relationship) { const bool requiresLock = isRunning(); - const auto conditionalLock = !requiresLock ? std::unique_lock() : std::unique_lock(relationship_mutex_); + const auto conditionalLock = !requiresLock ? std::unique_lock() : std::unique_lock < std::mutex > (relationship_mutex_); const auto &it = relationships_.find(relationship.getName()); if (it != relationships_.end()) { @@ -83,7 +83,7 @@ bool Connectable::setAutoTerminatedRelationships(std::set relation return false; } - std::lock_guard lock(relationship_mutex_); + std::lock_guard < std::mutex > lock(relationship_mutex_); auto_terminated_relationships_.clear(); for (auto item : relationships) { @@ -97,7 +97,7 @@ bool Connectable::setAutoTerminatedRelationships(std::set relation bool Connectable::isAutoTerminated(core::Relationship relationship) { const bool requiresLock = isRunning(); - const auto conditionalLock = !requiresLock ? std::unique_lock() : std::unique_lock(relationship_mutex_); + const auto conditionalLock = !requiresLock ? std::unique_lock() : std::unique_lock < std::mutex > (relationship_mutex_); const auto &it = auto_terminated_relationships_.find(relationship.getName()); if (it != auto_terminated_relationships_.end()) { @@ -111,7 +111,7 @@ void Connectable::waitForWork(uint64_t timeoutMs) { has_work_.store(isWorkAvailable()); if (!has_work_.load()) { - std::unique_lock lock(work_available_mutex_); + std::unique_lock < std::mutex > lock(work_available_mutex_); work_condition_.wait_for(lock, std::chrono::milliseconds(timeoutMs), [&] {return has_work_.load();}); } } @@ -143,7 +143,7 @@ std::set> Connectable::getOutGoingConnections(std:: } std::shared_ptr Connectable::getNextIncomingConnection() { - std::lock_guard lock(relationship_mutex_); + std::lock_guard < std::mutex > lock(relationship_mutex_); if (_incomingConnections.size() == 0) return NULL; diff --git a/libminifi/src/core/Core.cpp b/libminifi/src/core/Core.cpp index 304d4ce0b1..995c001573 100644 --- a/libminifi/src/core/Core.cpp +++ b/libminifi/src/core/Core.cpp @@ -35,6 +35,11 @@ void CoreComponent::setUUID(uuid_t uuid) { uuid_unparse_lower(uuid_, uuidStr); uuidStr_ = uuidStr; } + +void CoreComponent::setUUIDStr(const std::string uuidStr) { + uuid_parse(uuidStr.c_str(), uuid_); + uuidStr_ = uuidStr; +} // Get UUID bool CoreComponent::getUUID(uuid_t uuid) { if (uuid) { diff --git a/libminifi/src/core/FlowConfiguration.cpp b/libminifi/src/core/FlowConfiguration.cpp index c32add699a..e8e746288f 100644 --- a/libminifi/src/core/FlowConfiguration.cpp +++ b/libminifi/src/core/FlowConfiguration.cpp @@ -35,7 +35,7 @@ std::shared_ptr FlowConfiguration::createProcessor(std::string if (nullptr == ptr) { logger_->log_error("No Processor defined for %s", name.c_str()); } - std::shared_ptr processor = std::static_pointer_cast(ptr); + std::shared_ptr processor = std::static_pointer_cast < core::Processor > (ptr); // initialize the processor processor->initialize(); @@ -53,18 +53,16 @@ std::shared_ptr FlowConfiguration::createProvenanceReportTask() return processor; } -std::unique_ptr FlowConfiguration::createRootProcessGroup( - std::string name, uuid_t uuid, int version) { - return std::unique_ptr( - new core::ProcessGroup(core::ROOT_PROCESS_GROUP, name, uuid, version)); +std::unique_ptr FlowConfiguration::createRootProcessGroup(std::string name, uuid_t uuid, int version) { + return std::unique_ptr < core::ProcessGroup > (new core::ProcessGroup(core::ROOT_PROCESS_GROUP, name, uuid, version)); } std::unique_ptr FlowConfiguration::createRemoteProcessGroup(std::string name, uuid_t uuid) { - return std::unique_ptr(new core::ProcessGroup(core::REMOTE_PROCESS_GROUP, name, uuid)); + return std::unique_ptr < core::ProcessGroup > (new core::ProcessGroup(core::REMOTE_PROCESS_GROUP, name, uuid)); } std::shared_ptr FlowConfiguration::createConnection(std::string name, uuid_t uuid) { - return std::make_shared(flow_file_repo_, name, uuid); + return std::make_shared < minifi::Connection > (flow_file_repo_, content_repo_, name, uuid); } std::shared_ptr FlowConfiguration::createControllerService(const std::string &class_name, const std::string &name, uuid_t uuid) { diff --git a/libminifi/src/core/ProcessGroup.cpp b/libminifi/src/core/ProcessGroup.cpp index 2cf3db04c1..db0fe08c20 100644 --- a/libminifi/src/core/ProcessGroup.cpp +++ b/libminifi/src/core/ProcessGroup.cpp @@ -39,8 +39,7 @@ namespace core { std::shared_ptr ProcessGroup::id_generator_ = utils::IdGenerator::getIdGenerator(); -ProcessGroup::ProcessGroup(ProcessGroupType type, std::string name, uuid_t uuid, int version, - ProcessGroup *parent) +ProcessGroup::ProcessGroup(ProcessGroupType type, std::string name, uuid_t uuid, int version, ProcessGroup *parent) : logger_(logging::LoggerFactory::getLogger()), name_(name), type_(type), @@ -55,7 +54,7 @@ ProcessGroup::ProcessGroup(ProcessGroupType type, std::string name, uuid_t uuid, yield_period_msec_ = 0; transmitting_ = false; - logger_->log_info("ProcessGroup %s created", name_.c_str()); + logger_->log_info("ProcessGroup %s created", name_); } ProcessGroup::~ProcessGroup() { @@ -70,12 +69,12 @@ ProcessGroup::~ProcessGroup() { } bool ProcessGroup::isRootProcessGroup() { - std::lock_guard lock(mutex_); + std::lock_guard < std::recursive_mutex > lock(mutex_); return (type_ == ROOT_PROCESS_GROUP); } void ProcessGroup::addProcessor(std::shared_ptr processor) { - std::lock_guard lock(mutex_); + std::lock_guard < std::recursive_mutex > lock(mutex_); if (processors_.find(processor) == processors_.end()) { // We do not have the same processor in this process group yet @@ -85,7 +84,7 @@ void ProcessGroup::addProcessor(std::shared_ptr processor) { } void ProcessGroup::removeProcessor(std::shared_ptr processor) { - std::lock_guard lock(mutex_); + std::lock_guard < std::recursive_mutex > lock(mutex_); if (processors_.find(processor) != processors_.end()) { // We do have the same processor in this process group yet @@ -95,7 +94,7 @@ void ProcessGroup::removeProcessor(std::shared_ptr processor) { } void ProcessGroup::addProcessGroup(ProcessGroup *child) { - std::lock_guard lock(mutex_); + std::lock_guard < std::recursive_mutex > lock(mutex_); if (child_process_groups_.find(child) == child_process_groups_.end()) { // We do not have the same child process group in this process group yet @@ -105,7 +104,7 @@ void ProcessGroup::addProcessGroup(ProcessGroup *child) { } void ProcessGroup::removeProcessGroup(ProcessGroup *child) { - std::lock_guard lock(mutex_); + std::lock_guard < std::recursive_mutex > lock(mutex_); if (child_process_groups_.find(child) != child_process_groups_.end()) { // We do have the same child process group in this process group yet @@ -115,7 +114,7 @@ void ProcessGroup::removeProcessGroup(ProcessGroup *child) { } void ProcessGroup::startProcessing(TimerDrivenSchedulingAgent *timeScheduler, EventDrivenSchedulingAgent *eventScheduler) { - std::lock_guard lock(mutex_); + std::lock_guard < std::recursive_mutex > lock(mutex_); try { // Start all the processor node, input and output ports @@ -143,7 +142,7 @@ void ProcessGroup::startProcessing(TimerDrivenSchedulingAgent *timeScheduler, Ev } void ProcessGroup::stopProcessing(TimerDrivenSchedulingAgent *timeScheduler, EventDrivenSchedulingAgent *eventScheduler) { - std::lock_guard lock(mutex_); + std::lock_guard < std::recursive_mutex > lock(mutex_); try { // Stop all the processor node, input and output ports @@ -169,7 +168,7 @@ void ProcessGroup::stopProcessing(TimerDrivenSchedulingAgent *timeScheduler, Eve } std::shared_ptr ProcessGroup::findProcessor(uuid_t uuid) { - std::lock_guard lock(mutex_); + std::lock_guard < std::recursive_mutex > lock(mutex_); std::shared_ptr ret = NULL; for (auto processor : processors_) { logger_->log_info("find processor %s", processor->getName().c_str()); @@ -209,7 +208,7 @@ std::shared_ptr ProcessGroup::findContr } std::shared_ptr ProcessGroup::findProcessor(const std::string &processorName) { - std::lock_guard lock(mutex_); + std::lock_guard < std::recursive_mutex > lock(mutex_); std::shared_ptr ret = NULL; for (auto processor : processors_) { logger_->log_debug("Current processor is %s", processor->getName().c_str()); @@ -225,7 +224,7 @@ std::shared_ptr ProcessGroup::findProcessor(const std::string &proces } void ProcessGroup::updatePropertyValue(std::string processorName, std::string propertyName, std::string propertyValue) { - std::lock_guard lock(mutex_); + std::lock_guard < std::recursive_mutex > lock(mutex_); for (auto processor : processors_) { if (processor->getName() == processorName) { processor->setProperty(propertyName, propertyValue); @@ -247,7 +246,7 @@ void ProcessGroup::getConnections(std::map connection) { - std::lock_guard lock(mutex_); + std::lock_guard < std::recursive_mutex > lock(mutex_); if (connections_.find(connection) == connections_.end()) { // We do not have the same connection in this process group yet @@ -269,7 +268,7 @@ void ProcessGroup::addConnection(std::shared_ptr connection) { } void ProcessGroup::removeConnection(std::shared_ptr connection) { - std::lock_guard lock(mutex_); + std::lock_guard < std::recursive_mutex > lock(mutex_); if (connections_.find(connection) != connections_.end()) { // We do not have the same connection in this process group yet diff --git a/libminifi/src/core/ProcessSession.cpp b/libminifi/src/core/ProcessSession.cpp index df21a34fca..0ea1ae3541 100644 --- a/libminifi/src/core/ProcessSession.cpp +++ b/libminifi/src/core/ProcessSession.cpp @@ -38,19 +38,21 @@ namespace core { std::shared_ptr ProcessSession::create() { std::map empty; - std::shared_ptr record = std::make_shared(process_context_->getProvenanceRepository(), empty); + + std::shared_ptr record = std::make_shared(process_context_->getProvenanceRepository(), process_context_->getContentRepository(), empty); _addedFlowFiles[record->getUUIDStr()] = record; logger_->log_debug("Create FlowFile with UUID %s", record->getUUIDStr().c_str()); - std::string details = process_context_->getProcessorNode().getName() + " creates flow record " + record->getUUIDStr(); - provenance_report_->create(record, details); + std::stringstream details; + details << process_context_->getProcessorNode().getName() << " creates flow record " << record->getUUIDStr(); + provenance_report_->create(record, details.str()); return record; } std::shared_ptr ProcessSession::create(std::shared_ptr &&parent) { std::map empty; - std::shared_ptr record = std::make_shared(process_context_->getProvenanceRepository(), empty); + std::shared_ptr record = std::make_shared(process_context_->getProvenanceRepository(), process_context_->getContentRepository(), empty); if (record) { _addedFlowFiles[record->getUUIDStr()] = record; @@ -92,7 +94,7 @@ std::shared_ptr ProcessSession::clone(std::shared_ptr ProcessSession::cloneDuringTransfer(std::shared_ptr &parent) { std::map empty; - std::shared_ptr record = std::make_shared(process_context_->getProvenanceRepository(), empty); + std::shared_ptr record = std::make_shared(process_context_->getProvenanceRepository(), process_context_->getContentRepository(), empty); if (record) { this->_clonedFlowFiles[record->getUUIDStr()] = record; @@ -168,26 +170,30 @@ void ProcessSession::remove(std::shared_ptr &&flow) { void ProcessSession::putAttribute(std::shared_ptr &flow, std::string key, std::string value) { flow->setAttribute(key, value); - std::string details = process_context_->getProcessorNode().getName() + " modify flow record " + flow->getUUIDStr() + " attribute " + key + ":" + value; - provenance_report_->modifyAttributes(flow, details); + std::stringstream details; + details << process_context_->getProcessorNode().getName() << " modify flow record " << flow->getUUIDStr() << " attribute " << key << ":" << value; + provenance_report_->modifyAttributes(flow, details.str()); } void ProcessSession::removeAttribute(std::shared_ptr &flow, std::string key) { flow->removeAttribute(key); - std::string details = process_context_->getProcessorNode().getName() + " remove flow record " + flow->getUUIDStr() + " attribute " + key; - provenance_report_->modifyAttributes(flow, details); + std::stringstream details; + details << process_context_->getProcessorNode().getName() << " remove flow record " << flow->getUUIDStr() << " attribute " + key; + provenance_report_->modifyAttributes(flow, details.str()); } void ProcessSession::putAttribute(std::shared_ptr &&flow, std::string key, std::string value) { flow->setAttribute(key, value); - std::string details = process_context_->getProcessorNode().getName() + " modify flow record " + flow->getUUIDStr() + " attribute " + key + ":" + value; - provenance_report_->modifyAttributes(flow, details); + std::stringstream details; + details << process_context_->getProcessorNode().getName() << " modify flow record " << flow->getUUIDStr() << " attribute " << key << ":" << value; + provenance_report_->modifyAttributes(flow, details.str()); } void ProcessSession::removeAttribute(std::shared_ptr &&flow, std::string key) { flow->removeAttribute(key); - std::string details = process_context_->getProcessorNode().getName() + " remove flow record " + flow->getUUIDStr() + " attribute " + key; - provenance_report_->modifyAttributes(flow, details); + std::stringstream details; + details << process_context_->getProcessorNode().getName() << " remove flow record " << flow->getUUIDStr() << " attribute " << key; + provenance_report_->modifyAttributes(flow, details.str()); } void ProcessSession::penalize(std::shared_ptr &flow) { @@ -207,41 +213,42 @@ void ProcessSession::transfer(std::shared_ptr &&flow, Relationsh } void ProcessSession::write(std::shared_ptr &flow, OutputStreamCallback *callback) { - std::shared_ptr claim = std::make_shared( + std::shared_ptr claim = std::make_shared(process_context_->getContentRepository(), DEFAULT_CONTENT_DIRECTORY); try { - std::ofstream fs; uint64_t startTime = getTimeMillis(); - fs.open(claim->getContentFullPath().c_str(), std::fstream::out | std::fstream::binary | std::fstream::trunc); - if (fs.is_open()) { - // Call the callback to write the content - callback->process(&fs); - if (fs.good() && fs.tellp() >= 0) { - flow->setSize(fs.tellp()); - flow->setOffset(0); - std::shared_ptr flow_claim = flow->getResourceClaim(); - if (flow_claim != nullptr) { - // Remove the old claim - flow_claim->decreaseFlowFileRecordOwnedCount(); - flow->clearResourceClaim(); - } - flow->setResourceClaim(claim); - claim->increaseFlowFileRecordOwnedCount(); - /* - logger_->log_debug("Write offset %d length %d into content %s for FlowFile UUID %s", - flow->_offset, flow->_size, flow->_claim->getContentFullPath().c_str(), flow->getUUIDStr().c_str()); */ - fs.close(); - std::string details = process_context_->getProcessorNode().getName() + " modify flow record content " + flow->getUUIDStr(); - uint64_t endTime = getTimeMillis(); - provenance_report_->modifyContent(flow, details, endTime - startTime); - } else { - fs.close(); - throw Exception(FILE_OPERATION_EXCEPTION, "File Write Error"); - } - } else { - throw Exception(FILE_OPERATION_EXCEPTION, "File Open Error"); + claim->increaseFlowFileRecordOwnedCount(); +// fs.open(claim->getContentFullPath().c_str(), std::fstream::out | std::fstream::binary | std::fstream::trunc); + std::shared_ptr stream = process_context_->getContentRepository()->write(claim); + // Call the callback to write the content + if (nullptr == stream) { + rollback(); + return; + } + if (callback->process(stream) < 0) { + rollback(); + return; + } + + flow->setSize(stream->getSize()); + flow->setOffset(0); + std::shared_ptr flow_claim = flow->getResourceClaim(); + if (flow_claim != nullptr) { + // Remove the old claim + flow_claim->decreaseFlowFileRecordOwnedCount(); + flow->clearResourceClaim(); } + flow->setResourceClaim(claim); + + /* + logger_->log_debug("Write offset %d length %d into content %s for FlowFile UUID %s", + flow->_offset, flow->_size, flow->_claim->getContentFullPath().c_str(), flow->getUUIDStr().c_str()); */ + stream->closeStream(); + std::stringstream details; + details << process_context_->getProcessorNode().getName() << " modify flow record content " << flow->getUUIDStr(); + uint64_t endTime = getTimeMillis(); + provenance_report_->modifyContent(flow, details.str(), endTime - startTime); } catch (std::exception &exception) { if (flow && flow->getResourceClaim() == claim) { flow->getResourceClaim()->decreaseFlowFileRecordOwnedCount(); @@ -260,39 +267,34 @@ void ProcessSession::write(std::shared_ptr &flow, OutputStreamCa } void ProcessSession::write(std::shared_ptr &&flow, OutputStreamCallback *callback) { - std::shared_ptr claim = std::make_shared(); + std::shared_ptr claim = std::make_shared(process_context_->getContentRepository()); try { - std::ofstream fs; uint64_t startTime = getTimeMillis(); - fs.open(claim->getContentFullPath().c_str(), std::fstream::out | std::fstream::binary | std::fstream::trunc); - if (fs.is_open()) { - // Call the callback to write the content - callback->process(&fs); - if (fs.good() && fs.tellp() >= 0) { - flow->setSize(fs.tellp()); - flow->setOffset(0); - std::shared_ptr flow_claim = flow->getResourceClaim(); - if (flow_claim != nullptr) { - // Remove the old claim - flow_claim->decreaseFlowFileRecordOwnedCount(); - flow->clearResourceClaim(); - } - flow->setResourceClaim(claim); - claim->increaseFlowFileRecordOwnedCount(); - /* - logger_->log_debug("Write offset %d length %d into content %s for FlowFile UUID %s", - flow->_offset, flow->_size, flow->_claim->getContentFullPath().c_str(), flow->getUUIDStr().c_str()); */ - fs.close(); - std::string details = process_context_->getProcessorNode().getName() + " modify flow record content " + flow->getUUIDStr(); - uint64_t endTime = getTimeMillis(); - provenance_report_->modifyContent(flow, details, endTime - startTime); - } else { - fs.close(); - throw Exception(FILE_OPERATION_EXCEPTION, "File Write Error"); - } - } else { - throw Exception(FILE_OPERATION_EXCEPTION, "File Open Error"); + claim->increaseFlowFileRecordOwnedCount(); + std::shared_ptr stream = process_context_->getContentRepository()->write(claim); + if (nullptr == stream) { + rollback(); + return; + } + // Call the callback to write the content + if (callback->process(stream) < 0) { + rollback(); + return; + } + flow->setSize(stream->getSize()); + flow->setOffset(0); + std::shared_ptr flow_claim = flow->getResourceClaim(); + if (flow_claim != nullptr) { + // Remove the old claim + flow_claim->decreaseFlowFileRecordOwnedCount(); + flow->clearResourceClaim(); } + flow->setResourceClaim(claim); + + std::stringstream details; + details << process_context_->getProcessorNode().getName() << " modify flow record content " << flow->getUUIDStr(); + uint64_t endTime = getTimeMillis(); + provenance_report_->modifyContent(flow, details.str(), endTime - startTime); } catch (std::exception &exception) { if (flow && flow->getResourceClaim() == claim) { flow->getResourceClaim()->decreaseFlowFileRecordOwnedCount(); @@ -321,30 +323,25 @@ void ProcessSession::append(std::shared_ptr &&flow, OutputStream claim = flow->getResourceClaim(); try { - std::ofstream fs; uint64_t startTime = getTimeMillis(); - fs.open(claim->getContentFullPath().c_str(), std::fstream::out | std::fstream::binary | std::fstream::app); - if (fs.is_open()) { - // Call the callback to write the content - std::streampos oldPos = fs.tellp(); - callback->process(&fs); - if (fs.good() && fs.tellp() >= 0) { - uint64_t appendSize = fs.tellp() - oldPos; - flow->setSize(flow->getSize() + appendSize); - /* - logger_->log_debug("Append offset %d extra length %d to new size %d into content %s for FlowFile UUID %s", - flow->_offset, appendSize, flow->_size, claim->getContentFullPath().c_str(), flow->getUUIDStr().c_str()); */ - fs.close(); - std::string details = process_context_->getProcessorNode().getName() + " modify flow record content " + flow->getUUIDStr(); - uint64_t endTime = getTimeMillis(); - provenance_report_->modifyContent(flow, details, endTime - startTime); - } else { - fs.close(); - throw Exception(FILE_OPERATION_EXCEPTION, "File Write Error"); - } - } else { - throw Exception(FILE_OPERATION_EXCEPTION, "File Open Error"); - } + std::shared_ptr stream = process_context_->getContentRepository()->write(claim); + if (nullptr == stream) { + rollback(); + return; + } + // Call the callback to write the content + size_t oldPos = stream->getSize(); + stream->seek(oldPos + 1); + if (callback->process(stream) < 0) { + rollback(); + return; + } + uint64_t appendSize = stream->getSize() - oldPos; + flow->setSize(flow->getSize() + appendSize); + std::stringstream details; + details << process_context_->getProcessorNode().getName() << " modify flow record content " << flow->getUUIDStr(); + uint64_t endTime = getTimeMillis(); + provenance_report_->modifyContent(flow, details.str(), endTime - startTime); } catch (std::exception &exception) { logger_->log_debug("Caught Exception %s", exception.what()); throw; @@ -365,30 +362,26 @@ void ProcessSession::append(std::shared_ptr &flow, OutputStreamC claim = flow->getResourceClaim(); try { - std::ofstream fs; uint64_t startTime = getTimeMillis(); - fs.open(claim->getContentFullPath().c_str(), std::fstream::out | std::fstream::binary | std::fstream::app); - if (fs.is_open()) { - // Call the callback to write the content - std::streampos oldPos = fs.tellp(); - callback->process(&fs); - if (fs.good() && fs.tellp() >= 0) { - uint64_t appendSize = fs.tellp() - oldPos; - flow->setSize(flow->getSize() + appendSize); - /* - logger_->log_debug("Append offset %d extra length %d to new size %d into content %s for FlowFile UUID %s", - flow->_offset, appendSize, flow->_size, claim->getContentFullPath().c_str(), flow->getUUIDStr().c_str()); */ - fs.close(); - std::string details = process_context_->getProcessorNode().getName() + " modify flow record content " + flow->getUUIDStr(); - uint64_t endTime = getTimeMillis(); - provenance_report_->modifyContent(flow, details, endTime - startTime); - } else { - fs.close(); - throw Exception(FILE_OPERATION_EXCEPTION, "File Write Error"); - } - } else { - throw Exception(FILE_OPERATION_EXCEPTION, "File Open Error"); - } + std::shared_ptr stream = process_context_->getContentRepository()->write(claim); + if (nullptr == stream) { + rollback(); + return; + } + // Call the callback to write the content + size_t oldPos = stream->getSize(); + stream->seek(oldPos + 1); + if (callback->process(stream) < 0) { + rollback(); + return; + } + uint64_t appendSize = stream->getSize() - oldPos; + flow->setSize(flow->getSize() + appendSize); + + std::stringstream details; + details << process_context_->getProcessorNode().getName() << " modify flow record content " << flow->getUUIDStr(); + uint64_t endTime = getTimeMillis(); + provenance_report_->modifyContent(flow, details.str(), endTime - startTime); } catch (std::exception &exception) { logger_->log_debug("Caught Exception %s", exception.what()); throw; @@ -408,23 +401,18 @@ void ProcessSession::read(std::shared_ptr &flow, InputStreamCall } claim = flow->getResourceClaim(); - std::ifstream fs; - fs.open(claim->getContentFullPath().c_str(), std::fstream::in | std::fstream::binary); - if (fs.is_open()) { - fs.seekg(flow->getOffset(), fs.beg); - - if (fs.good()) { - callback->process(&fs); - /* - logger_->log_debug("Read offset %d size %d content %s for FlowFile UUID %s", - flow->_offset, flow->_size, claim->getContentFullPath().c_str(), flow->getUUIDStr().c_str()); */ - fs.close(); - } else { - fs.close(); - throw Exception(FILE_OPERATION_EXCEPTION, "File Read Error"); - } - } else { - throw Exception(FILE_OPERATION_EXCEPTION, "File Open Error"); + + std::shared_ptr stream = process_context_->getContentRepository()->read(claim); + + if (nullptr == stream) { + rollback(); + return; + } + stream->seek(flow->getOffset()); + + if (callback->process(stream) < 0) { + rollback(); + return; } } catch (std::exception &exception) { logger_->log_debug("Caught Exception %s", exception.what()); @@ -445,23 +433,17 @@ void ProcessSession::read(std::shared_ptr &&flow, InputStreamCal } claim = flow->getResourceClaim(); - std::ifstream fs; - fs.open(claim->getContentFullPath().c_str(), std::fstream::in | std::fstream::binary); - if (fs.is_open()) { - fs.seekg(flow->getOffset(), fs.beg); - - if (fs.good()) { - callback->process(&fs); - /* - logger_->log_debug("Read offset %d size %d content %s for FlowFile UUID %s", - flow->_offset, flow->_size, claim->getContentFullPath().c_str(), flow->getUUIDStr().c_str()); */ - fs.close(); - } else { - fs.close(); - throw Exception(FILE_OPERATION_EXCEPTION, "File Read Error"); - } - } else { - throw Exception(FILE_OPERATION_EXCEPTION, "File Open Error"); + std::shared_ptr stream = process_context_->getContentRepository()->read(claim); + + if (nullptr == stream) { + rollback(); + return; + } + stream->seek(flow->getOffset()); + + if (callback->process(stream) < 0) { + rollback(); + return; } } catch (std::exception &exception) { logger_->log_debug("Caught Exception %s", exception.what()); @@ -479,60 +461,55 @@ void ProcessSession::read(std::shared_ptr &&flow, InputStreamCal * */ void ProcessSession::importFrom(io::DataStream &stream, std::shared_ptr &&flow) { - std::shared_ptr claim = std::make_shared(); - + std::shared_ptr claim = std::make_shared(process_context_->getContentRepository()); int max_read = getpagesize(); std::vector charBuffer; charBuffer.resize(max_read); try { - std::ofstream fs; uint64_t startTime = getTimeMillis(); - fs.open(claim->getContentFullPath().c_str(), std::fstream::out | std::fstream::binary | std::fstream::trunc); - - if (fs.is_open()) { - size_t position = 0; - const size_t max_size = stream.getSize(); - size_t read_size = max_read; - while (position < max_size) { - if ((max_size - position) > max_read) { - read_size = max_read; - } else { - read_size = max_size - position; - } - charBuffer.clear(); - stream.readData(charBuffer, read_size); - - fs.write((const char*) charBuffer.data(), read_size); - position += read_size; + claim->increaseFlowFileRecordOwnedCount(); + std::shared_ptr content_stream = process_context_->getContentRepository()->write(claim); + + if (nullptr == content_stream) { + logger_->log_debug("Could not obtain claim for %s", claim->getContentFullPath()); + rollback(); + return; + } + 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; } - // Open the source file and stream to the flow file + charBuffer.clear(); + stream.readData(charBuffer, read_size); - if (fs.good() && fs.tellp() >= 0) { - flow->setSize(fs.tellp()); - flow->setOffset(0); - if (flow->getResourceClaim() != nullptr) { - // Remove the old claim - flow->getResourceClaim()->decreaseFlowFileRecordOwnedCount(); - flow->clearResourceClaim(); - } - flow->setResourceClaim(claim); - claim->increaseFlowFileRecordOwnedCount(); - - logger_->log_debug("Import offset %d length %d into content %s for FlowFile UUID %s", flow->getOffset(), flow->getSize(), flow->getResourceClaim()->getContentFullPath().c_str(), - flow->getUUIDStr().c_str()); + content_stream->write(charBuffer.data(), read_size); + position += read_size; + } + // Open the source file and stream to the flow file - fs.close(); - std::string details = process_context_->getProcessorNode().getName() + " modify flow record content " + flow->getUUIDStr(); - uint64_t endTime = getTimeMillis(); - provenance_report_->modifyContent(flow, details, endTime - startTime); - } else { - fs.close(); - throw Exception(FILE_OPERATION_EXCEPTION, "File Import Error"); - } - } else { - throw Exception(FILE_OPERATION_EXCEPTION, "File Import Error"); + flow->setSize(content_stream->getSize()); + flow->setOffset(0); + if (flow->getResourceClaim() != nullptr) { + // Remove the old claim + flow->getResourceClaim()->decreaseFlowFileRecordOwnedCount(); + flow->clearResourceClaim(); } + flow->setResourceClaim(claim); + + logger_->log_debug("Import offset %d length %d into content %s for FlowFile UUID %s", flow->getOffset(), flow->getSize(), flow->getResourceClaim()->getContentFullPath().c_str(), + flow->getUUIDStr().c_str()); + + content_stream->closeStream(); + std::stringstream details; + details << process_context_->getProcessorNode().getName() << " modify flow record content " << flow->getUUIDStr(); + uint64_t endTime = getTimeMillis(); + provenance_report_->modifyContent(flow, details.str(), endTime - startTime); } catch (std::exception &exception) { if (flow && flow->getResourceClaim() == claim) { flow->getResourceClaim()->decreaseFlowFileRecordOwnedCount(); @@ -550,34 +527,45 @@ void ProcessSession::importFrom(io::DataStream &stream, std::shared_ptr &flow, -bool keepSource, - uint64_t offset) { - std::shared_ptr claim = std::make_shared(); +void ProcessSession::import(std::string source, std::shared_ptr &flow, bool keepSource, uint64_t offset) { + std::shared_ptr claim = std::make_shared(process_context_->getContentRepository()); char *buf = NULL; int size = 4096; buf = new char[size]; try { - std::ofstream fs; + // std::ofstream fs; uint64_t startTime = getTimeMillis(); - fs.open(claim->getContentFullPath().c_str(), std::fstream::out | std::fstream::binary | std::fstream::trunc); std::ifstream input; input.open(source.c_str(), std::fstream::in | std::fstream::binary); - - if (fs.is_open() && input.is_open()) { + claim->increaseFlowFileRecordOwnedCount(); + std::shared_ptr stream = process_context_->getContentRepository()->write(claim); + if (nullptr == stream) { + rollback(); + return; + } + if (input.is_open()) { // Open the source file and stream to the flow file - input.seekg(offset, fs.beg); + input.seekg(offset); + bool invalidWrite = false; while (input.good()) { input.read(buf, size); - if (input) - fs.write(buf, size); - else - fs.write(buf, input.gcount()); + if (input) { + if (stream->write(reinterpret_cast(buf), size) < 0) { + invalidWrite = true; + break; + } + } else { + if (stream->write(reinterpret_cast(buf), input.gcount()) < 0) { + invalidWrite = true; + break; + } + } } - if (fs.good() && fs.tellp() >= 0) { - flow->setSize(fs.tellp()); + if (!invalidWrite) { + std::cout << "Import " << stream->getSize() << std::endl; + flow->setSize(stream->getSize()); flow->setOffset(0); if (flow->getResourceClaim() != nullptr) { // Remove the old claim @@ -585,20 +573,20 @@ bool keepSource, flow->clearResourceClaim(); } flow->setResourceClaim(claim); - claim->increaseFlowFileRecordOwnedCount(); logger_->log_debug("Import offset %d length %d into content %s for FlowFile UUID %s", flow->getOffset(), flow->getSize(), flow->getResourceClaim()->getContentFullPath().c_str(), flow->getUUIDStr().c_str()); - fs.close(); + stream->closeStream(); input.close(); if (!keepSource) std::remove(source.c_str()); - std::string details = process_context_->getProcessorNode().getName() + " modify flow record content " + flow->getUUIDStr(); + std::stringstream details; + details << process_context_->getProcessorNode().getName() << " modify flow record content " << flow->getUUIDStr(); uint64_t endTime = getTimeMillis(); - provenance_report_->modifyContent(flow, details, endTime - startTime); + provenance_report_->modifyContent(flow, details.str(), endTime - startTime); } else { - fs.close(); + stream->closeStream(); input.close(); throw Exception(FILE_OPERATION_EXCEPTION, "File Import Error"); } @@ -627,7 +615,8 @@ bool keepSource, } void ProcessSession::import(std::string source, std::vector> flows, - bool keepSource, uint64_t offset, char inputDelimiter) { +bool keepSource, + uint64_t offset, char inputDelimiter) { std::shared_ptr claim; std::shared_ptr flowFile; @@ -643,45 +632,57 @@ void ProcessSession::import(std::string source, std::vector(create()); - claim = std::make_shared(); + claim = std::make_shared(process_context_->getContentRepository()); uint64_t startTime = getTimeMillis(); input.getline(buf, size, inputDelimiter); - std::ofstream fs; - fs.open(claim->getContentFullPath().c_str(), std::fstream::out | std::fstream::binary | std::fstream::trunc); - - if (fs.is_open()) { - if (input) - fs.write(buf, strlen(buf)); - else - fs.write(buf, input.gcount()); - - if (fs.good() && fs.tellp() >= 0) { - flowFile->setSize(fs.tellp()); - flowFile->setOffset(0); - if (flowFile->getResourceClaim() != nullptr) { - // Remove the old claim - flowFile->getResourceClaim()->decreaseFlowFileRecordOwnedCount(); - flowFile->clearResourceClaim(); - } - flowFile->setResourceClaim(claim); - claim->increaseFlowFileRecordOwnedCount(); - logger_->log_debug("Import offset %d length %d into content %s for FlowFile UUID %s", flowFile->getOffset(), - flowFile->getSize(), flowFile->getResourceClaim()->getContentFullPath().c_str(), - flowFile->getUUIDStr().c_str()); + std::shared_ptr stream = process_context_->getContentRepository()->write(claim); + if (nullptr == stream) { + rollback(); + return; + } - fs.close(); - std::string details = process_context_->getProcessorNode().getName() + " modify flow record content " + flowFile->getUUIDStr(); - uint64_t endTime = getTimeMillis(); - provenance_report_->modifyContent(flowFile, details, endTime - startTime); - flows.push_back(flowFile); + if (input) { + if (stream->write(reinterpret_cast(buf), size) < 0) { + invalidWrite = true; + break; + } + } else { + if (stream->write(reinterpret_cast(buf), input.gcount()) < 0) { + invalidWrite = true; + break; + } + } - } else { - fs.close(); - throw Exception(FILE_OPERATION_EXCEPTION, "File Export Error creating Flowfile"); + if (!invalidWrite) { + flowFile->setSize(stream->getSize()); + flowFile->setOffset(0); + if (flowFile->getResourceClaim() != nullptr) { + // Remove the old claim + flowFile->getResourceClaim()->decreaseFlowFileRecordOwnedCount(); + flowFile->clearResourceClaim(); } + flowFile->setResourceClaim(claim); + claim->increaseFlowFileRecordOwnedCount(); + + logger_->log_debug("Import offset %d length %d into content %s for FlowFile UUID %s", flowFile->getOffset(), + flowFile->getSize(), + flowFile->getResourceClaim()->getContentFullPath().c_str(), + flowFile->getUUIDStr().c_str()); + + stream->closeStream(); + std::string details = process_context_->getProcessorNode().getName() + " modify flow record content " + flowFile->getUUIDStr(); + uint64_t endTime = getTimeMillis(); + provenance_report_->modifyContent(flowFile, details, endTime - startTime); + flows.push_back(flowFile); + + } else { + stream->closeStream(); + throw Exception(FILE_OPERATION_EXCEPTION, "File Export Error creating Flowfile"); } + } input.close(); if (!keepSource) @@ -711,35 +712,45 @@ void ProcessSession::import(std::string source, std::vector &&flow, -bool keepSource, - uint64_t offset) { - std::shared_ptr claim = std::make_shared(); - +void ProcessSession::import(std::string source, std::shared_ptr &&flow, bool keepSource, uint64_t offset) { + std::shared_ptr claim = std::make_shared(process_context_->getContentRepository()); char *buf = NULL; int size = 4096; buf = new char[size]; try { - std::ofstream fs; + // std::ofstream fs; uint64_t startTime = getTimeMillis(); - fs.open(claim->getContentFullPath().c_str(), std::fstream::out | std::fstream::binary | std::fstream::trunc); std::ifstream input; input.open(source.c_str(), std::fstream::in | std::fstream::binary); - - if (fs.is_open() && input.is_open()) { + claim->increaseFlowFileRecordOwnedCount(); + std::shared_ptr stream = process_context_->getContentRepository()->write(claim); + if (nullptr == stream) { + rollback(); + return; + } + if (input.is_open()) { // Open the source file and stream to the flow file - input.seekg(offset, fs.beg); + input.seekg(offset); + int sizeWritten = 0; + bool invalidWrite = false; while (input.good()) { input.read(buf, size); - if (input) - fs.write(buf, size); - else - fs.write(buf, input.gcount()); + if (input) { + if (stream->write(reinterpret_cast(buf), size) < 0) { + invalidWrite = true; + break; + } + } else { + if (stream->write(reinterpret_cast(buf), input.gcount()) < 0) { + invalidWrite = true; + break; + } + } } + if (!invalidWrite) { - if (fs.good() && fs.tellp() >= 0) { - flow->setSize(fs.tellp()); + flow->setSize(stream->getSize()); flow->setOffset(0); if (flow->getResourceClaim() != nullptr) { // Remove the old claim @@ -747,20 +758,20 @@ bool keepSource, flow->clearResourceClaim(); } flow->setResourceClaim(claim); - claim->increaseFlowFileRecordOwnedCount(); logger_->log_debug("Import offset %d length %d into content %s for FlowFile UUID %s", flow->getOffset(), flow->getSize(), flow->getResourceClaim()->getContentFullPath().c_str(), flow->getUUIDStr().c_str()); - fs.close(); + stream->closeStream(); input.close(); if (!keepSource) std::remove(source.c_str()); - std::string details = process_context_->getProcessorNode().getName() + " modify flow record content " + flow->getUUIDStr(); + std::stringstream details; + details << process_context_->getProcessorNode().getName() << " modify flow record content " << flow->getUUIDStr(); uint64_t endTime = getTimeMillis(); - provenance_report_->modifyContent(flow, details, endTime - startTime); + provenance_report_->modifyContent(flow, details.str(), endTime - startTime); } else { - fs.close(); + stream->closeStream(); input.close(); throw Exception(FILE_OPERATION_EXCEPTION, "File Import Error"); } @@ -947,7 +958,7 @@ void ProcessSession::rollback() { _addedFlowFiles.clear(); _updatedFlowFiles.clear(); _deletedFlowFiles.clear(); - logger_->log_trace("ProcessSession rollback for %s", process_context_->getProcessorNode().getName().c_str()); + logger_->log_debug("ProcessSession rollback for %s", process_context_->getProcessorNode().getName().c_str()); } catch (std::exception &exception) { logger_->log_debug("Caught Exception %s", exception.what()); throw; @@ -960,8 +971,9 @@ void ProcessSession::rollback() { std::shared_ptr ProcessSession::get() { std::shared_ptr first = process_context_->getProcessorNode().getNextIncomingConnection(); - if (first == NULL) + if (first == NULL) { return NULL; + } std::shared_ptr current = std::static_pointer_cast(first); @@ -972,8 +984,9 @@ std::shared_ptr ProcessSession::get() { // Remove expired flow record for (std::set >::iterator it = expired.begin(); it != expired.end(); ++it) { std::shared_ptr record = *it; - std::string details = process_context_->getProcessorNode().getName() + " expire flow record " + record->getUUIDStr(); - provenance_report_->expire(record, details); + std::stringstream details; + details << process_context_->getProcessorNode().getName() << " expire flow record " << record->getUUIDStr(); + provenance_report_->expire(record, details.str()); } } if (ret) { @@ -981,10 +994,9 @@ std::shared_ptr ProcessSession::get() { ret->setDeleted(false); _updatedFlowFiles[ret->getUUIDStr()] = ret; std::map empty; - std::shared_ptr snapshot = std::make_shared(process_context_->getProvenanceRepository(), empty); + std::shared_ptr snapshot = std::make_shared(process_context_->getProvenanceRepository(), process_context_->getContentRepository(), empty); logger_->log_debug("Create Snapshot FlowFile with UUID %s", snapshot->getUUIDStr().c_str()); snapshot = ret; -// snapshot->duplicate(ret); // save a snapshot _originalFlowFiles[snapshot->getUUIDStr()] = snapshot; return ret; diff --git a/libminifi/src/core/ProcessSessionFactory.cpp b/libminifi/src/core/ProcessSessionFactory.cpp index 31b7481691..570d895fd8 100644 --- a/libminifi/src/core/ProcessSessionFactory.cpp +++ b/libminifi/src/core/ProcessSessionFactory.cpp @@ -28,7 +28,7 @@ namespace minifi { namespace core { std::unique_ptr ProcessSessionFactory::createSession() { - return std::unique_ptr(new ProcessSession(process_context_)); + return std::unique_ptr < ProcessSession > (new ProcessSession(process_context_)); } } /* namespace core */ diff --git a/libminifi/src/core/Processor.cpp b/libminifi/src/core/Processor.cpp index 7b076384ff..0c2e7cf65f 100644 --- a/libminifi/src/core/Processor.cpp +++ b/libminifi/src/core/Processor.cpp @@ -62,7 +62,7 @@ Processor::Processor(std::string name, uuid_t uuid) active_tasks_ = 0; yield_expiration_ = 0; incoming_connections_Iter = this->_incomingConnections.begin(); - logger_->log_info("Processor %s created UUID %s", name_.c_str(), uuidStr_.c_str()); + logger_->log_info("Processor %s created UUID %s", name_, uuidStr_); } bool Processor::isRunning() { @@ -80,8 +80,8 @@ bool Processor::addConnection(std::shared_ptr conn) { logger_->log_info("Can not add connection while the process %s is running", name_.c_str()); return false; } - std::shared_ptr connection = std::static_pointer_cast(conn); - std::lock_guard lock(mutex_); + std::shared_ptr connection = std::static_pointer_cast < Connection > (conn); + std::lock_guard < std::mutex > lock(mutex_); uuid_t srcUUID; uuid_t destUUID; @@ -141,12 +141,12 @@ void Processor::removeConnection(std::shared_ptr conn) { return; } - std::lock_guard lock(mutex_); + std::lock_guard < std::mutex > lock(mutex_); uuid_t srcUUID; uuid_t destUUID; - std::shared_ptr connection = std::static_pointer_cast(conn); + std::shared_ptr connection = std::static_pointer_cast < Connection > (conn); connection->getSourceUUID(srcUUID); connection->getDestinationUUID(destUUID); @@ -178,13 +178,13 @@ void Processor::removeConnection(std::shared_ptr conn) { } bool Processor::flowFilesQueued() { - std::lock_guard lock(mutex_); + std::lock_guard < std::mutex > lock(mutex_); if (_incomingConnections.size() == 0) return false; for (auto &&conn : _incomingConnections) { - std::shared_ptr connection = std::static_pointer_cast(conn); + std::shared_ptr connection = std::static_pointer_cast < Connection > (conn); if (connection->getQueueSize() > 0) return true; } @@ -193,13 +193,13 @@ bool Processor::flowFilesQueued() { } bool Processor::flowFilesOutGoingFull() { - std::lock_guard lock(mutex_); + std::lock_guard < std::mutex > lock(mutex_); for (auto &&connection : out_going_connections_) { // We already has connection for this relationship std::set> existedConnection = connection.second; for (const auto conn : existedConnection) { - std::shared_ptr connection = std::static_pointer_cast(conn); + std::shared_ptr < Connection > connection = std::static_pointer_cast < Connection > (conn); if (connection->isFull()) return true; } @@ -232,7 +232,7 @@ bool Processor::isWorkAvailable() { try { for (const auto &conn : _incomingConnections) { - std::shared_ptr connection = std::static_pointer_cast(conn); + std::shared_ptr connection = std::static_pointer_cast < Connection > (conn); if (connection->getQueueSize() > 0) { hasWork = true; break; diff --git a/libminifi/src/core/Repository.cpp b/libminifi/src/core/Repository.cpp index 50e8cd289f..33dfd53b3e 100644 --- a/libminifi/src/core/Repository.cpp +++ b/libminifi/src/core/Repository.cpp @@ -19,13 +19,14 @@ #include #include #include + +#include "../../include/core/repository/FlowFileRepository.h" #include "io/DataStream.h" #include "io/Serializable.h" #include "core/Relationship.h" #include "core/logging/Logger.h" #include "FlowController.h" #include "provenance/Provenance.h" -#include "core/repository/FlowFileRepository.h" namespace org { namespace apache { diff --git a/libminifi/src/core/RepositoryFactory.cpp b/libminifi/src/core/RepositoryFactory.cpp index cf18601397..9e997186fb 100644 --- a/libminifi/src/core/RepositoryFactory.cpp +++ b/libminifi/src/core/RepositoryFactory.cpp @@ -18,13 +18,17 @@ #include #include #include +#include "core/ContentRepository.h" +#include "core/repository/FileSystemRepository.h" +#include "core/repository/VolatileContentRepository.h" #include "core/Repository.h" #ifdef LEVELDB_SUPPORT #include "core/repository/FlowFileRepository.h" #include "provenance/ProvenanceRepository.h" #endif -#include "core/repository/VolatileRepository.h" +#include "core/repository/VolatileProvenanceRepository.h" +#include "core/repository/VolatileFlowFileRepository.h" namespace org { namespace apache { @@ -48,14 +52,14 @@ std::shared_ptr createRepository(const std::string configurati try { std::shared_ptr return_obj = nullptr; if (class_name_lc == "flowfilerepository") { - std::cout << "creating flow" << std::endl; return_obj = instantiate(repo_name); } else if (class_name_lc == "provenancerepository") { return_obj = instantiate(repo_name); - } else if (class_name_lc == "volatilerepository") { - return_obj = instantiate(repo_name); + } else if (class_name_lc == "volatileflowfilerepository") { + return_obj = instantiate(repo_name); + } else if (class_name_lc == "volatileprovenancefilerepository") { + return_obj = instantiate(repo_name); } else if (class_name_lc == "nooprepository") { - std::cout << "creating noop" << std::endl; return_obj = instantiate(repo_name); } @@ -63,13 +67,42 @@ std::shared_ptr createRepository(const std::string configurati return return_obj; } if (fail_safe) { - return std::make_shared("fail_safe", "fail_safe", 1, 1, 1); + return std::make_shared < core::Repository > ("fail_safe", "fail_safe", 1, 1, 1); } else { throw std::runtime_error("Support for the provided configuration class could not be found"); } } catch (const std::runtime_error &r) { if (fail_safe) { - return std::make_shared("fail_safe", "fail_safe", 1, 1, 1); + return std::make_shared < core::Repository > ("fail_safe", "fail_safe", 1, 1, 1); + } + } + + throw std::runtime_error("Support for the provided configuration class could not be found"); +} + +std::shared_ptr createContentRepository(const std::string configuration_class_name, bool fail_safe, const std::string repo_name) { + std::shared_ptr return_obj = nullptr; + std::string class_name_lc = configuration_class_name; + std::transform(class_name_lc.begin(), class_name_lc.end(), class_name_lc.begin(), ::tolower); + try { + std::shared_ptr return_obj = nullptr; + if (class_name_lc == "volatilecontentrepository") { + return_obj = instantiate(repo_name); + } else { + return_obj = instantiate(repo_name); + } + + if (return_obj) { + return return_obj; + } + if (fail_safe) { + return std::make_shared < core::repository::FileSystemRepository > ("fail_safe"); + } else { + throw std::runtime_error("Support for the provided configuration class could not be found"); + } + } catch (const std::runtime_error &r) { + if (fail_safe) { + return std::make_shared < core::repository::FileSystemRepository > ("fail_safe"); } } diff --git a/libminifi/src/core/controller/StandardControllerServiceNode.cpp b/libminifi/src/core/controller/StandardControllerServiceNode.cpp index 5c4aa707ec..69004c1f23 100644 --- a/libminifi/src/core/controller/StandardControllerServiceNode.cpp +++ b/libminifi/src/core/controller/StandardControllerServiceNode.cpp @@ -27,12 +27,12 @@ namespace minifi { namespace core { namespace controller { std::shared_ptr &StandardControllerServiceNode::getProcessGroup() { - std::lock_guard lock(mutex_); + std::lock_guard < std::mutex > lock(mutex_); return process_group_; } void StandardControllerServiceNode::setProcessGroup(std::shared_ptr &processGroup) { - std::lock_guard lock(mutex_); + std::lock_guard < std::mutex > lock(mutex_); process_group_ = processGroup; } @@ -45,7 +45,7 @@ bool StandardControllerServiceNode::enable() { for (auto linked_service : property.getValues()) { std::shared_ptr csNode = provider->getControllerServiceNode(linked_service); if (nullptr != csNode) { - std::lock_guard lock(mutex_); + std::lock_guard < std::mutex > lock(mutex_); linked_controller_services_.push_back(csNode); } } diff --git a/libminifi/src/core/logging/LoggerConfiguration.cpp b/libminifi/src/core/logging/LoggerConfiguration.cpp index c06239be7e..4b97055092 100644 --- a/libminifi/src/core/logging/LoggerConfiguration.cpp +++ b/libminifi/src/core/logging/LoggerConfiguration.cpp @@ -56,19 +56,19 @@ std::vector LoggerProperties::get_keys_of_type(const std::string &t LoggerConfiguration::LoggerConfiguration() : root_namespace_(create_default_root()), loggers(std::vector>()), - formatter_(std::make_shared(spdlog_default_pattern)) { - logger_ = std::shared_ptr(new LoggerImpl(core::getClassName(), get_logger(nullptr, root_namespace_, core::getClassName(), formatter_))); + formatter_(std::make_shared < spdlog::pattern_formatter > (spdlog_default_pattern)) { + logger_ = std::shared_ptr < LoggerImpl > (new LoggerImpl(core::getClassName(), get_logger(nullptr, root_namespace_, core::getClassName(), formatter_))); loggers.push_back(logger_); } void LoggerConfiguration::initialize(const std::shared_ptr &logger_properties) { - std::lock_guard lock(mutex); + std::lock_guard < std::mutex > lock(mutex); root_namespace_ = initialize_namespaces(logger_properties); std::string spdlog_pattern; if (!logger_properties->get("spdlog.pattern", spdlog_pattern)) { spdlog_pattern = spdlog_default_pattern; } - formatter_ = std::make_shared(spdlog_pattern); + formatter_ = std::make_shared < spdlog::pattern_formatter > (spdlog_pattern); std::map> spdloggers; for (auto const & logger_impl : loggers) { std::shared_ptr spdlogger; @@ -85,8 +85,8 @@ void LoggerConfiguration::initialize(const std::shared_ptr &lo } std::shared_ptr LoggerConfiguration::getLogger(const std::string &name) { - std::lock_guard lock(mutex); - std::shared_ptr result = std::make_shared(name, get_logger(logger_, root_namespace_, name, formatter_)); + std::lock_guard < std::mutex > lock(mutex); + std::shared_ptr result = std::make_shared < LoggerImpl > (name, get_logger(logger_, root_namespace_, name, formatter_)); loggers.push_back(result); return result; } @@ -130,7 +130,7 @@ std::shared_ptr LoggerConfiguration::initialize_names } catch (const std::out_of_range &oor) { } } - sink_map[appender_name] = std::make_shared(file_name, max_file_size, max_files); + sink_map[appender_name] = std::make_shared < spdlog::sinks::rotating_file_sink_mt > (file_name, max_file_size, max_files); } else if ("stdout" == appender_type) { sink_map[appender_name] = spdlog::sinks::stdout_sink_mt::instance(); } else { @@ -227,7 +227,7 @@ std::shared_ptr LoggerConfiguration::get_logger(std::shared_ptr< if (logger != nullptr) { logger->log_debug("%s logger got sinks from namespace %s and level %s from namespace %s", name, sink_namespace_str, spdlog::level::level_names[level], level_namespace_str); } - spdlogger = std::make_shared(name, begin(sinks), end(sinks)); + spdlogger = std::make_shared < spdlog::logger > (name, begin(sinks), end(sinks)); spdlogger->set_level(level); spdlogger->set_formatter(formatter); spdlogger->flush_on(std::max(spdlog::level::info, current_namespace->level)); diff --git a/libminifi/src/core/reporting/SiteToSiteProvenanceReportingTask.cpp b/libminifi/src/core/reporting/SiteToSiteProvenanceReportingTask.cpp index 02ddb52a60..d4059d677c 100644 --- a/libminifi/src/core/reporting/SiteToSiteProvenanceReportingTask.cpp +++ b/libminifi/src/core/reporting/SiteToSiteProvenanceReportingTask.cpp @@ -24,8 +24,10 @@ #include #include #include +#include #include #include +#include "core/Repository.h" #include "core/reporting/SiteToSiteProvenanceReportingTask.h" #include "../include/io/StreamFactory.h" #include "io/ClientSocket.h" @@ -51,10 +53,14 @@ void SiteToSiteProvenanceReportingTask::initialize() { RemoteProcessorGroupPort::initialize(); } -void SiteToSiteProvenanceReportingTask::getJsonReport(core::ProcessContext *context, core::ProcessSession *session, std::vector> &records, +void SiteToSiteProvenanceReportingTask::getJsonReport(core::ProcessContext *context, core::ProcessSession *session, std::vector> &records, std::string &report) { Json::Value array; - for (auto record : records) { + for (auto sercomp : records) { + std::shared_ptr record = std::dynamic_pointer_cast < provenance::ProvenanceEventRecord > (sercomp); + if (nullptr == record) { + break; + } Json::Value recordJson; Json::Value updatedAttributesJson; Json::Value parentUuidJson; @@ -108,23 +114,32 @@ void SiteToSiteProvenanceReportingTask::onTrigger(core::ProcessContext *context, return; } + logger_->log_debug("SiteToSiteProvenanceReportingTask -- onTrigger"); + if (!protocol_->bootstrap()) { // bootstrap the client protocol if needeed context->yield(); - std::shared_ptr processor = std::static_pointer_cast(context->getProcessorNode().getProcessor()); + std::shared_ptr processor = std::static_pointer_cast < Processor > (context->getProcessorNode().getProcessor()); logger_->log_error("Site2Site bootstrap failed yield period %d peer ", processor->getYieldPeriodMsec()); returnProtocol(std::move(protocol_)); return; } - std::vector> records; - std::shared_ptr repo = std::static_pointer_cast(context->getProvenanceRepository()); - repo->getProvenanceRecord(records, batch_size_); - if (records.size() <= 0) { + std::vector> records; + + logger_->log_debug("batch size %d records", batch_size_); + size_t deserialized = batch_size_; + std::shared_ptr repo = context->getProvenanceRepository(); + std::function < std::shared_ptr() > constructor = []() {return std::make_shared();}; + if (!repo->DeSerialize(records, deserialized, constructor) && deserialized == 0) { + logger_->log_debug("Not sending because deserialized is %d", deserialized); returnProtocol(std::move(protocol_)); return; } + logger_->log_debug("batch size %d records", batch_size_, deserialized); + + logger_->log_debug("Captured %d records", deserialized); std::string jsonStr; this->getJsonReport(context, session, records, jsonStr); if (jsonStr.length() <= 0) { @@ -141,7 +156,7 @@ void SiteToSiteProvenanceReportingTask::onTrigger(core::ProcessContext *context, } // we transfer the record, purge the record from DB - repo->purgeProvenanceRecord(records); + repo->Delete(records); returnProtocol(std::move(protocol_)); } diff --git a/libminifi/src/core/repository/FileSystemRepository.cpp b/libminifi/src/core/repository/FileSystemRepository.cpp new file mode 100644 index 0000000000..67fab3e8a2 --- /dev/null +++ b/libminifi/src/core/repository/FileSystemRepository.cpp @@ -0,0 +1,49 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "core/repository/FileSystemRepository.h" +#include +#include "io/FileStream.h" + +namespace org { +namespace apache { +namespace nifi { +namespace minifi { +namespace core { +namespace repository { + +bool FileSystemRepository::initialize(const std::shared_ptr &configuration) { + return true; +} +void FileSystemRepository::stop() { +} + +std::shared_ptr FileSystemRepository::write(const std::shared_ptr &claim) { + return std::make_shared < io::FileStream > (claim->getContentFullPath()); +} + +std::shared_ptr FileSystemRepository::read(const std::shared_ptr &claim) { + return std::make_shared < io::FileStream > (claim->getContentFullPath()); +} + +} /* namespace repository */ +} /* namespace core */ +} /* namespace minifi */ +} /* namespace nifi */ +} /* namespace apache */ +} /* namespace org */ diff --git a/libminifi/src/core/repository/FlowFileRepository.cpp b/libminifi/src/core/repository/FlowFileRepository.cpp index e6d561aadc..dd40ca0c17 100644 --- a/libminifi/src/core/repository/FlowFileRepository.cpp +++ b/libminifi/src/core/repository/FlowFileRepository.cpp @@ -15,10 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include "core/repository/FlowFileRepository.h" +#include "../../../include/core/repository/FlowFileRepository.h" + #include #include #include + #include "FlowFileRecord.h" namespace org { @@ -40,19 +42,19 @@ void FlowFileRepository::run() { leveldb::Iterator* it = db_->NewIterator(leveldb::ReadOptions()); for (it->SeekToFirst(); it->Valid(); it->Next()) { - std::shared_ptr eventRead = std::make_shared(shared_from_this()); + std::shared_ptr eventRead = std::make_shared < FlowFileRecord > (shared_from_this(), content_repo_); std::string key = it->key().ToString(); if (eventRead->DeSerialize(reinterpret_cast(it->value().data()), it->value().size())) { if ((curTime - eventRead->getEventTime()) > max_partition_millis_) purgeList.push_back(key); } else { - logger_->log_debug("NiFi %s retrieve event %s fail", name_.c_str(), key.c_str()); + logger_->log_debug("NiFi %s retrieve event %s fail", name_, key); purgeList.push_back(key); } } delete it; for (auto eventId : purgeList) { - logger_->log_info("Repository Repo %s Purge %s", name_.c_str(), eventId.c_str()); + logger_->log_info("Repository Repo %s Purge %s", name_, eventId); Delete(eventId); } } @@ -64,25 +66,26 @@ void FlowFileRepository::run() { return; } -void FlowFileRepository::loadComponent() { +void FlowFileRepository::loadComponent(const std::shared_ptr &content_repo) { + content_repo_ = content_repo; std::vector purgeList; leveldb::Iterator* it = db_->NewIterator(leveldb::ReadOptions()); for (it->SeekToFirst(); it->Valid(); it->Next()) { - std::shared_ptr eventRead = std::make_shared(shared_from_this()); + std::shared_ptr eventRead = std::make_shared < FlowFileRecord > (shared_from_this(), content_repo_); std::string key = it->key().ToString(); if (eventRead->DeSerialize(reinterpret_cast(it->value().data()), it->value().size())) { auto search = connectionMap.find(eventRead->getConnectionUuid()); if (search != connectionMap.end()) { // we find the connection for the persistent flowfile, create the flowfile and enqueue that - std::shared_ptr flow_file_ref = std::static_pointer_cast(eventRead); - std::shared_ptr record = std::make_shared(shared_from_this(), flow_file_ref); + std::shared_ptr flow_file_ref = std::static_pointer_cast < core::FlowFile > (eventRead); + std::shared_ptr record = std::make_shared < FlowFileRecord > (shared_from_this(), content_repo_, flow_file_ref); // set store to repo to true so that we do need to persistent again in enqueue record->setStoredToRepository(true); search->second->put(record); } else { - if (eventRead->getContentFullPath().length() > 0) { - std::remove(eventRead->getContentFullPath().c_str()); + if (eventRead->getContentFullPath().length() > 0 && content_repo != nullptr) { + content_repo->remove(eventRead->getResourceClaim()); } purgeList.push_back(key); } diff --git a/libminifi/src/core/repository/VolatileContentRepository.cpp b/libminifi/src/core/repository/VolatileContentRepository.cpp new file mode 100644 index 0000000000..01f88d4105 --- /dev/null +++ b/libminifi/src/core/repository/VolatileContentRepository.cpp @@ -0,0 +1,129 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "core/repository/VolatileContentRepository.h" +#include +#include +#include "io/FileStream.h" + +namespace org { +namespace apache { +namespace nifi { +namespace minifi { +namespace core { +namespace repository { + +bool VolatileContentRepository::initialize(const std::shared_ptr &configure) { + VolatileRepository::initialize(configure); + resource_claim_comparator_ = [](std::shared_ptr lhsPtr, std::shared_ptr rhsPtr) { + return lhsPtr->getContentFullPath() == rhsPtr->getContentFullPath();}; + resource_claim_check_ = [](std::shared_ptr claim) { + return claim->getFlowFileRecordOwnedCount() <= 0;}; + claim_reclaimer_ = [&](std::shared_ptr claim) {if (claim->getFlowFileRecordOwnedCount() <= 0) { + remove(claim); + } + }; + start(); + + return true; +} + +void VolatileContentRepository::stop() { + running_ = false; +} + +void VolatileContentRepository::run() { +} + +void VolatileContentRepository::start() { + if (this->purge_period_ <= 0) + return; + if (running_) + return; + thread_ = std::thread(&VolatileContentRepository::run, shared_from_parent()); + thread_.detach(); + running_ = true; + logger_->log_info("%s Repository Monitor Thread Start", name_); +} + +std::shared_ptr VolatileContentRepository::write(const std::shared_ptr &claim) { + logger_->log_debug("enter write"); + { + std::lock_guard < std::mutex > lock(map_mutex_); + auto claim_check = master_list_.find(claim->getContentFullPath()); + if (claim_check != master_list_.end()) { + return std::make_shared < io::AtomicEntryStream>>(claim, claim_check->second); + } + } + + int size = 0; + for (auto ent : value_vector_) { + if (ent->testAndSetKey(claim, nullptr, nullptr, resource_claim_comparator_)) { + std::lock_guard < std::mutex > lock(map_mutex_); + master_list_[claim->getContentFullPath()] = ent; + return std::make_shared < io::AtomicEntryStream>>(claim, ent); + } + size++; + } + return nullptr; +} + +std::shared_ptr VolatileContentRepository::read(const std::shared_ptr &claim) { + logger_->log_debug("enter read"); + int size = 0; + { + std::lock_guard < std::mutex > lock(map_mutex_); + auto claim_check = master_list_.find(claim->getContentFullPath()); + if (claim_check != master_list_.end()) { + return std::make_shared < io::AtomicEntryStream>>(claim, claim_check->second); + } + } + + for (auto ent : value_vector_) { + RepoValue> *repo_value; + + if (ent->getValue(claim, &repo_value)) { + return std::make_shared < io::AtomicEntryStream>>(claim, ent); + } + size++; + } + logger_->log_debug("enter read for %s after %d", claim->getContentFullPath(), size); + return nullptr; +} + +bool VolatileContentRepository::remove(const std::shared_ptr &claim) { + logger_->log_debug("enter remove"); + + for (auto ent : value_vector_) { + // let the destructor do the cleanup + if (ent->freeValue(claim)) { + std::lock_guard < std::mutex > lock(map_mutex_); + master_list_.erase(claim->getContentFullPath()); + logger_->log_debug("removed %s", claim->getContentFullPath()); + return true; + } + } + return false; +} + +} /* namespace repository */ +} /* namespace core */ +} /* namespace minifi */ +} /* namespace nifi */ +} /* namespace apache */ +} /* namespace org */ diff --git a/libminifi/src/core/repository/VolatileRepository.cpp b/libminifi/src/core/repository/VolatileRepository.cpp index a7e3a51bf1..6e3e1c6988 100644 --- a/libminifi/src/core/repository/VolatileRepository.cpp +++ b/libminifi/src/core/repository/VolatileRepository.cpp @@ -16,7 +16,9 @@ * limitations under the License. */ #include "core/repository/VolatileRepository.h" +#include #include +#include #include #include #include "FlowFileRecord.h" @@ -28,33 +30,6 @@ namespace minifi { namespace core { namespace repository { -const char *VolatileRepository::volatile_repo_max_count = "max.count"; - -void VolatileRepository::run() { - repo_full_ = false; -} - -/** - * Purge - */ -void VolatileRepository::purge() { - while (current_size_ > max_size_) { - for (auto ent : value_vector_) { - // let the destructor do the cleanup - RepoValue value; - if (ent->getValue(value)) { - current_size_ -= value.size(); - logger_->log_info("VolatileRepository -- purge %s %d %d %d", value.getKey(), current_size_.load(), max_size_, current_index_.load()); - } - if (current_size_ < max_size_) - break; - } - } -} - -void VolatileRepository::loadComponent() { -} - } /* namespace repository */ } /* namespace core */ } /* namespace minifi */ diff --git a/libminifi/src/core/yaml/YamlConfiguration.cpp b/libminifi/src/core/yaml/YamlConfiguration.cpp index 4ce944eab6..d205f7f0fa 100644 --- a/libminifi/src/core/yaml/YamlConfiguration.cpp +++ b/libminifi/src/core/yaml/YamlConfiguration.cpp @@ -35,7 +35,8 @@ core::ProcessGroup *YamlConfiguration::parseRootProcessGroupYaml(YAML::Node root uuid_t uuid; int64_t version = 0; - checkRequiredField(&rootFlowNode, "name", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY); + checkRequiredField(&rootFlowNode, "name", + CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY); std::string flowName = rootFlowNode["name"].as(); std::string id = getOrGenerateId(&rootFlowNode); uuid_parse(id.c_str(), uuid); @@ -47,10 +48,8 @@ core::ProcessGroup *YamlConfiguration::parseRootProcessGroupYaml(YAML::Node root } } - logger_->log_debug( - "parseRootProcessGroup: id => [%s], name => [%s]", id, flowName); - std::unique_ptr group = - FlowConfiguration::createRootProcessGroup(flowName, uuid, version); + logger_->log_debug("parseRootProcessGroup: id => [%s], name => [%s]", id, flowName); + std::unique_ptr group = FlowConfiguration::createRootProcessGroup(flowName, uuid, version); this->name_ = flowName; @@ -77,7 +76,8 @@ void YamlConfiguration::parseProcessorNodeYaml(YAML::Node processorsNode, core:: core::ProcessorConfig procCfg; YAML::Node procNode = iter->as(); - checkRequiredField(&procNode, "name", CONFIG_YAML_PROCESSORS_KEY); + checkRequiredField(&procNode, "name", + CONFIG_YAML_PROCESSORS_KEY); procCfg.name = procNode["name"].as(); procCfg.id = getOrGenerateId(&procNode); uuid_parse(procCfg.id.c_str(), uuid); @@ -101,11 +101,13 @@ void YamlConfiguration::parseProcessorNodeYaml(YAML::Node processorsNode, core:: } processor->setName(procCfg.name); - checkRequiredField(&procNode, "scheduling strategy", CONFIG_YAML_PROCESSORS_KEY); + checkRequiredField(&procNode, "scheduling strategy", + CONFIG_YAML_PROCESSORS_KEY); procCfg.schedulingStrategy = procNode["scheduling strategy"].as(); logger_->log_debug("parseProcessorNode: scheduling strategy => [%s]", procCfg.schedulingStrategy); - checkRequiredField(&procNode, "scheduling period", CONFIG_YAML_PROCESSORS_KEY); + checkRequiredField(&procNode, "scheduling period", + CONFIG_YAML_PROCESSORS_KEY); procCfg.schedulingPeriod = procNode["scheduling period"].as(); logger_->log_debug("parseProcessorNode: scheduling period => [%s]", procCfg.schedulingPeriod); @@ -224,13 +226,15 @@ void YamlConfiguration::parseRemoteProcessGroupYaml(YAML::Node *rpgNode, core::P for (YAML::const_iterator iter = rpgNode->begin(); iter != rpgNode->end(); ++iter) { YAML::Node currRpgNode = iter->as(); - checkRequiredField(&currRpgNode, "name", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY); + checkRequiredField(&currRpgNode, "name", + CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY); auto name = currRpgNode["name"].as(); id = getOrGenerateId(&currRpgNode); logger_->log_debug("parseRemoteProcessGroupYaml: name => [%s], id => [%s]", name, id); - checkRequiredField(&currRpgNode, "url", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY); + checkRequiredField(&currRpgNode, "url", + CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY); std::string url = currRpgNode["url"].as(); logger_->log_debug("parseRemoteProcessGroupYaml: url => [%s]", url); @@ -266,7 +270,8 @@ void YamlConfiguration::parseRemoteProcessGroupYaml(YAML::Node *rpgNode, core::P group->setTransmitting(true); group->setURL(url); - checkRequiredField(&currRpgNode, "Input Ports", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY); + checkRequiredField(&currRpgNode, "Input Ports", + CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY); YAML::Node inputPorts = currRpgNode["Input Ports"].as(); if (inputPorts && inputPorts.IsSequence()) { for (YAML::const_iterator portIter = inputPorts.begin(); portIter != inputPorts.end(); ++portIter) { @@ -308,13 +313,15 @@ void YamlConfiguration::parseProvenanceReportingYaml(YAML::Node *reportNode, cor std::shared_ptr processor = nullptr; processor = createProvenanceReportTask(); - std::shared_ptr reportTask = std::static_pointer_cast(processor); + std::shared_ptr reportTask = std::static_pointer_cast < core::reporting::SiteToSiteProvenanceReportingTask > (processor); YAML::Node node = reportNode->as(); - checkRequiredField(&node, "scheduling strategy", CONFIG_YAML_PROVENANCE_REPORT_KEY); + checkRequiredField(&node, "scheduling strategy", + CONFIG_YAML_PROVENANCE_REPORT_KEY); auto schedulingStrategyStr = node["scheduling strategy"].as(); - checkRequiredField(&node, "scheduling period", CONFIG_YAML_PROVENANCE_REPORT_KEY); + checkRequiredField(&node, "scheduling period", + CONFIG_YAML_PROVENANCE_REPORT_KEY); auto schedulingPeriodStr = node["scheduling period"].as(); core::TimeUnit unit; @@ -394,9 +401,9 @@ void YamlConfiguration::parseControllerServices(YAML::Node *controllerServicesNo controller_service_node->initialize(); YAML::Node propertiesNode = controllerServiceNode["Properties"]; // we should propogate propertiets to the node and to the implementation - parsePropertiesNodeYaml(&propertiesNode, std::static_pointer_cast(controller_service_node)); + parsePropertiesNodeYaml(&propertiesNode, std::static_pointer_cast < core::ConfigurableComponent > (controller_service_node)); if (controller_service_node->getControllerServiceImplementation() != nullptr) { - parsePropertiesNodeYaml(&propertiesNode, std::static_pointer_cast(controller_service_node->getControllerServiceImplementation())); + parsePropertiesNodeYaml(&propertiesNode, std::static_pointer_cast < core::ConfigurableComponent > (controller_service_node->getControllerServiceImplementation())); } } controller_services_->put(id, controller_service_node); @@ -423,7 +430,8 @@ void YamlConfiguration::parseConnectionYaml(YAML::Node *connectionsNode, core::P // Configure basic connection uuid_t uuid; - checkRequiredField(&connectionNode, "name", CONFIG_YAML_CONNECTIONS_KEY); + checkRequiredField(&connectionNode, "name", + CONFIG_YAML_CONNECTIONS_KEY); std::string name = connectionNode["name"].as(); std::string id = getOrGenerateId(&connectionNode); uuid_parse(id.c_str(), uuid); @@ -431,7 +439,8 @@ void YamlConfiguration::parseConnectionYaml(YAML::Node *connectionsNode, core::P logger_->log_debug("Created connection with UUID %s and name %s", id, name); // Configure connection source - checkRequiredField(&connectionNode, "source relationship name", CONFIG_YAML_CONNECTIONS_KEY); + checkRequiredField(&connectionNode, "source relationship name", + CONFIG_YAML_CONNECTIONS_KEY); auto rawRelationship = connectionNode["source relationship name"].as(); core::Relationship relationship(rawRelationship, ""); logger_->log_debug("parseConnection: relationship => [%s]", rawRelationship); @@ -449,7 +458,8 @@ void YamlConfiguration::parseConnectionYaml(YAML::Node *connectionsNode, core::P name, connectionSrcProcId); } else { // if we don't have a source id, try to resolve using source name. config schema v2 will make this unnecessary - checkRequiredField(&connectionNode, "source name", CONFIG_YAML_CONNECTIONS_KEY); + checkRequiredField(&connectionNode, "source name", + CONFIG_YAML_CONNECTIONS_KEY); std::string connectionSrcProcName = connectionNode["source name"].as(); uuid_t tmpUUID; if (!uuid_parse(connectionSrcProcName.c_str(), tmpUUID) && NULL != parent->findProcessor(tmpUUID)) { @@ -486,7 +496,8 @@ void YamlConfiguration::parseConnectionYaml(YAML::Node *connectionsNode, core::P } else { // we use the same logic as above for resolving the source processor // for looking up the destination processor in absence of a processor id - checkRequiredField(&connectionNode, "destination name", CONFIG_YAML_CONNECTIONS_KEY); + checkRequiredField(&connectionNode, "destination name", + CONFIG_YAML_CONNECTIONS_KEY); std::string connectionDestProcName = connectionNode["destination name"].as(); uuid_t tmpUUID; if (!uuid_parse(connectionDestProcName.c_str(), tmpUUID) && @@ -534,7 +545,8 @@ void YamlConfiguration::parsePortYaml(YAML::Node *portNode, core::ProcessGroup * YAML::Node inputPortsObj = portNode->as(); // Check for required fields - checkRequiredField(&inputPortsObj, "name", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY); + checkRequiredField(&inputPortsObj, "name", + CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY); auto nameStr = inputPortsObj["name"].as(); checkRequiredField(&inputPortsObj, "id", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY, @@ -548,7 +560,7 @@ void YamlConfiguration::parsePortYaml(YAML::Node *portNode, core::ProcessGroup * port = std::make_shared(stream_factory_, nameStr, parent->getURL(), this->configuration_, uuid); - processor = std::static_pointer_cast(port); + processor = std::static_pointer_cast < core::Processor > (port); port->setDirection(direction); port->setTimeOut(parent->getTimeOut()); port->setTransmitting(true); @@ -558,7 +570,7 @@ void YamlConfiguration::parsePortYaml(YAML::Node *portNode, core::ProcessGroup * // handle port properties YAML::Node nodeVal = portNode->as(); YAML::Node propertiesNode = nodeVal["Properties"]; - parsePropertiesNodeYaml(&propertiesNode, std::static_pointer_cast(processor)); + parsePropertiesNodeYaml(&propertiesNode, std::static_pointer_cast < core::ConfigurableComponent > (processor)); // add processor to parent parent->addProcessor(processor); @@ -590,7 +602,7 @@ void YamlConfiguration::parsePropertiesNodeYaml(YAML::Node *propertiesNode, std: std::string rawValueString = propertiesNode.as(); logger_->log_info("Found %s=%s", propertyName, rawValueString); if (!processor->updateProperty(propertyName, rawValueString)) { - std::shared_ptr proc = std::dynamic_pointer_cast(processor); + std::shared_ptr proc = std::dynamic_pointer_cast < core::Connectable > (processor); if (proc != 0) { logger_->log_warn("Received property %s with value %s but is not one of the properties for %s", propertyName, rawValueString, proc->getName()); } @@ -600,7 +612,7 @@ void YamlConfiguration::parsePropertiesNodeYaml(YAML::Node *propertiesNode, std: } else { std::string rawValueString = propertyValueNode.as(); if (!processor->setProperty(propertyName, rawValueString)) { - std::shared_ptr proc = std::dynamic_pointer_cast(processor); + std::shared_ptr proc = std::dynamic_pointer_cast < core::Connectable > (processor); if (proc != 0) { logger_->log_warn("Received property %s with value %s but is not one of the properties for %s", propertyName, rawValueString, proc->getName()); } diff --git a/libminifi/src/io/AtomicEntryStream.cpp b/libminifi/src/io/AtomicEntryStream.cpp new file mode 100644 index 0000000000..aac9723d33 --- /dev/null +++ b/libminifi/src/io/AtomicEntryStream.cpp @@ -0,0 +1,34 @@ +/** + * + * 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 "io/AtomicEntryStream.h" +#include +#include +#include +#include "io/Serializable.h" + +namespace org { +namespace apache { +namespace nifi { +namespace minifi { +namespace io { + +} /* namespace io */ +} /* namespace minifi */ +} /* namespace nifi */ +} /* namespace apache */ +} /* namespace org */ diff --git a/libminifi/src/io/ClientSocket.cpp b/libminifi/src/io/ClientSocket.cpp index 57d6f03308..5f7f5f4e32 100644 --- a/libminifi/src/io/ClientSocket.cpp +++ b/libminifi/src/io/ClientSocket.cpp @@ -170,6 +170,9 @@ int16_t Socket::initialize() { int hh_errno; gethostbyname_r(host, &he, buf, sizeof(buf), &h, &hh_errno); #endif + if (h == nullptr) { + return -1; + } memcpy(reinterpret_cast(&addr), h->h_addr_list[0], h->h_length); auto p = addr_info_; @@ -197,7 +200,7 @@ int16_t Socket::select_descriptor(const uint16_t msec) { tv.tv_sec = msec / 1000; tv.tv_usec = (msec % 1000) * 1000; - std::lock_guard guard(selection_mutex_); + std::lock_guard < std::recursive_mutex > guard(selection_mutex_); if (msec > 0) retval = select(socket_max_ + 1, &read_fds_, NULL, NULL, &tv); @@ -241,14 +244,12 @@ int16_t Socket::setSocketOptions(const int sock) { bool nagle_off = true; #ifndef __MACH__ if (nagle_off) { - if (setsockopt(sock, SOL_TCP, TCP_NODELAY, static_cast(&opt), sizeof(opt)) - < 0) { + if (setsockopt(sock, SOL_TCP, TCP_NODELAY, static_cast(&opt), sizeof(opt)) < 0) { logger_->log_error("setsockopt() TCP_NODELAY failed"); close(sock); return -1; } - if (setsockopt(sock, SOL_SOCKET, SO_REUSEADDR, reinterpret_cast(&opt), - sizeof(opt)) < 0) { + if (setsockopt(sock, SOL_SOCKET, SO_REUSEADDR, reinterpret_cast(&opt), sizeof(opt)) < 0) { logger_->log_error("setsockopt() SO_REUSEADDR failed"); close(sock); return -1; @@ -256,8 +257,7 @@ int16_t Socket::setSocketOptions(const int sock) { } int sndsize = 256 * 1024; - if (setsockopt(sock, SOL_SOCKET, SO_SNDBUF, reinterpret_cast( &sndsize), - sizeof(sndsize)) < 0) { + if (setsockopt(sock, SOL_SOCKET, SO_SNDBUF, reinterpret_cast(&sndsize), sizeof(sndsize)) < 0) { logger_->log_error("setsockopt() SO_SNDBUF failed"); close(sock); return -1; diff --git a/libminifi/src/io/FileStream.cpp b/libminifi/src/io/FileStream.cpp new file mode 100644 index 0000000000..d22b06aac3 --- /dev/null +++ b/libminifi/src/io/FileStream.cpp @@ -0,0 +1,124 @@ +/** + * + * 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 "io/FileStream.h" +#include +#include +#include +#include +#include "io/validation.h" +namespace org { +namespace apache { +namespace nifi { +namespace minifi { +namespace io { + +FileStream::FileStream(const std::string &path, uint32_t offset) + : logger_(logging::LoggerFactory::getLogger()) { + file_stream_ = std::unique_ptr < std::fstream > (new std::fstream()); + file_stream_->open(path.c_str(), std::ios::in | std::ios::out | std::ios::binary); + file_stream_->seekg(0, file_stream_->end); + file_stream_->seekp(0, file_stream_->end); + int len = file_stream_->tellg(); + if (len > 0) { + length_ = len; + } else { + length_ = 0; + } + std::cout << "length is " << length_ << std::endl; + seek(offset); +} + +void FileStream::seek(uint64_t offset) { + std::lock_guard < std::recursive_mutex > lock(file_lock_); + offset_ = offset; + file_stream_->clear(); + file_stream_->seekg(offset_); + file_stream_->seekp(offset_); +} + +int FileStream::writeData(std::vector &buf, int buflen) { + if (buf.capacity() < buflen) + return -1; + return writeData(reinterpret_cast(&buf[0]), buflen); +} + +// data stream overrides + +int FileStream::writeData(uint8_t *value, int size) { + if (!IsNullOrEmpty(value)) { + std::lock_guard < std::recursive_mutex > lock(file_lock_); + file_stream_->write(reinterpret_cast(value), size); + offset_ += size; + if (offset_ > length_) { + length_ = offset_; + } + file_stream_->seekg(offset_); + file_stream_->flush(); + return size; + } else { + return 0; + } +} + +template +inline std::vector FileStream::readBuffer(const T& t) { + std::vector buf; + buf.resize(sizeof t); + readData(reinterpret_cast(&buf[0]), sizeof(t)); + return buf; +} + +int FileStream::readData(std::vector &buf, int buflen) { + if (buf.capacity() < buflen) { + buf.resize(buflen); + } + int ret = readData(reinterpret_cast(&buf[0]), buflen); + + if (ret < buflen) { + buf.resize(ret); + } + return ret; +} + +int FileStream::readData(uint8_t *buf, int buflen) { + if (!IsNullOrEmpty(buf)) { + std::lock_guard < std::recursive_mutex > lock(file_lock_); + file_stream_->read(reinterpret_cast(buf), buflen); + if ((file_stream_->rdstate() & (file_stream_->eofbit | file_stream_->failbit)) != 0) { + file_stream_->clear(); + offset_ = length_; + file_stream_->seekp(offset_); + return offset_; + } else { + offset_ += buflen; + file_stream_->seekp(offset_); + return buflen; + } + + } else { + return 0; + } +} + +} /* namespace io */ +} /* namespace minifi */ +} /* namespace nifi */ +} /* namespace apache */ +} /* namespace org */ + diff --git a/libminifi/src/io/StreamFactory.cpp b/libminifi/src/io/StreamFactory.cpp index 7990edd9e3..288f4d1759 100644 --- a/libminifi/src/io/StreamFactory.cpp +++ b/libminifi/src/io/StreamFactory.cpp @@ -47,11 +47,11 @@ class SocketCreator : public AbstractStreamFactory { public: template ContextTypeCheck> create(const std::shared_ptr &configure) { - return std::make_shared(configure); + return std::make_shared < V > (configure); } template ContextTypeCheck> create(const std::shared_ptr &configure) { - return std::make_shared(configure); + return std::make_shared < SocketContext > (configure); } SocketCreator(std::shared_ptr configure) { @@ -69,7 +69,7 @@ class SocketCreator : public AbstractStreamFactory { std::unique_ptr createSocket(const std::string &host, const uint16_t port) { T *socket = create(host, port); - return std::unique_ptr(socket); + return std::unique_ptr < Socket > (socket); } private: diff --git a/libminifi/src/processors/ExecuteProcess.cpp b/libminifi/src/processors/ExecuteProcess.cpp index 323d69a6f8..9c6f732f90 100644 --- a/libminifi/src/processors/ExecuteProcess.cpp +++ b/libminifi/src/processors/ExecuteProcess.cpp @@ -145,7 +145,7 @@ void ExecuteProcess::onTrigger(core::ProcessContext *context, core::ProcessSessi break; logger_->log_info("Execute Command Respond %d", numRead); ExecuteProcess::WriteCallback callback(buffer, numRead); - std::shared_ptr flowFile = std::static_pointer_cast(session->create()); + std::shared_ptr flowFile = std::static_pointer_cast < FlowFileRecord > (session->create()); if (!flowFile) continue; flowFile->addAttribute("command", _command.c_str()); @@ -167,7 +167,7 @@ void ExecuteProcess::onTrigger(core::ProcessContext *context, core::ProcessSessi // child exits and close the pipe ExecuteProcess::WriteCallback callback(buffer, totalRead); if (!flowFile) { - flowFile = std::static_pointer_cast(session->create()); + flowFile = std::static_pointer_cast < FlowFileRecord > (session->create()); if (!flowFile) break; flowFile->addAttribute("command", _command.c_str()); @@ -185,7 +185,7 @@ void ExecuteProcess::onTrigger(core::ProcessContext *context, core::ProcessSessi logger_->log_info("Execute Command Max Respond %d", sizeof(buffer)); ExecuteProcess::WriteCallback callback(buffer, sizeof(buffer)); if (!flowFile) { - flowFile = std::static_pointer_cast(session->create()); + flowFile = std::static_pointer_cast < FlowFileRecord > (session->create()); if (!flowFile) continue; flowFile->addAttribute("command", _command.c_str()); diff --git a/libminifi/src/processors/GenerateFlowFile.cpp b/libminifi/src/processors/GenerateFlowFile.cpp index 3741a8f3a4..2fee3f2782 100644 --- a/libminifi/src/processors/GenerateFlowFile.cpp +++ b/libminifi/src/processors/GenerateFlowFile.cpp @@ -91,7 +91,7 @@ void GenerateFlowFile::onTrigger(core::ProcessContext *context, core::ProcessSes } for (int i = 0; i < batchSize; i++) { // For each batch - std::shared_ptr flowFile = std::static_pointer_cast(session->create()); + std::shared_ptr flowFile = std::static_pointer_cast < FlowFileRecord > (session->create()); if (!flowFile) return; if (fileSize > 0) @@ -114,7 +114,7 @@ void GenerateFlowFile::onTrigger(core::ProcessContext *context, core::ProcessSes GenerateFlowFile::WriteCallback callback(_data, _dataSize); for (int i = 0; i < batchSize; i++) { // For each batch - std::shared_ptr flowFile = std::static_pointer_cast(session->create()); + std::shared_ptr flowFile = std::static_pointer_cast < FlowFileRecord > (session->create()); if (!flowFile) return; if (fileSize > 0) diff --git a/libminifi/src/processors/GetFile.cpp b/libminifi/src/processors/GetFile.cpp index f1dbb2147e..723d4613b4 100644 --- a/libminifi/src/processors/GetFile.cpp +++ b/libminifi/src/processors/GetFile.cpp @@ -150,7 +150,7 @@ void GetFile::onTrigger(core::ProcessContext *context, core::ProcessSession *ses std::string fileName = list.front(); list.pop(); logger_->log_info("GetFile process %s", fileName.c_str()); - std::shared_ptr flowFile = std::static_pointer_cast(session->create()); + std::shared_ptr flowFile = std::static_pointer_cast < FlowFileRecord > (session->create()); if (flowFile == nullptr) return; std::size_t found = fileName.find_last_of("/\\"); @@ -172,19 +172,19 @@ void GetFile::onTrigger(core::ProcessContext *context, core::ProcessSession *ses } bool GetFile::isListingEmpty() { - std::lock_guard lock(mutex_); + std::lock_guard < std::mutex > lock(mutex_); return _dirList.empty(); } void GetFile::putListing(std::string fileName) { - std::lock_guard lock(mutex_); + std::lock_guard < std::mutex > lock(mutex_); _dirList.push(fileName); } void GetFile::pollListing(std::queue &list, const GetFileRequest &request) { - std::lock_guard lock(mutex_); + std::lock_guard < std::mutex > lock(mutex_); while (!_dirList.empty() && (request.maxSize == 0 || list.size() < request.maxSize)) { std::string fileName = _dirList.front(); diff --git a/libminifi/src/processors/InvokeHTTP.cpp b/libminifi/src/processors/InvokeHTTP.cpp index d5097bb339..fb3893aa64 100644 --- a/libminifi/src/processors/InvokeHTTP.cpp +++ b/libminifi/src/processors/InvokeHTTP.cpp @@ -42,6 +42,8 @@ #include "io/StreamFactory.h" #include "ResourceClaim.h" #include "utils/StringUtils.h" +#include "utils/ByteInputCallBack.h" +#include "utils/HTTPUtils.h" namespace org { namespace apache { @@ -210,7 +212,7 @@ void InvokeHTTP::onSchedule(core::ProcessContext *context, core::ProcessSessionF if (context->getProperty(SSLContext.getName(), context_name) && !IsNullOrEmpty(context_name)) { std::shared_ptr service = context->getControllerService(context_name); if (nullptr != service) { - ssl_context_service_ = std::static_pointer_cast(service); + ssl_context_service_ = std::static_pointer_cast < minifi::controllers::SSLContextService > (service); } } } @@ -283,14 +285,14 @@ void InvokeHTTP::configure_secure_connection(CURL *http_session) { } void InvokeHTTP::onTrigger(core::ProcessContext *context, core::ProcessSession *session) { - std::shared_ptr flowFile = std::static_pointer_cast(session->get()); + std::shared_ptr flowFile = std::static_pointer_cast < FlowFileRecord > (session->get()); logger_->log_info("onTrigger InvokeHTTP with %s", method_.c_str()); if (flowFile == nullptr) { if (!emitFlowFile(method_)) { logger_->log_info("InvokeHTTP -- create flow file with %s", method_.c_str()); - flowFile = std::static_pointer_cast(session->create()); + flowFile = std::static_pointer_cast < FlowFileRecord > (session->create()); } else { logger_->log_info("exiting because method is %s", method_.c_str()); return; @@ -317,9 +319,9 @@ void InvokeHTTP::onTrigger(core::ProcessContext *context, core::ProcessSession * if (read_timeout_ > 0) { curl_easy_setopt(http_session, CURLOPT_TIMEOUT, read_timeout_); } + utils::HTTPRequestResponse content; - curl_easy_setopt(http_session, CURLOPT_WRITEFUNCTION, - &utils::HTTPRequestResponse::recieve_write); + curl_easy_setopt(http_session, CURLOPT_WRITEFUNCTION, &utils::HTTPRequestResponse::recieve_write); curl_easy_setopt(http_session, CURLOPT_WRITEDATA, static_cast(&content)); @@ -334,12 +336,10 @@ void InvokeHTTP::onTrigger(core::ProcessContext *context, core::ProcessSession * callbackObj->pos = 0; logger_->log_info("InvokeHTTP -- Setting callback"); curl_easy_setopt(http_session, CURLOPT_UPLOAD, 1L); - curl_easy_setopt(http_session, CURLOPT_INFILESIZE_LARGE, - (curl_off_t)callback->getBufferSize()); - curl_easy_setopt(http_session, CURLOPT_READFUNCTION, - &utils::HTTPRequestResponse::send_write); - curl_easy_setopt(http_session, CURLOPT_READDATA, - static_cast(callbackObj)); + curl_easy_setopt(http_session, CURLOPT_INFILESIZE_LARGE, (curl_off_t)callback->getBufferSize()); + curl_easy_setopt(http_session, CURLOPT_READFUNCTION, &utils::HTTPRequestResponse::send_write); + curl_easy_setopt(http_session, CURLOPT_READDATA, static_cast(callbackObj)); + } else { logger_->log_error("InvokeHTTP -- no resource claim"); } @@ -382,9 +382,9 @@ void InvokeHTTP::onTrigger(core::ProcessContext *context, core::ProcessSession * if (output_body_to_content) { if (flowFile != nullptr) { - response_flow = std::static_pointer_cast(session->create(flowFile)); + response_flow = std::static_pointer_cast < FlowFileRecord > (session->create(flowFile)); } else { - response_flow = std::static_pointer_cast(session->create()); + response_flow = std::static_pointer_cast < FlowFileRecord > (session->create()); } std::string ct = content_type; @@ -398,7 +398,7 @@ void InvokeHTTP::onTrigger(core::ProcessContext *context, core::ProcessSession * session->importFrom(stream, response_flow); } else { logger_->log_info("Cannot output body to content"); - response_flow = std::static_pointer_cast(session->create()); + response_flow = std::static_pointer_cast < FlowFileRecord > (session->create()); } route(flowFile, response_flow, session, context, isSuccess, http_code); } else { diff --git a/libminifi/src/processors/ListenHTTP.cpp b/libminifi/src/processors/ListenHTTP.cpp index c26b41dc73..d410547d07 100644 --- a/libminifi/src/processors/ListenHTTP.cpp +++ b/libminifi/src/processors/ListenHTTP.cpp @@ -201,7 +201,7 @@ ListenHTTP::~ListenHTTP() { } void ListenHTTP::onTrigger(core::ProcessContext *context, core::ProcessSession *session) { - std::shared_ptr flowFile = std::static_pointer_cast(session->get()); + std::shared_ptr flowFile = std::static_pointer_cast < FlowFileRecord > (session->get()); // Do nothing if there are no incoming files if (!flowFile) { @@ -243,7 +243,7 @@ bool ListenHTTP::Handler::handlePost(CivetServer *server, struct mg_connection * auto session = _processSessionFactory->createSession(); ListenHTTP::WriteCallback callback(conn, req_info); - auto flowFile = std::static_pointer_cast(session->create()); + auto flowFile = std::static_pointer_cast < FlowFileRecord > (session->create()); if (!flowFile) { sendErrorResponse(conn); @@ -295,11 +295,11 @@ ListenHTTP::WriteCallback::WriteCallback(struct mg_connection *conn, const struc _reqInfo = reqInfo; } -void ListenHTTP::WriteCallback::process(std::ofstream *stream) { +int64_t ListenHTTP::WriteCallback::process(std::shared_ptr stream) { int64_t rlen; int64_t nlen = 0; int64_t tlen = _reqInfo->content_length; - char buf[16384]; + uint8_t buf[16384]; while (nlen < tlen) { rlen = tlen - nlen; @@ -320,6 +320,8 @@ void ListenHTTP::WriteCallback::process(std::ofstream *stream) { nlen += rlen; } + + return nlen; } } /* namespace processors */ diff --git a/libminifi/src/processors/ListenSyslog.cpp b/libminifi/src/processors/ListenSyslog.cpp index 054d5858c2..a5fdf2851f 100644 --- a/libminifi/src/processors/ListenSyslog.cpp +++ b/libminifi/src/processors/ListenSyslog.cpp @@ -279,7 +279,7 @@ void ListenSyslog::onTrigger(core::ProcessContext *context, core::ProcessSession SysLogEvent event = eventQueue.front(); eventQueue.pop(); if (firstEvent) { - flowFile = std::static_pointer_cast(session->create()); + flowFile = std::static_pointer_cast < FlowFileRecord > (session->create()); if (!flowFile) return; ListenSyslog::WriteCallback callback(event.payload, event.len); diff --git a/libminifi/src/processors/LogAttribute.cpp b/libminifi/src/processors/LogAttribute.cpp index e30890112a..cddcbbe468 100644 --- a/libminifi/src/processors/LogAttribute.cpp +++ b/libminifi/src/processors/LogAttribute.cpp @@ -107,10 +107,10 @@ void LogAttribute::onTrigger(core::ProcessContext *context, core::ProcessSession message << "\n" << "Payload:" << "\n"; ReadCallback callback(flow->getSize()); session->read(flow, &callback); - for (unsigned int i = 0, j = 0; i < callback._readSize; i++) { - message << std::hex << callback._buffer[i]; + for (unsigned int i = 0, j = 0; i < callback.read_size_; i++) { + message << std::hex << callback.buffer_[i]; j++; - if (j == 16) { + if (j == 80) { message << '\n'; j = 0; } diff --git a/libminifi/src/processors/PutFile.cpp b/libminifi/src/processors/PutFile.cpp index d72c56abc4..b425ae9271 100644 --- a/libminifi/src/processors/PutFile.cpp +++ b/libminifi/src/processors/PutFile.cpp @@ -17,20 +17,26 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include "processors/PutFile.h" -#include + +#include "../../include/processors/PutFile.h" + +#include +#include #include -#include -#include +#include +#include #include #include #include -#include -#include "io/validation.h" -#include "utils/StringUtils.h" -#include "utils/TimeUtil.h" -#include "core/ProcessContext.h" -#include "core/ProcessSession.h" +#include + +#include "../../include/core/logging/Logger.h" +#include "../../include/core/ProcessContext.h" +#include "../../include/core/Property.h" +#include "../../include/core/Relationship.h" +#include "../../include/io/BaseStream.h" +#include "../../include/io/DataStream.h" +#include "../../include/io/validation.h" namespace org { namespace apache { @@ -76,7 +82,7 @@ void PutFile::onTrigger(core::ProcessContext *context, core::ProcessSession *ses return; } - std::shared_ptr flowFile = std::static_pointer_cast(session->get()); + std::shared_ptr flowFile = std::static_pointer_cast < FlowFileRecord > (session->get()); // Do nothing if there are no incoming files if (!flowFile) { @@ -142,10 +148,23 @@ PutFile::ReadCallback::ReadCallback(const std::string &tmpFile, const std::strin } // Copy the entire file contents to the temporary file -void PutFile::ReadCallback::process(std::ifstream *stream) { +int64_t PutFile::ReadCallback::process(std::shared_ptr stream) { // Copy file contents into tmp file _writeSucceeded = false; - _tmpFileOs << stream->rdbuf(); + size_t size = 0; + uint8_t buffer[1024]; + do { + int read = stream->read(buffer, 1024); + if (read < 0) { + return -1; + } + if (read == 0) { + break; + } + _tmpFileOs.write(reinterpret_cast(buffer), read); + size += read; + } while (size < stream->getSize()); + return size; _writeSucceeded = true; } diff --git a/libminifi/src/processors/TailFile.cpp b/libminifi/src/processors/TailFile.cpp index 46ed1fb8f5..f87f4ecfc4 100644 --- a/libminifi/src/processors/TailFile.cpp +++ b/libminifi/src/processors/TailFile.cpp @@ -51,7 +51,8 @@ core::Property TailFile::StateFile("State File", "Specifies the file that should " what data has been ingested so that upon restart NiFi can resume from where it left off", "TailFileState"); core::Property TailFile::Delimiter("Input Delimiter", "Specifies the character that should be used for delimiting the data being tailed" - "from the incoming file.", ""); + "from the incoming file.", + ""); core::Relationship TailFile::Success("success", "All files are routed to success"); void TailFile::initialize() { @@ -240,10 +241,9 @@ void TailFile::onTrigger(core::ProcessContext *context, core::ProcessSession *se context->yield(); return; } - - std::size_t found = _currentTailFileName.find_last_of("."); - std::string baseName = _currentTailFileName.substr(0, found); - std::string extension = _currentTailFileName.substr(found + 1); + std::size_t found = _currentTailFileName.find_last_of("."); + std::string baseName = _currentTailFileName.substr(0, found); + std::string extension = _currentTailFileName.substr(found + 1); if (!this->_delimiter.empty()) { char delim = this->_delimiter.c_str()[0]; @@ -254,20 +254,20 @@ void TailFile::onTrigger(core::ProcessContext *context, core::ProcessSession *se for (std::shared_ptr ffr : flowFiles) { logger_->log_info("TailFile %s for %d bytes", _currentTailFileName, ffr->getSize()); std::string logName = baseName + "." + std::to_string(_currentTailFilePosition) + "-" + std::to_string(_currentTailFilePosition + ffr->getSize()) + "." + extension; - ffr->updateKeyedAttribute(PATH, fileLocation); - ffr->addKeyedAttribute(ABSOLUTE_PATH, fullPath); + ffr->updateKeyedAttribute(PATH, fileLocation); + ffr->addKeyedAttribute(ABSOLUTE_PATH, fullPath); ffr->updateKeyedAttribute(FILENAME, logName); - session->transfer(ffr, Success); + session->transfer(ffr, Success); this->_currentTailFilePosition += ffr->getSize() + 1; storeState(); } } else { - std::shared_ptr flowFile = std::static_pointer_cast(session->create()); - if (!flowFile) - return; - flowFile->updateKeyedAttribute(PATH, fileLocation); - flowFile->addKeyedAttribute(ABSOLUTE_PATH, fullPath); + std::shared_ptr flowFile = std::static_pointer_cast(session->create()); + if (!flowFile) + return; + flowFile->updateKeyedAttribute(PATH, fileLocation); + flowFile->addKeyedAttribute(ABSOLUTE_PATH, fullPath); session->import(fullPath, flowFile, true, this->_currentTailFilePosition); session->transfer(flowFile, Success); logger_->log_info("TailFile %s for %d bytes", _currentTailFileName, flowFile->getSize()); diff --git a/libminifi/src/provenance/Provenance.cpp b/libminifi/src/provenance/Provenance.cpp index 8686a58cc2..3e42a5a137 100644 --- a/libminifi/src/provenance/Provenance.cpp +++ b/libminifi/src/provenance/Provenance.cpp @@ -17,13 +17,12 @@ */ #include "provenance/Provenance.h" - #include #include #include #include #include - +#include "core/Repository.h" #include "io/DataStream.h" #include "io/Serializable.h" #include "core/logging/Logger.h" @@ -42,30 +41,35 @@ std::shared_ptr ProvenanceEventRecord::logger_ = logging::Logge const char *ProvenanceEventRecord::ProvenanceEventTypeStr[REPLAY + 1] = { "CREATE", "RECEIVE", "FETCH", "SEND", "DOWNLOAD", "DROP", "EXPIRE", "FORK", "JOIN", "CLONE", "CONTENT_MODIFIED", "ATTRIBUTES_MODIFIED", "ROUTE", "ADDINFO", "REPLAY" }; -ProvenanceEventRecord::ProvenanceEventRecord(ProvenanceEventRecord::ProvenanceEventType event, std::string componentId, std::string componentType) { +ProvenanceEventRecord::ProvenanceEventRecord(ProvenanceEventRecord::ProvenanceEventType event, std::string componentId, std::string componentType) + : core::SerializableComponent(core::getClassName()), + _eventDuration(0), + _entryDate(0), + _lineageStartDate(0) { _eventType = event; _componentId = componentId; _componentType = componentType; _eventTime = getTimeMillis(); - char eventIdStr[37]; - // Generate the global UUID for th event - id_generator_->generate(_eventId); - uuid_unparse_lower(_eventId, eventIdStr); - _eventIdStr = eventIdStr; } // DeSerialize -bool ProvenanceEventRecord::DeSerialize(const std::shared_ptr &repo, std::string key) { +bool ProvenanceEventRecord::DeSerialize(const std::shared_ptr &store) { std::string value; bool ret; - ret = repo->Get(key, value); + const std::shared_ptr repo = std::dynamic_pointer_cast(store); + + if (nullptr == repo || IsNullOrEmpty(uuidStr_)) { + logger_->log_error("Repo could not be assigned"); + return false; + } + ret = repo->Get(uuidStr_, value); if (!ret) { - logger_->log_error("NiFi Provenance Store event %s can not found", key.c_str()); + logger_->log_error("NiFi Provenance Store event %s can not be found", uuidStr_); return false; } else { - logger_->log_debug("NiFi Provenance Read event %s length %d", key.c_str(), value.length()); + logger_->log_debug("NiFi Provenance Read event %s length %d", uuidStr_, value.length()); } org::apache::nifi::minifi::io::DataStream stream((const uint8_t*) value.data(), value.length()); @@ -73,20 +77,20 @@ bool ProvenanceEventRecord::DeSerialize(const std::shared_ptr ret = DeSerialize(stream); if (ret) { - logger_->log_debug("NiFi Provenance retrieve event %s size %d eventType %d success", _eventIdStr.c_str(), stream.getSize(), _eventType); + logger_->log_debug("NiFi Provenance retrieve event %s size %d eventType %d success", uuidStr_, stream.getSize(), _eventType); } else { - logger_->log_debug("NiFi Provenance retrieve event %s size %d eventType %d fail", _eventIdStr.c_str(), stream.getSize(), _eventType); + logger_->log_debug("NiFi Provenance retrieve event %s size %d eventType %d fail", uuidStr_, stream.getSize(), _eventType); } return ret; } -bool ProvenanceEventRecord::Serialize(const std::shared_ptr &repo) { +bool ProvenanceEventRecord::Serialize(const std::shared_ptr &repo) { org::apache::nifi::minifi::io::DataStream outStream; int ret; - ret = writeUTF(this->_eventIdStr, &outStream); + ret = writeUTF(this->uuidStr_, &outStream); if (ret <= 0) { return false; } @@ -127,7 +131,7 @@ bool ProvenanceEventRecord::Serialize(const std::shared_ptr &r return false; } - ret = writeUTF(this->uuid_, &outStream); + ret = writeUTF(this->flow_uuid_, &outStream); if (ret <= 0) { return false; } @@ -215,20 +219,20 @@ bool ProvenanceEventRecord::Serialize(const std::shared_ptr &r } } // Persistent to the DB - if (repo->Put(_eventIdStr, const_cast(outStream.getBuffer()), outStream.getSize())) { - logger_->log_debug("NiFi Provenance Store event %s size %d success", _eventIdStr.c_str(), outStream.getSize()); + if (repo->Serialize(uuidStr_, const_cast(outStream.getBuffer()), outStream.getSize())) { + logger_->log_debug("NiFi Provenance Store event %s size %d success", uuidStr_, outStream.getSize()); } else { - logger_->log_error("NiFi Provenance Store event %s size %d fail", _eventIdStr.c_str(), outStream.getSize()); + logger_->log_error("NiFi Provenance Store event %s size %d fail", uuidStr_, outStream.getSize()); } return true; } -bool ProvenanceEventRecord::DeSerialize(const uint8_t *buffer, const int bufferSize) { +bool ProvenanceEventRecord::DeSerialize(const uint8_t *buffer, const size_t bufferSize) { int ret; org::apache::nifi::minifi::io::DataStream outStream(buffer, bufferSize); - ret = readUTF(this->_eventIdStr, &outStream); + ret = readUTF(this->uuidStr_, &outStream); if (ret <= 0) { return false; @@ -271,7 +275,7 @@ bool ProvenanceEventRecord::DeSerialize(const uint8_t *buffer, const int bufferS return false; } - ret = readUTF(this->uuid_, &outStream); + ret = readUTF(this->flow_uuid_, &outStream); if (ret <= 0) { return false; } diff --git a/libminifi/src/provenance/ProvenanceRepository.cpp b/libminifi/src/provenance/ProvenanceRepository.cpp index e4a8ffa631..b2e3031f2d 100644 --- a/libminifi/src/provenance/ProvenanceRepository.cpp +++ b/libminifi/src/provenance/ProvenanceRepository.cpp @@ -30,7 +30,6 @@ void ProvenanceRepository::run() { // threshold for purge uint64_t purgeThreshold = max_partition_bytes_ * 3 / 4; while (running_) { - std::this_thread::sleep_for(std::chrono::milliseconds(purge_period_)); std::this_thread::sleep_for(std::chrono::milliseconds(purge_period_)); uint64_t curTime = getTimeMillis(); uint64_t size = repoSize(); diff --git a/libminifi/test/TestServer.h b/libminifi/test/TestServer.h new file mode 100644 index 0000000000..263a6b317d --- /dev/null +++ b/libminifi/test/TestServer.h @@ -0,0 +1,137 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#ifndef LIBMINIFI_TEST_TESTSERVER_H_ +#define LIBMINIFI_TEST_TESTSERVER_H_ +#include +#include +#include +#include "civetweb.h" + +/* Server context handle */ +static struct mg_context *ctx; +static std::string resp_str; + +static int responder(struct mg_connection *conn, void *response) { + const char *msg = resp_str.c_str(); + + + mg_printf(conn, "HTTP/1.1 200 OK\r\n" + "Content-Length: %lu\r\n" + "Content-Type: text/plain\r\n" + "Connection: close\r\n\r\n", + resp_str.size()); + + mg_write(conn, msg, resp_str.size()); + + return 200; +} + +void init_webserver() { + mg_init_library(0); +} + +void start_webserver(std::string &port, std::string &rooturi, const std::string &response, struct mg_callbacks *callbacks, std::string &cert) { + + std::cout << "root uri is " << rooturi << ":" << port << "/" << std::endl; + resp_str = response; + const char *options[] = { "listening_ports", port.c_str(), "ssl_certificate", cert.c_str(), "ssl_protocol_version", "3", "ssl_cipher_list", + "ECDHE-RSA-AES256-GCM-SHA384:DES-CBC3-SHA:AES128-SHA:AES128-GCM-SHA256", 0 }; + + if (!mg_check_feature(2)) { + std::cerr << "Error: Embedded example built with SSL support, " << "but civetweb library build without" << std::endl; + exit(1); + } + + ctx = mg_start(callbacks, 0, options); + if (ctx == nullptr) { + std::cerr << "Cannot start CivetWeb - mg_start failed." << std::endl; + exit(1); + } + + mg_set_request_handler(ctx, rooturi.c_str(), responder, (void*) &resp_str); + +} + +void start_webserver(std::string &port, std::string &rooturi, const std::string &response) { + + std::cout << "root uri is " << rooturi << ":" << port << "/" << std::endl; + resp_str = response; + + const char *options[] = { "listening_ports", port.c_str(), 0 }; + ctx = mg_start(nullptr, 0, options); + + if (ctx == nullptr) { + std::cerr << "Cannot start CivetWeb - mg_start failed." << std::endl; + exit(1); + } + + mg_set_request_handler(ctx, rooturi.c_str(), responder, (void*) &resp_str); + +} + +bool parse_http_components(const std::string &url, std::string &port, std::string &scheme, std::string &path) { + regex_t regex; + + const char *regexstr = "^(http|https)://(localhost:)([0-9]+)?(/.*)$"; + + int ret = regcomp(®ex, regexstr, REG_EXTENDED); + if (ret) { + return false; + } + + size_t potentialGroups = regex.re_nsub + 1; + regmatch_t groups[potentialGroups]; + if (regexec(®ex, url.c_str(), potentialGroups, groups, 0) == 0) { + for (int i = 0; i < potentialGroups; i++) { + if (groups[i].rm_so == -1) + break; + + std::string str(url.data() + groups[i].rm_so, groups[i].rm_eo - groups[i].rm_so); + switch (i) { + case 1: + scheme = str; + break; + case 3: + port = str; + break; + case 4: + path = str; + break; + default: + break; + } + } + } + if (path.empty() || scheme.empty() || port.empty()) + return false; + + regfree(®ex); + + return true; + +} + +static void stop_webserver() { + /* Stop the server */ + mg_stop(ctx); + + /* Un-initialize the library */ + mg_exit_library(); +} + +#endif diff --git a/libminifi/test/integration/ControllerServiceIntegrationTests.cpp b/libminifi/test/integration/ControllerServiceIntegrationTests.cpp index 3f27b66049..15720eb3a6 100644 --- a/libminifi/test/integration/ControllerServiceIntegrationTests.cpp +++ b/libminifi/test/integration/ControllerServiceIntegrationTests.cpp @@ -80,18 +80,21 @@ int main(int argc, char **argv) { configuration->set(minifi::Configure::nifi_default_directory, key_dir); std::shared_ptr stream_factory = std::make_shared(configuration); - - std::unique_ptr yaml_ptr = std::unique_ptr(new core::YamlConfiguration(test_repo, test_repo, stream_factory, configuration, test_file_location)); + std::shared_ptr content_repo = std::make_shared(); + content_repo->initialize(configuration); + std::unique_ptr yaml_ptr = std::unique_ptr( + new core::YamlConfiguration(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location)); std::shared_ptr repo = std::static_pointer_cast(test_repo); std::shared_ptr controller = std::make_shared(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), - DEFAULT_ROOT_GROUP_NAME, + content_repo, + DEFAULT_ROOT_GROUP_NAME, true); disabled = false; std::shared_ptr map = std::make_shared(); - core::YamlConfiguration yaml_config(test_repo, test_repo, stream_factory, configuration, test_file_location); + core::YamlConfiguration yaml_config(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location); std::unique_ptr ptr = yaml_config.getRoot(test_file_location); std::shared_ptr pg = std::shared_ptr(ptr.get()); diff --git a/libminifi/test/integration/HttpConfigurationListenerTest.cpp b/libminifi/test/integration/HttpConfigurationListenerTest.cpp index a86b884a31..b559f418a3 100644 --- a/libminifi/test/integration/HttpConfigurationListenerTest.cpp +++ b/libminifi/test/integration/HttpConfigurationListenerTest.cpp @@ -46,7 +46,7 @@ void waitToVerifyProcessor() { std::this_thread::sleep_for(std::chrono::seconds(10)); } -class ConfigHandler: public CivetHandler { +class ConfigHandler : public CivetHandler { public: bool handleGet(CivetServer *server, struct mg_connection *conn) { std::ifstream myfile(test_file_location_.c_str()); @@ -57,8 +57,8 @@ class ConfigHandler: public CivetHandler { std::string str = buffer.str(); myfile.close(); mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: " - "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n", - str.length()); + "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n", + str.length()); mg_printf(conn, "%s", str.c_str()); } else { mg_printf(conn, "HTTP/1.1 500 Internal Server Error\r\n"); @@ -75,7 +75,7 @@ int main(int argc, char **argv) { LogTestController::getInstance().setInfo(); const char *options[] = { "document_root", ".", "listening_ports", "9090", 0 }; - std::vector < std::string > cpp_options; + std::vector cpp_options; for (int i = 0; i < (sizeof(options) / sizeof(options[0]) - 1); i++) { cpp_options.push_back(options[i]); } @@ -89,45 +89,32 @@ int main(int argc, char **argv) { h_ex.test_file_location_ = test_file_location = argv[1]; key_dir = argv[2]; } - std::shared_ptr configuration = std::make_shared< - minifi::Configure>(); + std::shared_ptr configuration = std::make_shared(); configuration->set(minifi::Configure::nifi_default_directory, key_dir); - configuration->set(minifi::Configure::nifi_configuration_listener_type, - "http"); - configuration->set( - minifi::Configure::nifi_configuration_listener_pull_interval, "1 sec"); - configuration->set(minifi::Configure::nifi_configuration_listener_http_url, - "http://localhost:9090/config"); + configuration->set(minifi::Configure::nifi_configuration_listener_type, "http"); + configuration->set(minifi::Configure::nifi_configuration_listener_pull_interval, "1 sec"); + configuration->set(minifi::Configure::nifi_configuration_listener_http_url, "http://localhost:9090/config"); mkdir("content_repository", S_IRWXU | S_IRWXG | S_IROTH | S_IXOTH); - std::shared_ptr test_repo = - std::make_shared(); - std::shared_ptr test_flow_repo = std::make_shared< - TestFlowRepository>(); - - configuration->set(minifi::Configure::nifi_flow_configuration_file, - test_file_location); - - std::shared_ptr stream_factory = std::make_shared - < minifi::io::StreamFactory > (configuration); - std::unique_ptr yaml_ptr = std::unique_ptr - < core::YamlConfiguration - > (new core::YamlConfiguration(test_repo, test_repo, stream_factory, - configuration, test_file_location)); - std::shared_ptr repo = std::static_pointer_cast - < TestRepository > (test_repo); - - std::shared_ptr controller = - std::make_shared < minifi::FlowController - > (test_repo, test_flow_repo, configuration, std::move(yaml_ptr), DEFAULT_ROOT_GROUP_NAME, true); - - core::YamlConfiguration yaml_config(test_repo, test_repo, stream_factory, - configuration, test_file_location); - - std::unique_ptr ptr = yaml_config.getRoot( - test_file_location); - std::shared_ptr pg = std::shared_ptr < core::ProcessGroup - > (ptr.get()); + std::shared_ptr test_repo = std::make_shared(); + std::shared_ptr test_flow_repo = std::make_shared(); + + configuration->set(minifi::Configure::nifi_flow_configuration_file, test_file_location); + + std::shared_ptr stream_factory = std::make_shared < minifi::io::StreamFactory > (configuration); + + std::shared_ptr content_repo = std::make_shared(); + std::unique_ptr yaml_ptr = std::unique_ptr < core::YamlConfiguration + > (new core::YamlConfiguration(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location)); + std::shared_ptr repo = std::static_pointer_cast < TestRepository > (test_repo); + + std::shared_ptr controller = std::make_shared < minifi::FlowController + > (test_repo, test_flow_repo, configuration, std::move(yaml_ptr), content_repo, DEFAULT_ROOT_GROUP_NAME, true); + + core::YamlConfiguration yaml_config(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location); + + std::unique_ptr ptr = yaml_config.getRoot(test_file_location); + std::shared_ptr pg = std::shared_ptr < core::ProcessGroup > (ptr.get()); ptr.release(); controller->load(); diff --git a/libminifi/test/integration/HttpGetIntegrationTest.cpp b/libminifi/test/integration/HttpGetIntegrationTest.cpp index ae60dc129d..a235759001 100644 --- a/libminifi/test/integration/HttpGetIntegrationTest.cpp +++ b/libminifi/test/integration/HttpGetIntegrationTest.cpp @@ -26,6 +26,7 @@ #include #include #include +#include "../TestServer.h" #include "../TestBase.h" #include "utils/StringUtils.h" #include "core/Core.h" @@ -41,9 +42,23 @@ void waitToVerifyProcessor() { std::this_thread::sleep_for(std::chrono::seconds(10)); } +int log_message(const struct mg_connection *conn, const char *message) { + puts(message); + return 1; +} + +int ssl_enable(void *ssl_context, void *user_data) { + struct ssl_ctx_st *ctx = (struct ssl_ctx_st *) ssl_context; + return 0; +} + int main(int argc, char **argv) { - LogTestController::getInstance().setInfo(); - LogTestController::getInstance().setInfo(); + init_webserver(); + LogTestController::getInstance().setDebug(); + LogTestController::getInstance().setDebug(); + LogTestController::getInstance().setDebug(); + LogTestController::getInstance().setDebug(); + LogTestController::getInstance().setDebug(); std::string key_dir, test_file_location; if (argc > 1) { test_file_location = argv[1]; @@ -59,27 +74,61 @@ int main(int argc, char **argv) { configuration->set(minifi::Configure::nifi_flow_configuration_file, test_file_location); std::shared_ptr stream_factory = std::make_shared(configuration); - std::unique_ptr yaml_ptr = std::unique_ptr(new core::YamlConfiguration(test_repo, test_repo, stream_factory, configuration, test_file_location)); + + std::shared_ptr content_repo = std::make_shared(); + + content_repo->initialize(configuration); + + std::unique_ptr yaml_ptr = std::unique_ptr( + new core::YamlConfiguration(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location)); std::shared_ptr repo = std::static_pointer_cast(test_repo); - std::shared_ptr controller = std::make_shared(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), DEFAULT_ROOT_GROUP_NAME, - true); + std::shared_ptr controller = std::make_shared(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), + content_repo, + DEFAULT_ROOT_GROUP_NAME, + true); - core::YamlConfiguration yaml_config(test_repo, test_repo, stream_factory, configuration, test_file_location); + core::YamlConfiguration yaml_config(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location); std::unique_ptr ptr = yaml_config.getRoot(test_file_location); std::shared_ptr pg = std::shared_ptr(ptr.get()); - ptr.release(); + std::shared_ptr proc = ptr->findProcessor("invoke"); + assert(proc != nullptr); + std::shared_ptr inv = std::dynamic_pointer_cast(proc); + + assert(inv != nullptr); + std::string url = ""; + inv->getProperty(minifi::processors::InvokeHTTP::URL.getName(), url); + ptr.release(); + std::string port, scheme, path; + parse_http_components(url, port, scheme, path); + struct mg_callbacks callback; + if (url.find("localhost") != std::string::npos) { + if (scheme == "https") { + std::string cert = ""; + cert = key_dir + "nifi-cert.pem"; + memset(&callback, 0, sizeof(callback)); + callback.init_ssl = ssl_enable; + callback.log_message = log_message; + std::cout << cert << std::endl; + start_webserver(port, path, "hi this is a get test", &callback, cert); + } else { + start_webserver(port, path, "hi this is a get test"); + } + } controller->load(); controller->start(); waitToVerifyProcessor(); controller->waitUnload(60000); + if (url.find("localhost") != std::string::npos) { + stop_webserver(); + } std::string logs = LogTestController::getInstance().log_output.str(); + assert(logs.find("key:filename value:") != std::string::npos); - assert(logs.find("key:invokehttp.request.url value:https://raw.githubusercontent.com/curl/curl/master/docs/examples/httpput.c") != std::string::npos); - assert(logs.find("Size:3734 Offset:0") != std::string::npos); + assert(logs.find("key:invokehttp.request.url value:" + url) != std::string::npos); assert(logs.find("key:invokehttp.status.code value:200") != std::string::npos); std::string stringtofind = "Resource Claim created ./content_repository/"; diff --git a/libminifi/test/integration/HttpPostIntegrationTest.cpp b/libminifi/test/integration/HttpPostIntegrationTest.cpp index dfa284fb63..9a4657426f 100644 --- a/libminifi/test/integration/HttpPostIntegrationTest.cpp +++ b/libminifi/test/integration/HttpPostIntegrationTest.cpp @@ -28,6 +28,7 @@ #include #include "utils/StringUtils.h" #include "core/Core.h" +#include "../TestServer.h" #include "../include/core/logging/Logger.h" #include "core/ProcessGroup.h" #include "core/yaml/YamlConfiguration.h" @@ -42,6 +43,7 @@ void waitToVerifyProcessor() { } int main(int argc, char **argv) { + init_webserver(); LogTestController::getInstance().setDebug(); LogTestController::getInstance().setDebug(); std::string test_file_location; @@ -51,7 +53,6 @@ int main(int argc, char **argv) { mkdir("/tmp/aljr39/", S_IRWXU | S_IRWXG | S_IROTH | S_IXOTH); std::ofstream myfile; myfile.open("/tmp/aljr39/example.txt"); - myfile << "Hello world" << std::endl; myfile.close(); mkdir("content_repository", S_IRWXU | S_IRWXG | S_IROTH | S_IXOTH); @@ -62,31 +63,44 @@ int main(int argc, char **argv) { configuration->set(minifi::Configure::nifi_flow_configuration_file, test_file_location); std::shared_ptr stream_factory = std::make_shared(configuration); - - std::unique_ptr yaml_ptr = std::unique_ptr(new core::YamlConfiguration(test_repo, test_repo, stream_factory, configuration, test_file_location)); + std::shared_ptr content_repo = std::make_shared(); + content_repo->initialize(configuration); + std::unique_ptr yaml_ptr = std::unique_ptr( + new core::YamlConfiguration(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location)); std::shared_ptr repo = std::static_pointer_cast(test_repo); - std::shared_ptr controller = std::make_shared(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), - DEFAULT_ROOT_GROUP_NAME, - true); + std::shared_ptr controller = std::make_shared(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), content_repo, DEFAULT_ROOT_GROUP_NAME, + true); - core::YamlConfiguration yaml_config(test_repo, test_repo, stream_factory, configuration, test_file_location); + core::YamlConfiguration yaml_config(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location); std::unique_ptr ptr = yaml_config.getRoot(test_file_location); std::shared_ptr pg = std::shared_ptr(ptr.get()); - ptr.release(); + std::shared_ptr proc = ptr->findProcessor("OhJeez"); + assert(proc != nullptr); + + std::shared_ptr inv = std::dynamic_pointer_cast(proc); + assert(inv != nullptr); + std::string url = ""; + inv->getProperty(minifi::processors::InvokeHTTP::URL.getName(), url); + ptr.release(); + std::string port, scheme, path; + parse_http_components(url, port, scheme, path); + start_webserver(port, path, "hi this is a post test"); controller->load(); controller->start(); waitToVerifyProcessor(); controller->waitUnload(60000); + std::string logs = LogTestController::getInstance().log_output.str(); + // stop webserver + stop_webserver(); assert(LogTestController::getInstance().contains("curl performed") == true); - assert(LogTestController::getInstance().contains("Import offset 0 length 12") == true); + assert(LogTestController::getInstance().contains("Import offset 0 length 22") == true); std::string stringtofind = "Resource Claim created ./content_repository/"; - std::string logs = LogTestController::getInstance().log_output.str(); size_t loc = logs.find(stringtofind); while (loc > 0 && loc != std::string::npos) { std::string id = logs.substr(loc + stringtofind.size(), 36); diff --git a/libminifi/test/integration/ProvenanceReportingTest.cpp b/libminifi/test/integration/ProvenanceReportingTest.cpp index a7bcc2bfa9..a6dc377eb5 100644 --- a/libminifi/test/integration/ProvenanceReportingTest.cpp +++ b/libminifi/test/integration/ProvenanceReportingTest.cpp @@ -53,21 +53,20 @@ int main(int argc, char **argv) { LogTestController::getInstance().setDebug(); std::shared_ptr configuration = std::make_shared(); - std::shared_ptr test_repo = std::make_shared(); std::shared_ptr test_flow_repo = std::make_shared(); configuration->set(minifi::Configure::nifi_flow_configuration_file, test_file_location); std::shared_ptr stream_factory = std::make_shared(configuration); - - std::unique_ptr yaml_ptr = std::unique_ptr(new core::YamlConfiguration(test_repo, test_repo, stream_factory, configuration, test_file_location)); + std::shared_ptr content_repo = std::make_shared(); + std::unique_ptr yaml_ptr = std::unique_ptr( + new core::YamlConfiguration(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location)); std::shared_ptr repo = std::static_pointer_cast(test_repo); - std::shared_ptr controller = std::make_shared(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), - DEFAULT_ROOT_GROUP_NAME, + std::shared_ptr controller = std::make_shared(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), content_repo, DEFAULT_ROOT_GROUP_NAME, true); - core::YamlConfiguration yaml_config(test_repo, test_repo, stream_factory, configuration, test_file_location); + core::YamlConfiguration yaml_config(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location); std::unique_ptr ptr = yaml_config.getRoot(test_file_location); std::shared_ptr pg = std::shared_ptr(ptr.get()); diff --git a/libminifi/test/integration/Site2SiteRestTest.cpp b/libminifi/test/integration/Site2SiteRestTest.cpp index 01aa7a80c1..1773cdba1b 100644 --- a/libminifi/test/integration/Site2SiteRestTest.cpp +++ b/libminifi/test/integration/Site2SiteRestTest.cpp @@ -45,22 +45,22 @@ void waitToVerifyProcessor() { std::this_thread::sleep_for(std::chrono::seconds(10)); } -class ConfigHandler: public CivetHandler { +class ConfigHandler : public CivetHandler { public: bool handleGet(CivetServer *server, struct mg_connection *conn) { static const std::string site2site_rest_resp = "{" - "\"revision\": {" - "\"clientId\": \"483d53eb-53ec-4e93-b4d4-1fc3d23dae6f\"" - "}," - "\"controller\": {" - "\"id\": \"fe4a3a42-53b6-4af1-a80d-6fdfe60de97f\"," - "\"name\": \"NiFi Flow\"," - "\"remoteSiteListeningPort\": 10001," - "\"siteToSiteSecure\": false" - "}}"; + "\"revision\": {" + "\"clientId\": \"483d53eb-53ec-4e93-b4d4-1fc3d23dae6f\"" + "}," + "\"controller\": {" + "\"id\": \"fe4a3a42-53b6-4af1-a80d-6fdfe60de97f\"," + "\"name\": \"NiFi Flow\"," + "\"remoteSiteListeningPort\": 10001," + "\"siteToSiteSecure\": false" + "}}"; mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: " - "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n", - site2site_rest_resp.length()); + "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n", + site2site_rest_resp.length()); mg_printf(conn, "%s", site2site_rest_resp.c_str()); return true; } @@ -71,7 +71,7 @@ int main(int argc, char **argv) { LogTestController::getInstance().setInfo(); const char *options[] = { "document_root", ".", "listening_ports", "8082", 0 }; - std::vector < std::string > cpp_options; + std::vector cpp_options; for (int i = 0; i < (sizeof(options) / sizeof(options[0]) - 1); i++) { cpp_options.push_back(options[i]); } @@ -106,28 +106,31 @@ int main(int argc, char **argv) { TestFlowRepository>(); configuration->set(minifi::Configure::nifi_flow_configuration_file, - test_file_location); + test_file_location); std::shared_ptr stream_factory = std::make_shared - < minifi::io::StreamFactory > (configuration); - std::unique_ptr yaml_ptr = std::unique_ptr - < core::YamlConfiguration - > (new core::YamlConfiguration(test_repo, test_repo, stream_factory, - configuration, test_file_location)); - std::shared_ptr repo = std::static_pointer_cast - < TestRepository > (test_repo); + (configuration); + std::shared_ptr content_repo = std::make_shared(); - std::shared_ptr controller = - std::make_shared < minifi::FlowController - > (test_repo, test_flow_repo, configuration, std::move(yaml_ptr), DEFAULT_ROOT_GROUP_NAME, true); + content_repo->initialize(configuration); - core::YamlConfiguration yaml_config(test_repo, test_repo, stream_factory, - configuration, test_file_location); + std::unique_ptr yaml_ptr = std::unique_ptr( + new core::YamlConfiguration(test_repo, test_repo, content_repo, stream_factory, configuration, test_file_location)); + std::shared_ptr repo = std::static_pointer_cast(test_repo); + + std::shared_ptr controller = std::make_shared(test_repo, test_flow_repo, configuration, std::move(yaml_ptr), + content_repo, + DEFAULT_ROOT_GROUP_NAME, + true); + + core::YamlConfiguration yaml_config(test_repo, test_repo, content_repo, stream_factory, + configuration, + test_file_location); std::unique_ptr ptr = yaml_config.getRoot( - test_file_location); - std::shared_ptr pg = std::shared_ptr < core::ProcessGroup - > (ptr.get()); + test_file_location); + std::shared_ptr pg = std::shared_ptr(ptr.get()); ptr.release(); controller->load(); diff --git a/libminifi/test/integration/TestExecuteProcess.cpp b/libminifi/test/integration/TestExecuteProcess.cpp index ef0d113370..16cb133430 100644 --- a/libminifi/test/integration/TestExecuteProcess.cpp +++ b/libminifi/test/integration/TestExecuteProcess.cpp @@ -27,7 +27,7 @@ #include #include #include - +#include "core/repository/VolatileContentRepository.h" #include "../unit/ProvenanceTestHelper.h" #include "FlowController.h" #include "processors/GetFile.h" @@ -47,15 +47,17 @@ int main(int argc, char **argv) { std::shared_ptr processor = std::make_shared("executeProcess"); processor->setMaxConcurrentTasks(1); - std::shared_ptr test_repo = std::make_shared(); - - std::shared_ptr repo = std::static_pointer_cast(test_repo); - std::shared_ptr controller = std::make_shared(test_repo, test_repo); + std::shared_ptr test_repo = + std::make_shared(); + std::shared_ptr content_repo = std::make_shared(); + std::shared_ptr repo = + std::static_pointer_cast(test_repo); + std::shared_ptr controller = std::make_shared< + TestFlowController>(test_repo, test_repo, content_repo); uuid_t processoruuid; assert(true == processor->getUUID(processoruuid)); - - std::shared_ptr connection = std::make_shared(test_repo, "executeProcessConnection"); + std::shared_ptr connection = std::make_shared(test_repo, content_repo, "executeProcessConnection"); connection->setRelationship(core::Relationship("success", "description")); // link the connections so that we can test results at the end for this diff --git a/libminifi/test/resources/TestHTTPGet.yml b/libminifi/test/resources/TestHTTPGet.yml index 2f64f2a3d4..58f95d99e9 100644 --- a/libminifi/test/resources/TestHTTPGet.yml +++ b/libminifi/test/resources/TestHTTPGet.yml @@ -32,7 +32,7 @@ Processors: auto-terminated relationships list: Properties: HTTP Method: GET - Remote URL: https://raw.githubusercontent.com/curl/curl/master/docs/examples/httpput.c + Remote URL: http://localhost:10003/geturl - name: OhJeez id: 2438e3c8-015a-1000-79ca-83af40ec1992 class: org.apache.nifi.processors.standard.LogAttribute diff --git a/libminifi/test/resources/TestHTTPGetSecure.yml b/libminifi/test/resources/TestHTTPGetSecure.yml index f3a23e55ac..9d19632369 100644 --- a/libminifi/test/resources/TestHTTPGetSecure.yml +++ b/libminifi/test/resources/TestHTTPGetSecure.yml @@ -33,7 +33,7 @@ Processors: Properties: SSL Context Service: SSLContextService HTTP Method: GET - Remote URL: https://raw.githubusercontent.com/curl/curl/master/docs/examples/httpput.c + Remote URL: https://raw.githubusercontent.com/apache/nifi-minifi-cpp/master/docs/minifi-logo.png - name: OhJeez id: 2438e3c8-015a-1000-79ca-83af40ec1992 class: org.apache.nifi.processors.standard.LogAttribute diff --git a/libminifi/test/resources/TestHTTPPost.yml b/libminifi/test/resources/TestHTTPPost.yml index 837194d6b2..c76069a319 100644 --- a/libminifi/test/resources/TestHTTPPost.yml +++ b/libminifi/test/resources/TestHTTPPost.yml @@ -46,7 +46,7 @@ Processors: auto-terminated relationships list: response Properties: HTTP Method: POST - Remote URL: http://requestb.in/u8ax9uu8 + Remote URL: http://localhost:10003/urlofchampions - name: Loggit id: 2438e3c8-015a-1000-79ca-83af40ec1993 diff --git a/libminifi/test/resources/cn.ckey.pem b/libminifi/test/resources/cn.ckey.pem index 23017fa7b0..fc42f06fa4 100644 --- a/libminifi/test/resources/cn.ckey.pem +++ b/libminifi/test/resources/cn.ckey.pem @@ -1,5 +1,4 @@ Bag Attributes - friendlyName: nifi-key localKeyID: 73 E6 90 32 31 08 F5 87 C2 CE 8D 17 10 32 05 F2 95 6A 9E 9C Key Attributes: -----BEGIN RSA PRIVATE KEY----- diff --git a/libminifi/test/resources/cn.crt.pem b/libminifi/test/resources/cn.crt.pem index 3a786db6f6..60a38ac9e6 100644 --- a/libminifi/test/resources/cn.crt.pem +++ b/libminifi/test/resources/cn.crt.pem @@ -1,5 +1,4 @@ Bag Attributes - friendlyName: nifi-key localKeyID: 73 E6 90 32 31 08 F5 87 C2 CE 8D 17 10 32 05 F2 95 6A 9E 9C subject=/OU=NIFI/CN=test issuer=/OU=NIFI/CN=localhost diff --git a/libminifi/test/resources/nifi-cert.pem b/libminifi/test/resources/nifi-cert.pem index 4e404cd7ab..0c3b7dae75 100644 --- a/libminifi/test/resources/nifi-cert.pem +++ b/libminifi/test/resources/nifi-cert.pem @@ -18,3 +18,30 @@ lvrRtWOqyGHiRoaRE5+VUjyO+0ToEgj9E+3rV8JL66BT7SWQusLGqbX1OoANCMTj BRYeqB0g0PrXU+6chh6StpNSnYzkQdoxLUIDYYZx2XGsbkjDh/k6ni6bgJEKEOCu T3Z2tyvGpc+PjLRXW/WyXCpg/xfr3+GSVKI6ark= -----END CERTIFICATE----- +-----BEGIN RSA PRIVATE KEY----- +MIIEpAIBAAKCAQEAwCF6Tchue7tR66BPg886WOYNPgSwNaq1KJQSuGcEHK2wlAEu +YfiYz9LbjFLZRLRY2CF9mIGb683byrnvOMcq6a+YdXDaOHZnkKBSsI/xTzScXTv3 +EKSueZ0sMuD7L0y/2Cs2lf8heBUEUqmNe15J9yvEQ1GpJ0j7iCCneKYjjezFWglR +Sv/9suvqVCxIxr4j9gXODgyU3wdwIxkQUBJXk4GtDp03Rxcx6Ch0VBwjcGkYHhcs +GHRzg6dcr795tLfOQNA/Vlje0+RtH/KU/WXgzl9nKtxD7XUwZyhoElzNcehN0WmK +DgAmASncvy7+YYzKU69H14Q+2n/apdoqx/kTQQIDAQABAoIBAQCz7eY69+y4BXo3 +nz84Ipby8CcQoJVg/QiBAwLxHNCWBvdp9B069PQvFLo1FNWSaQ8XAW48p4yc7YHb +vftRgfwnMyIlQdWrsP9WSz6FSZhkY9HX4rODK6aWD+J3l4jFCCxVxkpteKwgaBZP +T6hHE8tTJfK8VLqEJu4g0uvjqjt7ydJT69lThdyf3VE0v6ZeSjsya5qqw+9RK+uC +q5T/8FxeFZgpfR6UXXnoLAmAkfcMZNIBo6cOJWi/BQHjZdpCOVXUBtu0/lC8bffa +4/ESaxRS8kOp+WEb64pT7u6F7yhD/kve6ZnJj/SX1EvN+RzB3zoVG42WUs/+/SwN +dU1ERz+tAoGBAPbgZPDnWuKxW7Cam/Aqmvux624C1lNfhfXEGURhyc+wHWjjhWRe +2vEPJOVxG5pN/FAo+lFoGiLe3QsLRLPlQrGfT/92W28QEcRrRSutjRZOL3wKezQA +DkAPU9HX3lACR5yQD6+a0HHgMr1MqeNFPi9MPPjywGywTyWzHd4WQqvTAoGBAMc7 +J4fpr5uPVq9mKemK67i7meJ8AxjjU7oNe8EN+2XfCYcQUmgIo+dLzV9+DTrYkoTz +iqjA6Ph2DNs6YHI/JNwsdSbAz6KVDteimt3t+uyNpiMGuyLmfOgpYEMJcHp+q6I6 +7PGKVS4c5iPFiYuIo23Is9ZMxOVQp76+UOy09rwbAoGBAOM5Za7VQjGkTGAf7ab/ +j+ZZu/dlZR8XrJSoCRmHZ9hgoLEJuJzJMXruFWeY028SmEivbrW+u0+dEJY5qOJr +ARe7KkZXCZEPmUrP8Lpi4pjFHa9tdjhGVNdhRCTAKz442vCfJ9DZDUHCuPDCvxsP +gEzIPtZjl/hxzmdElRj0JClBAoGAaXmfzAyjs6+HLQThW4r4kKyBI66T1TFEulM5 +GVPVrHEQEjlJ51nrrCAtckjBqE3QBCMLXZwDusaEt+uH8/QKB6Zhv0qEooZXfUHQ +y32aQnIbap+9oxRzPFXraJIuwisdop2fo6Cgx/D0xitmTkDghNaknue1tdGlfQ40 +uZx0o9ECgYBeKeNbMnWoO46ZOrhaz8On+fIY7xtboV2bALy7lvUbWd9B41ntqYUm +NHlYXDDU+Izs5wnNJnNnx4vECuUzYbpeY82dvMewlQwfl5aiyKrjo7VxLm//2U/K +hlID6DU5wi9O+TAQ319DhxT7Ja+AQxO/OFS/mfrtwJEevxXqJLu55Q== +-----END RSA PRIVATE KEY----- diff --git a/libminifi/test/unit/FileStreamTests.cpp b/libminifi/test/unit/FileStreamTests.cpp new file mode 100644 index 0000000000..f5af5e3e14 --- /dev/null +++ b/libminifi/test/unit/FileStreamTests.cpp @@ -0,0 +1,210 @@ +/** + * + * 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. + */ +#define CATCH_CONFIG_MAIN // This tells Catch to provide a main() - only do this in one cpp file +#include "io/FileStream.h" +#include +#include +#include +#include "../TestBase.h" + +TEST_CASE("TestFileOverWrite", "[TestFiles]") { + TestController testController; + char format[] = "/tmp/gt.XXXXXX"; + char *dir = testController.createTempDirectory(format); + + std::fstream file; + std::stringstream ss; + ss << dir << "/" << "tstFile.ext"; + std::string path = ss.str(); + file.open(path, std::ios::out); + file << "tempFile"; + file.close(); + + minifi::io::FileStream stream(path); + std::vector readBuffer; + REQUIRE(stream.readData(readBuffer, stream.getSize()) == stream.getSize()); + + uint8_t* data = readBuffer.data(); + + REQUIRE(std::string(reinterpret_cast(data), readBuffer.size()) == "tempFile"); + + stream.seek(4); + + stream.write(reinterpret_cast(const_cast("file")), 4); + + stream.seek(0); + + std::vector verifybuffer; + + REQUIRE(stream.readData(verifybuffer, stream.getSize()) == stream.getSize()); + + data = verifybuffer.data(); + + REQUIRE(std::string(reinterpret_cast(data), verifybuffer.size()) == "tempfile"); + + unlink(ss.str().c_str()); +} + +TEST_CASE("TestFileBadArgumentNoChange", "[TestLoader]") { + TestController testController; + char format[] = "/tmp/gt.XXXXXX"; + char *dir = testController.createTempDirectory(format); + + std::fstream file; + std::stringstream ss; + ss << dir << "/" << "tstFile.ext"; + std::string path = ss.str(); + file.open(path, std::ios::out); + file << "tempFile"; + file.close(); + + minifi::io::FileStream stream(path); + std::vector readBuffer; + REQUIRE(stream.readData(readBuffer, stream.getSize()) == stream.getSize()); + + uint8_t* data = readBuffer.data(); + + REQUIRE(std::string(reinterpret_cast(data), readBuffer.size()) == "tempFile"); + + stream.seek(4); + + stream.write(reinterpret_cast(const_cast("file")), 0); + + stream.seek(0); + + std::vector verifybuffer; + + REQUIRE(stream.readData(verifybuffer, stream.getSize()) == stream.getSize()); + + data = verifybuffer.data(); + + REQUIRE(std::string(reinterpret_cast(data), verifybuffer.size()) == "tempFile"); + + unlink(ss.str().c_str()); +} + +TEST_CASE("TestFileBadArgumentNoChange2", "[TestLoader]") { + TestController testController; + char format[] = "/tmp/gt.XXXXXX"; + char *dir = testController.createTempDirectory(format); + + std::fstream file; + std::stringstream ss; + ss << dir << "/" << "tstFile.ext"; + std::string path = ss.str(); + file.open(path, std::ios::out); + file << "tempFile"; + file.close(); + + minifi::io::FileStream stream(path); + std::vector readBuffer; + REQUIRE(stream.readData(readBuffer, stream.getSize()) == stream.getSize()); + + uint8_t* data = readBuffer.data(); + + REQUIRE(std::string(reinterpret_cast(data), readBuffer.size()) == "tempFile"); + + stream.seek(4); + + stream.write(nullptr, 0); + + stream.seek(0); + + std::vector verifybuffer; + + REQUIRE(stream.readData(verifybuffer, stream.getSize()) == stream.getSize()); + + data = verifybuffer.data(); + + REQUIRE(std::string(reinterpret_cast(data), verifybuffer.size()) == "tempFile"); + + unlink(ss.str().c_str()); +} + +TEST_CASE("TestFileBadArgumentNoChange3", "[TestLoader]") { + TestController testController; + char format[] = "/tmp/gt.XXXXXX"; + char *dir = testController.createTempDirectory(format); + + std::fstream file; + std::stringstream ss; + ss << dir << "/" << "tstFile.ext"; + std::string path = ss.str(); + file.open(path, std::ios::out); + file << "tempFile"; + file.close(); + + minifi::io::FileStream stream(path); + std::vector readBuffer; + REQUIRE(stream.readData(readBuffer, stream.getSize()) == stream.getSize()); + + uint8_t* data = readBuffer.data(); + + REQUIRE(std::string(reinterpret_cast(data), readBuffer.size()) == "tempFile"); + + stream.seek(4); + + stream.write(nullptr, 0); + + stream.seek(0); + + std::vector verifybuffer; + + REQUIRE(stream.readData(nullptr, stream.getSize()) == 0); + + data = verifybuffer.data(); + + REQUIRE(std::string(reinterpret_cast(data), verifybuffer.size()) == ""); + + unlink(ss.str().c_str()); +} + + +TEST_CASE("TestFileBeyondEnd3", "[TestLoader]") { + TestController testController; + char format[] = "/tmp/gt.XXXXXX"; + char *dir = testController.createTempDirectory(format); + + std::fstream file; + std::stringstream ss; + ss << dir << "/" << "tstFile.ext"; + std::string path = ss.str(); + file.open(path, std::ios::out); + file << "tempFile"; + file.close(); + + minifi::io::FileStream stream(path); + std::vector readBuffer; + REQUIRE(stream.readData(readBuffer, stream.getSize()) == stream.getSize()); + + uint8_t* data = readBuffer.data(); + + REQUIRE(std::string(reinterpret_cast(data), readBuffer.size()) == "tempFile"); + + stream.seek(0); + + std::vector verifybuffer; + + REQUIRE(stream.readData(verifybuffer, 8192) == 8); + + data = verifybuffer.data(); + + REQUIRE(std::string(reinterpret_cast(data), verifybuffer.size()) == "tempFile"); + + unlink(ss.str().c_str()); +} diff --git a/libminifi/test/unit/InvokeHTTPTests.cpp b/libminifi/test/unit/InvokeHTTPTests.cpp index 705ac846e0..34661aa99f 100644 --- a/libminifi/test/unit/InvokeHTTPTests.cpp +++ b/libminifi/test/unit/InvokeHTTPTests.cpp @@ -25,6 +25,7 @@ #include #include #include "FlowController.h" +#include "io/BaseStream.h" #include "../TestBase.h" #include "processors/GetFile.h" #include "core/Core.h" @@ -37,6 +38,7 @@ TEST_CASE("HTTPTestsPostNoResourceClaim", "[httptest1]") { TestController testController; + std::shared_ptr content_repo = std::make_shared(); LogTestController::getInstance().setInfo(); std::shared_ptr repo = std::make_shared(); @@ -50,10 +52,11 @@ TEST_CASE("HTTPTestsPostNoResourceClaim", "[httptest1]") { uuid_t invokehttp_uuid; REQUIRE(true == invokehttp->getUUID(invokehttp_uuid)); - std::shared_ptr connection = std::make_shared(repo, "getfileCreate2Connection"); + + std::shared_ptr connection = std::make_shared(repo, content_repo, "getfileCreate2Connection"); connection->setRelationship(core::Relationship("success", "description")); - std::shared_ptr connection2 = std::make_shared(repo, "listenhttp"); + std::shared_ptr connection2 = std::make_shared(repo, content_repo, "listenhttp"); connection2->setRelationship(core::Relationship("No Retry", "description")); @@ -134,6 +137,7 @@ TEST_CASE("HTTPTestsPostNoResourceClaim", "[httptest1]") { TEST_CASE("HTTPTestsWithNoResourceClaimPOST", "[httptest1]") { TestController testController; + std::shared_ptr content_repo = std::make_shared(); LogTestController::getInstance().setInfo(); std::shared_ptr repo = std::make_shared(); @@ -154,16 +158,16 @@ TEST_CASE("HTTPTestsWithNoResourceClaimPOST", "[httptest1]") { uuid_t invokehttp_uuid; REQUIRE(true == invokehttp->getUUID(invokehttp_uuid)); - std::shared_ptr gcConnection = std::make_shared(repo, "getfileCreate2Connection"); + std::shared_ptr gcConnection = std::make_shared(repo, content_repo, "getfileCreate2Connection"); gcConnection->setRelationship(core::Relationship("success", "description")); - std::shared_ptr laConnection = std::make_shared(repo, "logattribute"); + std::shared_ptr laConnection = std::make_shared(repo, content_repo, "logattribute"); laConnection->setRelationship(core::Relationship("success", "description")); - std::shared_ptr connection = std::make_shared(repo, "getfileCreate2Connection"); + std::shared_ptr connection = std::make_shared(repo, content_repo, "getfileCreate2Connection"); connection->setRelationship(core::Relationship("success", "description")); - std::shared_ptr connection2 = std::make_shared(repo, "listenhttp"); + std::shared_ptr connection2 = std::make_shared(repo, content_repo, "listenhttp"); connection2->setRelationship(core::Relationship("No Retry", "description")); @@ -242,9 +246,10 @@ class CallBack : public minifi::OutputStreamCallback { } virtual ~CallBack() { } - virtual void process(std::ofstream *stream) { + virtual int64_t process(std::shared_ptr stream) { + // leaving the typo for posterity sake std::string st = "we're gnna write some test stuff"; - stream->write(st.c_str(), st.length()); + return stream->write(reinterpret_cast(const_cast(st.c_str())), st.length()); } }; @@ -270,16 +275,18 @@ TEST_CASE("HTTPTestsWithResourceClaimPOST", "[httptest1]") { uuid_t invokehttp_uuid; REQUIRE(true == invokehttp->getUUID(invokehttp_uuid)); - std::shared_ptr gcConnection = std::make_shared(repo, "getfileCreate2Connection"); + std::shared_ptr content_repo = std::make_shared(); + + std::shared_ptr gcConnection = std::make_shared(repo, content_repo, "getfileCreate2Connection"); gcConnection->setRelationship(core::Relationship("success", "description")); - std::shared_ptr laConnection = std::make_shared(repo, "logattribute"); + std::shared_ptr laConnection = std::make_shared(repo, content_repo, "logattribute"); laConnection->setRelationship(core::Relationship("success", "description")); - std::shared_ptr connection = std::make_shared(repo, "getfileCreate2Connection"); + std::shared_ptr connection = std::make_shared(repo, content_repo, "getfileCreate2Connection"); connection->setRelationship(core::Relationship("success", "description")); - std::shared_ptr connection2 = std::make_shared(repo, "listenhttp"); + std::shared_ptr connection2 = std::make_shared(repo, content_repo, "listenhttp"); connection2->setRelationship(core::Relationship("No Retry", "description")); @@ -317,14 +324,9 @@ TEST_CASE("HTTPTestsWithResourceClaimPOST", "[httptest1]") { CallBack callback; - /* - explicit FlowFileRecord(std::shared_ptr flow_repository, - std::map attributes, - std::shared_ptr claim = nullptr); - */ std::map attributes; attributes["testy"] = "test"; - std::shared_ptr flow = std::make_shared(repo, attributes); + std::shared_ptr flow = std::make_shared(repo, content_repo, attributes); session2.write(flow, &callback); invokehttp->incrementActiveTasks(); diff --git a/libminifi/test/unit/ProcessorTests.cpp b/libminifi/test/unit/ProcessorTests.cpp index 9e2d50c13d..ad15da24dc 100644 --- a/libminifi/test/unit/ProcessorTests.cpp +++ b/libminifi/test/unit/ProcessorTests.cpp @@ -44,6 +44,7 @@ TEST_CASE("Test Creation of GetFile", "[getfileCreate]") { TEST_CASE("Test Find file", "[getfileCreate2]") { TestController testController; + std::shared_ptr content_repo = std::make_shared(); std::shared_ptr processor = std::make_shared("getfileCreate2"); std::shared_ptr configure = std::make_shared(); @@ -61,7 +62,7 @@ TEST_CASE("Test Find file", "[getfileCreate2]") { uuid_t processoruuid; REQUIRE(true == processor->getUUID(processoruuid)); - std::shared_ptr connection = std::make_shared(test_repo, "getfileCreate2Connection"); + std::shared_ptr connection = std::make_shared(test_repo, content_repo, "getfileCreate2Connection"); connection->setRelationship(core::Relationship("success", "description")); // link the connections so that we can test results at the end for this @@ -149,11 +150,13 @@ TEST_CASE("Test Find file", "[getfileCreate2]") { std::shared_ptr taskReport = std::static_pointer_cast< org::apache::nifi::minifi::core::reporting::SiteToSiteProvenanceReportingTask>(processorReport); taskReport->setBatchSize(1); - std::vector> recordsReport; + std::vector> recordsReport; + recordsReport.push_back(std::make_shared()); processorReport->incrementActiveTasks(); processorReport->setScheduledState(core::ScheduledState::RUNNING); std::string jsonStr; - repo->getProvenanceRecord(recordsReport, 1); + std::size_t deserialized = 0; + repo->DeSerialize(recordsReport, deserialized); taskReport->getJsonReport(&contextReport, &sessionReport, recordsReport, jsonStr); REQUIRE(recordsReport.size() == 1); REQUIRE(taskReport->getName() == std::string(org::apache::nifi::minifi::core::reporting::SiteToSiteProvenanceReportingTask::ReportTaskName)); @@ -162,11 +165,10 @@ TEST_CASE("Test Find file", "[getfileCreate2]") { TEST_CASE("Test GetFileLikeIt'sThreaded", "[getfileCreate3]") { TestController testController; - + std::shared_ptr content_repo = std::make_shared(); std::shared_ptr processor = std::make_shared("getfileCreate2"); std::shared_ptr test_repo = std::make_shared(); - std::shared_ptr repo = std::static_pointer_cast(test_repo); char format[] = "/tmp/gt.XXXXXX"; @@ -175,7 +177,8 @@ TEST_CASE("Test GetFileLikeIt'sThreaded", "[getfileCreate3]") { uuid_t processoruuid; REQUIRE(true == processor->getUUID(processoruuid)); - std::shared_ptr connection = std::make_shared(test_repo, "getfileCreate2Connection"); + std::shared_ptr connection = std::make_shared(test_repo, content_repo, "getfileCreate2Connection"); + connection->setRelationship(core::Relationship("success", "description")); // link the connections so that we can test results at the end for this @@ -243,6 +246,7 @@ TEST_CASE("Test GetFileLikeIt'sThreaded", "[getfileCreate3]") { TEST_CASE("LogAttributeTest", "[getfileCreate3]") { TestController testController; + std::shared_ptr content_repo = std::make_shared(); LogTestController::getInstance().setDebug(); std::shared_ptr repo = std::make_shared(); @@ -260,10 +264,11 @@ TEST_CASE("LogAttributeTest", "[getfileCreate3]") { uuid_t logattribute_uuid; REQUIRE(true == logAttribute->getUUID(logattribute_uuid)); - std::shared_ptr connection = std::make_shared(repo, "getfileCreate2Connection"); + std::shared_ptr connection = std::make_shared(repo, content_repo, "getfileCreate2Connection"); connection->setRelationship(core::Relationship("success", "description")); - std::shared_ptr connection2 = std::make_shared(repo, "logattribute"); + std::shared_ptr connection2 = std::make_shared(repo, content_repo, "logattribute"); + connection2->setRelationship(core::Relationship("success", "description")); // link the connections so that we can test results at the end for this @@ -335,6 +340,7 @@ TEST_CASE("LogAttributeTest", "[getfileCreate3]") { records = reporter->getEvents(); + std::cout << LogTestController::getInstance().log_output.str() << std::endl; REQUIRE(true == LogTestController::getInstance().contains("key:absolute.path value:" + ss.str())); REQUIRE(true == LogTestController::getInstance().contains("Size:8 Offset:0")); REQUIRE(true == LogTestController::getInstance().contains("key:path value:" + std::string(dir))); diff --git a/libminifi/test/unit/ProvenanceTestHelper.h b/libminifi/test/unit/ProvenanceTestHelper.h index 17e6078a84..1b39700cdb 100644 --- a/libminifi/test/unit/ProvenanceTestHelper.h +++ b/libminifi/test/unit/ProvenanceTestHelper.h @@ -18,11 +18,22 @@ #ifndef LIBMINIFI_TEST_UNIT_PROVENANCETESTHELPER_H_ #define LIBMINIFI_TEST_UNIT_PROVENANCETESTHELPER_H_ -#include "provenance/Provenance.h" -#include "FlowController.h" -#include "core/Repository.h" -#include "core/repository/FlowFileRepository.h" -#include "core/Core.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include "core/repository/VolatileContentRepository.h" +#include "../../include/core/Processor.h" +#include "../../include/core/repository/FlowFileRepository.h" +#include "../../include/Connection.h" +#include "../../include/FlowController.h" +#include "../../include/properties/Configure.h" +#include "../../include/provenance/Provenance.h" + /** * Test repository */ @@ -41,17 +52,22 @@ class TestRepository : public core::Repository { } - bool Put(std::string key, uint8_t *buf, int bufLen) { + bool Put(std::string key, const uint8_t *buf, size_t bufLen) { repositoryResults.insert(std::pair(key, std::string((const char*) buf, bufLen))); return true; } + + virtual bool Serialize(const std::string &key, const uint8_t *buffer, const size_t bufferSize) { + return Put(key, buffer, bufferSize); + } + // Delete bool Delete(std::string key) { repositoryResults.erase(key); return true; } // Get - bool Get(std::string key, std::string &value) { + bool Get(const std::string &key, std::string &value) { auto result = repositoryResults.find(key); if (result != repositoryResults.end()) { value = result->second; @@ -61,6 +77,39 @@ class TestRepository : public core::Repository { } } + virtual bool Serialize(std::vector> &store, size_t max_size) { + return false; + } + + virtual bool DeSerialize(std::vector> &store, size_t &max_size) { + max_size = 0; + for (auto entry : repositoryResults) { + std::shared_ptr eventRead = store.at(max_size); + + if (eventRead->DeSerialize((uint8_t*) entry.second.data(), entry.second.length())) { + } + if (+max_size >= store.size()) { + break; + } + } + return true; + } + + virtual bool Serialize(const std::shared_ptr &store) { + return false; + } + + virtual bool DeSerialize(const std::shared_ptr &store) { + std::string value; + Get(store->getUUIDStr(), value); + store->DeSerialize(reinterpret_cast(const_cast(value.c_str())), value.size()); + return true; + } + + virtual bool DeSerialize(const uint8_t *buffer, const size_t bufferSize) { + return false; + } + const std::map &getRepoMap() const { return repositoryResults; } @@ -134,6 +183,9 @@ class TestFlowRepository : public core::repository::FlowFileRepository { } } } + + void loadComponent(const std::shared_ptr &content_repo) { + } void run() { // do nothing @@ -145,8 +197,8 @@ class TestFlowRepository : public core::repository::FlowFileRepository { class TestFlowController : public minifi::FlowController { public: - TestFlowController(std::shared_ptr repo, std::shared_ptr flow_file_repo) - : minifi::FlowController(repo, flow_file_repo, std::make_shared(), nullptr, "", true) { + TestFlowController(std::shared_ptr repo, std::shared_ptr flow_file_repo, std::shared_ptr content_repo) + : minifi::FlowController(repo, flow_file_repo,std::make_shared(), nullptr, std::make_shared(), "", true) { } ~TestFlowController() { diff --git a/libminifi/test/unit/ProvenanceTests.cpp b/libminifi/test/unit/ProvenanceTests.cpp index 6a585973f3..97cb6467d6 100644 --- a/libminifi/test/unit/ProvenanceTests.cpp +++ b/libminifi/test/unit/ProvenanceTests.cpp @@ -26,8 +26,8 @@ #include "provenance/Provenance.h" #include "FlowFileRecord.h" #include "core/Core.h" -#include "core/repository/FlowFileRepository.h" -#include "core/repository/VolatileRepository.h" +#include "core/repository/AtomicRepoEntries.h" +#include "core/repository/VolatileProvenanceRepository.h" TEST_CASE("Test Provenance record create", "[Testprovenance::ProvenanceEventRecord]") { provenance::ProvenanceEventRecord record1(provenance::ProvenanceEventRecord::ProvenanceEventType::CREATE, "blah", "blahblah"); @@ -49,7 +49,8 @@ TEST_CASE("Test Provenance record serialization", "[Testprovenance::ProvenanceEv record1.Serialize(testRepository); provenance::ProvenanceEventRecord record2; - REQUIRE(record2.DeSerialize(testRepository, eventId) == true); + record2.setEventId(eventId); + REQUIRE(record2.DeSerialize(testRepository) == true); REQUIRE(record2.getEventId() == record1.getEventId()); REQUIRE(record2.getComponentId() == record1.getComponentId()); REQUIRE(record2.getComponentType() == record1.getComponentType()); @@ -60,12 +61,13 @@ TEST_CASE("Test Provenance record serialization", "[Testprovenance::ProvenanceEv TEST_CASE("Test Flowfile record added to provenance", "[TestFlowAndProv1]") { provenance::ProvenanceEventRecord record1(provenance::ProvenanceEventRecord::ProvenanceEventType::CLONE, "componentid", "componenttype"); + std::shared_ptr content_repo = std::make_shared(); std::string eventId = record1.getEventId(); std::map attributes; attributes.insert(std::pair("potato", "potatoe")); attributes.insert(std::pair("tomato", "tomatoe")); std::shared_ptr frepo = std::make_shared("ff", "./content_repository", 0, 0, 0); - std::shared_ptr ffr1 = std::make_shared(frepo, attributes); + std::shared_ptr ffr1 = std::make_shared(frepo, content_repo, attributes); record1.addChildFlowFile(ffr1); @@ -75,7 +77,8 @@ TEST_CASE("Test Flowfile record added to provenance", "[TestFlowAndProv1]") { record1.Serialize(testRepository); provenance::ProvenanceEventRecord record2; - REQUIRE(record2.DeSerialize(testRepository, eventId) == true); + record2.setEventId(eventId); + REQUIRE(record2.DeSerialize(testRepository) == true); REQUIRE(record1.getChildrenUuids().size() == 1); REQUIRE(record2.getChildrenUuids().size() == 1); std::string childId = record2.getChildrenUuids().at(0); @@ -94,13 +97,14 @@ TEST_CASE("Test Provenance record serialization Volatile", "[Testprovenance::Pro uint64_t sample = 65555; - std::shared_ptr testRepository = std::make_shared(); + std::shared_ptr testRepository = std::make_shared(); testRepository->initialize(0); record1.setEventDuration(sample); record1.Serialize(testRepository); provenance::ProvenanceEventRecord record2; - REQUIRE(record2.DeSerialize(testRepository, eventId) == true); + record2.setEventId(eventId); + REQUIRE(record2.DeSerialize(testRepository) == true); REQUIRE(record2.getEventId() == record1.getEventId()); REQUIRE(record2.getComponentId() == record1.getComponentId()); REQUIRE(record2.getComponentType() == record1.getComponentType()); @@ -111,24 +115,26 @@ TEST_CASE("Test Provenance record serialization Volatile", "[Testprovenance::Pro TEST_CASE("Test Flowfile record added to provenance using Volatile Repo", "[TestFlowAndProv1]") { provenance::ProvenanceEventRecord record1(provenance::ProvenanceEventRecord::ProvenanceEventType::CLONE, "componentid", "componenttype"); + std::shared_ptr content_repo = std::make_shared(); std::string eventId = record1.getEventId(); std::map attributes; attributes.insert(std::pair("potato", "potatoe")); attributes.insert(std::pair("tomato", "tomatoe")); - std::shared_ptr frepo = std::make_shared(); + std::shared_ptr frepo = std::make_shared(); frepo->initialize(0); - std::shared_ptr ffr1 = std::make_shared(frepo, attributes); + std::shared_ptr ffr1 = std::make_shared(frepo, content_repo, attributes); record1.addChildFlowFile(ffr1); uint64_t sample = 65555; - std::shared_ptr testRepository = std::make_shared(); + std::shared_ptr testRepository = std::make_shared(); testRepository->initialize(0); record1.setEventDuration(sample); record1.Serialize(testRepository); provenance::ProvenanceEventRecord record2; - REQUIRE(record2.DeSerialize(testRepository, eventId) == true); + record2.setEventId(eventId); + REQUIRE(record2.DeSerialize(testRepository) == true); REQUIRE(record1.getChildrenUuids().size() == 1); REQUIRE(record2.getChildrenUuids().size() == 1); std::string childId = record2.getChildrenUuids().at(0); @@ -151,7 +157,8 @@ TEST_CASE("Test Provenance record serialization NoOp", "[Testprovenance::Provena testRepository->initialize(0); record1.setEventDuration(sample); - record1.Serialize(testRepository); + REQUIRE(record1.Serialize(testRepository) == true); provenance::ProvenanceEventRecord record2; - REQUIRE(record2.DeSerialize(testRepository, eventId) == false); + record2.setEventId(eventId); + REQUIRE(record2.DeSerialize(testRepository) == false); } diff --git a/libminifi/test/unit/RepoTests.cpp b/libminifi/test/unit/RepoTests.cpp index 4424a934b3..3b18310f9e 100644 --- a/libminifi/test/unit/RepoTests.cpp +++ b/libminifi/test/unit/RepoTests.cpp @@ -23,7 +23,7 @@ #include "provenance/Provenance.h" #include "FlowFileRecord.h" #include "core/Core.h" -#include "core/repository/FlowFileRepository.h" +#include "../../include/core/repository/AtomicRepoEntries.h" #include "properties/Configure.h" TEST_CASE("Test Repo Empty Value Attribute", "[TestFFR1]") { @@ -34,7 +34,8 @@ TEST_CASE("Test Repo Empty Value Attribute", "[TestFFR1]") { repository->initialize(std::make_shared()); - minifi::FlowFileRecord record(repository); + std::shared_ptr content_repo = std::make_shared(); + minifi::FlowFileRecord record(repository, content_repo); record.addAttribute("keyA", ""); @@ -50,8 +51,8 @@ TEST_CASE("Test Repo Empty Key Attribute ", "[TestFFR2]") { std::shared_ptr repository = std::make_shared("ff", dir, 0, 0, 1); repository->initialize(std::make_shared()); - - minifi::FlowFileRecord record(repository); + std::shared_ptr content_repo = std::make_shared(); + minifi::FlowFileRecord record(repository, content_repo); record.addAttribute("keyA", "hasdgasdgjsdgasgdsgsadaskgasd"); @@ -70,9 +71,10 @@ TEST_CASE("Test Repo Key Attribute Verify ", "[TestFFR3]") { repository->initialize(std::make_shared()); - minifi::FlowFileRecord record(repository); + std::shared_ptr content_repo = std::make_shared(); + minifi::FlowFileRecord record(repository, content_repo); - minifi::FlowFileRecord record2(repository); + minifi::FlowFileRecord record2(repository, content_repo); std::string uuid = record.getUUIDStr(); diff --git a/libminifi/test/unit/TailFileTests.cpp b/libminifi/test/unit/TailFileTests.cpp index e800b4ce87..616505ba9b 100644 --- a/libminifi/test/unit/TailFileTests.cpp +++ b/libminifi/test/unit/TailFileTests.cpp @@ -62,7 +62,9 @@ TEST_CASE("TailFileWithDelimiter", "[tailfiletest1]") { uuid_t logAttributeuuid; REQUIRE(true == logAttributeProcessor->getUUID(logAttributeuuid)); - std::shared_ptr connection = std::make_shared(repo, "logattributeconnection"); + std::shared_ptr content_repo = std::make_shared(); + + std::shared_ptr connection = std::make_shared(repo,content_repo, "logattributeconnection"); connection->setRelationship(core::Relationship("success", "TailFile successful output")); // link the connections so that we can test results at the end for this @@ -128,7 +130,9 @@ TEST_CASE("TailFileWithoutDelimiter", "[tailfiletest2]") { uuid_t logAttributeuuid; REQUIRE(true == logAttributeProcessor->getUUID(logAttributeuuid)); - std::shared_ptr connection = std::make_shared(repo, "logattributeconnection"); + std::shared_ptr content_repo = std::make_shared(); + + std::shared_ptr connection = std::make_shared(repo, content_repo, "logattributeconnection"); connection->setRelationship(core::Relationship("success", "TailFile successful output")); // link the connections so that we can test results at the end for this diff --git a/libminifi/test/unit/YamlConfigurationTests.cpp b/libminifi/test/unit/YamlConfigurationTests.cpp index ba73a346ad..fc11d04a73 100644 --- a/libminifi/test/unit/YamlConfigurationTests.cpp +++ b/libminifi/test/unit/YamlConfigurationTests.cpp @@ -20,6 +20,7 @@ #include #include #include +#include "core/repository/VolatileContentRepository.h" #include #include "core/yaml/YamlConfiguration.h" #include "../TestBase.h" @@ -30,8 +31,9 @@ TEST_CASE("Test YAML Config Processing", "[YamlConfiguration]") { std::shared_ptr testProvRepo = core::createRepository("provenancerepository", true); std::shared_ptr testFlowFileRepo = core::createRepository("flowfilerepository", true); std::shared_ptr configuration = std::make_shared(); - std::shared_ptr streamFactory = std::make_shared(configuration); - core::YamlConfiguration *yamlConfig = new core::YamlConfiguration(testProvRepo, testFlowFileRepo, streamFactory, configuration); + std::shared_ptr streamFactory = std::make_shared < minifi::io::StreamFactory > (configuration); + std::shared_ptr content_repo = std::make_shared(); + core::YamlConfiguration *yamlConfig = new core::YamlConfiguration(testProvRepo, testFlowFileRepo, content_repo, streamFactory, configuration); SECTION("loading YAML without optional component IDs works") { static const std::string CONFIG_YAML_WITHOUT_IDS = "" diff --git a/main/MiNiFiMain.cpp b/main/MiNiFiMain.cpp index ba59623c5d..3afa06e565 100644 --- a/main/MiNiFiMain.cpp +++ b/main/MiNiFiMain.cpp @@ -90,6 +90,7 @@ int main(int argc, char **argv) { std::string prov_repo_class = "provenancerepository"; std::string flow_repo_class = "flowfilerepository"; std::string nifi_configuration_class_name = "yamlconfiguration"; + std::string content_repo_class = "filesystemrepository"; running = sem_open("MiNiFiMain", O_CREAT, 0644, 0); if (running == SEM_FAILED || running == 0) { @@ -108,14 +109,11 @@ int main(int argc, char **argv) { char full_path[PATH_MAX]; path = realpath(argv[0], full_path); std::string minifiHomePath(path); - minifiHomePath = minifiHomePath.substr(0, - minifiHomePath.find_last_of("/\\")); //Remove /minifi from path + minifiHomePath = minifiHomePath.substr(0, minifiHomePath.find_last_of("/\\")); //Remove /minifi from path minifiHome = minifiHomePath.substr(0, minifiHomePath.find_last_of("/\\")); //Remove /bin from path } - if (signal(SIGINT, sigHandler) == SIG_ERR - || signal(SIGTERM, sigHandler) == SIG_ERR - || signal(SIGPIPE, SIG_IGN) == SIG_ERR) { + if (signal(SIGINT, sigHandler) == SIG_ERR || signal(SIGTERM, sigHandler) == SIG_ERR || signal(SIGPIPE, SIG_IGN) == SIG_ERR) { logger->log_error("Can not install signal handler"); return -1; } @@ -132,56 +130,49 @@ int main(int argc, char **argv) { // Make a record of minifi home in the configured log file. logger->log_info("MINIFI_HOME=%s", minifiHome); - + std::shared_ptr configure = std::make_shared(); configure->setHome(minifiHome); configure->loadConfigureFile(DEFAULT_NIFI_PROPERTIES_FILE); - if (configure->get(minifi::Configure::nifi_graceful_shutdown_seconds, - graceful_shutdown_seconds)) { + if (configure->get(minifi::Configure::nifi_graceful_shutdown_seconds, graceful_shutdown_seconds)) { try { stop_wait_time = std::stoi(graceful_shutdown_seconds); } catch (const std::out_of_range &e) { - logger->log_error("%s is out of range. %s", - minifi::Configure::nifi_graceful_shutdown_seconds, - e.what()); + logger->log_error("%s is out of range. %s", minifi::Configure::nifi_graceful_shutdown_seconds, e.what()); } catch (const std::invalid_argument &e) { - logger->log_error("%s contains an invalid argument set. %s", - minifi::Configure::nifi_graceful_shutdown_seconds, - e.what()); + logger->log_error("%s contains an invalid argument set. %s", minifi::Configure::nifi_graceful_shutdown_seconds, e.what()); } } else { - logger->log_debug("%s not set, defaulting to %d", - minifi::Configure::nifi_graceful_shutdown_seconds, - STOP_WAIT_TIME_MS); + logger->log_debug("%s not set, defaulting to %d", minifi::Configure::nifi_graceful_shutdown_seconds, + STOP_WAIT_TIME_MS); } - configure->get(minifi::Configure::nifi_provenance_repository_class_name, - prov_repo_class); + configure->get(minifi::Configure::nifi_provenance_repository_class_name, prov_repo_class); // Create repos for flow record and provenance - std::shared_ptr prov_repo = core::createRepository( - prov_repo_class, true,"provenance"); + std::shared_ptr prov_repo = core::createRepository(prov_repo_class, true, "provenance"); prov_repo->initialize(configure); - configure->get(minifi::Configure::nifi_flow_repository_class_name, - flow_repo_class); + configure->get(minifi::Configure::nifi_flow_repository_class_name, flow_repo_class); - std::shared_ptr flow_repo = core::createRepository( - flow_repo_class, true, "flowfile"); + std::shared_ptr flow_repo = core::createRepository(flow_repo_class, true, "flowfile"); flow_repo->initialize(configure); - configure->get(minifi::Configure::nifi_configuration_class_name, - nifi_configuration_class_name); + configure->get(minifi::Configure::nifi_content_repository_class_name, content_repo_class); + + std::shared_ptr content_repo = core::createContentRepository(content_repo_class, true, "content"); + + content_repo->initialize(configure); + + configure->get(minifi::Configure::nifi_configuration_class_name, nifi_configuration_class_name); + std::shared_ptr stream_factory = std::make_shared(configure); - std::unique_ptr flow_configuration = std::move( - core::createFlowConfiguration(prov_repo, flow_repo, configure, stream_factory, - nifi_configuration_class_name)); + std::unique_ptr flow_configuration = std::move(core::createFlowConfiguration(prov_repo, flow_repo, content_repo, configure, stream_factory, nifi_configuration_class_name)); std::shared_ptr controller = std::unique_ptr( - new minifi::FlowController(prov_repo, flow_repo, configure, - std::move(flow_configuration))); + new minifi::FlowController(prov_repo, flow_repo, configure, std::move(flow_configuration), content_repo)); logger->log_info("Loading FlowController"); // Load flow from specified configuration file @@ -205,9 +196,9 @@ int main(int argc, char **argv) { * Trigger unload -- wait stop_wait_time */ controller->waitUnload(stop_wait_time); - + flow_repo = nullptr; - + prov_repo = nullptr; logger->log_info("MiNiFi exit"); From 70c9ef421a2034372df6a0e6b0f0ffe520d68262 Mon Sep 17 00:00:00 2001 From: Marc Parisi Date: Tue, 13 Jun 2017 12:32:15 -0400 Subject: [PATCH 2/6] MINIFI-249: Update Tests --- cmake/BuildTests.cmake | 4 +- .../core/repository/FlowFileRepository.h | 1 - libminifi/include/io/AtomicEntryStream.h | 2 - libminifi/src/Connection.cpp | 15 +- libminifi/src/core/ProcessSession.cpp | 10 +- libminifi/src/io/FileStream.cpp | 1 - libminifi/src/processors/LogAttribute.cpp | 3 +- libminifi/test/CPPLINT.cfg | 1 + libminifi/test/TestBase.cpp | 211 +++++++++++++ libminifi/test/TestBase.h | 102 ++++++- libminifi/test/unit/InvokeHTTPTests.cpp | 135 +++------ libminifi/test/unit/ProcessorTests.cpp | 286 ++++++------------ 12 files changed, 460 insertions(+), 311 deletions(-) create mode 100644 libminifi/test/TestBase.cpp diff --git a/cmake/BuildTests.cmake b/cmake/BuildTests.cmake index 59f1d59ba2..9500792eae 100644 --- a/cmake/BuildTests.cmake +++ b/cmake/BuildTests.cmake @@ -67,7 +67,7 @@ GETSOURCEFILES(INTEGRATION_TESTS "${TEST_DIR}/integration/") SET(UNIT_TEST_COUNT 0) FOREACH(testfile ${UNIT_TESTS}) get_filename_component(testfilename "${testfile}" NAME_WE) - add_executable("${testfilename}" "${TEST_DIR}/unit/${testfile}" ${SPD_SOURCES}) + add_executable("${testfilename}" "${TEST_DIR}/unit/${testfile}" ${SPD_SOURCES} "${TEST_DIR}/TestBase.cpp") createTests("${testfilename}") MATH(EXPR UNIT_TEST_COUNT "${UNIT_TEST_COUNT}+1") add_test(NAME "${testfilename}" COMMAND "${testfilename}" WORKING_DIRECTORY ${TEST_DIR}) @@ -77,7 +77,7 @@ message("-- Finished building ${UNIT_TEST_COUNT} unit test file(s)...") SET(INT_TEST_COUNT 0) FOREACH(testfile ${INTEGRATION_TESTS}) get_filename_component(testfilename "${testfile}" NAME_WE) - add_executable("${testfilename}" "${TEST_DIR}/integration/${testfile}" ${SPD_SOURCES}) + add_executable("${testfilename}" "${TEST_DIR}/integration/${testfile}" ${SPD_SOURCES} "${TEST_DIR}/TestBase.cpp") createTests("${testfilename}") #message("Adding ${testfilename} from ${testfile}") MATH(EXPR INT_TEST_COUNT "${INT_TEST_COUNT}+1") diff --git a/libminifi/include/core/repository/FlowFileRepository.h b/libminifi/include/core/repository/FlowFileRepository.h index 6022e654b7..d3f1a34af9 100644 --- a/libminifi/include/core/repository/FlowFileRepository.h +++ b/libminifi/include/core/repository/FlowFileRepository.h @@ -64,7 +64,6 @@ class FlowFileRepository : public core::Repository, public std::enable_shared_fr // initialize virtual bool initialize(const std::shared_ptr &configure) { std::string value; - std::cout << "initialize " << std::endl; if (configure->get(Configure::nifi_flowfile_repository_directory_default, value)) { directory_ = value; diff --git a/libminifi/include/io/AtomicEntryStream.h b/libminifi/include/io/AtomicEntryStream.h index 48322a42fb..1171e9fc7c 100644 --- a/libminifi/include/io/AtomicEntryStream.h +++ b/libminifi/include/io/AtomicEntryStream.h @@ -42,7 +42,6 @@ class AtomicEntryStream : public BaseStream { core::repository::RepoValue *value; if (entry_->getValue(key, &value)) { length_ = value->getBufferSize(); - std::cout << "Length is " << length_ << std::endl; } else { throw Exception(FILE_OPERATION_EXCEPTION, "Could not create valid entry"); } @@ -59,7 +58,6 @@ class AtomicEntryStream : public BaseStream { void seek(uint64_t offset); virtual const uint32_t getSize() const { - std::cout << "Length is " << length_ << std::endl; return length_; } diff --git a/libminifi/src/Connection.cpp b/libminifi/src/Connection.cpp index 841d941a7f..37e4652401 100644 --- a/libminifi/src/Connection.cpp +++ b/libminifi/src/Connection.cpp @@ -62,13 +62,13 @@ Connection::Connection(const std::shared_ptr &flow_repository, } bool Connection::isEmpty() { - std::lock_guard < std::mutex > lock(mutex_); + std::lock_guard lock(mutex_); return queue_.empty(); } bool Connection::isFull() { - std::lock_guard < std::mutex > lock(mutex_); + std::lock_guard lock(mutex_); if (max_queue_size_ <= 0 && max_data_queue_size_ <= 0) // No back pressure setting @@ -85,7 +85,7 @@ bool Connection::isFull() { void Connection::put(std::shared_ptr flow) { { - std::lock_guard < std::mutex > lock(mutex_); + std::lock_guard lock(mutex_); queue_.push(flow); @@ -104,12 +104,13 @@ void Connection::put(std::shared_ptr flow) { // Notify receiving processor that work may be available if (dest_connectable_) { + logger_->log_debug("Notifying %s", dest_connectable_->getName()); dest_connectable_->notifyWork(); } } std::shared_ptr Connection::poll(std::set> &expiredFlowRecords) { - std::lock_guard < std::mutex > lock(mutex_); + std::lock_guard lock(mutex_); while (!queue_.empty()) { std::shared_ptr item = queue_.front(); @@ -132,7 +133,7 @@ std::shared_ptr Connection::poll(std::setgetSize(); break; } - std::shared_ptr connectable = std::static_pointer_cast < Connectable > (shared_from_this()); + std::shared_ptr connectable = std::static_pointer_cast(shared_from_this()); item->setOriginalConnection(connectable); logger_->log_debug("Dequeue flow file UUID %s from connection %s", item->getUUIDStr().c_str(), name_.c_str()); @@ -151,7 +152,7 @@ std::shared_ptr Connection::poll(std::setgetSize(); break; } - std::shared_ptr connectable = std::static_pointer_cast < Connectable > (shared_from_this()); + std::shared_ptr connectable = std::static_pointer_cast(shared_from_this()); item->setOriginalConnection(connectable); logger_->log_debug("Dequeue flow file UUID %s from connection %s", item->getUUIDStr().c_str(), name_.c_str()); // delete from the flowfile repo @@ -167,7 +168,7 @@ std::shared_ptr Connection::poll(std::set lock(mutex_); + std::lock_guard lock(mutex_); while (!queue_.empty()) { auto &&item = queue_.front(); diff --git a/libminifi/src/core/ProcessSession.cpp b/libminifi/src/core/ProcessSession.cpp index 0ea1ae3541..680bcd202a 100644 --- a/libminifi/src/core/ProcessSession.cpp +++ b/libminifi/src/core/ProcessSession.cpp @@ -564,7 +564,6 @@ void ProcessSession::import(std::string source, std::shared_ptr } if (!invalidWrite) { - std::cout << "Import " << stream->getSize() << std::endl; flow->setSize(stream->getSize()); flow->setOffset(0); if (flow->getResourceClaim() != nullptr) { @@ -749,7 +748,6 @@ void ProcessSession::import(std::string source, std::shared_ptr } } if (!invalidWrite) { - flow->setSize(stream->getSize()); flow->setOffset(0); if (flow->getResourceClaim() != nullptr) { @@ -845,13 +843,14 @@ void ProcessSession::commit() { } } - // Do the samething for added flow file + // Do the same thing for added flow file for (const auto it : _addedFlowFiles) { std::shared_ptr record = it.second; if (record->isDeleted()) continue; std::map::iterator itRelationship = this->_transferRelationship.find(record->getUUIDStr()); if (itRelationship != _transferRelationship.end()) { + logger_->log_debug("size is %d for there", _addedFlowFiles.size()); Relationship relationship = itRelationship->second; // Find the relationship, we need to find the connections for that relationship std::set> connections = process_context_->getProcessorNode().getOutGoingConnections(relationship.getName()); @@ -862,17 +861,21 @@ void ProcessSession::commit() { std::string message = "Connect empty for non auto terminated relationship " + relationship.getName(); throw Exception(PROCESS_SESSION_EXCEPTION, message.c_str()); } else { + logger_->log_debug("added flow file is auto terminated"); // Autoterminated remove(record); } } else { + logger_->log_debug("size is %d for here", _addedFlowFiles.size()); // We connections, clone the flow and assign the connection accordingly for (std::set>::iterator itConnection = connections.begin(); itConnection != connections.end(); ++itConnection) { std::shared_ptr connection(*itConnection); + logger_->log_debug("size is %d for here %s", _addedFlowFiles.size(), connection->getName()); if (itConnection == connections.begin()) { // First connection which the flow need be routed to record->setConnection(connection); } else { + logger_->log_debug("clone is %d for here %s", _addedFlowFiles.size(), connection->getName()); // Clone the flow file and route to the connection std::shared_ptr cloneRecord; cloneRecord = this->cloneDuringTransfer(record); @@ -972,6 +975,7 @@ std::shared_ptr ProcessSession::get() { std::shared_ptr first = process_context_->getProcessorNode().getNextIncomingConnection(); if (first == NULL) { + logger_->log_debug("Get is null for %s", process_context_->getProcessorNode().getName()); return NULL; } diff --git a/libminifi/src/io/FileStream.cpp b/libminifi/src/io/FileStream.cpp index d22b06aac3..721c5f486e 100644 --- a/libminifi/src/io/FileStream.cpp +++ b/libminifi/src/io/FileStream.cpp @@ -40,7 +40,6 @@ FileStream::FileStream(const std::string &path, uint32_t offset) } else { length_ = 0; } - std::cout << "length is " << length_ << std::endl; seek(offset); } diff --git a/libminifi/src/processors/LogAttribute.cpp b/libminifi/src/processors/LogAttribute.cpp index cddcbbe468..ad8e664282 100644 --- a/libminifi/src/processors/LogAttribute.cpp +++ b/libminifi/src/processors/LogAttribute.cpp @@ -71,8 +71,9 @@ void LogAttribute::onTrigger(core::ProcessContext *context, core::ProcessSession std::shared_ptr flow = session->get(); - if (!flow) + if (!flow) { return; + } std::string value; if (context->getProperty(LogLevel.getName(), value)) { diff --git a/libminifi/test/CPPLINT.cfg b/libminifi/test/CPPLINT.cfg index beed48affa..a1e22ada37 100644 --- a/libminifi/test/CPPLINT.cfg +++ b/libminifi/test/CPPLINT.cfg @@ -1,3 +1,4 @@ set noparent filter=-build/include_order,-build/include_alpha exclude_files=Server.cpp +exclude_files=TestBase.cpp \ No newline at end of file diff --git a/libminifi/test/TestBase.cpp b/libminifi/test/TestBase.cpp new file mode 100644 index 0000000000..c773299d48 --- /dev/null +++ b/libminifi/test/TestBase.cpp @@ -0,0 +1,211 @@ +/** + * + * 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 +#include +#include +#include + +TestPlan::TestPlan(std::shared_ptr content_repo, std::shared_ptr flow_repo, std::shared_ptr prov_repo) + : + content_repo_(content_repo), + flow_repo_(flow_repo), + prov_repo_(prov_repo), + location(-1), + finalized(false), + current_flowfile_(nullptr) { +} + +std::shared_ptr TestPlan::addProcessor(const std::shared_ptr &processor, const std::string &name, core::Relationship relationship, +bool linkToPrevious) { + if (finalized) { + return nullptr; + } + std::lock_guard guard(mutex); + + uuid_t uuid; + uuid_generate(uuid); + + // initialize the processor + processor->initialize(); + + processor_mapping_[processor->getUUIDStr()] = processor; + + if (!linkToPrevious) { + termination_ = relationship; + } else { + std::shared_ptr last = processor_queue_.back(); + + if (last == nullptr) { + last = processor; + termination_ = relationship; + } + + std::stringstream connection_name; + connection_name << last->getUUIDStr() << "-to-" << processor->getUUIDStr(); + std::shared_ptr connection = std::make_shared(flow_repo_, content_repo_, connection_name.str()); + connection->setRelationship(relationship); + + // link the connections so that we can test results at the end for this + connection->setSource(last); + connection->setDestination(processor); + + uuid_t uuid_copy, uuid_copy_next; + last->getUUID(uuid_copy); + connection->setSourceUUID(uuid_copy); + processor->getUUID(uuid_copy_next); + connection->setDestinationUUID(uuid_copy_next); + last->addConnection(connection); + if (last != processor) { + processor->addConnection(connection); + } + relationships_.push_back(connection); + } + + std::shared_ptr node = std::make_shared(processor); + + processor_nodes_.push_back(node); + + std::shared_ptr context = std::make_shared(*(node.get()), controller_services_provider_, flow_repo_, content_repo_); + processor_contexts_.push_back(context); + + processor_queue_.push_back(processor); + + return processor; +} + +std::shared_ptr TestPlan::addProcessor(const std::string &processor_name, const std::string &name, core::Relationship relationship, +bool linkToPrevious) { + if (finalized) { + return nullptr; + } + std::lock_guard guard(mutex); + + uuid_t uuid; + uuid_generate(uuid); + + auto ptr = core::ClassLoader::getDefaultClassLoader().instantiate(processor_name, uuid); + if (nullptr == ptr) { + throw std::exception(); + } + std::shared_ptr processor = std::static_pointer_cast(ptr); + + processor->setName(name); + + return addProcessor(processor, name, relationship, linkToPrevious); +} + +bool TestPlan::setProperty(const std::shared_ptr proc, const std::string &prop, const std::string &value) { + std::lock_guard guard(mutex); + int i = 0; + for (i = 0; i < processor_queue_.size(); i++) { + if (processor_queue_.at(i) == proc) { + break; + } + } + if (i >= processor_queue_.size() || i < 0 || i >= processor_contexts_.size()) { + return false; + } + + return processor_contexts_.at(i)->setProperty(prop, value); +} + +void TestPlan::reset() { + std::lock_guard guard(mutex); + process_sessions_.clear(); + factories_.clear(); + location = -1; + for (auto proc : processor_queue_) { + while (proc->getActiveTasks() > 0) { + proc->decrementActiveTask(); + } + } +} + +bool TestPlan::runNextProcessor(std::function verify) { + if (!finalized) { + finalize(); + } + std::lock_guard guard(mutex); + location++; + std::shared_ptr processor = processor_queue_.at(location); + std::shared_ptr context = processor_contexts_.at(location); + std::shared_ptr factory = std::make_shared(context.get()); + factories_.push_back(factory); + if (std::find(configured_processors_.begin(), configured_processors_.end(), processor) == configured_processors_.end()) { + processor->onSchedule(context.get(), factory.get()); + configured_processors_.push_back(processor); + } + std::shared_ptr current_session = std::make_shared(context.get()); + process_sessions_.push_back(current_session); + processor->incrementActiveTasks(); + processor->setScheduledState(core::ScheduledState::RUNNING); + if (verify != nullptr) { + verify(context.get(), current_session.get()); + } else { + processor->onTrigger(context.get(), current_session.get()); + } + current_session->commit(); + current_flowfile_ = current_session->get(); + return location + 1 < processor_queue_.size(); +} + +std::set TestPlan::getProvenanceRecords() { + return process_sessions_.at(location)->getProvenanceReporter()->getEvents(); +} + +std::shared_ptr TestPlan::getCurrentFlowFile() { + return current_flowfile_; +} + +std::shared_ptr TestPlan::buildFinalConnection(std::shared_ptr processor, bool setDest) { + std::stringstream connection_name; + std::shared_ptr last = processor; + connection_name << last->getUUIDStr() << "-to-" << processor->getUUIDStr(); + std::shared_ptr connection = std::make_shared(flow_repo_, content_repo_, connection_name.str()); + connection->setRelationship(termination_); + + // link the connections so that we can test results at the end for this + connection->setSource(last); + if (setDest) + connection->setDestination(processor); + + uuid_t uuid_copy; + last->getUUID(uuid_copy); + connection->setSourceUUID(uuid_copy); + if (setDest) + connection->setDestinationUUID(uuid_copy); + + processor->addConnection(connection); + return connection; +} + +void TestPlan::finalize() { + std::lock_guard guard(mutex); + if (relationships_.size() > 0) { + relationships_.push_back(buildFinalConnection(processor_queue_.back())); + } else { + for (auto processor : processor_queue_) { + relationships_.push_back(buildFinalConnection(processor, true)); + } + } + + finalized = true; +} + diff --git a/libminifi/test/TestBase.h b/libminifi/test/TestBase.h index 331df088d2..47db4c3e1f 100644 --- a/libminifi/test/TestBase.h +++ b/libminifi/test/TestBase.h @@ -25,6 +25,8 @@ #include "ResourceClaim.h" #include "catch.hpp" #include +#include +#include #include "core/logging/Logger.h" #include "core/Core.h" #include "properties/Configure.h" @@ -33,6 +35,14 @@ #include "utils/Id.h" #include "spdlog/sinks/ostream_sink.h" #include "spdlog/sinks/dist_sink.h" +#include "unit/ProvenanceTestHelper.h" +#include "core/Core.h" +#include "core/FlowFile.h" +#include "core/Processor.h" +#include "core/ProcessContext.h" +#include "core/ProcessSession.h" +#include "core/ProcessorNode.h" +#include "core/reporting/SiteToSiteProvenanceReportingTask.h" class LogTestController { public: @@ -105,7 +115,7 @@ class LogTestController { std::ostringstream log_output; std::shared_ptr logger_; - private: + private: class TestBootstrapLogger : public logging::Logger { public: TestBootstrapLogger(std::shared_ptr logger) @@ -138,6 +148,73 @@ class LogTestController { std::vector modified_loggers; }; +class TestPlan { + public: + + explicit TestPlan(std::shared_ptr content_repo, std::shared_ptr flow_repo, std::shared_ptr prov_repo); + + std::shared_ptr addProcessor(const std::shared_ptr &processor, const std::string &name, + core::Relationship relationship = core::Relationship("success", "description"), + bool linkToPrevious = false); + + std::shared_ptr addProcessor(const std::string &processor_name, const std::string &name, core::Relationship relationship = core::Relationship("success", "description"), + bool linkToPrevious = false); + + bool setProperty(const std::shared_ptr proc, const std::string &prop, const std::string &value); + + void reset(); + + bool runNextProcessor(std::function verify = nullptr); + + std::set getProvenanceRecords(); + + std::shared_ptr getCurrentFlowFile(); + + std::shared_ptr getFlowRepo() { + return flow_repo_; + } + + std::shared_ptr getProvenanceRepo() { + return prov_repo_; + } + + std::shared_ptr getContentRepo() { + return content_repo_; + } + + protected: + + void finalize(); + + std::shared_ptr buildFinalConnection(std::shared_ptr processor, bool setDest = false); + + std::atomic finalized; + + std::shared_ptr content_repo_; + + std::shared_ptr flow_repo_; + std::shared_ptr prov_repo_; + + std::shared_ptr controller_services_provider_; + + std::recursive_mutex mutex; + + int location; + + std::shared_ptr current_session_; + std::shared_ptr current_flowfile_; + + std::map> processor_mapping_; + std::vector> processor_queue_; + std::vector> configured_processors_; + std::vector> processor_nodes_; + std::vector> processor_contexts_; + std::vector> process_sessions_; + std::vector> factories_; + std::vector> relationships_; + core::Relationship termination_; +}; + class TestController { public: @@ -148,6 +225,25 @@ class TestController { utils::IdGenerator::getIdGenerator()->initialize(std::make_shared()); } + std::shared_ptr createPlan() + { + std::shared_ptr configuration = std::make_shared(); + std::shared_ptr content_repo = std::make_shared(); + + content_repo->initialize(configuration); + + std::shared_ptr repo = std::make_shared(); + return std::make_shared(content_repo, repo, repo); + } + + void runSession(std::shared_ptr &plan, bool runToCompletion = true, std::function verify = nullptr) { + + while (plan->runNextProcessor(verify) && runToCompletion) + { + + } + } + ~TestController() { for (auto dir : directories) { DIR *created_dir; @@ -176,6 +272,10 @@ class TestController { } protected: + + std::mutex test_mutex; + //std::map + LogTestController &log; std::vector directories; diff --git a/libminifi/test/unit/InvokeHTTPTests.cpp b/libminifi/test/unit/InvokeHTTPTests.cpp index 34661aa99f..8d0004f798 100644 --- a/libminifi/test/unit/InvokeHTTPTests.cpp +++ b/libminifi/test/unit/InvokeHTTPTests.cpp @@ -36,105 +36,6 @@ #include "core/ProcessSession.h" #include "core/ProcessorNode.h" -TEST_CASE("HTTPTestsPostNoResourceClaim", "[httptest1]") { - TestController testController; - std::shared_ptr content_repo = std::make_shared(); - LogTestController::getInstance().setInfo(); - - std::shared_ptr repo = std::make_shared(); - - std::shared_ptr processor = std::make_shared("listenhttp"); - - std::shared_ptr invokehttp = std::make_shared("invokehttp"); - uuid_t processoruuid; - REQUIRE(true == processor->getUUID(processoruuid)); - - uuid_t invokehttp_uuid; - REQUIRE(true == invokehttp->getUUID(invokehttp_uuid)); - - - std::shared_ptr connection = std::make_shared(repo, content_repo, "getfileCreate2Connection"); - connection->setRelationship(core::Relationship("success", "description")); - - std::shared_ptr connection2 = std::make_shared(repo, content_repo, "listenhttp"); - - connection2->setRelationship(core::Relationship("No Retry", "description")); - - // link the connections so that we can test results at the end for this - connection->setSource(processor); - - // link the connections so that we can test results at the end for this - connection->setDestination(invokehttp); - - connection2->setSource(invokehttp); - - connection2->setSourceUUID(invokehttp_uuid); - connection->setSourceUUID(processoruuid); - connection->setDestinationUUID(invokehttp_uuid); - - processor->addConnection(connection); - invokehttp->addConnection(connection); - invokehttp->addConnection(connection2); - - core::ProcessorNode node(processor); - core::ProcessorNode node2(invokehttp); - - std::shared_ptr controller_services_provider = nullptr; - core::ProcessContext context(node, controller_services_provider, repo); - core::ProcessContext context2(node2, controller_services_provider, repo); - context.setProperty(org::apache::nifi::minifi::processors::ListenHTTP::Port, "8685"); - context.setProperty(org::apache::nifi::minifi::processors::ListenHTTP::BasePath, "/testytesttest"); - - context2.setProperty(org::apache::nifi::minifi::processors::InvokeHTTP::Method, "POST"); - context2.setProperty(org::apache::nifi::minifi::processors::InvokeHTTP::URL, "http://localhost:8685/testytesttest"); - core::ProcessSession session(&context); - core::ProcessSession session2(&context2); - - REQUIRE(processor->getName() == "listenhttp"); - - core::ProcessSessionFactory factory(&context); - - std::shared_ptr record; - processor->setScheduledState(core::ScheduledState::RUNNING); - processor->onSchedule(&context, &factory); - processor->onTrigger(&context, &session); - - invokehttp->incrementActiveTasks(); - invokehttp->setScheduledState(core::ScheduledState::RUNNING); - core::ProcessSessionFactory factory2(&context2); - invokehttp->onSchedule(&context2, &factory2); - invokehttp->onTrigger(&context2, &session2); - - provenance::ProvenanceReporter *reporter = session.getProvenanceReporter(); - std::set records = reporter->getEvents(); - record = session.get(); - REQUIRE(record == nullptr); - REQUIRE(records.size() == 0); - - processor->incrementActiveTasks(); - processor->setScheduledState(core::ScheduledState::RUNNING); - processor->onTrigger(&context, &session); - - reporter = session.getProvenanceReporter(); - - records = reporter->getEvents(); - session.commit(); - - invokehttp->incrementActiveTasks(); - invokehttp->setScheduledState(core::ScheduledState::RUNNING); - invokehttp->onTrigger(&context2, &session2); - - session2.commit(); - records = reporter->getEvents(); - - for (provenance::ProvenanceEventRecord *provEventRecord : records) { - REQUIRE(provEventRecord->getComponentType() == processor->getName()); - } - std::shared_ptr ffr = session2.get(); - REQUIRE(true == LogTestController::getInstance().contains("exiting because method is POST")); - LogTestController::getInstance().reset(); -} - TEST_CASE("HTTPTestsWithNoResourceClaimPOST", "[httptest1]") { TestController testController; std::shared_ptr content_repo = std::make_shared(); @@ -370,3 +271,39 @@ TEST_CASE("HTTPTestsWithResourceClaimPOST", "[httptest1]") { LogTestController::getInstance().reset(); } +TEST_CASE("HTTPTestsPostNoResourceClaim", "[httptest1]") { + TestController testController; + LogTestController::getInstance().setInfo(); + LogTestController::getInstance().setInfo(); + LogTestController::getInstance().setInfo(); + + std::shared_ptr plan = testController.createPlan(); + std::shared_ptr processor = plan->addProcessor("ListenHTTP", "listenhttp", core::Relationship("No Retry", "description"), false); + std::shared_ptr invokehttp = plan->addProcessor("InvokeHTTP", "invokehttp", core::Relationship("success", "description"), true); + + REQUIRE(true == plan->setProperty(processor, org::apache::nifi::minifi::processors::ListenHTTP::Port.getName(), "8685")); + REQUIRE(true == plan->setProperty(processor, org::apache::nifi::minifi::processors::ListenHTTP::BasePath.getName(), "/testytesttest")); + + REQUIRE(true == plan->setProperty(invokehttp, org::apache::nifi::minifi::processors::InvokeHTTP::Method.getName(), "POST")); + REQUIRE(true == plan->setProperty(invokehttp, org::apache::nifi::minifi::processors::InvokeHTTP::URL.getName(), "http://localhost:8685/testytesttest")); + plan->reset(); + testController.runSession(plan, true); + + std::set records = plan->getProvenanceRecords(); + std::shared_ptr record = plan->getCurrentFlowFile(); + REQUIRE(record == nullptr); + REQUIRE(records.size() == 0); + + plan->reset(); + testController.runSession(plan, true); + + records = plan->getProvenanceRecords(); + record = plan->getCurrentFlowFile(); + + for (provenance::ProvenanceEventRecord *provEventRecord : records) { + REQUIRE(provEventRecord->getComponentType() == processor->getName()); + } + std::shared_ptr ffr = plan->getCurrentFlowFile(); + REQUIRE(true == LogTestController::getInstance().contains("exiting because method is POST")); + LogTestController::getInstance().reset(); +} diff --git a/libminifi/test/unit/ProcessorTests.cpp b/libminifi/test/unit/ProcessorTests.cpp index ad15da24dc..10ddd6f32a 100644 --- a/libminifi/test/unit/ProcessorTests.cpp +++ b/libminifi/test/unit/ProcessorTests.cpp @@ -23,11 +23,12 @@ #include #include #include -#include "../unit/ProvenanceTestHelper.h" + #include "../TestBase.h" #include "processors/ListenHTTP.h" #include "processors/LogAttribute.h" #include "processors/GetFile.h" +#include "../unit/ProvenanceTestHelper.h" #include "core/Core.h" #include "core/FlowFile.h" #include "core/Processor.h" @@ -42,126 +43,6 @@ TEST_CASE("Test Creation of GetFile", "[getfileCreate]") { REQUIRE(processor->getName() == "processorname"); } -TEST_CASE("Test Find file", "[getfileCreate2]") { - TestController testController; - std::shared_ptr content_repo = std::make_shared(); - - std::shared_ptr processor = std::make_shared("getfileCreate2"); - std::shared_ptr configure = std::make_shared(); - - std::shared_ptr processorReport = std::make_shared( - std::make_shared(configure), configure); - - std::shared_ptr test_repo = std::make_shared(); - - std::shared_ptr repo = std::static_pointer_cast(test_repo); - - char format[] = "/tmp/gt.XXXXXX"; - char *dir = testController.createTempDirectory(format); - - uuid_t processoruuid; - REQUIRE(true == processor->getUUID(processoruuid)); - - std::shared_ptr connection = std::make_shared(test_repo, content_repo, "getfileCreate2Connection"); - connection->setRelationship(core::Relationship("success", "description")); - - // link the connections so that we can test results at the end for this - connection->setSource(processor); - connection->setDestination(processor); - - connection->setSourceUUID(processoruuid); - connection->setDestinationUUID(processoruuid); - - processor->addConnection(connection); - REQUIRE(dir != NULL); - - core::ProcessorNode node(processor); - std::shared_ptr controller_services_provider = nullptr; - core::ProcessContext context(node, controller_services_provider, test_repo); - core::ProcessSessionFactory factory(&context); - context.setProperty(org::apache::nifi::minifi::processors::GetFile::Directory, dir); - core::ProcessSession session(&context); - - processor->onSchedule(&context, &factory); - REQUIRE(processor->getName() == "getfileCreate2"); - - std::shared_ptr record; - processor->setScheduledState(core::ScheduledState::RUNNING); - processor->onTrigger(&context, &session); - - provenance::ProvenanceReporter *reporter = session.getProvenanceReporter(); - std::set records = reporter->getEvents(); - record = session.get(); - REQUIRE(record == nullptr); - REQUIRE(records.size() == 0); - - std::fstream file; - std::stringstream ss; - ss << dir << "/" << "tstFile.ext"; - file.open(ss.str(), std::ios::out); - file << "tempFile"; - file.close(); - - processor->incrementActiveTasks(); - processor->setScheduledState(core::ScheduledState::RUNNING); - processor->onTrigger(&context, &session); - unlink(ss.str().c_str()); - reporter = session.getProvenanceReporter(); - - REQUIRE(processor->getName() == "getfileCreate2"); - - records = reporter->getEvents(); - - for (provenance::ProvenanceEventRecord *provEventRecord : records) { - REQUIRE(provEventRecord->getComponentType() == processor->getName()); - } - session.commit(); - std::shared_ptr ffr = session.get(); - - ffr->getResourceClaim()->decreaseFlowFileRecordOwnedCount(); - REQUIRE(2 == repo->getRepoMap().size()); - - for (auto entry : repo->getRepoMap()) { - provenance::ProvenanceEventRecord newRecord; - newRecord.DeSerialize(reinterpret_cast(const_cast(entry.second.data())), entry.second.length()); - - bool found = false; - for (auto provRec : records) { - if (provRec->getEventId() == newRecord.getEventId()) { - REQUIRE(provRec->getEventId() == newRecord.getEventId()); - REQUIRE(provRec->getComponentId() == newRecord.getComponentId()); - REQUIRE(provRec->getComponentType() == newRecord.getComponentType()); - REQUIRE(provRec->getDetails() == newRecord.getDetails()); - REQUIRE(provRec->getEventDuration() == newRecord.getEventDuration()); - found = true; - break; - } - } - if (!found) { - throw std::runtime_error("Did not find record"); - } - } - - core::ProcessorNode nodeReport(processorReport); - core::ProcessContext contextReport(nodeReport, controller_services_provider, test_repo); - core::ProcessSessionFactory factoryReport(&contextReport); - core::ProcessSession sessionReport(&contextReport); - processorReport->onSchedule(&contextReport, &factoryReport); - std::shared_ptr taskReport = std::static_pointer_cast< - org::apache::nifi::minifi::core::reporting::SiteToSiteProvenanceReportingTask>(processorReport); - taskReport->setBatchSize(1); - std::vector> recordsReport; - recordsReport.push_back(std::make_shared()); - processorReport->incrementActiveTasks(); - processorReport->setScheduledState(core::ScheduledState::RUNNING); - std::string jsonStr; - std::size_t deserialized = 0; - repo->DeSerialize(recordsReport, deserialized); - taskReport->getJsonReport(&contextReport, &sessionReport, recordsReport, jsonStr); - REQUIRE(recordsReport.size() == 1); - REQUIRE(taskReport->getName() == std::string(org::apache::nifi::minifi::core::reporting::SiteToSiteProvenanceReportingTask::ReportTaskName)); - REQUIRE(jsonStr.find("\"componentType\" : \"getfileCreate2\"") != std::string::npos); -} TEST_CASE("Test GetFileLikeIt'sThreaded", "[getfileCreate3]") { TestController testController; @@ -246,75 +127,61 @@ TEST_CASE("Test GetFileLikeIt'sThreaded", "[getfileCreate3]") { TEST_CASE("LogAttributeTest", "[getfileCreate3]") { TestController testController; - std::shared_ptr content_repo = std::make_shared(); LogTestController::getInstance().setDebug(); - std::shared_ptr repo = std::make_shared(); + std::shared_ptr plan = testController.createPlan(); + std::shared_ptr getfile = plan->addProcessor("GetFile", "getfileCreate2"); - std::shared_ptr processor = std::make_shared("getfileCreate2"); - - std::shared_ptr logAttribute = std::make_shared("logattribute"); + plan->addProcessor("LogAttribute", "logattribute", core::Relationship("success", "description"), true); char format[] = "/tmp/gt.XXXXXX"; char *dir = testController.createTempDirectory(format); - uuid_t processoruuid; - REQUIRE(true == processor->getUUID(processoruuid)); - - uuid_t logattribute_uuid; - REQUIRE(true == logAttribute->getUUID(logattribute_uuid)); - - std::shared_ptr connection = std::make_shared(repo, content_repo, "getfileCreate2Connection"); - connection->setRelationship(core::Relationship("success", "description")); - - std::shared_ptr connection2 = std::make_shared(repo, content_repo, "logattribute"); - - connection2->setRelationship(core::Relationship("success", "description")); - - // link the connections so that we can test results at the end for this - connection->setSource(processor); - - // link the connections so that we can test results at the end for this - connection->setDestination(logAttribute); - - connection2->setSource(logAttribute); - - connection2->setSourceUUID(logattribute_uuid); - connection->setSourceUUID(processoruuid); - connection->setDestinationUUID(logattribute_uuid); + plan->setProperty(getfile, org::apache::nifi::minifi::processors::GetFile::Directory.getName(), dir); + testController.runSession(plan, false); + std::set records = plan->getProvenanceRecords(); + std::shared_ptr record = plan->getCurrentFlowFile(); + REQUIRE(record == nullptr); + REQUIRE(records.size() == 0); - processor->addConnection(connection); - logAttribute->addConnection(connection); - logAttribute->addConnection(connection2); - REQUIRE(dir != NULL); + std::fstream file; + std::stringstream ss; + ss << dir << "/" << "tstFile.ext"; + file.open(ss.str(), std::ios::out); + file << "tempFile"; + file.close(); + plan->reset(); + testController.runSession(plan, false); - core::ProcessorNode node(processor); - core::ProcessorNode node2(logAttribute); - std::shared_ptr controller_services_provider = nullptr; - core::ProcessContext context(node, controller_services_provider, repo); - core::ProcessContext context2(node2, controller_services_provider, repo); - context.setProperty(org::apache::nifi::minifi::processors::GetFile::Directory, dir); - core::ProcessSession session(&context); - core::ProcessSession session2(&context2); + unlink(ss.str().c_str()); - REQUIRE(processor->getName() == "getfileCreate2"); + records = plan->getProvenanceRecords(); + record = plan->getCurrentFlowFile(); + testController.runSession(plan, false); - std::shared_ptr record; - processor->setScheduledState(core::ScheduledState::RUNNING); + records = plan->getProvenanceRecords(); + record = plan->getCurrentFlowFile(); - core::ProcessSessionFactory factory(&context); - processor->onSchedule(&context, &factory); - processor->onTrigger(&context, &session); + REQUIRE(true == LogTestController::getInstance().contains("key:absolute.path value:" + ss.str())); + REQUIRE(true == LogTestController::getInstance().contains("Size:8 Offset:0")); + REQUIRE(true == LogTestController::getInstance().contains("key:path value:" + std::string(dir))); + LogTestController::getInstance().reset(); +} - logAttribute->incrementActiveTasks(); - logAttribute->setScheduledState(core::ScheduledState::RUNNING); - core::ProcessSessionFactory factory2(&context2); - logAttribute->onSchedule(&context2, &factory2); - logAttribute->onTrigger(&context2, &session2); +TEST_CASE("Test Find file", "[getfileCreate3]") { + TestController testController; + std::shared_ptr plan = testController.createPlan(); + std::shared_ptr processor = plan->addProcessor("GetFile", "getfileCreate2"); + std::shared_ptr processorReport = std::make_shared( + std::make_shared(std::make_shared())); + plan->addProcessor(processorReport, "reporter", core::Relationship("success", "description"), false); + char format[] = "/tmp/gt.XXXXXX"; + char *dir = testController.createTempDirectory(format); - provenance::ProvenanceReporter *reporter = session.getProvenanceReporter(); - std::set records = reporter->getEvents(); - record = session.get(); + plan->setProperty(processor, org::apache::nifi::minifi::processors::GetFile::Directory.getName(), dir); + testController.runSession(plan, false); + std::set records = plan->getProvenanceRecords(); + std::shared_ptr record = plan->getCurrentFlowFile(); REQUIRE(record == nullptr); REQUIRE(records.size() == 0); @@ -324,27 +191,58 @@ TEST_CASE("LogAttributeTest", "[getfileCreate3]") { file.open(ss.str(), std::ios::out); file << "tempFile"; file.close(); + plan->reset(); + testController.runSession(plan, false); - processor->incrementActiveTasks(); - processor->setScheduledState(core::ScheduledState::RUNNING); - processor->onTrigger(&context, &session); - unlink(ss.str().c_str()); - reporter = session.getProvenanceReporter(); - - records = reporter->getEvents(); - session.commit(); - - logAttribute->incrementActiveTasks(); - logAttribute->setScheduledState(core::ScheduledState::RUNNING); - logAttribute->onTrigger(&context2, &session2); + records = plan->getProvenanceRecords(); + record = plan->getCurrentFlowFile(); + for (provenance::ProvenanceEventRecord *provEventRecord : records) { + REQUIRE(provEventRecord->getComponentType() == processor->getName()); + } + std::shared_ptr ffr = plan->getCurrentFlowFile(); + REQUIRE(ffr != nullptr); + ffr->getResourceClaim()->decreaseFlowFileRecordOwnedCount(); + auto repo = std::static_pointer_cast(plan->getProvenanceRepo()); + REQUIRE(2 == repo->getRepoMap().size()); - records = reporter->getEvents(); + for (auto entry : repo->getRepoMap()) { + provenance::ProvenanceEventRecord newRecord; + newRecord.DeSerialize(reinterpret_cast(const_cast(entry.second.data())), entry.second.length()); - std::cout << LogTestController::getInstance().log_output.str() << std::endl; - REQUIRE(true == LogTestController::getInstance().contains("key:absolute.path value:" + ss.str())); - REQUIRE(true == LogTestController::getInstance().contains("Size:8 Offset:0")); - REQUIRE(true == LogTestController::getInstance().contains("key:path value:" + std::string(dir))); - LogTestController::getInstance().reset(); + bool found = false; + for (auto provRec : records) { + if (provRec->getEventId() == newRecord.getEventId()) { + REQUIRE(provRec->getEventId() == newRecord.getEventId()); + REQUIRE(provRec->getComponentId() == newRecord.getComponentId()); + REQUIRE(provRec->getComponentType() == newRecord.getComponentType()); + REQUIRE(provRec->getDetails() == newRecord.getDetails()); + REQUIRE(provRec->getEventDuration() == newRecord.getEventDuration()); + found = true; + break; + } + } + if (!found) { + throw std::runtime_error("Did not find record"); + } + } + std::shared_ptr taskReport = std::static_pointer_cast< + org::apache::nifi::minifi::core::reporting::SiteToSiteProvenanceReportingTask>(processorReport); + taskReport->setBatchSize(1); + std::vector> recordsReport; + recordsReport.push_back(std::make_shared()); + processorReport->incrementActiveTasks(); + processorReport->setScheduledState(core::ScheduledState::RUNNING); + std::string jsonStr; + std::size_t deserialized = 0; + repo->DeSerialize(recordsReport, deserialized); + std::function verifyReporter = [&](core::ProcessContext *context, core::ProcessSession *session) { + taskReport->getJsonReport(context, session, recordsReport, jsonStr); + REQUIRE(recordsReport.size() == 1); + REQUIRE(taskReport->getName() == std::string(org::apache::nifi::minifi::core::reporting::SiteToSiteProvenanceReportingTask::ReportTaskName)); + REQUIRE(jsonStr.find("\"componentType\" : \"getfileCreate2\"") != std::string::npos); + }; + + testController.runSession(plan, false, verifyReporter); } int fileSize(const char *add) { From 101397974ba7e0ffaee832df72891cca5b7a5c9b Mon Sep 17 00:00:00 2001 From: Marc Date: Thu, 15 Jun 2017 10:48:47 -0400 Subject: [PATCH 3/6] MINIFI-249: Allow for different volatile configurations --- libminifi/include/Connection.h | 6 +- .../include/EventDrivenSchedulingAgent.h | 4 +- libminifi/include/ResourceClaim.h | 11 +- libminifi/include/SchedulingAgent.h | 5 +- libminifi/include/Site2SiteClientProtocol.h | 3 +- libminifi/include/ThreadedSchedulingAgent.h | 3 +- .../include/TimerDrivenSchedulingAgent.h | 3 +- libminifi/include/core/ProcessContext.h | 8 +- .../core/repository/AtomicRepoEntries.h | 89 +++++++++++++-- .../core/repository/FileSystemRepository.h | 7 +- .../repository/VolatileContentRepository.h | 17 ++- .../core/repository/VolatileRepository.h | 14 +-- libminifi/include/io/AtomicEntryStream.h | 27 ++++- libminifi/include/io/FileStream.h | 18 +-- libminifi/include/provenance/Provenance.h | 1 + libminifi/src/Connection.cpp | 2 +- libminifi/src/FlowController.cpp | 4 +- libminifi/src/RemoteProcessorGroupPort.cpp | 1 + libminifi/src/ResourceClaim.cpp | 1 - libminifi/src/SchedulingAgent.cpp | 8 +- libminifi/src/ThreadedSchedulingAgent.cpp | 2 +- libminifi/src/core/FlowFile.cpp | 2 +- libminifi/src/core/ProcessSession.cpp | 9 +- .../core/repository/FileSystemRepository.cpp | 9 +- .../repository/VolatileContentRepository.cpp | 108 +++++++++++++----- libminifi/src/core/yaml/YamlConfiguration.cpp | 32 ++++-- libminifi/src/io/FileStream.cpp | 75 +++++++++--- libminifi/test/TestBase.cpp | 2 +- .../test/integration/TestExecuteProcess.cpp | 4 +- libminifi/test/unit/FileStreamTests.cpp | 12 +- libminifi/test/unit/InvokeHTTPTests.cpp | 8 +- libminifi/test/unit/ProcessorTests.cpp | 2 +- 32 files changed, 365 insertions(+), 132 deletions(-) diff --git a/libminifi/include/Connection.h b/libminifi/include/Connection.h index e40fe75a15..ff32baf30e 100644 --- a/libminifi/include/Connection.h +++ b/libminifi/include/Connection.h @@ -47,7 +47,9 @@ class Connection : public core::Connectable, public std::enable_shared_from_this /* * Create a new processor */ - explicit Connection(const std::shared_ptr &flow_repository, const std::shared_ptr &content_repo, std::string name, uuid_t uuid = NULL, uuid_t srcUUID = NULL,uuid_t destUUID = NULL); + explicit Connection(const std::shared_ptr &flow_repository, const std::shared_ptr &content_repo, std::string name, uuid_t uuid = NULL, + uuid_t srcUUID = NULL, + uuid_t destUUID = NULL); // Destructor virtual ~Connection() { } @@ -123,7 +125,7 @@ class Connection : public core::Connectable, public std::enable_shared_from_this bool isFull(); // Get queue size uint64_t getQueueSize() { - std::lock_guard < std::mutex > lock(mutex_); + std::lock_guard lock(mutex_); return queue_.size(); } // Get queue data size diff --git a/libminifi/include/EventDrivenSchedulingAgent.h b/libminifi/include/EventDrivenSchedulingAgent.h index b7c21c7f73..c838b11f2b 100644 --- a/libminifi/include/EventDrivenSchedulingAgent.h +++ b/libminifi/include/EventDrivenSchedulingAgent.h @@ -39,8 +39,8 @@ class EventDrivenSchedulingAgent : public ThreadedSchedulingAgent { * Create a new event driven scheduling agent. */ EventDrivenSchedulingAgent(std::shared_ptr controller_service_provider, std::shared_ptr repo, - std::shared_ptr content_repo, std::shared_ptr configuration) - : ThreadedSchedulingAgent(controller_service_provider, repo, content_repo, configuration) { + std::shared_ptr flow_repo, std::shared_ptr content_repo, std::shared_ptr configuration) + : ThreadedSchedulingAgent(controller_service_provider, repo, flow_repo, content_repo, configuration) { } // Destructor virtual ~EventDrivenSchedulingAgent() { diff --git a/libminifi/include/ResourceClaim.h b/libminifi/include/ResourceClaim.h index b235ba225d..19a67fa8d0 100644 --- a/libminifi/include/ResourceClaim.h +++ b/libminifi/include/ResourceClaim.h @@ -53,7 +53,7 @@ class ResourceClaim : public std::enable_shared_from_this { */ ResourceClaim(std::shared_ptr> claim_manager, const std::string contentDirectory = default_directory_path); - ResourceClaim(const std::string path, std::shared_ptr> claim_manager , bool deleted = false); + ResourceClaim(const std::string path, std::shared_ptr> claim_manager, bool deleted = false); // Destructor virtual ~ResourceClaim() { } @@ -90,6 +90,15 @@ class ResourceClaim : public std::enable_shared_from_this { } + friend std::ostream& operator<<(std::ostream& stream, const ResourceClaim& claim) { + stream << claim._contentFullPath; + return stream; + } + + friend std::ostream& operator<<(std::ostream& stream, const std::shared_ptr& claim) { + stream << claim->_contentFullPath; + return stream; + } protected: std::atomic deleted_; // Full path to the content diff --git a/libminifi/include/SchedulingAgent.h b/libminifi/include/SchedulingAgent.h index 7333cc57c3..1ff3faca76 100644 --- a/libminifi/include/SchedulingAgent.h +++ b/libminifi/include/SchedulingAgent.h @@ -53,6 +53,7 @@ class SchedulingAgent { * Create a new scheduling agent. */ SchedulingAgent(std::shared_ptr controller_service_provider, std::shared_ptr repo, + std::shared_ptr flow_repo, std::shared_ptr content_repo, std::shared_ptr configuration) : configure_(configuration), @@ -63,6 +64,7 @@ class SchedulingAgent { logger_(logging::LoggerFactory::getLogger()) { running_ = false; repo_ = repo; + flow_repo_ = flow_repo; utils::ThreadPool pool = utils::ThreadPool(configure_->getInt(Configure::nifi_flow_engine_threads, 8), true); component_lifecycle_thread_pool_ = std::move(pool); component_lifecycle_thread_pool_.start(); @@ -80,7 +82,6 @@ class SchedulingAgent { // start void start() { running_ = true; - } // stop void stop() { @@ -111,6 +112,8 @@ class SchedulingAgent { std::shared_ptr configure_; std::shared_ptr repo_; + + std::shared_ptr flow_repo_; std::shared_ptr content_repo_; // thread pool for components. diff --git a/libminifi/include/Site2SiteClientProtocol.h b/libminifi/include/Site2SiteClientProtocol.h index c47254e1a5..dcb551aee0 100644 --- a/libminifi/include/Site2SiteClientProtocol.h +++ b/libminifi/include/Site2SiteClientProtocol.h @@ -584,8 +584,7 @@ class Site2SiteClientProtocol { if (readSize == 0) { break; } - if (readSize < 0) - { + if (readSize < 0) { return -1; } int ret = _packet->_transaction->getStream().writeData(buffer, readSize); diff --git a/libminifi/include/ThreadedSchedulingAgent.h b/libminifi/include/ThreadedSchedulingAgent.h index ea6696f1ab..b4db4bf301 100644 --- a/libminifi/include/ThreadedSchedulingAgent.h +++ b/libminifi/include/ThreadedSchedulingAgent.h @@ -43,9 +43,10 @@ class ThreadedSchedulingAgent : public SchedulingAgent { * Create a new threaded scheduling agent. */ ThreadedSchedulingAgent(std::shared_ptr controller_service_provider, std::shared_ptr repo, + std::shared_ptr flow_repo, std::shared_ptr content_repo, std::shared_ptr configuration) - : SchedulingAgent(controller_service_provider, repo, content_repo, configuration), + : SchedulingAgent(controller_service_provider, repo, flow_repo, content_repo, configuration), logger_(logging::LoggerFactory::getLogger()) { } // Destructor diff --git a/libminifi/include/TimerDrivenSchedulingAgent.h b/libminifi/include/TimerDrivenSchedulingAgent.h index 602b2e4a7c..816bcec656 100644 --- a/libminifi/include/TimerDrivenSchedulingAgent.h +++ b/libminifi/include/TimerDrivenSchedulingAgent.h @@ -38,9 +38,10 @@ class TimerDrivenSchedulingAgent : public ThreadedSchedulingAgent { * Create a new processor */ TimerDrivenSchedulingAgent(std::shared_ptr controller_service_provider, std::shared_ptr repo, + std::shared_ptr flow_repo, std::shared_ptr content_repo, std::shared_ptr configure) - : ThreadedSchedulingAgent(controller_service_provider, repo, content_repo, configure) { + : ThreadedSchedulingAgent(controller_service_provider, repo, flow_repo, content_repo, configure) { } // Destructor virtual ~TimerDrivenSchedulingAgent() { diff --git a/libminifi/include/core/ProcessContext.h b/libminifi/include/core/ProcessContext.h index 7b269ca34b..f6aaf5e668 100644 --- a/libminifi/include/core/ProcessContext.h +++ b/libminifi/include/core/ProcessContext.h @@ -49,11 +49,12 @@ class ProcessContext : public controller::ControllerServiceLookup { * Create a new process context associated with the processor/controller service/state manager */ ProcessContext(ProcessorNode &processor, std::shared_ptr &controller_service_provider, std::shared_ptr repo, + std::shared_ptr flow_repo, std::shared_ptr content_repo = std::make_shared()) : processor_node_(processor), controller_service_provider_(controller_service_provider), logger_(logging::LoggerFactory::getLogger()), - content_repo_(content_repo) { + content_repo_(content_repo), flow_repo_(flow_repo) { repo_ = repo; } // Destructor @@ -103,6 +104,10 @@ class ProcessContext : public controller::ControllerServiceLookup { std::shared_ptr getContentRepository() { return content_repo_; } + + std::shared_ptr getFlowFileRepository() { + return flow_repo_; + } // Prevent default copy constructor and assignment operation // Only support pass by reference or pointer @@ -157,6 +162,7 @@ class ProcessContext : public controller::ControllerServiceLookup { std::shared_ptr controller_service_provider_; // repository shared pointer. std::shared_ptr repo_; + std::shared_ptr flow_repo_; // repository shared pointer. std::shared_ptr content_repo_; diff --git a/libminifi/include/core/repository/AtomicRepoEntries.h b/libminifi/include/core/repository/AtomicRepoEntries.h index a6bf39c00b..c7057a7682 100644 --- a/libminifi/include/core/repository/AtomicRepoEntries.h +++ b/libminifi/include/core/repository/AtomicRepoEntries.h @@ -2,7 +2,7 @@ * * 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. + * this work for additional information regarding copyright ref_count_hip. * 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 @@ -186,7 +186,9 @@ class AtomicEntry { : write_pending_(false), has_value_(false), total_size_(total_size), - max_size_(max_size) { + max_size_(max_size), + ref_count_(0), + free_required(false) { } @@ -200,8 +202,8 @@ class AtomicEntry { bool setRepoValue(RepoValue &new_value, RepoValue &old_value, size_t &prev_size) { // delete the underlying pointer bool lock = false; - if (!write_pending_.compare_exchange_weak(lock, true) && !lock) - { + if (!write_pending_.compare_exchange_weak(lock, true)) + { return false; } if (has_value_) { @@ -213,7 +215,20 @@ class AtomicEntry { try_unlock(); return true; } - + + + AtomicEntry *takeOwnership() + { + bool lock = false; + if (!write_pending_.compare_exchange_weak(lock, true) ) + return nullptr; + + ref_count_++; + + try_unlock(); + + return this; + } /** * A test and set operation, which is used to allow a function to test * if an item can be released and a function used for reclaiming memory associated @@ -221,10 +236,9 @@ class AtomicEntry { * A custom comparator can be provided to augment the key being added into value_ */ bool testAndSetKey(const T str, std::function releaseTest = nullptr, std::function reclaimer = nullptr, std::function comparator = nullptr) { - // delete the underlying pointer bool lock = false; - if (!write_pending_.compare_exchange_weak(lock, true) && !lock) + if (!write_pending_.compare_exchange_weak(lock, true) ) return false; if (has_value_) { @@ -234,12 +248,23 @@ class AtomicEntry { { reclaimer(value_.getKey()); } + else if (free_required && ref_count_ == 0) + { + size_t bufferSize = value_.getBufferSize(); + value_.clearBuffer(); + has_value_ = false; + if (total_size_ != nullptr) { + *total_size_ -= bufferSize; + } + free_required = false; + } else { try_unlock(); return false; } } + ref_count_=1; value_.setKey(str, comparator); has_value_ = true; try_unlock(); @@ -283,6 +308,30 @@ class AtomicEntry { try_unlock(); return true; } + + void decrementOwnership(){ + try_lock(); + if (!has_value_) { + try_unlock(); + return; + } + if (ref_count_ > 0){ + ref_count_--; + } + if (ref_count_ == 0 && free_required) + { + size_t bufferSize = value_.getBufferSize(); + value_.clearBuffer(); + has_value_ = false; + if (total_size_ != nullptr) { + *total_size_ -= bufferSize; + } + free_required = false; + } + else{ + } + try_unlock(); + } /** * Moved the value into the argument @@ -299,6 +348,7 @@ class AtomicEntry { try_unlock(); return false; } + ref_count_++; *value = &value_; try_unlock(); return true; @@ -332,6 +382,16 @@ class AtomicEntry { try_unlock(); return ref; } + + size_t getLength() + { + size_t size = 0; + try_lock(); + size = value_.getBufferSize(); + try_unlock(); + return size; + + } /** * sets has_value to false; however, does not call @@ -347,12 +407,19 @@ class AtomicEntry { try_unlock(); return false; } + if (ref_count_ > 0) + { + free_required = true; + try_unlock(); + return true; + } size_t bufferSize = value_.getBufferSize(); value_.clearBuffer(); has_value_ = false; if (total_size_ != nullptr) { *total_size_ -= bufferSize; } + free_required = false; try_unlock(); return true; } @@ -393,7 +460,8 @@ class AtomicEntry { */ inline void try_lock() { bool lock = false; - while (!write_pending_.compare_exchange_weak(lock, true) && !lock) { + while (!write_pending_.compare_exchange_weak(lock, true,std::memory_order_acquire)) { + lock = false; // attempt again } } @@ -403,7 +471,8 @@ class AtomicEntry { */ inline void try_unlock() { bool lock = true; - while (!write_pending_.compare_exchange_weak(lock, false) && lock) { + while (!write_pending_.compare_exchange_weak(lock, false,std::memory_order_acquire)) { + lock = true; // attempt again } } @@ -416,6 +485,8 @@ class AtomicEntry { std::atomic write_pending_; // used to determine if a value is present in this atomic entry. std::atomic has_value_; + std::atomic ref_count_; + std::atomic free_required; // repo value. RepoValue value_; }; diff --git a/libminifi/include/core/repository/FileSystemRepository.h b/libminifi/include/core/repository/FileSystemRepository.h index 3792ebe7aa..84bf01e064 100644 --- a/libminifi/include/core/repository/FileSystemRepository.h +++ b/libminifi/include/core/repository/FileSystemRepository.h @@ -43,7 +43,6 @@ class FileSystemRepository : public core::ContentRepository, public core::CoreCo } - virtual bool initialize(const std::shared_ptr &configuration); virtual void stop(); @@ -53,12 +52,10 @@ class FileSystemRepository : public core::ContentRepository, public core::CoreCo virtual std::shared_ptr read(const std::shared_ptr &claim); virtual bool close(const std::shared_ptr &claim) { - return true; + return remove(claim); } - virtual bool remove(const std::shared_ptr &claim) { - return true; - } + virtual bool remove(const std::shared_ptr &claim); private: diff --git a/libminifi/include/core/repository/VolatileContentRepository.h b/libminifi/include/core/repository/VolatileContentRepository.h index 9ea6e52acb..73d9c6f820 100644 --- a/libminifi/include/core/repository/VolatileContentRepository.h +++ b/libminifi/include/core/repository/VolatileContentRepository.h @@ -38,12 +38,23 @@ namespace repository { */ class VolatileContentRepository : public core::ContentRepository, public core::repository::VolatileRepository> { public: - VolatileContentRepository(std::string name = getClassName()) + + static const char *minimal_locking; + + explicit VolatileContentRepository(std::string name = getClassName()) : core::repository::VolatileRepository>(name), - logger_(logging::LoggerFactory::getLogger()) { + logger_(logging::LoggerFactory::getLogger()), minimize_locking_(true) { max_count_ = 15000; } virtual ~VolatileContentRepository() { + if (!minimize_locking_){ + std::lock_guard < std::mutex > lock(map_mutex_); + for(const auto &item : master_list_) + { + delete item.second; + } + master_list_.clear(); + } } @@ -98,6 +109,8 @@ class VolatileContentRepository : public core::ContentRepository, public core::r } private: + + bool minimize_locking_; // function pointers that are associated with the claims. std::function, std::shared_ptr)> resource_claim_comparator_; diff --git a/libminifi/include/core/repository/VolatileRepository.h b/libminifi/include/core/repository/VolatileRepository.h index 07d9cc46a1..295a61deb2 100644 --- a/libminifi/include/core/repository/VolatileRepository.h +++ b/libminifi/include/core/repository/VolatileRepository.h @@ -140,6 +140,7 @@ class VolatileRepository : public core::Repository, public std::enable_shared_fr std::atomic current_index_; // value vector. std::vector*> value_vector_; + // max count we are allowed to store. uint32_t max_count_; // maximum estimated size @@ -160,12 +161,7 @@ template const char *VolatileRepository::volatile_repo_max_count = "max.count"; template const char *VolatileRepository::volatile_repo_max_bytes = "max.bytes"; -/* - template - void VolatileRepository::run() { - repo_full_ = false; - } - */ + template void VolatileRepository::loadComponent(const std::shared_ptr &content_repo) { } @@ -243,10 +239,12 @@ bool VolatileRepository::Put(T key, const uint8_t *buf, size_t bufLen) { continue; } } - logger_->log_debug("Set repo value at %d out of %d", private_index, max_count_); + updated = value_vector_.at(private_index)->setRepoValue(new_value, old_value, reclaimed_size); - if (updated) + logger_->log_debug("Set repo value at %d out of %d updated %d current_size %d, adding %d to %d", private_index, max_count_,updated==true,reclaimed_size,size, current_size_.load()); + if (updated && reclaimed_size > 0) { + std::lock_guard lock(mutex_); purge_list_.push_back(old_value.getKey()); } if (reclaimed_size > 0) { diff --git a/libminifi/include/io/AtomicEntryStream.h b/libminifi/include/io/AtomicEntryStream.h index 1171e9fc7c..5f200f050a 100644 --- a/libminifi/include/io/AtomicEntryStream.h +++ b/libminifi/include/io/AtomicEntryStream.h @@ -42,10 +42,14 @@ class AtomicEntryStream : public BaseStream { core::repository::RepoValue *value; if (entry_->getValue(key, &value)) { length_ = value->getBufferSize(); + entry_->decrementOwnership(); + invalid_stream_ = false; } else { - throw Exception(FILE_OPERATION_EXCEPTION, "Could not create valid entry"); + invalid_stream_ = true; } } + + virtual ~AtomicEntryStream(); virtual void closeStream() { @@ -103,6 +107,7 @@ class AtomicEntryStream : public BaseStream { size_t offset_; T key_; core::repository::AtomicEntry *entry_; + std::atomic invalid_stream_; std::recursive_mutex entry_lock_; // Logger @@ -110,6 +115,12 @@ class AtomicEntryStream : public BaseStream { }; +template +AtomicEntryStream::~AtomicEntryStream(){ + logger_->log_debug("Decrementing"); + entry_->decrementOwnership(); +} + template void AtomicEntryStream::seek(uint64_t offset) { std::lock_guard lock(entry_lock_); @@ -118,7 +129,7 @@ void AtomicEntryStream::seek(uint64_t offset) { template int AtomicEntryStream::writeData(std::vector &buf, int buflen) { - if (buf.capacity() < buflen) + if (buf.capacity() < buflen || invalid_stream_) return -1; return writeData(reinterpret_cast(&buf[0]), buflen); } @@ -126,7 +137,7 @@ int AtomicEntryStream::writeData(std::vector &buf, int buflen) { // data stream overrides template int AtomicEntryStream::writeData(uint8_t *value, int size) { - if (nullptr != value) { + if (nullptr != value && !invalid_stream_) { std::lock_guard lock(entry_lock_); if (entry_->insert(key_, value, size)) { offset_ += size; @@ -147,6 +158,9 @@ int AtomicEntryStream::writeData(uint8_t *value, int size) { template int AtomicEntryStream::readData(std::vector &buf, int buflen) { + if (invalid_stream_){ + return -1; + } if (buf.capacity() < buflen) { buf.resize(buflen); } @@ -160,7 +174,7 @@ int AtomicEntryStream::readData(std::vector &buf, int buflen) { template int AtomicEntryStream::readData(uint8_t *buf, int buflen) { - if (nullptr != buf) { + if (nullptr != buf && !invalid_stream_) { std::lock_guard lock(entry_lock_); int len = buflen; core::repository::RepoValue *value; @@ -168,17 +182,18 @@ int AtomicEntryStream::readData(uint8_t *buf, int buflen) { if (offset_ + len > value->getBufferSize()) { len = value->getBufferSize() - offset_; if (len <= 0) { + entry_->decrementOwnership(); return 0; } } std::memcpy(buf, reinterpret_cast(const_cast(value->getBuffer()) + offset_), len); offset_ += len; - + entry_->decrementOwnership(); return len; } } - return 0; + return -1; } } /* namespace io */ diff --git a/libminifi/include/io/FileStream.h b/libminifi/include/io/FileStream.h index 06cb58ffb7..23a1f0bf66 100644 --- a/libminifi/include/io/FileStream.h +++ b/libminifi/include/io/FileStream.h @@ -45,20 +45,19 @@ class FileStream : public io::BaseStream { * File Stream constructor that accepts an fstream shared pointer. * It must already be initialized for read and write. */ - explicit FileStream(const std::string &path, uint32_t offset = 0); + explicit FileStream(const std::string &path, uint32_t offset, bool write_enable = false); + + /** + * File Stream constructor that accepts an fstream shared pointer. + * It must already be initialized for read and write. + */ + explicit FileStream(const std::string &path); virtual ~FileStream() { closeStream(); } - virtual void closeStream() { - std::lock_guard lock(file_lock_); - if (file_stream_ != nullptr) { - file_stream_->close(); - file_stream_ = nullptr; - } - } - + virtual void closeStream(); /** * Skip to the specified offset. * @param offset offset to which we will skip @@ -119,6 +118,7 @@ class FileStream : public io::BaseStream { std::recursive_mutex file_lock_; std::unique_ptr file_stream_; size_t offset_; + std::string path_; size_t length_; private: diff --git a/libminifi/include/provenance/Provenance.h b/libminifi/include/provenance/Provenance.h index d7c30e092e..ac4059565b 100644 --- a/libminifi/include/provenance/Provenance.h +++ b/libminifi/include/provenance/Provenance.h @@ -440,6 +440,7 @@ class ProvenanceReporter { // Add event void add(ProvenanceEventRecord *event) { _events.insert(event); + logger_->log_debug("Prove reporter now %d",_events.size()); } // Remove event void remove(ProvenanceEventRecord *event) { diff --git a/libminifi/src/Connection.cpp b/libminifi/src/Connection.cpp index 37e4652401..1d937b4ea9 100644 --- a/libminifi/src/Connection.cpp +++ b/libminifi/src/Connection.cpp @@ -91,7 +91,7 @@ void Connection::put(std::shared_ptr flow) { queued_data_size_ += flow->getSize(); - logger_->log_debug("Enqueue flow file UUID %s to connection %s", flow->getUUIDStr().c_str(), name_.c_str()); + logger_->log_debug("Enqueue flow file UUID %s to connection %s %d", flow->getUUIDStr(), name_, queue_.size()); } if (!flow->isStored()) { diff --git a/libminifi/src/FlowController.cpp b/libminifi/src/FlowController.cpp index 798db715b1..6358ed0515 100644 --- a/libminifi/src/FlowController.cpp +++ b/libminifi/src/FlowController.cpp @@ -252,11 +252,11 @@ void FlowController::load() { logger_->log_info("Initializing timers"); if (nullptr == timer_scheduler_) { timer_scheduler_ = std::make_shared < TimerDrivenSchedulingAgent - > (std::static_pointer_cast < core::controller::ControllerServiceProvider > (shared_from_this()), provenance_repo_, content_repo_, configuration_); + > (std::static_pointer_cast < core::controller::ControllerServiceProvider > (shared_from_this()), provenance_repo_, flow_file_repo_, content_repo_, configuration_); } if (nullptr == event_scheduler_) { event_scheduler_ = std::make_shared < EventDrivenSchedulingAgent - > (std::static_pointer_cast < core::controller::ControllerServiceProvider > (shared_from_this()), provenance_repo_, content_repo_, configuration_); + > (std::static_pointer_cast < core::controller::ControllerServiceProvider > (shared_from_this()), provenance_repo_, flow_file_repo_, content_repo_, configuration_); } logger_->log_info("Load Flow Controller from file %s", configuration_filename_.c_str()); diff --git a/libminifi/src/RemoteProcessorGroupPort.cpp b/libminifi/src/RemoteProcessorGroupPort.cpp index 86f8360e8b..3c88e8f870 100644 --- a/libminifi/src/RemoteProcessorGroupPort.cpp +++ b/libminifi/src/RemoteProcessorGroupPort.cpp @@ -204,6 +204,7 @@ void RemoteProcessorGroupPort::onTrigger(core::ProcessContext *context, core::Pr session->rollback(); } + throw std::exception(); } diff --git a/libminifi/src/ResourceClaim.cpp b/libminifi/src/ResourceClaim.cpp index 0986ca7741..67c0e4cc22 100644 --- a/libminifi/src/ResourceClaim.cpp +++ b/libminifi/src/ResourceClaim.cpp @@ -42,7 +42,6 @@ ResourceClaim::ResourceClaim(std::shared_ptr> claim_manager_(claim_manager), deleted_(false), logger_(logging::LoggerFactory::getLogger()) { - // Create the full content path for the content _contentFullPath = contentDirectory + "/" + non_repeating_string_generator_.generate(); logger_->log_debug("Resource Claim created %s", _contentFullPath); diff --git a/libminifi/src/SchedulingAgent.cpp b/libminifi/src/SchedulingAgent.cpp index 26dd165375..1060830ae7 100644 --- a/libminifi/src/SchedulingAgent.cpp +++ b/libminifi/src/SchedulingAgent.cpp @@ -77,13 +77,15 @@ bool SchedulingAgent::onTrigger(std::shared_ptr processor, core // No need to yield, reset yield expiration to 0 processor->clearYield(); - if (!hasWorkToDo(processor)) + if (!hasWorkToDo(processor)) { // No work to do, yield return true; - - if (hasTooMuchOutGoing(processor)) + } + if (hasTooMuchOutGoing(processor)) { + logger_->log_debug("backpressure applied because too much outgoing"); // need to apply backpressure return true; + } processor->incrementActiveTasks(); try { diff --git a/libminifi/src/ThreadedSchedulingAgent.cpp b/libminifi/src/ThreadedSchedulingAgent.cpp index 2bd7f13439..7b4ce85687 100644 --- a/libminifi/src/ThreadedSchedulingAgent.cpp +++ b/libminifi/src/ThreadedSchedulingAgent.cpp @@ -68,7 +68,7 @@ void ThreadedSchedulingAgent::schedule(std::shared_ptr processo } core::ProcessorNode processor_node(processor); - auto processContext = std::make_shared < core::ProcessContext > (processor_node, controller_service_provider_, repo_, content_repo_); + auto processContext = std::make_shared < core::ProcessContext > (processor_node, controller_service_provider_, repo_, flow_repo_, content_repo_); auto sessionFactory = std::make_shared < core::ProcessSessionFactory > (processContext.get()); processor->onSchedule(processContext.get(), sessionFactory.get()); diff --git a/libminifi/src/core/FlowFile.cpp b/libminifi/src/core/FlowFile.cpp index d9057c5078..6afd0fe2b3 100644 --- a/libminifi/src/core/FlowFile.cpp +++ b/libminifi/src/core/FlowFile.cpp @@ -47,7 +47,7 @@ FlowFile::FlowFile() entry_date_ = getTimeMillis(); lineage_start_date_ = entry_date_; - char uuidStr[37]; + char uuidStr[37] = { 0 }; // Generate the global UUID for the flow record id_generator_->generate(uuid_); diff --git a/libminifi/src/core/ProcessSession.cpp b/libminifi/src/core/ProcessSession.cpp index 680bcd202a..0402efefd9 100644 --- a/libminifi/src/core/ProcessSession.cpp +++ b/libminifi/src/core/ProcessSession.cpp @@ -39,7 +39,7 @@ namespace core { std::shared_ptr ProcessSession::create() { std::map empty; - std::shared_ptr record = std::make_shared(process_context_->getProvenanceRepository(), process_context_->getContentRepository(), empty); + std::shared_ptr record = std::make_shared(process_context_->getFlowFileRepository(), process_context_->getContentRepository(), empty); _addedFlowFiles[record->getUUIDStr()] = record; logger_->log_debug("Create FlowFile with UUID %s", record->getUUIDStr().c_str()); @@ -52,7 +52,7 @@ std::shared_ptr ProcessSession::create() { std::shared_ptr ProcessSession::create(std::shared_ptr &&parent) { std::map empty; - std::shared_ptr record = std::make_shared(process_context_->getProvenanceRepository(), process_context_->getContentRepository(), empty); + std::shared_ptr record = std::make_shared(process_context_->getFlowFileRepository(), process_context_->getContentRepository(), empty); if (record) { _addedFlowFiles[record->getUUIDStr()] = record; @@ -94,7 +94,7 @@ std::shared_ptr ProcessSession::clone(std::shared_ptr ProcessSession::cloneDuringTransfer(std::shared_ptr &parent) { std::map empty; - std::shared_ptr record = std::make_shared(process_context_->getProvenanceRepository(), process_context_->getContentRepository(), empty); + std::shared_ptr record = std::make_shared(process_context_->getFlowFileRepository(), process_context_->getContentRepository(), empty); if (record) { this->_clonedFlowFiles[record->getUUIDStr()] = record; @@ -408,6 +408,7 @@ void ProcessSession::read(std::shared_ptr &flow, InputStreamCall rollback(); return; } + stream->seek(flow->getOffset()); if (callback->process(stream) < 0) { @@ -998,7 +999,7 @@ std::shared_ptr ProcessSession::get() { ret->setDeleted(false); _updatedFlowFiles[ret->getUUIDStr()] = ret; std::map empty; - std::shared_ptr snapshot = std::make_shared(process_context_->getProvenanceRepository(), process_context_->getContentRepository(), empty); + std::shared_ptr snapshot = std::make_shared(process_context_->getFlowFileRepository(), process_context_->getContentRepository(), empty); logger_->log_debug("Create Snapshot FlowFile with UUID %s", snapshot->getUUIDStr().c_str()); snapshot = ret; // save a snapshot diff --git a/libminifi/src/core/repository/FileSystemRepository.cpp b/libminifi/src/core/repository/FileSystemRepository.cpp index 67fab3e8a2..fba1fe3648 100644 --- a/libminifi/src/core/repository/FileSystemRepository.cpp +++ b/libminifi/src/core/repository/FileSystemRepository.cpp @@ -34,11 +34,16 @@ void FileSystemRepository::stop() { } std::shared_ptr FileSystemRepository::write(const std::shared_ptr &claim) { - return std::make_shared < io::FileStream > (claim->getContentFullPath()); + return std::make_shared(claim->getContentFullPath()); } std::shared_ptr FileSystemRepository::read(const std::shared_ptr &claim) { - return std::make_shared < io::FileStream > (claim->getContentFullPath()); + return std::make_shared(claim->getContentFullPath(), 0, false); +} + +bool FileSystemRepository::remove(const std::shared_ptr &claim) { + std::remove(claim->getContentFullPath().c_str()); + return true; } } /* namespace repository */ diff --git a/libminifi/src/core/repository/VolatileContentRepository.cpp b/libminifi/src/core/repository/VolatileContentRepository.cpp index 01f88d4105..30d44b13b1 100644 --- a/libminifi/src/core/repository/VolatileContentRepository.cpp +++ b/libminifi/src/core/repository/VolatileContentRepository.cpp @@ -17,8 +17,11 @@ */ #include "core/repository/VolatileContentRepository.h" +#include +#include #include #include +#include "utils/StringUtils.h" #include "io/FileStream.h" namespace org { @@ -28,9 +31,14 @@ namespace minifi { namespace core { namespace repository { +const char *VolatileContentRepository::minimal_locking = "minimal.locking"; + bool VolatileContentRepository::initialize(const std::shared_ptr &configure) { VolatileRepository::initialize(configure); resource_claim_comparator_ = [](std::shared_ptr lhsPtr, std::shared_ptr rhsPtr) { + if (lhsPtr == nullptr || rhsPtr == nullptr) { + return false; + } return lhsPtr->getContentFullPath() == rhsPtr->getContentFullPath();}; resource_claim_check_ = [](std::shared_ptr claim) { return claim->getFlowFileRecordOwnedCount() <= 0;}; @@ -38,6 +46,23 @@ bool VolatileContentRepository::initialize(const std::shared_ptr &con remove(claim); } }; + + if (configure != nullptr) { + bool minimize_locking = false; + std::string value; + std::stringstream strstream; + strstream << Configure::nifi_volatile_repository_options << getName() << "." << minimal_locking; + if (configure->get(strstream.str(), value)) { + utils::StringUtils::StringToBool(value, minimize_locking); + minimize_locking_ = minimize_locking; + } + } + if (!minimize_locking_) { + for (auto ent : value_vector_) { + delete ent; + } + value_vector_.clear(); + } start(); return true; @@ -64,22 +89,46 @@ void VolatileContentRepository::start() { std::shared_ptr VolatileContentRepository::write(const std::shared_ptr &claim) { logger_->log_debug("enter write"); { - std::lock_guard < std::mutex > lock(map_mutex_); + std::lock_guard lock(map_mutex_); auto claim_check = master_list_.find(claim->getContentFullPath()); if (claim_check != master_list_.end()) { - return std::make_shared < io::AtomicEntryStream>>(claim, claim_check->second); + logger_->log_debug("Creating copy of atomic entry"); + auto ent = claim_check->second->takeOwnership(); + if (ent == nullptr) { + logger_->log_debug("write returns nullptr"); + return nullptr; + } + return std::make_shared>>(claim, ent); } } int size = 0; - for (auto ent : value_vector_) { - if (ent->testAndSetKey(claim, nullptr, nullptr, resource_claim_comparator_)) { - std::lock_guard < std::mutex > lock(map_mutex_); - master_list_[claim->getContentFullPath()] = ent; - return std::make_shared < io::AtomicEntryStream>>(claim, ent); + if (__builtin_expect(minimize_locking_ == true, 1)) { + logger_->log_debug("Minimize lkocking"); + for (auto ent : value_vector_) { + if (ent->testAndSetKey(claim, nullptr, nullptr, resource_claim_comparator_)) { + std::lock_guard lock(map_mutex_); + master_list_[claim->getContentFullPath()] = ent; + return std::make_shared>>(claim, ent); + } + size++; + } + } else { + std::lock_guard < std::mutex > lock(map_mutex_); + auto claim_check = master_list_.find(claim->getContentFullPath()); + if (claim_check != master_list_.end()) { + logger_->log_debug("Creating copy of atomic entry"); + return std::make_shared < io::AtomicEntryStream>>(claim, claim_check->second); + } else { + logger_->log_debug("Creating new atomic entry"); + AtomicEntry> *ent = new AtomicEntry>(¤t_size_, &max_size_); + if (ent->testAndSetKey(claim, nullptr, nullptr, resource_claim_comparator_)) { + master_list_[claim->getContentFullPath()] = ent; + return std::make_shared< io::AtomicEntryStream>>(claim, ent); + } } - size++; } + logger_->log_debug("write returns nullptr %d", size); return nullptr; } @@ -87,37 +136,42 @@ std::shared_ptr VolatileContentRepository::read(const std::share logger_->log_debug("enter read"); int size = 0; { - std::lock_guard < std::mutex > lock(map_mutex_); + std::lock_guard lock(map_mutex_); auto claim_check = master_list_.find(claim->getContentFullPath()); if (claim_check != master_list_.end()) { - return std::make_shared < io::AtomicEntryStream>>(claim, claim_check->second); + auto ent = claim_check->second->takeOwnership(); + if (ent == nullptr) { + return nullptr; + } + return std::make_shared>>(claim, ent); } } - - for (auto ent : value_vector_) { - RepoValue> *repo_value; - - if (ent->getValue(claim, &repo_value)) { - return std::make_shared < io::AtomicEntryStream>>(claim, ent); - } - size++; - } logger_->log_debug("enter read for %s after %d", claim->getContentFullPath(), size); return nullptr; } bool VolatileContentRepository::remove(const std::shared_ptr &claim) { - logger_->log_debug("enter remove"); - - for (auto ent : value_vector_) { - // let the destructor do the cleanup - if (ent->freeValue(claim)) { - std::lock_guard < std::mutex > lock(map_mutex_); + logger_->log_debug("enter remove for %s, reducing %d", claim->getContentFullPath(), current_size_.load()); + if (__builtin_expect(minimize_locking_ == true, 1)) { + std::lock_guard lock(map_mutex_); + auto ent = master_list_.find(claim->getContentFullPath()); + if (ent != master_list_.end()) { + // if we cannot remove the entry we will let the owner's destructor + // decrement the reference count and free it + if (ent->second->freeValue(claim)) { + logger_->log_debug("removed %s", claim->getContentFullPath()); + return true; + } master_list_.erase(claim->getContentFullPath()); - logger_->log_debug("removed %s", claim->getContentFullPath()); - return true; } + } else { + std::lock_guard lock(map_mutex_); + delete master_list_[claim->getContentFullPath()]; + master_list_.erase(claim->getContentFullPath()); + return true; } + + logger_->log_debug("could not remove %s", claim->getContentFullPath()); return false; } diff --git a/libminifi/src/core/yaml/YamlConfiguration.cpp b/libminifi/src/core/yaml/YamlConfiguration.cpp index d205f7f0fa..b5d9a8f984 100644 --- a/libminifi/src/core/yaml/YamlConfiguration.cpp +++ b/libminifi/src/core/yaml/YamlConfiguration.cpp @@ -313,7 +313,7 @@ void YamlConfiguration::parseProvenanceReportingYaml(YAML::Node *reportNode, cor std::shared_ptr processor = nullptr; processor = createProvenanceReportTask(); - std::shared_ptr reportTask = std::static_pointer_cast < core::reporting::SiteToSiteProvenanceReportingTask > (processor); + std::shared_ptr reportTask = std::static_pointer_cast(processor); YAML::Node node = reportNode->as(); @@ -401,9 +401,9 @@ void YamlConfiguration::parseControllerServices(YAML::Node *controllerServicesNo controller_service_node->initialize(); YAML::Node propertiesNode = controllerServiceNode["Properties"]; // we should propogate propertiets to the node and to the implementation - parsePropertiesNodeYaml(&propertiesNode, std::static_pointer_cast < core::ConfigurableComponent > (controller_service_node)); + parsePropertiesNodeYaml(&propertiesNode, std::static_pointer_cast(controller_service_node)); if (controller_service_node->getControllerServiceImplementation() != nullptr) { - parsePropertiesNodeYaml(&propertiesNode, std::static_pointer_cast < core::ConfigurableComponent > (controller_service_node->getControllerServiceImplementation())); + parsePropertiesNodeYaml(&propertiesNode, std::static_pointer_cast(controller_service_node->getControllerServiceImplementation())); } } controller_services_->put(id, controller_service_node); @@ -450,6 +450,24 @@ void YamlConfiguration::parseConnectionYaml(YAML::Node *connectionsNode, core::P uuid_t srcUUID; + if (connectionNode["max work queue size"]) { + auto max_work_queue_str = connectionNode["max work queue size"].as(); + int64_t max_work_queue_size = 0; + if (core::Property::StringToInt(max_work_queue_str, max_work_queue_size)) { + connection->setMaxQueueSize(max_work_queue_size); + } + logger_->log_debug("Setting %d as the max queue size for %s", max_work_queue_size, name); + } + + if (connectionNode["max work queue data size"]) { + auto max_work_queue_str = connectionNode["max work queue data size"].as(); + int64_t max_work_queue_data_size = 0; + if (core::Property::StringToInt(max_work_queue_str, max_work_queue_data_size)) { + connection->setMaxQueueDataSize(max_work_queue_data_size); + } + logger_->log_debug("Setting %d as the max queue data size for %s", max_work_queue_data_size, name); + } + if (connectionNode["source id"]) { std::string connectionSrcProcId = connectionNode["source id"].as(); uuid_parse(connectionSrcProcId.c_str(), srcUUID); @@ -560,7 +578,7 @@ void YamlConfiguration::parsePortYaml(YAML::Node *portNode, core::ProcessGroup * port = std::make_shared(stream_factory_, nameStr, parent->getURL(), this->configuration_, uuid); - processor = std::static_pointer_cast < core::Processor > (port); + processor = std::static_pointer_cast(port); port->setDirection(direction); port->setTimeOut(parent->getTimeOut()); port->setTransmitting(true); @@ -570,7 +588,7 @@ void YamlConfiguration::parsePortYaml(YAML::Node *portNode, core::ProcessGroup * // handle port properties YAML::Node nodeVal = portNode->as(); YAML::Node propertiesNode = nodeVal["Properties"]; - parsePropertiesNodeYaml(&propertiesNode, std::static_pointer_cast < core::ConfigurableComponent > (processor)); + parsePropertiesNodeYaml(&propertiesNode, std::static_pointer_cast(processor)); // add processor to parent parent->addProcessor(processor); @@ -602,7 +620,7 @@ void YamlConfiguration::parsePropertiesNodeYaml(YAML::Node *propertiesNode, std: std::string rawValueString = propertiesNode.as(); logger_->log_info("Found %s=%s", propertyName, rawValueString); if (!processor->updateProperty(propertyName, rawValueString)) { - std::shared_ptr proc = std::dynamic_pointer_cast < core::Connectable > (processor); + std::shared_ptr proc = std::dynamic_pointer_cast(processor); if (proc != 0) { logger_->log_warn("Received property %s with value %s but is not one of the properties for %s", propertyName, rawValueString, proc->getName()); } @@ -612,7 +630,7 @@ void YamlConfiguration::parsePropertiesNodeYaml(YAML::Node *propertiesNode, std: } else { std::string rawValueString = propertyValueNode.as(); if (!processor->setProperty(propertyName, rawValueString)) { - std::shared_ptr proc = std::dynamic_pointer_cast < core::Connectable > (processor); + std::shared_ptr proc = std::dynamic_pointer_cast(processor); if (proc != 0) { logger_->log_warn("Received property %s with value %s but is not one of the properties for %s", propertyName, rawValueString, proc->getName()); } diff --git a/libminifi/src/io/FileStream.cpp b/libminifi/src/io/FileStream.cpp index 721c5f486e..3b2bfe1614 100644 --- a/libminifi/src/io/FileStream.cpp +++ b/libminifi/src/io/FileStream.cpp @@ -28,10 +28,32 @@ namespace nifi { namespace minifi { namespace io { -FileStream::FileStream(const std::string &path, uint32_t offset) - : logger_(logging::LoggerFactory::getLogger()) { - file_stream_ = std::unique_ptr < std::fstream > (new std::fstream()); - file_stream_->open(path.c_str(), std::ios::in | std::ios::out | std::ios::binary); +FileStream::FileStream(const std::string &path) + : logger_(logging::LoggerFactory::getLogger()), + path_(path), + offset_(0) { + file_stream_ = std::unique_ptr(new std::fstream()); + file_stream_->open(path.c_str(), std::fstream::out | std::fstream::binary); + file_stream_->seekg(0, file_stream_->end); + file_stream_->seekp(0, file_stream_->end); + int len = file_stream_->tellg(); + if (len > 0) { + length_ = len; + } else { + length_ = 0; + } + seek(offset_); +} + +FileStream::FileStream(const std::string &path, uint32_t offset, bool write_enable) + : logger_(logging::LoggerFactory::getLogger()), + path_(path) { + file_stream_ = std::unique_ptr(new std::fstream()); + if (write_enable) { + file_stream_->open(path.c_str(), std::fstream::in | std::fstream::out | std::fstream::binary); + } else { + file_stream_->open(path.c_str(), std::fstream::in | std::fstream::binary); + } file_stream_->seekg(0, file_stream_->end); file_stream_->seekp(0, file_stream_->end); int len = file_stream_->tellg(); @@ -43,8 +65,16 @@ FileStream::FileStream(const std::string &path, uint32_t offset) seek(offset); } +void FileStream::closeStream() { + std::lock_guard lock(file_lock_); + if (file_stream_ != nullptr) { + file_stream_->close(); + file_stream_ = nullptr; + } +} + void FileStream::seek(uint64_t offset) { - std::lock_guard < std::recursive_mutex > lock(file_lock_); + std::lock_guard lock(file_lock_); offset_ = offset; file_stream_->clear(); file_stream_->seekg(offset_); @@ -52,8 +82,9 @@ void FileStream::seek(uint64_t offset) { } int FileStream::writeData(std::vector &buf, int buflen) { - if (buf.capacity() < buflen) + if (buf.capacity() < buflen) { return -1; + } return writeData(reinterpret_cast(&buf[0]), buflen); } @@ -61,17 +92,20 @@ int FileStream::writeData(std::vector &buf, int buflen) { int FileStream::writeData(uint8_t *value, int size) { if (!IsNullOrEmpty(value)) { - std::lock_guard < std::recursive_mutex > lock(file_lock_); - file_stream_->write(reinterpret_cast(value), size); - offset_ += size; - if (offset_ > length_) { - length_ = offset_; + std::lock_guard lock(file_lock_); + if (file_stream_->write(reinterpret_cast(value), size)) { + offset_ += size; + if (offset_ > length_) { + length_ = offset_; + } + file_stream_->seekg(offset_); + file_stream_->flush(); + return size; + } else { + return -1; } - file_stream_->seekg(offset_); - file_stream_->flush(); - return size; } else { - return 0; + return -1; } } @@ -97,12 +131,15 @@ int FileStream::readData(std::vector &buf, int buflen) { int FileStream::readData(uint8_t *buf, int buflen) { if (!IsNullOrEmpty(buf)) { - std::lock_guard < std::recursive_mutex > lock(file_lock_); + std::lock_guard lock(file_lock_); file_stream_->read(reinterpret_cast(buf), buflen); if ((file_stream_->rdstate() & (file_stream_->eofbit | file_stream_->failbit)) != 0) { file_stream_->clear(); - offset_ = length_; - file_stream_->seekp(offset_); + file_stream_->seekg(0, file_stream_->end); + file_stream_->seekp(0, file_stream_->end); + int len = file_stream_->tellg(); + offset_ = len; + length_ = len; return offset_; } else { offset_ += buflen; @@ -111,7 +148,7 @@ int FileStream::readData(uint8_t *buf, int buflen) { } } else { - return 0; + return -1; } } diff --git a/libminifi/test/TestBase.cpp b/libminifi/test/TestBase.cpp index c773299d48..4c0814dfeb 100644 --- a/libminifi/test/TestBase.cpp +++ b/libminifi/test/TestBase.cpp @@ -82,7 +82,7 @@ bool linkToPrevious) { processor_nodes_.push_back(node); - std::shared_ptr context = std::make_shared(*(node.get()), controller_services_provider_, flow_repo_, content_repo_); + std::shared_ptr context = std::make_shared(*(node.get()), controller_services_provider_, flow_repo_, prov_repo_, content_repo_); processor_contexts_.push_back(context); processor_queue_.push_back(processor); diff --git a/libminifi/test/integration/TestExecuteProcess.cpp b/libminifi/test/integration/TestExecuteProcess.cpp index 16cb133430..5506c321c8 100644 --- a/libminifi/test/integration/TestExecuteProcess.cpp +++ b/libminifi/test/integration/TestExecuteProcess.cpp @@ -81,7 +81,7 @@ int main(int argc, char **argv) { core::ProcessorNode node2(processor); std::shared_ptr controller_services_provider = nullptr; - std::shared_ptr contextset = std::make_shared(node2, controller_services_provider, test_repo); + std::shared_ptr contextset = std::make_shared(node2, controller_services_provider, test_repo, test_repo); core::ProcessSessionFactory factory(contextset.get()); processor->onSchedule(contextset.get(), &factory); @@ -89,7 +89,7 @@ int main(int argc, char **argv) { processor_workers.push_back(std::thread([processor, test_repo, &is_ready]() { core::ProcessorNode node(processor); std::shared_ptr controller_services_provider = nullptr; - std::shared_ptr context = std::make_shared(node, controller_services_provider, test_repo); + std::shared_ptr context = std::make_shared(node, controller_services_provider, test_repo, test_repo); context->setProperty(org::apache::nifi::minifi::processors::ExecuteProcess::Command, "sleep 0.5"); std::shared_ptr session = std::make_shared(context.get()); while (!is_ready.load(std::memory_order_relaxed)) { diff --git a/libminifi/test/unit/FileStreamTests.cpp b/libminifi/test/unit/FileStreamTests.cpp index f5af5e3e14..5c86f1917f 100644 --- a/libminifi/test/unit/FileStreamTests.cpp +++ b/libminifi/test/unit/FileStreamTests.cpp @@ -35,7 +35,7 @@ TEST_CASE("TestFileOverWrite", "[TestFiles]") { file << "tempFile"; file.close(); - minifi::io::FileStream stream(path); + minifi::io::FileStream stream(path, 0, true); std::vector readBuffer; REQUIRE(stream.readData(readBuffer, stream.getSize()) == stream.getSize()); @@ -73,7 +73,7 @@ TEST_CASE("TestFileBadArgumentNoChange", "[TestLoader]") { file << "tempFile"; file.close(); - minifi::io::FileStream stream(path); + minifi::io::FileStream stream(path, 0, true); std::vector readBuffer; REQUIRE(stream.readData(readBuffer, stream.getSize()) == stream.getSize()); @@ -111,7 +111,7 @@ TEST_CASE("TestFileBadArgumentNoChange2", "[TestLoader]") { file << "tempFile"; file.close(); - minifi::io::FileStream stream(path); + minifi::io::FileStream stream(path, 0, true); std::vector readBuffer; REQUIRE(stream.readData(readBuffer, stream.getSize()) == stream.getSize()); @@ -149,7 +149,7 @@ TEST_CASE("TestFileBadArgumentNoChange3", "[TestLoader]") { file << "tempFile"; file.close(); - minifi::io::FileStream stream(path); + minifi::io::FileStream stream(path, 0, true); std::vector readBuffer; REQUIRE(stream.readData(readBuffer, stream.getSize()) == stream.getSize()); @@ -165,7 +165,7 @@ TEST_CASE("TestFileBadArgumentNoChange3", "[TestLoader]") { std::vector verifybuffer; - REQUIRE(stream.readData(nullptr, stream.getSize()) == 0); + REQUIRE(stream.readData(nullptr, stream.getSize()) == -1); data = verifybuffer.data(); @@ -188,7 +188,7 @@ TEST_CASE("TestFileBeyondEnd3", "[TestLoader]") { file << "tempFile"; file.close(); - minifi::io::FileStream stream(path); + minifi::io::FileStream stream(path, 0, true); std::vector readBuffer; REQUIRE(stream.readData(readBuffer, stream.getSize()) == stream.getSize()); diff --git a/libminifi/test/unit/InvokeHTTPTests.cpp b/libminifi/test/unit/InvokeHTTPTests.cpp index 8d0004f798..2ef3c17ccf 100644 --- a/libminifi/test/unit/InvokeHTTPTests.cpp +++ b/libminifi/test/unit/InvokeHTTPTests.cpp @@ -86,8 +86,8 @@ TEST_CASE("HTTPTestsWithNoResourceClaimPOST", "[httptest1]") { core::ProcessorNode node(listenhttp); core::ProcessorNode node2(invokehttp); std::shared_ptr controller_services_provider = nullptr; - core::ProcessContext context(node, controller_services_provider, repo); - core::ProcessContext context2(node2, controller_services_provider, repo); + core::ProcessContext context(node, controller_services_provider, repo, repo, content_repo); + core::ProcessContext context2(node2, controller_services_provider, repo, repo, content_repo); context.setProperty(org::apache::nifi::minifi::processors::ListenHTTP::Port, "8686"); context.setProperty(org::apache::nifi::minifi::processors::ListenHTTP::BasePath, "/testytesttest"); @@ -207,8 +207,8 @@ TEST_CASE("HTTPTestsWithResourceClaimPOST", "[httptest1]") { core::ProcessorNode node(invokehttp); core::ProcessorNode node2(listenhttp); std::shared_ptr controller_services_provider = nullptr; - core::ProcessContext context(node, controller_services_provider, repo); - core::ProcessContext context2(node2, controller_services_provider, repo); + core::ProcessContext context(node, controller_services_provider, repo, repo, content_repo); + core::ProcessContext context2(node2, controller_services_provider, repo, repo, content_repo); context.setProperty(org::apache::nifi::minifi::processors::ListenHTTP::Port, "8680"); context.setProperty(org::apache::nifi::minifi::processors::ListenHTTP::BasePath, "/testytesttest"); diff --git a/libminifi/test/unit/ProcessorTests.cpp b/libminifi/test/unit/ProcessorTests.cpp index 10ddd6f32a..80801bc1ec 100644 --- a/libminifi/test/unit/ProcessorTests.cpp +++ b/libminifi/test/unit/ProcessorTests.cpp @@ -74,7 +74,7 @@ TEST_CASE("Test GetFileLikeIt'sThreaded", "[getfileCreate3]") { core::ProcessorNode node(processor); std::shared_ptr controller_services_provider = nullptr; - core::ProcessContext context(node, controller_services_provider, test_repo); + core::ProcessContext context(node, controller_services_provider, test_repo, test_repo); core::ProcessSessionFactory factory(&context); context.setProperty(org::apache::nifi::minifi::processors::GetFile::Directory, dir); // replicate 10 threads From 6ec2ae139b64b486fd8bad4bc642e0006289a43c Mon Sep 17 00:00:00 2001 From: Marc Parisi Date: Wed, 21 Jun 2017 11:26:15 -0400 Subject: [PATCH 4/6] MINIFI-249: Update per review comments --- libminifi/include/core/FlowFile.h | 4 --- .../core/repository/AtomicRepoEntries.h | 28 +++++++++---------- .../core/repository/VolatileRepository.h | 3 +- 3 files changed, 16 insertions(+), 19 deletions(-) diff --git a/libminifi/include/core/FlowFile.h b/libminifi/include/core/FlowFile.h index ff8e8cb6de..95b901b216 100644 --- a/libminifi/include/core/FlowFile.h +++ b/libminifi/include/core/FlowFile.h @@ -224,10 +224,6 @@ class FlowFile { void setStoredToRepository(bool storedInRepository) { stored = storedInRepository; - if (!stored && nullptr != claim_) - { - claim_->decreaseFlowFileRecordOwnedCount(); - } } bool isStored() { diff --git a/libminifi/include/core/repository/AtomicRepoEntries.h b/libminifi/include/core/repository/AtomicRepoEntries.h index c7057a7682..c68106054c 100644 --- a/libminifi/include/core/repository/AtomicRepoEntries.h +++ b/libminifi/include/core/repository/AtomicRepoEntries.h @@ -185,8 +185,8 @@ class AtomicEntry { explicit AtomicEntry(std::atomic *total_size, size_t *max_size) : write_pending_(false), has_value_(false), - total_size_(total_size), - max_size_(max_size), + accumulated_repo_size_(total_size), + max_repo_size_(max_size), ref_count_(0), free_required(false) { @@ -253,8 +253,8 @@ class AtomicEntry { size_t bufferSize = value_.getBufferSize(); value_.clearBuffer(); has_value_ = false; - if (total_size_ != nullptr) { - *total_size_ -= bufferSize; + if (accumulated_repo_size_ != nullptr) { + *accumulated_repo_size_ -= bufferSize; } free_required = false; } @@ -323,8 +323,8 @@ class AtomicEntry { size_t bufferSize = value_.getBufferSize(); value_.clearBuffer(); has_value_ = false; - if (total_size_ != nullptr) { - *total_size_ -= bufferSize; + if (accumulated_repo_size_ != nullptr) { + *accumulated_repo_size_ -= bufferSize; } free_required = false; } @@ -374,8 +374,8 @@ class AtomicEntry { value_.clearBuffer(); ref = value_.getKey(); has_value_ = false; - if (total_size_ != nullptr) { - *total_size_ -= bufferSize; + if (accumulated_repo_size_ != nullptr) { + *accumulated_repo_size_ -= bufferSize; } } @@ -416,8 +416,8 @@ class AtomicEntry { size_t bufferSize = value_.getBufferSize(); value_.clearBuffer(); has_value_ = false; - if (total_size_ != nullptr) { - *total_size_ -= bufferSize; + if (accumulated_repo_size_ != nullptr) { + *accumulated_repo_size_ -= bufferSize; } free_required = false; try_unlock(); @@ -441,14 +441,14 @@ class AtomicEntry { return false; } - if ((total_size_ != nullptr && max_size_ != nullptr) && (*total_size_ + size > *max_size_)) { + if ((accumulated_repo_size_ != nullptr && max_repo_size_ != nullptr) && (*accumulated_repo_size_ + size > *max_repo_size_)) { // can't support this write try_unlock(); return false; } value_.append(buffer, size); - (*total_size_) += size; + (*accumulated_repo_size_) += size; try_unlock(); return true; } @@ -478,9 +478,9 @@ class AtomicEntry { } // atomic size pointer. - std::atomic *total_size_; + std::atomic *accumulated_repo_size_; // max size - size_t *max_size_; + size_t *max_repo_size_; // determines if a write is pending. std::atomic write_pending_; // used to determine if a value is present in this atomic entry. diff --git a/libminifi/include/core/repository/VolatileRepository.h b/libminifi/include/core/repository/VolatileRepository.h index 295a61deb2..dc22aa03dc 100644 --- a/libminifi/include/core/repository/VolatileRepository.h +++ b/libminifi/include/core/repository/VolatileRepository.h @@ -138,7 +138,8 @@ class VolatileRepository : public core::Repository, public std::enable_shared_fr std::atomic current_size_; // current index. std::atomic current_index_; - // value vector. + // value vector that exists for non blocking iteration over + // objects that store data for this repo instance. std::vector*> value_vector_; // max count we are allowed to store. From 853773a6c7df7632e287cfd69b7eb2ff44972529 Mon Sep 17 00:00:00 2001 From: Marc Parisi Date: Thu, 29 Jun 2017 20:05:24 -0400 Subject: [PATCH 5/6] MINIFI-249: Correct linter errors --- libminifi/src/ResourceClaim.cpp | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/libminifi/src/ResourceClaim.cpp b/libminifi/src/ResourceClaim.cpp index 67c0e4cc22..e7d455752f 100644 --- a/libminifi/src/ResourceClaim.cpp +++ b/libminifi/src/ResourceClaim.cpp @@ -42,18 +42,15 @@ ResourceClaim::ResourceClaim(std::shared_ptr> claim_manager_(claim_manager), deleted_(false), logger_(logging::LoggerFactory::getLogger()) { - // Create the full content path for the content + // Create the full content path for the content _contentFullPath = contentDirectory + "/" + non_repeating_string_generator_.generate(); logger_->log_debug("Resource Claim created %s", _contentFullPath); } ResourceClaim::ResourceClaim(const std::string path, std::shared_ptr> claim_manager, bool deleted) : claim_manager_(claim_manager), - deleted_(deleted) -{ + deleted_(deleted) { _contentFullPath = path; - - } } /* namespace minifi */ From dde7d18f47b422d10459c6fcee540853cb45ceee Mon Sep 17 00:00:00 2001 From: Marc Parisi Date: Thu, 6 Jul 2017 11:39:13 -0400 Subject: [PATCH 6/6] MINIFI-249: Update removals and link FlowFileRepo to content repo --- .../core/repository/FlowFileRepository.h | 10 +- .../repository/VolatileContentRepository.h | 17 +- .../core/repository/VolatileRepository.h | 3 +- libminifi/include/provenance/Provenance.h | 33 ++- .../include/provenance/ProvenanceRepository.h | 3 +- libminifi/include/utils/ByteInputCallBack.h | 14 +- libminifi/src/FlowFileRecord.cpp | 3 + libminifi/src/core/ProcessSession.cpp | 16 +- libminifi/src/core/Repository.cpp | 3 +- .../core/repository/FlowFileRepository.cpp | 63 +++--- .../repository/VolatileContentRepository.cpp | 2 +- libminifi/src/processors/InvokeHTTP.cpp | 5 +- .../src/provenance/ProvenanceRepository.cpp | 17 +- libminifi/test/unit/ProcessorTests.cpp | 3 +- libminifi/test/unit/TailFileTests.cpp | 191 +++++++++--------- 15 files changed, 210 insertions(+), 173 deletions(-) diff --git a/libminifi/include/core/repository/FlowFileRepository.h b/libminifi/include/core/repository/FlowFileRepository.h index d3f1a34af9..28b9c05073 100644 --- a/libminifi/include/core/repository/FlowFileRepository.h +++ b/libminifi/include/core/repository/FlowFileRepository.h @@ -98,6 +98,7 @@ class FlowFileRepository : public core::Repository, public std::enable_shared_fr // persistent to the DB leveldb::Slice value((const char *) buf, bufLen); leveldb::Status status; + repo_size_+=bufLen; status = db_->Put(leveldb::WriteOptions(), key, value); if (status.ok()) return true; @@ -113,7 +114,9 @@ class FlowFileRepository : public core::Repository, public std::enable_shared_fr leveldb::Status status; status = db_->Delete(leveldb::WriteOptions(), key); if (status.ok()) + { return true; + } else return false; } @@ -137,12 +140,15 @@ class FlowFileRepository : public core::Repository, public std::enable_shared_fr void start() { if (this->purge_period_ <= 0) + { return; + } if (running_) + { return; - thread_ = std::thread(&FlowFileRepository::run, shared_from_this()); - thread_.detach(); + } running_ = true; + thread_ = std::thread(&FlowFileRepository::run, shared_from_this()); logger_->log_info("%s Repository Monitor Thread Start", name_.c_str()); } diff --git a/libminifi/include/core/repository/VolatileContentRepository.h b/libminifi/include/core/repository/VolatileContentRepository.h index 73d9c6f820..306a81248b 100644 --- a/libminifi/include/core/repository/VolatileContentRepository.h +++ b/libminifi/include/core/repository/VolatileContentRepository.h @@ -38,20 +38,21 @@ namespace repository { */ class VolatileContentRepository : public core::ContentRepository, public core::repository::VolatileRepository> { public: - + static const char *minimal_locking; - + explicit VolatileContentRepository(std::string name = getClassName()) : core::repository::VolatileRepository>(name), - logger_(logging::LoggerFactory::getLogger()), minimize_locking_(true) { + logger_(logging::LoggerFactory::getLogger()), + minimize_locking_(true) { max_count_ = 15000; } virtual ~VolatileContentRepository() { - if (!minimize_locking_){ - std::lock_guard < std::mutex > lock(map_mutex_); - for(const auto &item : master_list_) + if (!minimize_locking_) { + std::lock_guard lock(map_mutex_); + for (const auto &item : master_list_) { - delete item.second; + delete item.second; } master_list_.clear(); } @@ -109,7 +110,7 @@ class VolatileContentRepository : public core::ContentRepository, public core::r } private: - + bool minimize_locking_; // function pointers that are associated with the claims. diff --git a/libminifi/include/core/repository/VolatileRepository.h b/libminifi/include/core/repository/VolatileRepository.h index dc22aa03dc..958d91a9e6 100644 --- a/libminifi/include/core/repository/VolatileRepository.h +++ b/libminifi/include/core/repository/VolatileRepository.h @@ -362,9 +362,8 @@ void VolatileRepository::start() { return; if (running_) return; - thread_ = std::thread(&VolatileRepository::run, std::enable_shared_from_this>::shared_from_this()); - thread_.detach(); running_ = true; + thread_ = std::thread(&VolatileRepository::run, std::enable_shared_from_this>::shared_from_this()); logger_->log_info("%s Repository Monitor Thread Start", name_); } diff --git a/libminifi/include/provenance/Provenance.h b/libminifi/include/provenance/Provenance.h index ac4059565b..b9415dc79b 100644 --- a/libminifi/include/provenance/Provenance.h +++ b/libminifi/include/provenance/Provenance.h @@ -164,9 +164,8 @@ class ProvenanceEventRecord : public core::SerializableComponent { */ ProvenanceEventRecord(ProvenanceEventType event, std::string componentId, std::string componentType); - ProvenanceEventRecord() - : core::SerializableComponent(core::getClassName()){ + : core::SerializableComponent(core::getClassName()) { _eventTime = getTimeMillis(); } @@ -361,6 +360,34 @@ class ProvenanceEventRecord : public core::SerializableComponent { // DeSerialize bool DeSerialize(const std::shared_ptr &repo); + uint64_t getEventTime(const uint8_t *buffer, const size_t bufferSize) { + + int size = bufferSize > 72 ? 72 : bufferSize; + org::apache::nifi::minifi::io::DataStream outStream(buffer, size); + + std::string uuid; + int ret = readUTF(uuid, &outStream); + + if (ret <= 0) { + return 0; + } + + uint32_t eventType; + ret = read(eventType, &outStream); + if (ret != 4) { + return 0; + } + + uint64_t event_time; + + ret = read(event_time, &outStream); + if (ret != 8) { + return 0; + } + + return event_time; + } + protected: // Event type @@ -440,7 +467,7 @@ class ProvenanceReporter { // Add event void add(ProvenanceEventRecord *event) { _events.insert(event); - logger_->log_debug("Prove reporter now %d",_events.size()); + logger_->log_debug("Prove reporter now %d", _events.size()); } // Remove event void remove(ProvenanceEventRecord *event) { diff --git a/libminifi/include/provenance/ProvenanceRepository.h b/libminifi/include/provenance/ProvenanceRepository.h index 8f669678e3..ea78a3ce09 100644 --- a/libminifi/include/provenance/ProvenanceRepository.h +++ b/libminifi/include/provenance/ProvenanceRepository.h @@ -61,9 +61,8 @@ class ProvenanceRepository : public core::Repository, public std::enable_shared_ return; if (running_) return; - thread_ = std::thread(&ProvenanceRepository::run, shared_from_this()); - thread_.detach(); running_ = true; + thread_ = std::thread(&ProvenanceRepository::run, shared_from_this()); logger_->log_info("%s Repository Monitor Thread Start", name_.c_str()); } diff --git a/libminifi/include/utils/ByteInputCallBack.h b/libminifi/include/utils/ByteInputCallBack.h index 059de99e1e..86aae0998c 100644 --- a/libminifi/include/utils/ByteInputCallBack.h +++ b/libminifi/include/utils/ByteInputCallBack.h @@ -42,11 +42,15 @@ class ByteInputCallBack : public InputStreamCallback { int64_t process(std::shared_ptr stream) { - std::vector nv = std::vector(reinterpret_cast(const_cast(stream->getBuffer())), - reinterpret_cast(const_cast(stream->getBuffer())) + stream->getSize()); - vec = std::move(nv); + stream->seek(0); - ptr = &vec[0]; + if (stream->getSize() > 0) { + vec.resize(stream->getSize()); + + stream->readData(vec, stream->getSize()); + } + + ptr = (char*) &vec[0]; return vec.size(); @@ -62,7 +66,7 @@ class ByteInputCallBack : public InputStreamCallback { private: char *ptr; - std::vector vec; + std::vector vec; }; } /* namespace utils */ diff --git a/libminifi/src/FlowFileRecord.cpp b/libminifi/src/FlowFileRecord.cpp index dfe0d07811..efd6fa7f3e 100644 --- a/libminifi/src/FlowFileRecord.cpp +++ b/libminifi/src/FlowFileRecord.cpp @@ -324,6 +324,9 @@ bool FlowFileRecord::DeSerialize(const uint8_t *buffer, const int bufferSize) { return false; } + if (nullptr == claim_) { + claim_ = std::make_shared(content_full_fath_, content_repo_, true); + } return true; } diff --git a/libminifi/src/core/ProcessSession.cpp b/libminifi/src/core/ProcessSession.cpp index 0402efefd9..c69b361957 100644 --- a/libminifi/src/core/ProcessSession.cpp +++ b/libminifi/src/core/ProcessSession.cpp @@ -213,8 +213,7 @@ void ProcessSession::transfer(std::shared_ptr &&flow, Relationsh } void ProcessSession::write(std::shared_ptr &flow, OutputStreamCallback *callback) { - std::shared_ptr claim = std::make_shared(process_context_->getContentRepository(), - DEFAULT_CONTENT_DIRECTORY); + std::shared_ptr claim = std::make_shared(process_context_->getContentRepository()); try { uint64_t startTime = getTimeMillis(); @@ -614,9 +613,7 @@ void ProcessSession::import(std::string source, std::shared_ptr } } -void ProcessSession::import(std::string source, std::vector> flows, -bool keepSource, - uint64_t offset, char inputDelimiter) { +void ProcessSession::import(std::string source, std::vector> flows, bool keepSource, uint64_t offset, char inputDelimiter) { std::shared_ptr claim; std::shared_ptr flowFile; @@ -628,6 +625,7 @@ bool keepSource, try { // Open the input file and seek to the appropriate location. std::ifstream input; + logger_->log_debug("Opening %s", source); input.open(source.c_str(), std::fstream::in | std::fstream::binary); if (input.is_open()) { input.seekg(offset, input.beg); @@ -640,6 +638,7 @@ bool keepSource, std::shared_ptr stream = process_context_->getContentRepository()->write(claim); if (nullptr == stream) { + logger_->log_debug("Stream is null"); rollback(); return; } @@ -677,12 +676,11 @@ bool keepSource, uint64_t endTime = getTimeMillis(); provenance_report_->modifyContent(flowFile, details, endTime - startTime); flows.push_back(flowFile); - } else { + logger_->log_debug("Error while writing"); stream->closeStream(); throw Exception(FILE_OPERATION_EXCEPTION, "File Export Error creating Flowfile"); } - } input.close(); if (!keepSource) @@ -851,7 +849,6 @@ void ProcessSession::commit() { continue; std::map::iterator itRelationship = this->_transferRelationship.find(record->getUUIDStr()); if (itRelationship != _transferRelationship.end()) { - logger_->log_debug("size is %d for there", _addedFlowFiles.size()); Relationship relationship = itRelationship->second; // Find the relationship, we need to find the connections for that relationship std::set> connections = process_context_->getProcessorNode().getOutGoingConnections(relationship.getName()); @@ -867,16 +864,13 @@ void ProcessSession::commit() { remove(record); } } else { - logger_->log_debug("size is %d for here", _addedFlowFiles.size()); // We connections, clone the flow and assign the connection accordingly for (std::set>::iterator itConnection = connections.begin(); itConnection != connections.end(); ++itConnection) { std::shared_ptr connection(*itConnection); - logger_->log_debug("size is %d for here %s", _addedFlowFiles.size(), connection->getName()); if (itConnection == connections.begin()) { // First connection which the flow need be routed to record->setConnection(connection); } else { - logger_->log_debug("clone is %d for here %s", _addedFlowFiles.size(), connection->getName()); // Clone the flow file and route to the connection std::shared_ptr cloneRecord; cloneRecord = this->cloneDuringTransfer(record); diff --git a/libminifi/src/core/Repository.cpp b/libminifi/src/core/Repository.cpp index 33dfd53b3e..cf26a0d158 100644 --- a/libminifi/src/core/Repository.cpp +++ b/libminifi/src/core/Repository.cpp @@ -39,9 +39,8 @@ void Repository::start() { return; if (running_) return; - thread_ = std::thread(&Repository::threadExecutor, this); - thread_.detach(); running_ = true; + thread_ = std::thread(&Repository::threadExecutor, this); logger_->log_info("%s Repository Monitor Thread Start", name_.c_str()); } diff --git a/libminifi/src/core/repository/FlowFileRepository.cpp b/libminifi/src/core/repository/FlowFileRepository.cpp index dd40ca0c17..ac092ea340 100644 --- a/libminifi/src/core/repository/FlowFileRepository.cpp +++ b/libminifi/src/core/repository/FlowFileRepository.cpp @@ -15,12 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include "../../../include/core/repository/FlowFileRepository.h" - +#include "core/repository/FlowFileRepository.h" #include #include +#include #include - #include "FlowFileRecord.h" namespace org { @@ -38,24 +37,36 @@ void FlowFileRepository::run() { uint64_t curTime = getTimeMillis(); uint64_t size = repoSize(); if (size >= purgeThreshold) { - std::vector purgeList; + std::vector> purgeList; + std::vector> keyRemovalList; leveldb::Iterator* it = db_->NewIterator(leveldb::ReadOptions()); for (it->SeekToFirst(); it->Valid(); it->Next()) { - std::shared_ptr eventRead = std::make_shared < FlowFileRecord > (shared_from_this(), content_repo_); + std::shared_ptr eventRead = std::make_shared(shared_from_this(), content_repo_); std::string key = it->key().ToString(); if (eventRead->DeSerialize(reinterpret_cast(it->value().data()), it->value().size())) { - if ((curTime - eventRead->getEventTime()) > max_partition_millis_) - purgeList.push_back(key); + if ((curTime - eventRead->getEventTime()) > max_partition_millis_) { + purgeList.push_back(eventRead); + keyRemovalList.push_back(std::make_pair(key, it->value().size())); + } } else { - logger_->log_debug("NiFi %s retrieve event %s fail", name_, key); - purgeList.push_back(key); + logger_->log_debug("NiFi %s retrieve event %s fail", name_.c_str(), key.c_str()); + keyRemovalList.push_back(std::make_pair(key, it->value().size())); } } delete it; - for (auto eventId : purgeList) { - logger_->log_info("Repository Repo %s Purge %s", name_, eventId); - Delete(eventId); + for (auto eventId : keyRemovalList) { + logger_->log_info("Repository Repo %s Purge %s", name_.c_str(), eventId.first.c_str()); + if (Delete(eventId.first)) { + repo_size_ -= eventId.second; + } + } + + for (const auto &ffr : purgeList) { + auto claim = ffr->getResourceClaim(); + if (claim != nullptr) { + content_repo_->remove(claim); + } } } if (size > max_partition_bytes_) @@ -63,43 +74,43 @@ void FlowFileRepository::run() { else repo_full_ = false; } - return; } void FlowFileRepository::loadComponent(const std::shared_ptr &content_repo) { content_repo_ = content_repo; - std::vector purgeList; + std::vector> purgeList; leveldb::Iterator* it = db_->NewIterator(leveldb::ReadOptions()); for (it->SeekToFirst(); it->Valid(); it->Next()) { - std::shared_ptr eventRead = std::make_shared < FlowFileRecord > (shared_from_this(), content_repo_); + std::shared_ptr eventRead = std::make_shared(shared_from_this(), content_repo_); std::string key = it->key().ToString(); + repo_size_ += it->value().size(); if (eventRead->DeSerialize(reinterpret_cast(it->value().data()), it->value().size())) { auto search = connectionMap.find(eventRead->getConnectionUuid()); if (search != connectionMap.end()) { // we find the connection for the persistent flowfile, create the flowfile and enqueue that - std::shared_ptr flow_file_ref = std::static_pointer_cast < core::FlowFile > (eventRead); - std::shared_ptr record = std::make_shared < FlowFileRecord > (shared_from_this(), content_repo_, flow_file_ref); + std::shared_ptr flow_file_ref = std::static_pointer_cast(eventRead); + std::shared_ptr record = std::make_shared(shared_from_this(), content_repo_); // set store to repo to true so that we do need to persistent again in enqueue record->setStoredToRepository(true); search->second->put(record); } else { - if (eventRead->getContentFullPath().length() > 0 && content_repo != nullptr) { - content_repo->remove(eventRead->getResourceClaim()); + if (eventRead->getContentFullPath().length() > 0) { + std::remove(eventRead->getContentFullPath().c_str()); } - purgeList.push_back(key); + purgeList.push_back(std::make_pair(key, it->value().size())); } } else { - purgeList.push_back(key); + purgeList.push_back(std::make_pair(key, it->value().size())); } } delete it; - std::vector::iterator itPurge; - for (itPurge = purgeList.begin(); itPurge != purgeList.end(); itPurge++) { - std::string eventId = *itPurge; - logger_->log_info("Repository Repo %s Purge %s", name_.c_str(), eventId.c_str()); - Delete(eventId); + for (auto eventId : purgeList) { + logger_->log_info("Repository Repo %s Purge %s", name_.c_str(), eventId.first.c_str()); + if (Delete(eventId.first)) { + repo_size_ -= eventId.second; + } } return; diff --git a/libminifi/src/core/repository/VolatileContentRepository.cpp b/libminifi/src/core/repository/VolatileContentRepository.cpp index 30d44b13b1..ac575c56b5 100644 --- a/libminifi/src/core/repository/VolatileContentRepository.cpp +++ b/libminifi/src/core/repository/VolatileContentRepository.cpp @@ -104,7 +104,7 @@ std::shared_ptr VolatileContentRepository::write(const std::shar int size = 0; if (__builtin_expect(minimize_locking_ == true, 1)) { - logger_->log_debug("Minimize lkocking"); + logger_->log_debug("Minimize locking"); for (auto ent : value_vector_) { if (ent->testAndSetKey(claim, nullptr, nullptr, resource_claim_comparator_)) { std::lock_guard lock(map_mutex_); diff --git a/libminifi/src/processors/InvokeHTTP.cpp b/libminifi/src/processors/InvokeHTTP.cpp index fb3893aa64..7dc75d2463 100644 --- a/libminifi/src/processors/InvokeHTTP.cpp +++ b/libminifi/src/processors/InvokeHTTP.cpp @@ -115,7 +115,7 @@ void InvokeHTTP::set_request_method(CURL *curl, const std::string &method) { if (my_method == "POST") { curl_easy_setopt(curl, CURLOPT_POST, 1); } else if (my_method == "PUT") { - curl_easy_setopt(curl, CURLOPT_UPLOAD, 1); + curl_easy_setopt(curl, CURLOPT_PUT, 1); } else if (my_method == "GET") { } else { curl_easy_setopt(curl, CURLOPT_CUSTOMREQUEST, my_method.c_str()); @@ -335,7 +335,6 @@ void InvokeHTTP::onTrigger(core::ProcessContext *context, core::ProcessSession * callbackObj->ptr = callback; callbackObj->pos = 0; logger_->log_info("InvokeHTTP -- Setting callback"); - curl_easy_setopt(http_session, CURLOPT_UPLOAD, 1L); curl_easy_setopt(http_session, CURLOPT_INFILESIZE_LARGE, (curl_off_t)callback->getBufferSize()); curl_easy_setopt(http_session, CURLOPT_READFUNCTION, &utils::HTTPRequestResponse::send_write); curl_easy_setopt(http_session, CURLOPT_READDATA, static_cast(callbackObj)); @@ -377,7 +376,7 @@ void InvokeHTTP::onTrigger(core::ProcessContext *context, core::ProcessSession * bool output_body_to_content = isSuccess && !putToAttribute; bool body_empty = IsNullOrEmpty(content.data); - logger_->log_info("isSuccess: %d", isSuccess); + logger_->log_info("isSuccess: %d, response code %d ", isSuccess, http_code); std::shared_ptr response_flow = nullptr; if (output_body_to_content) { diff --git a/libminifi/src/provenance/ProvenanceRepository.cpp b/libminifi/src/provenance/ProvenanceRepository.cpp index b2e3031f2d..ce19fe41ce 100644 --- a/libminifi/src/provenance/ProvenanceRepository.cpp +++ b/libminifi/src/provenance/ProvenanceRepository.cpp @@ -34,27 +34,20 @@ void ProvenanceRepository::run() { uint64_t curTime = getTimeMillis(); uint64_t size = repoSize(); if (size >= purgeThreshold) { - std::vector purgeList; leveldb::Iterator* it = db_->NewIterator(leveldb::ReadOptions()); for (it->SeekToFirst(); it->Valid(); it->Next()) { ProvenanceEventRecord eventRead; std::string key = it->key().ToString(); - if (eventRead.DeSerialize(reinterpret_cast(const_cast(it->value().data())), it->value().size())) { - if ((curTime - eventRead.getEventTime()) > max_partition_millis_) - purgeList.push_back(key); + uint64_t eventTime = eventRead.getEventTime(reinterpret_cast(const_cast(it->value().data())), it->value().size()); + if (eventTime > 0) { + if ((curTime - eventTime) > max_partition_millis_) + Delete(key); } else { logger_->log_debug("NiFi Provenance retrieve event %s fail", key.c_str()); - purgeList.push_back(key); + Delete(key); } } delete it; - std::vector::iterator itPurge; - - for (itPurge = purgeList.begin(); itPurge != purgeList.end(); itPurge++) { - std::string eventId = *itPurge; - logger_->log_info("ProvenanceRepository Repo Purge %s", eventId.c_str()); - Delete(eventId); - } } if (size > max_partition_bytes_) repo_full_ = true; diff --git a/libminifi/test/unit/ProcessorTests.cpp b/libminifi/test/unit/ProcessorTests.cpp index 80801bc1ec..7f34ba45b1 100644 --- a/libminifi/test/unit/ProcessorTests.cpp +++ b/libminifi/test/unit/ProcessorTests.cpp @@ -43,7 +43,6 @@ TEST_CASE("Test Creation of GetFile", "[getfileCreate]") { REQUIRE(processor->getName() == "processorname"); } - TEST_CASE("Test GetFileLikeIt'sThreaded", "[getfileCreate3]") { TestController testController; std::shared_ptr content_repo = std::make_shared(); @@ -173,7 +172,7 @@ TEST_CASE("Test Find file", "[getfileCreate3]") { std::shared_ptr plan = testController.createPlan(); std::shared_ptr processor = plan->addProcessor("GetFile", "getfileCreate2"); std::shared_ptr processorReport = std::make_shared( - std::make_shared(std::make_shared())); + std::make_shared(std::make_shared()), std::make_shared()); plan->addProcessor(processorReport, "reporter", core::Relationship("success", "description"), false); char format[] = "/tmp/gt.XXXXXX"; char *dir = testController.createTempDirectory(format); diff --git a/libminifi/test/unit/TailFileTests.cpp b/libminifi/test/unit/TailFileTests.cpp index 616505ba9b..eb33f8cc69 100644 --- a/libminifi/test/unit/TailFileTests.cpp +++ b/libminifi/test/unit/TailFileTests.cpp @@ -42,134 +42,137 @@ static const char *TMP_FILE = "/tmp/minifi-tmpfile.txt"; static const char *STATE_FILE = "/tmp/minifi-state-file.txt"; TEST_CASE("TailFileWithDelimiter", "[tailfiletest1]") { - try { - // Create and write to the test file - std::ofstream tmpfile; - tmpfile.open(TMP_FILE); - tmpfile << NEWLINE_FILE; - tmpfile.close(); + try { + // Create and write to the test file + std::ofstream tmpfile; + tmpfile.open(TMP_FILE); + tmpfile << NEWLINE_FILE; + tmpfile.close(); - TestController testController; - LogTestController::getInstance().setInfo(); + TestController testController; + LogTestController::getInstance().setDebug(); + LogTestController::getInstance().setDebug(); + LogTestController::getInstance().setDebug(); - std::shared_ptr repo = std::make_shared(); + std::shared_ptr repo = std::make_shared(); - std::shared_ptr processor = std::make_shared("tailfile"); - std::shared_ptr logAttributeProcessor = std::make_shared("logattribute"); + std::shared_ptr processor = std::make_shared("tailfile"); + std::shared_ptr logAttributeProcessor = std::make_shared("logattribute"); - uuid_t processoruuid; - REQUIRE(true == processor->getUUID(processoruuid)); - uuid_t logAttributeuuid; - REQUIRE(true == logAttributeProcessor->getUUID(logAttributeuuid)); + uuid_t processoruuid; + REQUIRE(true == processor->getUUID(processoruuid)); + uuid_t logAttributeuuid; + REQUIRE(true == logAttributeProcessor->getUUID(logAttributeuuid)); - std::shared_ptr content_repo = std::make_shared(); + std::shared_ptr content_repo = std::make_shared(); + content_repo->initialize(std::make_shared()); + std::shared_ptr connection = std::make_shared(repo, content_repo, "logattributeconnection"); + connection->setRelationship(core::Relationship("success", "TailFile successful output")); - std::shared_ptr connection = std::make_shared(repo,content_repo, "logattributeconnection"); - connection->setRelationship(core::Relationship("success", "TailFile successful output")); + // link the connections so that we can test results at the end for this + connection->setDestination(connection); - // link the connections so that we can test results at the end for this - connection->setDestination(connection); + connection->setSourceUUID(processoruuid); - connection->setSourceUUID(processoruuid); + processor->addConnection(connection); - processor->addConnection(connection); + core::ProcessorNode node(processor); - core::ProcessorNode node(processor); + std::shared_ptr controller_services_provider = nullptr; + core::ProcessContext context(node, controller_services_provider, repo, repo, content_repo); + context.setProperty(org::apache::nifi::minifi::processors::TailFile::Delimiter, "\n"); + context.setProperty(org::apache::nifi::minifi::processors::TailFile::FileName, TMP_FILE); + context.setProperty(org::apache::nifi::minifi::processors::TailFile::StateFile, STATE_FILE); - std::shared_ptr controller_services_provider = nullptr; - core::ProcessContext context(node, controller_services_provider, repo); - context.setProperty(org::apache::nifi::minifi::processors::TailFile::Delimiter, "\n"); - context.setProperty(org::apache::nifi::minifi::processors::TailFile::FileName, TMP_FILE); - context.setProperty(org::apache::nifi::minifi::processors::TailFile::StateFile, STATE_FILE); + core::ProcessSession session(&context); - core::ProcessSession session(&context); + REQUIRE(processor->getName() == "tailfile"); - REQUIRE(processor->getName() == "tailfile"); + core::ProcessSessionFactory factory(&context); - core::ProcessSessionFactory factory(&context); + std::shared_ptr record; + processor->setScheduledState(core::ScheduledState::RUNNING); + processor->onSchedule(&context, &factory); + processor->onTrigger(&context, &session); - std::shared_ptr record; - processor->setScheduledState(core::ScheduledState::RUNNING); - processor->onSchedule(&context, &factory); - processor->onTrigger(&context, &session); + provenance::ProvenanceReporter *reporter = session.getProvenanceReporter(); + std::set provRecords = reporter->getEvents(); + record = session.get(); + REQUIRE(record == nullptr); + std::shared_ptr ff = session.get(); + REQUIRE(provRecords.size() == 4); // 2 creates and 2 modifies for flowfiles - provenance::ProvenanceReporter *reporter = session.getProvenanceReporter(); - std::set provRecords = reporter->getEvents(); - record = session.get(); - REQUIRE(record == nullptr); - std::shared_ptr ff = session.get(); - REQUIRE(provRecords.size() == 4); // 2 creates and 2 modifies for flowfiles + LogTestController::getInstance().reset(); + } catch (...) { + } - LogTestController::getInstance().reset(); - } catch (...) { } - - // Delete the test and state file. - std::remove(TMP_FILE); - std::remove(STATE_FILE); + // Delete the test and state file. + std::remove(TMP_FILE); + std::remove(STATE_FILE); } - TEST_CASE("TailFileWithoutDelimiter", "[tailfiletest2]") { - try { - // Create and write to the test file - std::ofstream tmpfile; - tmpfile.open(TMP_FILE); - tmpfile << NEWLINE_FILE; - tmpfile.close(); - - TestController testController; - LogTestController::getInstance().setInfo(); + try { + // Create and write to the test file + std::ofstream tmpfile; + tmpfile.open(TMP_FILE); + tmpfile << NEWLINE_FILE; + tmpfile.close(); - std::shared_ptr repo = std::make_shared(); + TestController testController; + LogTestController::getInstance().setInfo(); - std::shared_ptr processor = std::make_shared("tailfile"); - std::shared_ptr logAttributeProcessor = std::make_shared("logattribute"); + std::shared_ptr repo = std::make_shared(); - uuid_t processoruuid; - REQUIRE(true == processor->getUUID(processoruuid)); - uuid_t logAttributeuuid; - REQUIRE(true == logAttributeProcessor->getUUID(logAttributeuuid)); + std::shared_ptr processor = std::make_shared("tailfile"); + std::shared_ptr logAttributeProcessor = std::make_shared("logattribute"); - std::shared_ptr content_repo = std::make_shared(); + uuid_t processoruuid; + REQUIRE(true == processor->getUUID(processoruuid)); + uuid_t logAttributeuuid; + REQUIRE(true == logAttributeProcessor->getUUID(logAttributeuuid)); - std::shared_ptr connection = std::make_shared(repo, content_repo, "logattributeconnection"); - connection->setRelationship(core::Relationship("success", "TailFile successful output")); + std::shared_ptr content_repo = std::make_shared(); + content_repo->initialize(std::make_shared()); + std::shared_ptr connection = std::make_shared(repo, content_repo, "logattributeconnection"); + connection->setRelationship(core::Relationship("success", "TailFile successful output")); - // link the connections so that we can test results at the end for this - connection->setDestination(connection); - connection->setSourceUUID(processoruuid); + // link the connections so that we can test results at the end for this + connection->setDestination(connection); + connection->setSourceUUID(processoruuid); - processor->addConnection(connection); + processor->addConnection(connection); - core::ProcessorNode node(processor); + core::ProcessorNode node(processor); - std::shared_ptr controller_services_provider = nullptr; - core::ProcessContext context(node, controller_services_provider, repo); - context.setProperty(org::apache::nifi::minifi::processors::TailFile::FileName, TMP_FILE); - context.setProperty(org::apache::nifi::minifi::processors::TailFile::StateFile, STATE_FILE); + std::shared_ptr controller_services_provider = nullptr; + core::ProcessContext context(node, controller_services_provider, repo, repo, content_repo); + context.setProperty(org::apache::nifi::minifi::processors::TailFile::FileName, TMP_FILE); + context.setProperty(org::apache::nifi::minifi::processors::TailFile::StateFile, STATE_FILE); - core::ProcessSession session(&context); + core::ProcessSession session(&context); - REQUIRE(processor->getName() == "tailfile"); + REQUIRE(processor->getName() == "tailfile"); - core::ProcessSessionFactory factory(&context); + core::ProcessSessionFactory factory(&context); - std::shared_ptr record; - processor->setScheduledState(core::ScheduledState::RUNNING); - processor->onSchedule(&context, &factory); - processor->onTrigger(&context, &session); + std::shared_ptr record; + processor->setScheduledState(core::ScheduledState::RUNNING); + processor->onSchedule(&context, &factory); + processor->onTrigger(&context, &session); - provenance::ProvenanceReporter *reporter = session.getProvenanceReporter(); - std::set provRecords = reporter->getEvents(); - record = session.get(); - REQUIRE(record == nullptr); - std::shared_ptr ff = session.get(); - REQUIRE(provRecords.size() == 2); + provenance::ProvenanceReporter *reporter = session.getProvenanceReporter(); + std::set provRecords = reporter->getEvents(); + record = session.get(); + REQUIRE(record == nullptr); + std::shared_ptr ff = session.get(); + REQUIRE(provRecords.size() == 2); - LogTestController::getInstance().reset(); - } catch (...) { } + LogTestController::getInstance().reset(); + } catch (...) { + } - // Delete the test and state file. - std::remove(TMP_FILE); - std::remove(STATE_FILE); + // Delete the test and state file. + std::remove(TMP_FILE); + std::remove(STATE_FILE); }