[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #822: MINIFICPP-1253 - Do not hang on full cycles

2020-07-07 Thread GitBox


adamdebreceni commented on a change in pull request #822:
URL: https://github.com/apache/nifi-minifi-cpp/pull/822#discussion_r450927873



##
File path: libminifi/src/core/Processor.cpp
##
@@ -289,6 +303,109 @@ bool Processor::isWorkAvailable() {
   return hasWork;
 }
 
+// must hold the graphMutex
+void Processor::updateReachability(const std::lock_guard& 
graph_lock, bool force) {
+  bool didChange = force;
+  for (auto& outIt : out_going_connections_) {
+for (auto& outConn : outIt.second) {
+  auto connection = std::dynamic_pointer_cast(outConn);
+  if (!connection) {
+continue;
+  }
+  auto dest = std::dynamic_pointer_cast(connection->getDestination());
+  if (!dest) {
+continue;
+  }
+  if (reachable_processors_[connection].insert(dest).second) {
+didChange = true;
+  }
+  for (auto& reachedIt : dest->reachable_processors_) {
+for (auto _proc : reachedIt.second) {

Review comment:
   I'm all for a O(better) solution  but yeah it probably won't be the 
bottleneck





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #822: MINIFICPP-1253 - Do not hang on full cycles

2020-07-07 Thread GitBox


adamdebreceni commented on a change in pull request #822:
URL: https://github.com/apache/nifi-minifi-cpp/pull/822#discussion_r450927873



##
File path: libminifi/src/core/Processor.cpp
##
@@ -289,6 +303,109 @@ bool Processor::isWorkAvailable() {
   return hasWork;
 }
 
+// must hold the graphMutex
+void Processor::updateReachability(const std::lock_guard& 
graph_lock, bool force) {
+  bool didChange = force;
+  for (auto& outIt : out_going_connections_) {
+for (auto& outConn : outIt.second) {
+  auto connection = std::dynamic_pointer_cast(outConn);
+  if (!connection) {
+continue;
+  }
+  auto dest = std::dynamic_pointer_cast(connection->getDestination());
+  if (!dest) {
+continue;
+  }
+  if (reachable_processors_[connection].insert(dest).second) {
+didChange = true;
+  }
+  for (auto& reachedIt : dest->reachable_processors_) {
+for (auto _proc : reachedIt.second) {

Review comment:
   I'm all for a O(better) solution  





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #822: MINIFICPP-1253 - Do not hang on full cycles

2020-07-07 Thread GitBox


adamdebreceni commented on a change in pull request #822:
URL: https://github.com/apache/nifi-minifi-cpp/pull/822#discussion_r450885343



##
File path: libminifi/test/flow-tests/CustomProcessors.h
##
@@ -0,0 +1,106 @@
+/**
+ *
+ * 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 NIFI_MINIFI_CPP_CUSTOMPROCESSORS_H
+#define NIFI_MINIFI_CPP_CUSTOMPROCESSORS_H
+
+#include 
+#include 
+#include 
+#include 
+#include "core/Processor.h"
+#include "TestBase.h"
+#include "../../extensions/standard-processors/processors/GenerateFlowFile.h"

Review comment:
   done

##
File path: libminifi/src/core/Processor.cpp
##
@@ -101,14 +102,27 @@ void Processor::setScheduledState(ScheduledState state) {
 }
 
 bool Processor::addConnection(std::shared_ptr conn) {
-  bool ret = false;
+  enum class DidSet{

Review comment:
   done





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #822: MINIFICPP-1253 - Do not hang on full cycles

2020-07-07 Thread GitBox


adamdebreceni commented on a change in pull request #822:
URL: https://github.com/apache/nifi-minifi-cpp/pull/822#discussion_r450884786



##
File path: libminifi/test/flow-tests/CustomProcessors.h
##
@@ -0,0 +1,106 @@
+/**
+ *
+ * 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 NIFI_MINIFI_CPP_CUSTOMPROCESSORS_H
+#define NIFI_MINIFI_CPP_CUSTOMPROCESSORS_H
+
+#include 
+#include 
+#include 
+#include 
+#include "core/Processor.h"
+#include "TestBase.h"
+#include "../../extensions/standard-processors/processors/GenerateFlowFile.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+static core::Relationship Apple{"apple", ""};
+static core::Relationship Banana{"banana", ""};
+// The probability that this processor routes to Apple
+static core::Property AppleProbability = 
core::PropertyBuilder::createProperty("AppleProbability")->withDefaultValue(100)->build();
+// The probability that this processor routes to Banana
+static core::Property BananaProbability = 
core::PropertyBuilder::createProperty("BananaProbability")->withDefaultValue(0)->build();
+
+class ProcessorWithStatistics {
+ public:
+  std::atomic trigger_count{0};
+};
+
+class TestProcessor : public core::Processor, public ProcessorWithStatistics {
+ public:
+  TestProcessor(std::string name, utils::Identifier ) : Processor(name, 
uuid) {}
+  TestProcessor(std::string name) : Processor(name) {}
+  void initialize() override {
+setSupportedProperties({AppleProbability, BananaProbability});
+setSupportedRelationships({Apple, Banana});
+  }
+  void onTrigger(const std::shared_ptr , const 
std::shared_ptr ) override {
+++trigger_count;
+auto flowFile = session->get();
+if (!flowFile) return;
+std::random_device rd{};
+std::uniform_int_distribution dis(0, 100);
+int rand = dis(rd);
+if (rand <= apple_probability_) {
+  session->transfer(flowFile, Apple);
+  return;
+}
+rand -= apple_probability_;
+if (rand <= banana_probability_) {
+  session->transfer(flowFile, Banana);
+  return;
+}
+throw std::runtime_error("Couldn't route file");
+  }
+  void onSchedule(const std::shared_ptr , const 
std::shared_ptr ) override {
+int apple;
+assert(context->getProperty(AppleProbability.getName(), apple));
+int banana;
+assert(context->getProperty(BananaProbability.getName(), banana));

Review comment:
   done (although each test started with an `#undef NDEBUG`)





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #822: MINIFICPP-1253 - Do not hang on full cycles

2020-07-07 Thread GitBox


adamdebreceni commented on a change in pull request #822:
URL: https://github.com/apache/nifi-minifi-cpp/pull/822#discussion_r450884735



##
File path: libminifi/test/flow-tests/FlowBuilder.h
##
@@ -0,0 +1,73 @@
+/**
+ *
+ * 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 NIFI_MINIFI_CPP_FLOWCREATOR_H
+#define NIFI_MINIFI_CPP_FLOWCREATOR_H
+
+#include 
+#include 
+#include 
+#include 
+#include "core/Processor.h"
+#include "TestBase.h"
+
+struct Flow{
+  Flow(std::shared_ptr&& controller, 
std::shared_ptr&& root)
+  : controller_(std::move(controller)), root_(std::move(root)) {
+controller_->load(root_);
+controller_->start();
+  }
+  ~Flow() {
+controller_->stop(true);
+controller_->unload();
+  }
+  std::shared_ptr controller_;
+  std::shared_ptr root_;
+};
+
+Flow createFlow(const std::string& yamlPath) {
+  std::shared_ptr configuration = 
std::make_shared();
+  std::shared_ptr prov_repo = 
std::make_shared();
+  std::shared_ptr ff_repo = 
std::make_shared();
+  std::shared_ptr content_repo = 
std::make_shared();
+
+  configuration->set(minifi::Configure::nifi_flow_configuration_file, 
yamlPath);
+
+  std::shared_ptr stream_factory = 
minifi::io::StreamFactory::getInstance(configuration);
+  content_repo->initialize(configuration);
+
+  std::unique_ptr flow = 
utils::make_unique(prov_repo, ff_repo, content_repo, 
stream_factory, configuration, yamlPath);
+  std::shared_ptr root = flow->getRoot();
+
+  std::shared_ptr controller = 
std::make_shared(
+  prov_repo, ff_repo, configuration,
+  std::move(flow),
+  content_repo, DEFAULT_ROOT_GROUP_NAME, true);
+
+  return Flow{std::move(controller), std::move(root)};
+}
+
+double calculateCongestion(const std::shared_ptr& conn) {

Review comment:
   removed

##
File path: libminifi/test/flow-tests/CustomProcessors.h
##
@@ -0,0 +1,106 @@
+/**
+ *
+ * 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 NIFI_MINIFI_CPP_CUSTOMPROCESSORS_H
+#define NIFI_MINIFI_CPP_CUSTOMPROCESSORS_H
+
+#include 
+#include 
+#include 
+#include 
+#include "core/Processor.h"
+#include "TestBase.h"
+#include "../../extensions/standard-processors/processors/GenerateFlowFile.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+static core::Relationship Apple{"apple", ""};
+static core::Relationship Banana{"banana", ""};
+// The probability that this processor routes to Apple
+static core::Property AppleProbability = 
core::PropertyBuilder::createProperty("AppleProbability")->withDefaultValue(100)->build();
+// The probability that this processor routes to Banana
+static core::Property BananaProbability = 
core::PropertyBuilder::createProperty("BananaProbability")->withDefaultValue(0)->build();
+
+class ProcessorWithStatistics {
+ public:
+  std::atomic trigger_count{0};
+};
+
+class TestProcessor : public core::Processor, public ProcessorWithStatistics {
+ public:
+  TestProcessor(std::string name, utils::Identifier ) : Processor(name, 
uuid) {}
+  TestProcessor(std::string name) : Processor(name) {}
+  void initialize() override {
+setSupportedProperties({AppleProbability, BananaProbability});
+setSupportedRelationships({Apple, Banana});
+  }
+  void onTrigger(const std::shared_ptr , const 
std::shared_ptr ) override {
+++trigger_count;
+auto flowFile = session->get();
+if (!flowFile) return;
+std::random_device rd{};
+

[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #822: MINIFICPP-1253 - Do not hang on full cycles

2020-07-07 Thread GitBox


adamdebreceni commented on a change in pull request #822:
URL: https://github.com/apache/nifi-minifi-cpp/pull/822#discussion_r450881264



##
File path: libminifi/test/flow-tests/LoopTest.cpp
##
@@ -0,0 +1,102 @@
+/**
+ *
+ * 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.
+ */
+
+#undef NDEBUG
+#include "CustomProcessors.h"
+#include "FlowBuilder.h"
+
+// A flow with structure:
+// [Generator] ---> [A] ---|
+//   ^_|
+const char* flowConfigurationYaml =
+R"(
+Flow Controller:
+  name: MiNiFi Flow
+  id: 2438e3c8-015a-1001-79ca-83af40ec1990
+Processors:
+  - name: Generator
+id: 2438e3c8-015a-1001-79ca-83af40ec1991
+class: org.apache.nifi.processors.TestFlowFileGenerator
+max concurrent tasks: 1
+scheduling strategy: TIMER_DRIVEN
+scheduling period: 100 ms
+penalization period: 300 ms
+yield period: 100 ms
+run duration nanos: 0
+auto-terminated relationships list:
+  - name: A
+id: 2438e3c8-015a-1001-79ca-83af40ec1992
+class: org.apache.nifi.processors.TestProcessor
+max concurrent tasks: 1
+scheduling strategy: TIMER_DRIVEN
+scheduling period: 100 ms
+penalization period: 300 ms
+yield period: 100 ms
+run duration nanos: 0
+auto-terminated relationships list:
+Properties:
+  AppleProbability: 100
+  BananaProbability: 0
+
+Connections:
+  - name: Gen
+id: 2438e3c8-015a-1001-79ca-83af40ec1993
+source name: Generator
+destination name: A
+source relationship name: success
+max work queue size: 1
+max work queue data size: 1 MB
+flowfile expiration: 0
+  - name: Loop
+id: 2438e3c8-015a-1001-79ca-83af40ec1994
+source name: A
+destination name: A
+source relationship name: apple
+max work queue size: 1
+max work queue data size: 1 MB
+flowfile expiration: 0
+
+Remote Processing Groups:
+)";
+
+TEST_CASE("Flow with a single loop", "[SingleLoopFlow]") {
+  TestController testController;
+
+  LogTestController::getInstance().setTrace();
+  LogTestController::getInstance().setTrace();
+  LogTestController::getInstance().setTrace();
+
+  char format[] = "/tmp/flow.XX";
+  std::string dir = testController.createTempDirectory(format);
+  std::string yamlPath = utils::file::FileUtils::concat_path(dir, 
"config.yml");
+  std::ofstream{yamlPath} << flowConfigurationYaml;
+
+  Flow flow = createFlow(yamlPath);
+
+  auto procGenerator = 
std::static_pointer_cast(flow.root_->findProcessor("Generator"));
+  auto procA = 
std::static_pointer_cast(flow.root_->findProcessor("A"));
+
+  int tryCount = 0;
+  // wait for the procA to get triggered 15 times
+  while (tryCount++ < 10 && !(procA->trigger_count.load() >= 15)) {

Review comment:
   `tryCount` is our friend  





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #822: MINIFICPP-1253 - Do not hang on full cycles

2020-07-07 Thread GitBox


adamdebreceni commented on a change in pull request #822:
URL: https://github.com/apache/nifi-minifi-cpp/pull/822#discussion_r450873613



##
File path: libminifi/src/core/Processor.cpp
##
@@ -101,14 +102,27 @@ void Processor::setScheduledState(ScheduledState state) {
 }
 
 bool Processor::addConnection(std::shared_ptr conn) {
-  bool ret = false;
+  enum class DidSet{

Review comment:
   interesting because I recall that I first called it "SetAs" but the 
commit history doesn't confirm this, will make the change





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #822: MINIFICPP-1253 - Do not hang on full cycles

2020-07-07 Thread GitBox


adamdebreceni commented on a change in pull request #822:
URL: https://github.com/apache/nifi-minifi-cpp/pull/822#discussion_r450869926



##
File path: libminifi/include/core/Processor.h
##
@@ -305,6 +311,19 @@ class Processor : public Connectable, public 
ConfigurableComponent, public std::
   Processor =(const Processor );
 
  private:
+  static std::mutex& getGraphMutex() {
+static std::mutex mutex{};
+return mutex;
+  }
+
+  // must hold the graphMutex
+  void updateReachability(const std::lock_guard& graph_lock, bool 
force = false);

Review comment:
   if it turns out that there is a legit use-case where we would need to 
expose this method, I'm more than happy to make it non-private but I couldn't 
come up with a reason to do so





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org