You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2022/08/17 09:06:09 UTC

[GitHub] [nifi-minifi-cpp] adamdebreceni opened a new pull request, #1391: MINIFICPP-1846 - Json configuration support part 1

adamdebreceni opened a new pull request, #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391

   > :warning: To preserve you sanity during review diff the old `YamlConfiguration.h` to the new `StructuredConfiguration.h`, similarly do so for `YamlConfiguration.cpp` and `StructuredConfiguration.cpp` as git could not keep track of the lineage. Testing will happen in a separate PR as there will be some semantic changes to yaml parsing, better to keep those separate from this (mostly) syntactic change.
   
   Thank you for submitting a contribution to Apache NiFi - MiNiFi C++.
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced
        in the commit message?
   
   - [ ] Does your PR title start with MINIFICPP-XXXX where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically main)?
   
   - [ ] Is your initial contribution a single, squashed commit?
   
   ### For code changes:
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   - [ ] If applicable, have you updated the LICENSE file?
   - [ ] If applicable, have you updated the NOTICE file?
   
   ### For documentation related changes:
   - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check GitHub Actions CI results for build issues and submit an update to your PR as soon as possible.
   


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r979899914


##########
libminifi/include/core/flow/Node.h:
##########
@@ -0,0 +1,143 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string_view>
+#include <tuple>
+#include <optional>
+#include "nonstd/expected.hpp"
+
+namespace org::apache::nifi::minifi::core::flow {
+
+class Node {
+ public:
+  struct Cursor {
+    int line;
+    int column;
+    int pos;
+  };
+
+  class Impl;
+  class Iterator {
+   public:
+    class Value;
+
+    class Impl {
+     public:
+      virtual Impl& operator++() = 0;
+      virtual bool operator==(const Impl& other) const = 0;
+      virtual Value operator*() const = 0;
+      bool operator!=(const Impl& other) const {return !(*this == other);}
+
+      virtual std::unique_ptr<Impl> clone() const = 0;
+      virtual ~Impl() = default;
+    };
+
+    Iterator& operator++() {
+      impl_->operator++();
+      return *this;
+    }
+
+    explicit Iterator(std::unique_ptr<Impl> impl) : impl_(std::move(impl)) {}
+    Iterator(const Iterator& other): impl_(other.impl_->clone()) {}
+    Iterator(Iterator&&) = default;
+    Iterator& operator=(const Iterator& other) {
+      if (this == &other) {
+        return *this;
+      }
+      impl_ = other.impl_->clone();
+      return *this;
+    }
+    Iterator& operator=(Iterator&&) = default;
+
+    bool operator==(const Iterator& other) const {return impl_->operator==(*other.impl_);}
+    bool operator!=(const Iterator& other) const {return !(*this == other);}
+
+    Value operator*() const;
+
+   private:
+    std::unique_ptr<Impl> impl_;
+  };
+
+  class Impl {
+   public:
+    virtual explicit operator bool() const = 0;
+    virtual bool isSequence() const = 0;
+    virtual bool isMap() const = 0;
+    virtual bool isNull() const = 0;
+    virtual bool isScalar() const = 0;
+
+    virtual nonstd::expected<std::string, std::exception_ptr> getString() const = 0;
+    virtual nonstd::expected<int, std::exception_ptr> getInt() const = 0;
+    virtual nonstd::expected<unsigned int, std::exception_ptr> getUInt() const = 0;
+    virtual nonstd::expected<bool, std::exception_ptr> getBool() const = 0;
+    virtual nonstd::expected<int64_t, std::exception_ptr> getInt64() const = 0;
+    virtual nonstd::expected<uint64_t, std::exception_ptr> getUInt64() const = 0;
+
+    virtual std::string getDebugString() const = 0;
+
+    virtual size_t size() const = 0;
+    virtual Iterator begin() const = 0;
+    virtual Iterator end() const = 0;
+    virtual Node operator[](std::string_view key) const = 0;
+
+    virtual std::optional<Cursor> getCursor() const {return std::nullopt;}
+
+    virtual ~Impl() = default;
+  };
+
+  Node() = default;
+  explicit Node(std::shared_ptr<Impl> impl): impl_(std::move(impl)) {}
+
+  explicit operator bool() const {return impl_->operator bool();}
+  bool isSequence() const {return impl_->isSequence();}
+  bool isMap() const {return impl_->isMap();}
+  bool isNull() const {return impl_->isNull();}
+  bool isScalar() const {return impl_->isScalar();}
+
+  nonstd::expected<std::string, std::exception_ptr> getString() const {return impl_->getString();}
+  nonstd::expected<bool, std::exception_ptr> getBool() const {return impl_->getBool();}
+  nonstd::expected<int, std::exception_ptr> getInt() const {return impl_->getInt();}
+  nonstd::expected<unsigned int, std::exception_ptr> getUInt() const {return impl_->getUInt();}
+  nonstd::expected<int64_t, std::exception_ptr> getInt64() const {return impl_->getInt64();}
+  nonstd::expected<uint64_t, std::exception_ptr> getUInt64() const {return impl_->getUInt64();}
+
+  // return a string representation of the node (need not to be deserializable)
+  std::string getDebugString() const {return impl_->getDebugString();}
+
+  size_t size() const {return impl_->size();}
+  size_t empty() const {
+    return size() == 0;
+  }
+  Iterator begin() const {return impl_->begin();}
+  Iterator end() const {return impl_->end();}
+  Node operator[](std::string_view key) const {return impl_->operator[](key);}
+
+  std::optional<Cursor> getCursor() const {return impl_->getCursor();}
+
+ private:
+  std::shared_ptr<Impl> impl_;
+};
+
+class Node::Iterator::Value : public Node, public std::pair<Node, Node> {
+ public:
+  Value(Node node, Node key, Node value): Node{std::move(node)}, std::pair<Node, Node>{std::move(key), std::move(value)} {}
+};

Review Comment:
   I wanted to mimic the `yaml-cpp`'s implementation (`node/iterator.h`) allowing uniform iteration on both maps and sequences without having to write forwarding methods if the `Node`-case was a member



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
fgerlits commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r979882035


##########
libminifi/include/core/flow/README.md:
##########
@@ -0,0 +1,57 @@
+## Differences between JSON and YAML implementation
+
+### YAML
+
+The possible types of a `YAML::Node` are:
+* Undefined
+* Null
+* Map
+* Sequence
+* Scalar
+
+#### Undefined
+
+The result of querying any member of `Null`, querying non-existing members of `Map`,
+or non-existing indices of `Sequence`.
+
+Note that for `Map`s string conversion applies `map[0]` could be valid, given a key `"0"`,
+while for `Sequence`s string index parsing does NOT happen `seq["0"]`
+will return `Undefined` even if the sequence is non-empty.
+
+Querying or otherwise accessing an `Undefined` (other than `operator bool` or `IsDefined`) usually throws.
+
+#### Null
+
+The value of parsing an empty document, the value of a `Map` item with empty value,
+the value of an omitted `Sequence` item.
+
+```
+key1:      # this is a Null
+key2: ''   # this is a Scalar, the empty string
+arr:
+  - one
+  -        # Null as well
+  - three
+```
+
+#### Scalar
+
+A string value, all conversions to numbers happen on the fly.
+
+### Conversions
+
+#### 1. `::as&lt;std::string&gt;`
+
+* `Null` --> `"null"`
+* `Scalar` --> the string value
+* others --> throws
+
+#### 2. `::as&lt;YAML::Node&gt;`

Review Comment:
   You're right, I hadn't upgraded Clion in a while, and now that I've upgraded to 2022.2.3, it is now the other way round :(
   
   But on GitHub itself, `<...>` works and `&lt;...&gt;` doesn't, so I still think changing to `<...>` is the right way to go, even if it is going to look bad in Clion.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r991944136


##########
libminifi/include/core/json/JsonNode.h:
##########
@@ -0,0 +1,248 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string>
+#include <utility>
+#include <memory>
+
+#include "core/flow/Node.h"
+#include "rapidjson/document.h"
+#include "utils/gsl.h"
+#include "utils/ValueCaster.h"
+
+namespace org::apache::nifi::minifi::core {
+
+class JsonNode : public flow::Node::NodeImpl {
+ public:
+  explicit JsonNode(const rapidjson::Value* node): node_(node) {}
+
+  explicit operator bool() const override {
+    return node_ != nullptr;
+  }
+  bool isSequence() const override {
+    return node_ ? node_->IsArray() : false;
+  }
+  bool isMap() const override {
+    return node_ ? node_->IsObject() : false;
+  }
+  bool isNull() const override {
+    return node_ ? node_->IsNull() : false;
+  }
+
+  nonstd::expected<std::string, std::exception_ptr> getString() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get string of invalid json value");
+      }
+      if (!node_->IsString()) {
+        throw std::runtime_error("Cannot get string of non-string json value");
+      }
+      return std::string{node_->GetString(), node_->GetStringLength()};
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());
+    }
+  }
+
+  nonstd::expected<int, std::exception_ptr> getInt() const override {
+    return getNumber<int>("int");
+  }
+  nonstd::expected<unsigned int, std::exception_ptr> getUInt() const override {
+    return getNumber<unsigned int>("unsigned int");
+  }
+  nonstd::expected<int64_t, std::exception_ptr> getInt64() const override {
+    return getNumber<int64_t>("int64_t");
+  }
+  nonstd::expected<uint64_t, std::exception_ptr> getUInt64() const override {
+    return getNumber<uint64_t>("uint64_t");
+  }
+
+  nonstd::expected<bool, std::exception_ptr> getBool() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get bool of invalid json value");
+      }
+      if (!node_->IsBool()) {
+        throw std::runtime_error("Cannot get bool of non-bool json value");
+      }
+      return node_->GetBool();
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());
+    }
+  }
+
+  std::string getDebugString() const override {
+    if (!node_) return "<invalid>";
+    if (node_->IsObject()) return "<Map>";
+    if (node_->IsArray()) return "<Array>";
+    if (node_->IsNull()) return "null";
+    if (node_->IsInt()) return std::to_string(node_->GetInt());
+    if (node_->IsUint()) return std::to_string(node_->GetUint());
+    if (node_->IsInt64()) return std::to_string(node_->GetInt64());
+    if (node_->IsUint64()) return std::to_string(node_->GetUint64());

Review Comment:
   looking into the constructors of `rapidjson::GenericValue` they indeed set all the appropriate flags if the integers are in range, removed unnecessary smaller integer checks, also removed all but the `getInt64` from the `flow::Node` interface



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r979884544


##########
libminifi/include/core/flow/README.md:
##########
@@ -0,0 +1,57 @@
+## Differences between JSON and YAML implementation
+
+### YAML
+
+The possible types of a `YAML::Node` are:
+* Undefined
+* Null
+* Map
+* Sequence
+* Scalar
+
+#### Undefined
+
+The result of querying any member of `Null`, querying non-existing members of `Map`,
+or non-existing indices of `Sequence`.
+
+Note that for `Map`s string conversion applies `map[0]` could be valid, given a key `"0"`,
+while for `Sequence`s string index parsing does NOT happen `seq["0"]`
+will return `Undefined` even if the sequence is non-empty.
+
+Querying or otherwise accessing an `Undefined` (other than `operator bool` or `IsDefined`) usually throws.

Review Comment:
   the `operator bool()` is implemented as `IsDefined()`, so I replaced all `IsDefined()` calls with `operator bool()` calls, this part here describes the workings of the `YAML::Node`, I did not plan on exposing an `IsDefined`-like method



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r979881994


##########
libminifi/include/core/flow/Node.h:
##########
@@ -0,0 +1,143 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string_view>
+#include <tuple>
+#include <optional>
+#include "nonstd/expected.hpp"
+
+namespace org::apache::nifi::minifi::core::flow {
+
+class Node {
+ public:
+  struct Cursor {
+    int line;
+    int column;
+    int pos;
+  };
+
+  class Impl;
+  class Iterator {
+   public:
+    class Value;
+
+    class Impl {
+     public:
+      virtual Impl& operator++() = 0;
+      virtual bool operator==(const Impl& other) const = 0;
+      virtual Value operator*() const = 0;
+      bool operator!=(const Impl& other) const {return !(*this == other);}
+
+      virtual std::unique_ptr<Impl> clone() const = 0;
+      virtual ~Impl() = default;
+    };
+
+    Iterator& operator++() {
+      impl_->operator++();
+      return *this;
+    }
+
+    explicit Iterator(std::unique_ptr<Impl> impl) : impl_(std::move(impl)) {}
+    Iterator(const Iterator& other): impl_(other.impl_->clone()) {}
+    Iterator(Iterator&&) = default;
+    Iterator& operator=(const Iterator& other) {
+      if (this == &other) {
+        return *this;
+      }
+      impl_ = other.impl_->clone();
+      return *this;
+    }
+    Iterator& operator=(Iterator&&) = default;
+
+    bool operator==(const Iterator& other) const {return impl_->operator==(*other.impl_);}
+    bool operator!=(const Iterator& other) const {return !(*this == other);}
+
+    Value operator*() const;
+
+   private:
+    std::unique_ptr<Impl> impl_;
+  };
+
+  class Impl {

Review Comment:
   makes sense, renamed them



##########
libminifi/include/core/flow/Node.h:
##########
@@ -0,0 +1,143 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string_view>
+#include <tuple>
+#include <optional>
+#include "nonstd/expected.hpp"
+
+namespace org::apache::nifi::minifi::core::flow {
+
+class Node {
+ public:
+  struct Cursor {
+    int line;
+    int column;
+    int pos;
+  };
+
+  class Impl;
+  class Iterator {
+   public:
+    class Value;
+
+    class Impl {
+     public:
+      virtual Impl& operator++() = 0;
+      virtual bool operator==(const Impl& other) const = 0;
+      virtual Value operator*() const = 0;
+      bool operator!=(const Impl& other) const {return !(*this == other);}
+
+      virtual std::unique_ptr<Impl> clone() const = 0;
+      virtual ~Impl() = default;
+    };
+
+    Iterator& operator++() {
+      impl_->operator++();
+      return *this;
+    }
+
+    explicit Iterator(std::unique_ptr<Impl> impl) : impl_(std::move(impl)) {}
+    Iterator(const Iterator& other): impl_(other.impl_->clone()) {}
+    Iterator(Iterator&&) = default;
+    Iterator& operator=(const Iterator& other) {
+      if (this == &other) {
+        return *this;
+      }
+      impl_ = other.impl_->clone();
+      return *this;
+    }
+    Iterator& operator=(Iterator&&) = default;
+
+    bool operator==(const Iterator& other) const {return impl_->operator==(*other.impl_);}
+    bool operator!=(const Iterator& other) const {return !(*this == other);}
+
+    Value operator*() const;
+
+   private:
+    std::unique_ptr<Impl> impl_;
+  };
+
+  class Impl {
+   public:
+    virtual explicit operator bool() const = 0;
+    virtual bool isSequence() const = 0;
+    virtual bool isMap() const = 0;
+    virtual bool isNull() const = 0;
+    virtual bool isScalar() const = 0;
+
+    virtual nonstd::expected<std::string, std::exception_ptr> getString() const = 0;
+    virtual nonstd::expected<int, std::exception_ptr> getInt() const = 0;
+    virtual nonstd::expected<unsigned int, std::exception_ptr> getUInt() const = 0;
+    virtual nonstd::expected<bool, std::exception_ptr> getBool() const = 0;
+    virtual nonstd::expected<int64_t, std::exception_ptr> getInt64() const = 0;
+    virtual nonstd::expected<uint64_t, std::exception_ptr> getUInt64() const = 0;
+
+    virtual std::string getDebugString() const = 0;
+
+    virtual size_t size() const = 0;
+    virtual Iterator begin() const = 0;
+    virtual Iterator end() const = 0;
+    virtual Node operator[](std::string_view key) const = 0;
+
+    virtual std::optional<Cursor> getCursor() const {return std::nullopt;}

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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
fgerlits commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r997051098


##########
libminifi/src/core/flow/StructuredConfiguration.cpp:
##########
@@ -0,0 +1,896 @@
+/**
+ *
+ * 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 <memory>
+#include <vector>
+#include <set>
+#include <cinttypes>
+
+#include "core/flow/StructuredConfiguration.h"
+#include "core/flow/CheckRequiredField.h"
+#include "core/flow/StructuredConnectionParser.h"
+#include "core/state/Value.h"
+#include "Defaults.h"
+#include "utils/TimeUtil.h"
+#include "utils/RegexUtils.h"
+
+namespace org::apache::nifi::minifi::core::flow {
+
+std::shared_ptr<utils::IdGenerator> StructuredConfiguration::id_generator_ = utils::IdGenerator::getIdGenerator();
+
+StructuredConfiguration::StructuredConfiguration(ConfigurationContext ctx, std::shared_ptr<logging::Logger> logger)
+    : FlowConfiguration(std::move(ctx)),
+      logger_(std::move(logger)) {}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::parseRootProcessGroup(const Node& root_flow_node) {
+  auto flow_controller_node = root_flow_node[CONFIG_FLOW_CONTROLLER_KEY];
+  auto root_group = parseProcessGroup(flow_controller_node, root_flow_node, true);
+  this->name_ = root_group->getName();
+  return root_group;
+}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::createProcessGroup(const Node& node, bool is_root) {
+  int version = 0;
+
+  checkRequiredField(node, "name", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+  auto flowName = node["name"].getString().value();
+
+  utils::Identifier uuid;
+  // assignment throws on invalid uuid
+  uuid = getOrGenerateId(node);
+
+  if (node["version"]) {
+    version = node["version"].getInt().value();
+  }
+
+  logger_->log_debug("parseRootProcessGroup: id => [%s], name => [%s]", uuid.to_string(), flowName);
+  std::unique_ptr<core::ProcessGroup> group;
+  if (is_root) {
+    group = FlowConfiguration::createRootProcessGroup(flowName, uuid, version);
+  } else {
+    group = FlowConfiguration::createSimpleProcessGroup(flowName, uuid, version);
+  }
+
+  if (node["onschedule retry interval"]) {
+    auto onScheduleRetryPeriod = node["onschedule retry interval"].getString().value();
+    logger_->log_debug("parseRootProcessGroup: onschedule retry period => [%s]", onScheduleRetryPeriod);
+
+    auto on_schedule_retry_period_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(onScheduleRetryPeriod);
+    if (on_schedule_retry_period_value.has_value() && group) {
+      logger_->log_debug("parseRootProcessGroup: onschedule retry => [%" PRId64 "] ms", on_schedule_retry_period_value->count());
+      group->setOnScheduleRetryPeriod(on_schedule_retry_period_value->count());
+    }
+  }
+
+  return group;
+}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::parseProcessGroup(const Node& headerNode, const Node& yamlNode, bool is_root) {
+  auto group = createProcessGroup(headerNode, is_root);
+  Node processorsNode = yamlNode[CONFIG_PROCESSORS_KEY];
+  Node connectionsNode = yamlNode[StructuredConnectionParser::CONFIG_CONNECTIONS_KEY];
+  Node funnelsNode = yamlNode[CONFIG_FUNNELS_KEY];
+  Node remoteProcessingGroupsNode = [&] {
+    // assignment is not supported on invalid Yaml nodes
+    Node candidate = yamlNode[CONFIG_REMOTE_PROCESS_GROUP_KEY];
+    if (candidate) {
+      return candidate;
+    }
+    return yamlNode[CONFIG_REMOTE_PROCESS_GROUP_KEY_V3];
+  }();
+  Node childProcessGroupNodeSeq = yamlNode["Process Groups"];
+
+  parseProcessorNode(processorsNode, group.get());
+  parseRemoteProcessGroup(remoteProcessingGroupsNode, group.get());
+  parseFunnels(funnelsNode, group.get());
+  // parse connections last to give feedback if the source and/or destination
+  // is not in the same process group
+  parseConnection(connectionsNode, group.get());
+
+  if (childProcessGroupNodeSeq && childProcessGroupNodeSeq.isSequence()) {
+    for (const auto childProcessGroupNode : childProcessGroupNodeSeq) {
+      group->addProcessGroup(parseProcessGroup(childProcessGroupNode, childProcessGroupNode));
+    }
+  }
+  return group;
+}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::getRootFrom(const Node& rootYamlNode) {
+  uuids_.clear();
+  Node controllerServiceNode = rootYamlNode[CONFIG_CONTROLLER_SERVICES_KEY];
+  Node provenanceReportNode = rootYamlNode[CONFIG_PROVENANCE_REPORT_KEY];
+
+  parseControllerServices(controllerServiceNode);
+  // Create the root process group
+  std::unique_ptr<core::ProcessGroup> root = parseRootProcessGroup(rootYamlNode);
+  parseProvenanceReporting(provenanceReportNode, root.get());
+
+  // set the controller services into the root group.
+  for (const auto& controller_service : controller_services_->getAllControllerServices()) {
+    root->addControllerService(controller_service->getName(), controller_service);
+    root->addControllerService(controller_service->getUUIDStr(), controller_service);
+  }
+
+  return root;
+}
+
+void StructuredConfiguration::parseProcessorNode(const Node& processors_node, core::ProcessGroup* parentGroup) {
+  int64_t runDurationNanos = -1;
+  utils::Identifier uuid;
+  std::unique_ptr<core::Processor> processor;
+
+  if (!parentGroup) {
+    logger_->log_error("parseProcessNodeYaml: no parent group exists");
+    return;
+  }
+
+  if (!processors_node) {
+    throw std::invalid_argument("Cannot instantiate a MiNiFi instance without a defined Processors configuration node.");
+  }
+  if (!processors_node.isSequence()) {
+    throw std::invalid_argument(
+        "Cannot instantiate a MiNiFi instance without a defined Processors configuration node.");
+  }
+  // Evaluate sequence of processors
+  for (const auto procNode : processors_node) {
+    core::ProcessorConfig procCfg;
+
+    checkRequiredField(procNode, "name", CONFIG_PROCESSORS_KEY);
+    procCfg.name = procNode["name"].getString().value();
+    procCfg.id = getOrGenerateId(procNode);
+
+    uuid = procCfg.id;
+    logger_->log_debug("parseProcessorNode: name => [%s] id => [%s]", procCfg.name, procCfg.id);
+    checkRequiredField(procNode, "class", CONFIG_PROCESSORS_KEY);
+    procCfg.javaClass = procNode["class"].getString().value();
+    logger_->log_debug("parseProcessorNode: class => [%s]", procCfg.javaClass);
+
+    // Determine the processor name only from the Java class
+    auto lastOfIdx = procCfg.javaClass.find_last_of('.');
+    if (lastOfIdx != std::string::npos) {
+      lastOfIdx++;  // if a value is found, increment to move beyond the .
+      std::string processorName = procCfg.javaClass.substr(lastOfIdx);
+      processor = this->createProcessor(processorName, procCfg.javaClass, uuid);
+    } else {
+      // Allow unqualified class names for core processors
+      processor = this->createProcessor(procCfg.javaClass, uuid);
+    }
+
+    if (!processor) {
+      logger_->log_error("Could not create a processor %s with id %s", procCfg.name, procCfg.id);
+      throw std::invalid_argument("Could not create processor " + procCfg.name);
+    }
+
+    processor->setName(procCfg.name);
+
+    processor->setFlowIdentifier(flow_version_->getFlowIdentifier());
+
+    procCfg.schedulingStrategy = getOptionalField(procNode, "scheduling strategy", DEFAULT_SCHEDULING_STRATEGY, CONFIG_PROCESSORS_KEY);
+    logger_->log_debug("parseProcessorNode: scheduling strategy => [%s]", procCfg.schedulingStrategy);
+
+    procCfg.schedulingPeriod = getOptionalField(procNode, "scheduling period", DEFAULT_SCHEDULING_PERIOD_STR, CONFIG_PROCESSORS_KEY);
+
+    logger_->log_debug("parseProcessorNode: scheduling period => [%s]", procCfg.schedulingPeriod);
+
+    if (auto tasksNode = procNode["max concurrent tasks"]) {
+      if (auto int_val = tasksNode.getUInt64()) {
+        procCfg.maxConcurrentTasks = std::to_string(int_val.value());
+      } else {
+        procCfg.maxConcurrentTasks = tasksNode.getString().value();
+      }
+      logger_->log_debug("parseProcessorNode: max concurrent tasks => [%s]", procCfg.maxConcurrentTasks);
+    }
+
+    if (procNode["penalization period"]) {
+      procCfg.penalizationPeriod = procNode["penalization period"].getString().value();
+      logger_->log_debug("parseProcessorNode: penalization period => [%s]", procCfg.penalizationPeriod);
+    }
+
+    if (procNode["yield period"]) {
+      procCfg.yieldPeriod = procNode["yield period"].getString().value();
+      logger_->log_debug("parseProcessorNode: yield period => [%s]", procCfg.yieldPeriod);
+    }
+
+    if (auto runNode = procNode["run duration nanos"]) {
+      if (auto int_val = runNode.getUInt64()) {
+        procCfg.runDurationNanos = std::to_string(int_val.value());
+      } else {
+        procCfg.runDurationNanos = runNode.getString().value();
+      }
+      logger_->log_debug("parseProcessorNode: run duration nanos => [%s]", procCfg.runDurationNanos);
+    }
+
+    // handle auto-terminated relationships
+    if (procNode["auto-terminated relationships list"]) {
+      Node autoTerminatedSequence = procNode["auto-terminated relationships list"];
+      std::vector<std::string> rawAutoTerminatedRelationshipValues;
+      if (autoTerminatedSequence.isSequence() && autoTerminatedSequence.size() > 0) {
+        for (const auto autoTerminatedRel : autoTerminatedSequence) {
+          rawAutoTerminatedRelationshipValues.push_back(autoTerminatedRel.getString().value());
+        }
+      }
+      procCfg.autoTerminatedRelationships = rawAutoTerminatedRelationshipValues;
+    }
+
+    // handle processor properties
+    if (procNode["Properties"]) {
+      Node propertiesNode = procNode["Properties"];
+      parsePropertiesNode(propertiesNode, *processor, procCfg.name, CONFIG_PROCESSORS_KEY);
+    }
+
+    // Take care of scheduling
+
+    if (procCfg.schedulingStrategy == "TIMER_DRIVEN" || procCfg.schedulingStrategy == "EVENT_DRIVEN") {
+      if (auto scheduling_period = utils::timeutils::StringToDuration<std::chrono::nanoseconds>(procCfg.schedulingPeriod)) {
+        logger_->log_debug("convert: parseProcessorNode: schedulingPeriod => [%" PRId64 "] ns", scheduling_period->count());
+        processor->setSchedulingPeriodNano(*scheduling_period);
+      }
+    } else {
+      processor->setCronPeriod(procCfg.schedulingPeriod);
+    }
+
+    if (auto penalization_period = utils::timeutils::StringToDuration<std::chrono::milliseconds>(procCfg.penalizationPeriod)) {
+      logger_->log_debug("convert: parseProcessorNode: penalizationPeriod => [%" PRId64 "] ms", penalization_period->count());
+      processor->setPenalizationPeriod(penalization_period.value());
+    }
+
+    if (auto yield_period = utils::timeutils::StringToDuration<std::chrono::milliseconds>(procCfg.yieldPeriod)) {
+      logger_->log_debug("convert: parseProcessorNode: yieldPeriod => [%" PRId64 "] ms", yield_period->count());
+      processor->setYieldPeriodMsec(yield_period.value());
+    }
+
+    // Default to running
+    processor->setScheduledState(core::RUNNING);
+
+    if (procCfg.schedulingStrategy == "TIMER_DRIVEN") {
+      processor->setSchedulingStrategy(core::TIMER_DRIVEN);
+      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
+    } else if (procCfg.schedulingStrategy == "EVENT_DRIVEN") {
+      processor->setSchedulingStrategy(core::EVENT_DRIVEN);
+      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
+    } else {
+      processor->setSchedulingStrategy(core::CRON_DRIVEN);
+      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
+    }
+
+    int32_t maxConcurrentTasks;
+    if (core::Property::StringToInt(procCfg.maxConcurrentTasks, maxConcurrentTasks)) {
+      logger_->log_debug("parseProcessorNode: maxConcurrentTasks => [%d]", maxConcurrentTasks);
+      processor->setMaxConcurrentTasks((uint8_t) maxConcurrentTasks);
+    }
+
+    if (core::Property::StringToInt(procCfg.runDurationNanos, runDurationNanos)) {
+      logger_->log_debug("parseProcessorNode: runDurationNanos => [%d]", runDurationNanos);
+      processor->setRunDurationNano(std::chrono::nanoseconds(runDurationNanos));
+    }
+
+    std::vector<core::Relationship> autoTerminatedRelationships;
+    for (auto &&relString : procCfg.autoTerminatedRelationships) {
+      core::Relationship relationship(relString, "");
+      logger_->log_debug("parseProcessorNode: autoTerminatedRelationship  => [%s]", relString);
+      autoTerminatedRelationships.push_back(relationship);
+    }
+
+    processor->setAutoTerminatedRelationships(autoTerminatedRelationships);
+
+    parentGroup->addProcessor(std::move(processor));
+  }
+}
+
+void StructuredConfiguration::parseRemoteProcessGroup(const Node& rpg_node_seq, core::ProcessGroup* parentGroup) {
+  utils::Identifier uuid;
+  std::string id;
+
+  if (!parentGroup) {
+    logger_->log_error("parseRemoteProcessGroupYaml: no parent group exists");
+    return;
+  }
+
+  if (!rpg_node_seq || !rpg_node_seq.isSequence()) {
+    return;
+  }
+  for (const auto currRpgNode : rpg_node_seq) {
+    checkRequiredField(currRpgNode, "name", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+    auto name = currRpgNode["name"].getString().value();
+    id = getOrGenerateId(currRpgNode);
+
+    logger_->log_debug("parseRemoteProcessGroupYaml: name => [%s], id => [%s]", name, id);
+
+    auto url = getOptionalField(currRpgNode, "url", "", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+
+    logger_->log_debug("parseRemoteProcessGroupYaml: url => [%s]", url);
+
+    uuid = id;
+    auto group = createRemoteProcessGroup(name, uuid);
+    group->setParent(parentGroup);
+
+    if (currRpgNode["yield period"]) {
+      auto yieldPeriod = currRpgNode["yield period"].getString().value();
+      logger_->log_debug("parseRemoteProcessGroupYaml: yield period => [%s]", yieldPeriod);
+
+      auto yield_period_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(yieldPeriod);
+      if (yield_period_value.has_value() && group) {
+        logger_->log_debug("parseRemoteProcessGroupYaml: yieldPeriod => [%" PRId64 "] ms", yield_period_value->count());
+        group->setYieldPeriodMsec(*yield_period_value);
+      }
+    }
+
+    if (currRpgNode["timeout"]) {
+      auto timeout = currRpgNode["timeout"].getString().value();
+      logger_->log_debug("parseRemoteProcessGroupYaml: timeout => [%s]", timeout);
+
+      auto timeout_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(timeout);
+      if (timeout_value.has_value() && group) {
+        logger_->log_debug("parseRemoteProcessGroupYaml: timeoutValue => [%" PRId64 "] ms", timeout_value->count());
+        group->setTimeout(timeout_value->count());
+      }
+    }
+
+    if (currRpgNode["local network interface"]) {
+      auto interface = currRpgNode["local network interface"].getString().value();
+      logger_->log_debug("parseRemoteProcessGroupYaml: local network interface => [%s]", interface);
+      group->setInterface(interface);
+    }
+
+    if (currRpgNode["transport protocol"]) {
+      auto transport_protocol = currRpgNode["transport protocol"].getString().value();
+      logger_->log_debug("parseRemoteProcessGroupYaml: transport protocol => [%s]", transport_protocol);
+      if (transport_protocol == "HTTP") {
+        group->setTransportProtocol(transport_protocol);
+        if (currRpgNode["proxy host"]) {
+          auto http_proxy_host = currRpgNode["proxy host"].getString().value();
+          logger_->log_debug("parseRemoteProcessGroupYaml: proxy host => [%s]", http_proxy_host);
+          group->setHttpProxyHost(http_proxy_host);
+          if (currRpgNode["proxy user"]) {
+            auto http_proxy_username = currRpgNode["proxy user"].getString().value();
+            logger_->log_debug("parseRemoteProcessGroupYaml: proxy user => [%s]", http_proxy_username);
+            group->setHttpProxyUserName(http_proxy_username);
+          }
+          if (currRpgNode["proxy password"]) {
+            auto http_proxy_password = currRpgNode["proxy password"].getString().value();
+            logger_->log_debug("parseRemoteProcessGroupYaml: proxy password => [%s]", http_proxy_password);
+            group->setHttpProxyPassWord(http_proxy_password);
+          }
+          if (currRpgNode["proxy port"]) {
+            auto http_proxy_port = currRpgNode["proxy port"].getString().value();
+            int32_t port;
+            if (core::Property::StringToInt(http_proxy_port, port)) {
+              logger_->log_debug("parseRemoteProcessGroupYaml: proxy port => [%d]", port);
+              group->setHttpProxyPort(port);
+            }
+          }
+        }
+      } else if (transport_protocol == "RAW") {
+        group->setTransportProtocol(transport_protocol);
+      } else {
+        std::stringstream stream;
+        stream << "Invalid transport protocol " << transport_protocol;
+        throw minifi::Exception(ExceptionType::SITE2SITE_EXCEPTION, stream.str().c_str());
+      }
+    }
+
+    group->setTransmitting(true);
+    group->setURL(url);
+
+    checkRequiredField(currRpgNode, "Input Ports", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+    auto inputPorts = currRpgNode["Input Ports"];
+    if (inputPorts && inputPorts.isSequence()) {
+      for (const auto currPort : inputPorts) {
+        parsePort(currPort, group.get(), sitetosite::SEND);
+      }  // for node
+    }
+    auto outputPorts = currRpgNode["Output Ports"];
+    if (outputPorts && outputPorts.isSequence()) {
+      for (const auto currPort : outputPorts) {
+        logger_->log_debug("Got a current port, iterating...");
+
+        parsePort(currPort, group.get(), sitetosite::RECEIVE);
+      }  // for node
+    }
+    parentGroup->addProcessGroup(std::move(group));
+  }
+}
+
+void StructuredConfiguration::parseProvenanceReporting(const Node& node, core::ProcessGroup* parent_group) {
+  utils::Identifier port_uuid;
+
+  if (!parent_group) {
+    logger_->log_error("parseProvenanceReportingYaml: no parent group exists");
+    return;
+  }
+
+  if (!node || node.isNull()) {
+    logger_->log_debug("no provenance reporting task specified");
+    return;
+  }
+
+  auto reportTask = createProvenanceReportTask();
+
+  checkRequiredField(node, "scheduling strategy", CONFIG_PROVENANCE_REPORT_KEY);
+  auto schedulingStrategyStr = node["scheduling strategy"].getString().value();
+  checkRequiredField(node, "scheduling period", CONFIG_PROVENANCE_REPORT_KEY);
+  auto schedulingPeriodStr = node["scheduling period"].getString().value();
+
+  if (auto scheduling_period = utils::timeutils::StringToDuration<std::chrono::nanoseconds>(schedulingPeriodStr)) {
+    logger_->log_debug("ProvenanceReportingTask schedulingPeriod %" PRId64 " ns", scheduling_period->count());
+    reportTask->setSchedulingPeriodNano(*scheduling_period);
+  }
+
+  if (schedulingStrategyStr == "TIMER_DRIVEN") {
+    reportTask->setSchedulingStrategy(core::TIMER_DRIVEN);
+    logger_->log_debug("ProvenanceReportingTask scheduling strategy %s", schedulingStrategyStr);
+  } else {
+    throw std::invalid_argument("Invalid scheduling strategy " + schedulingStrategyStr);
+  }
+
+  int64_t lvalue;
+  if (node["host"] && node["port"]) {
+    auto hostStr = node["host"].getString().value();
+
+    std::string portStr;
+    if (auto int_val = node["port"].getInt()) {
+      portStr = std::to_string(int_val.value());
+    } else {
+      portStr = node["port"].getString().value();
+    }
+    if (core::Property::StringToInt(portStr, lvalue) && !hostStr.empty()) {
+      logger_->log_debug("ProvenanceReportingTask port %" PRId64, lvalue);
+      std::string url = hostStr + ":" + portStr;
+      reportTask->setURL(url);
+    }
+  }
+
+  if (node["url"]) {
+    auto urlStr = node["url"].getString().value();
+    if (!urlStr.empty()) {
+      reportTask->setURL(urlStr);
+      logger_->log_debug("ProvenanceReportingTask URL %s", urlStr);
+    }
+  }
+  checkRequiredField(node, "port uuid", CONFIG_PROVENANCE_REPORT_KEY);
+  auto portUUIDStr = node["port uuid"].getString().value();
+  checkRequiredField(node, "batch size", CONFIG_PROVENANCE_REPORT_KEY);
+  auto batchSizeStr = node["batch size"].getString().value();
+
+  logger_->log_debug("ProvenanceReportingTask port uuid %s", portUUIDStr);
+  port_uuid = portUUIDStr;
+  reportTask->setPortUUID(port_uuid);
+
+  if (core::Property::StringToInt(batchSizeStr, lvalue)) {
+    reportTask->setBatchSize(gsl::narrow<int>(lvalue));
+  }
+
+  reportTask->initialize();
+
+  // add processor to parent
+  reportTask->setScheduledState(core::RUNNING);
+  parent_group->addProcessor(std::move(reportTask));
+}
+
+void StructuredConfiguration::parseControllerServices(const Node& controllerServicesNode) {
+  if (!controllerServicesNode || !controllerServicesNode.isSequence()) {
+    return;
+  }
+  for (const auto& controllerServiceNode : controllerServicesNode) {
+    checkRequiredField(controllerServiceNode, "name", CONFIG_CONTROLLER_SERVICES_KEY);
+
+    auto type = getRequiredField(controllerServiceNode, std::vector<std::string>{"class", "type"}, CONFIG_CONTROLLER_SERVICES_KEY);
+    logger_->log_debug("Using type %s for controller service node", type);
+
+    std::string fullType = type;
+    auto lastOfIdx = type.find_last_of('.');
+    if (lastOfIdx != std::string::npos) {
+      lastOfIdx++;  // if a value is found, increment to move beyond the .
+      type = type.substr(lastOfIdx);
+    }
+
+    auto name = controllerServiceNode["name"].getString().value();
+    auto id = getRequiredIdField(controllerServiceNode, CONFIG_CONTROLLER_SERVICES_KEY);
+
+    utils::Identifier uuid;
+    uuid = id;
+    std::shared_ptr<core::controller::ControllerServiceNode> controller_service_node = createControllerService(type, fullType, name, uuid);
+    if (nullptr != controller_service_node) {
+      logger_->log_debug("Created Controller Service with UUID %s and name %s", id, name);
+      controller_service_node->initialize();
+      if (Node propertiesNode = controllerServiceNode["Properties"]) {
+        // we should propagate properties to the node and to the implementation
+        parsePropertiesNode(propertiesNode, *controller_service_node, name, CONFIG_CONTROLLER_SERVICES_KEY);
+        if (auto controllerServiceImpl = controller_service_node->getControllerServiceImplementation(); controllerServiceImpl) {
+          parsePropertiesNode(propertiesNode, *controllerServiceImpl, name, CONFIG_CONTROLLER_SERVICES_KEY);
+        }
+      }
+    } else {
+      logger_->log_debug("Could not locate %s", type);
+    }
+    controller_services_->put(id, controller_service_node);
+    controller_services_->put(name, controller_service_node);
+  }
+}
+
+void StructuredConfiguration::parseConnection(const Node& connection_node_seq, core::ProcessGroup* parent) {
+  if (!parent) {
+    logger_->log_error("parseProcessNode: no parent group was provided");
+    return;
+  }
+  if (!connection_node_seq || !connection_node_seq.isSequence()) {
+    return;
+  }
+
+  for (const auto& connection_node : connection_node_seq) {
+    if (!connection_node || !connection_node.isMap()) {
+      logger_->log_error("Invalid connection node, ignoring");
+      continue;
+    }

Review Comment:
   Can you add a comment to explain that this is done for backward compatibility, please?  I think future people reading the code will have the same question.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r979742491


##########
libminifi/include/core/flow/README.md:
##########
@@ -0,0 +1,57 @@
+## Differences between JSON and YAML implementation
+
+### YAML
+
+The possible types of a `YAML::Node` are:
+* Undefined
+* Null
+* Map
+* Sequence
+* Scalar
+
+#### Undefined
+
+The result of querying any member of `Null`, querying non-existing members of `Map`,
+or non-existing indices of `Sequence`.
+
+Note that for `Map`s string conversion applies `map[0]` could be valid, given a key `"0"`,
+while for `Sequence`s string index parsing does NOT happen `seq["0"]`
+will return `Undefined` even if the sequence is non-empty.
+
+Querying or otherwise accessing an `Undefined` (other than `operator bool` or `IsDefined`) usually throws.
+
+#### Null
+
+The value of parsing an empty document, the value of a `Map` item with empty value,
+the value of an omitted `Sequence` item.
+
+```
+key1:      # this is a Null
+key2: ''   # this is a Scalar, the empty string
+arr:
+  - one
+  -        # Null as well
+  - three
+```
+
+#### Scalar
+
+A string value, all conversions to numbers happen on the fly.
+
+### Conversions
+
+#### 1. `::as&lt;std::string&gt;`
+
+* `Null` --> `"null"`
+* `Scalar` --> the string value
+* others --> throws
+
+#### 2. `::as&lt;YAML::Node&gt;`

Review Comment:
   very interesting, for me it is the opposite, I am running clion version 2022.1.1, which version displays them correctly? (Visual Studio Code also displays the `<`/`>` correctly so I'll change to those



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r989817108


##########
libminifi/src/core/yaml/YamlConfiguration.cpp:
##########
@@ -22,900 +22,66 @@
 #include <cinttypes>
 
 #include "core/yaml/YamlConfiguration.h"
-#include "core/yaml/CheckRequiredField.h"
-#include "core/yaml/YamlConnectionParser.h"
 #include "core/state/Value.h"
 #include "Defaults.h"
 #include "utils/TimeUtil.h"
-
-#ifdef YAML_CONFIGURATION_USE_REGEX
+#include "yaml-cpp/yaml.h"
+#include "core/yaml/YamlNode.h"
 #include "utils/RegexUtils.h"
-#endif  // YAML_CONFIGURATION_USE_REGEX
 
 namespace org::apache::nifi::minifi::core {
 
-std::shared_ptr<utils::IdGenerator> YamlConfiguration::id_generator_ = utils::IdGenerator::getIdGenerator();
-
-YamlConfiguration::YamlConfiguration(const std::shared_ptr<core::Repository>& repo, const std::shared_ptr<core::Repository>& flow_file_repo,
-                                     const std::shared_ptr<core::ContentRepository>& content_repo, const std::shared_ptr<io::StreamFactory>& stream_factory,
-                                     const std::shared_ptr<Configure>& configuration, const std::optional<std::string>& path,
-                                     const std::shared_ptr<utils::file::FileSystem>& filesystem)
-    : FlowConfiguration(repo, flow_file_repo, content_repo, stream_factory, configuration,
-                        path.value_or(DEFAULT_NIFI_CONFIG_YML), filesystem),
-      stream_factory_(stream_factory),
-      logger_(logging::LoggerFactory<YamlConfiguration>::getLogger()) {}
-
-std::unique_ptr<core::ProcessGroup> YamlConfiguration::parseRootProcessGroupYaml(const YAML::Node& rootFlowNode) {
-  auto flowControllerNode = rootFlowNode[CONFIG_YAML_FLOW_CONTROLLER_KEY];
-  auto rootGroup = parseProcessGroupYaml(flowControllerNode, rootFlowNode, true);
-  this->name_ = rootGroup->getName();
-  return rootGroup;
-}
-
-std::unique_ptr<core::ProcessGroup> YamlConfiguration::createProcessGroup(const YAML::Node& yamlNode, bool is_root) {
-  int version = 0;
-
-  yaml::checkRequiredField(yamlNode, "name", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
-  auto flowName = yamlNode["name"].as<std::string>();
-
-  utils::Identifier uuid;
-  // assignment throws on invalid uuid
-  uuid = getOrGenerateId(yamlNode);
-
-  if (yamlNode["version"]) {
-    version = yamlNode["version"].as<int>();
-  }
-
-  logger_->log_debug("parseRootProcessGroup: id => [%s], name => [%s]", uuid.to_string(), flowName);
-  std::unique_ptr<core::ProcessGroup> group;
-  if (is_root) {
-    group = FlowConfiguration::createRootProcessGroup(flowName, uuid, version);
-  } else {
-    group = FlowConfiguration::createSimpleProcessGroup(flowName, uuid, version);
-  }
-
-  if (yamlNode["onschedule retry interval"]) {
-    auto onScheduleRetryPeriod = yamlNode["onschedule retry interval"].as<std::string>();
-    logger_->log_debug("parseRootProcessGroup: onschedule retry period => [%s]", onScheduleRetryPeriod);
-
-    auto on_schedule_retry_period_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(onScheduleRetryPeriod);
-    if (on_schedule_retry_period_value.has_value() && group) {
-      logger_->log_debug("parseRootProcessGroup: onschedule retry => [%" PRId64 "] ms", on_schedule_retry_period_value->count());
-      group->setOnScheduleRetryPeriod(on_schedule_retry_period_value->count());
-    }
-  }
-
-  return group;
-}
-
-std::unique_ptr<core::ProcessGroup> YamlConfiguration::parseProcessGroupYaml(const YAML::Node& headerNode, const YAML::Node& yamlNode, bool is_root) {
-  auto group = createProcessGroup(headerNode, is_root);
-  YAML::Node processorsNode = yamlNode[CONFIG_YAML_PROCESSORS_KEY];
-  YAML::Node connectionsNode = yamlNode[yaml::YamlConnectionParser::CONFIG_YAML_CONNECTIONS_KEY];
-  YAML::Node funnelsNode = yamlNode[CONFIG_YAML_FUNNELS_KEY];
-  YAML::Node remoteProcessingGroupsNode = [&] {
-    // assignment is not supported on invalid Yaml nodes
-    YAML::Node candidate = yamlNode[CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY];
-    if (candidate) {
-      return candidate;
-    }
-    return yamlNode[CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY_V3];
-  }();
-  YAML::Node childProcessGroupNodeSeq = yamlNode["Process Groups"];
-
-  parseProcessorNodeYaml(processorsNode, group.get());
-  parseRemoteProcessGroupYaml(remoteProcessingGroupsNode, group.get());
-  parseFunnelsYaml(funnelsNode, group.get());
-  // parse connections last to give feedback if the source and/or destination
-  // is not in the same process group
-  parseConnectionYaml(connectionsNode, group.get());
-
-  if (childProcessGroupNodeSeq && childProcessGroupNodeSeq.IsSequence()) {
-    for (YAML::const_iterator it = childProcessGroupNodeSeq.begin(); it != childProcessGroupNodeSeq.end(); ++it) {
-      auto childProcessGroupNode = it->as<YAML::Node>();
-      group->addProcessGroup(parseProcessGroupYaml(childProcessGroupNode, childProcessGroupNode));
-    }
-  }
-  return group;
-}
-
-std::unique_ptr<core::ProcessGroup> YamlConfiguration::getYamlRoot(const YAML::Node& rootYamlNode) {
-  uuids_.clear();
-  YAML::Node controllerServiceNode = rootYamlNode[CONFIG_YAML_CONTROLLER_SERVICES_KEY];
-  YAML::Node provenanceReportNode = rootYamlNode[CONFIG_YAML_PROVENANCE_REPORT_KEY];
-
-  parseControllerServices(controllerServiceNode);
-  // Create the root process group
-  std::unique_ptr<core::ProcessGroup> root = parseRootProcessGroupYaml(rootYamlNode);
-  parseProvenanceReportingYaml(provenanceReportNode, root.get());
-
-  // set the controller services into the root group.
-  for (const auto& controller_service : controller_services_->getAllControllerServices()) {
-    root->addControllerService(controller_service->getName(), controller_service);
-    root->addControllerService(controller_service->getUUIDStr(), controller_service);
-  }
-
-  return root;
-}
-
-void YamlConfiguration::parseProcessorNodeYaml(const YAML::Node& processorsNode, core::ProcessGroup* parentGroup) {
-  int64_t runDurationNanos = -1;
-  utils::Identifier uuid;
-  std::unique_ptr<core::Processor> processor;
-
-  if (!parentGroup) {
-    logger_->log_error("parseProcessNodeYaml: no parent group exists");
-    return;
-  }
-
-  if (!processorsNode) {
-    throw std::invalid_argument("Cannot instantiate a MiNiFi instance without a defined Processors configuration node.");
-  }
-  if (!processorsNode.IsSequence()) {
-    throw std::invalid_argument(
-        "Cannot instantiate a MiNiFi instance without a defined Processors configuration node.");
-  }
-  // Evaluate sequence of processors
-  for (YAML::const_iterator iter = processorsNode.begin(); iter != processorsNode.end(); ++iter) {
-    core::ProcessorConfig procCfg;
-    const auto procNode = iter->as<YAML::Node>();
-
-    yaml::checkRequiredField(procNode, "name", CONFIG_YAML_PROCESSORS_KEY);
-    procCfg.name = procNode["name"].as<std::string>();
-    procCfg.id = getOrGenerateId(procNode);
-
-    uuid = procCfg.id;
-    logger_->log_debug("parseProcessorNode: name => [%s] id => [%s]", procCfg.name, procCfg.id);
-    yaml::checkRequiredField(procNode, "class", CONFIG_YAML_PROCESSORS_KEY);
-    procCfg.javaClass = procNode["class"].as<std::string>();
-    logger_->log_debug("parseProcessorNode: class => [%s]", procCfg.javaClass);
-
-    // Determine the processor name only from the Java class
-    auto lastOfIdx = procCfg.javaClass.find_last_of('.');
-    if (lastOfIdx != std::string::npos) {
-      lastOfIdx++;  // if a value is found, increment to move beyond the .
-      std::string processorName = procCfg.javaClass.substr(lastOfIdx);
-      processor = this->createProcessor(processorName, procCfg.javaClass, uuid);
-    } else {
-      // Allow unqualified class names for core processors
-      processor = this->createProcessor(procCfg.javaClass, uuid);
-    }
-
-    if (!processor) {
-      logger_->log_error("Could not create a processor %s with id %s", procCfg.name, procCfg.id);
-      throw std::invalid_argument("Could not create processor " + procCfg.name);
-    }
-
-    processor->setName(procCfg.name);
-
-    processor->setFlowIdentifier(flow_version_->getFlowIdentifier());
-
-    auto strategyNode = getOptionalField(procNode, "scheduling strategy", YAML::Node(DEFAULT_SCHEDULING_STRATEGY),
-    CONFIG_YAML_PROCESSORS_KEY);
-    procCfg.schedulingStrategy = strategyNode.as<std::string>();
-    logger_->log_debug("parseProcessorNode: scheduling strategy => [%s]", procCfg.schedulingStrategy);
-
-    auto periodNode = getOptionalField(procNode, "scheduling period", YAML::Node(DEFAULT_SCHEDULING_PERIOD_STR),
-    CONFIG_YAML_PROCESSORS_KEY);
-
-    procCfg.schedulingPeriod = periodNode.as<std::string>();
-    logger_->log_debug("parseProcessorNode: scheduling period => [%s]", procCfg.schedulingPeriod);
-
-    if (procNode["max concurrent tasks"]) {
-      procCfg.maxConcurrentTasks = procNode["max concurrent tasks"].as<std::string>();
-      logger_->log_debug("parseProcessorNode: max concurrent tasks => [%s]", procCfg.maxConcurrentTasks);
-    }
-
-    if (procNode["penalization period"]) {
-      procCfg.penalizationPeriod = procNode["penalization period"].as<std::string>();
-      logger_->log_debug("parseProcessorNode: penalization period => [%s]", procCfg.penalizationPeriod);
-    }
-
-    if (procNode["yield period"]) {
-      procCfg.yieldPeriod = procNode["yield period"].as<std::string>();
-      logger_->log_debug("parseProcessorNode: yield period => [%s]", procCfg.yieldPeriod);
-    }
-
-    if (procNode["run duration nanos"]) {
-      procCfg.runDurationNanos = procNode["run duration nanos"].as<std::string>();
-      logger_->log_debug("parseProcessorNode: run duration nanos => [%s]", procCfg.runDurationNanos);
-    }
-
-    // handle auto-terminated relationships
-    if (procNode["auto-terminated relationships list"]) {
-      YAML::Node autoTerminatedSequence = procNode["auto-terminated relationships list"];
-      std::vector<std::string> rawAutoTerminatedRelationshipValues;
-      if (autoTerminatedSequence.IsSequence() && !autoTerminatedSequence.IsNull() && autoTerminatedSequence.size() > 0) {
-        for (YAML::const_iterator relIter = autoTerminatedSequence.begin(); relIter != autoTerminatedSequence.end(); ++relIter) {
-          auto autoTerminatedRel = relIter->as<std::string>();
-          rawAutoTerminatedRelationshipValues.push_back(autoTerminatedRel);
-        }
-      }
-      procCfg.autoTerminatedRelationships = rawAutoTerminatedRelationshipValues;
-    }
-
-    // handle processor properties
-    if (procNode["Properties"]) {
-      YAML::Node propertiesNode = procNode["Properties"];
-      parsePropertiesNodeYaml(propertiesNode, *processor, procCfg.name, CONFIG_YAML_PROCESSORS_KEY);
-    }
-
-    // Take care of scheduling
-
-    if (procCfg.schedulingStrategy == "TIMER_DRIVEN" || procCfg.schedulingStrategy == "EVENT_DRIVEN") {
-      if (auto scheduling_period = utils::timeutils::StringToDuration<std::chrono::nanoseconds>(procCfg.schedulingPeriod)) {
-        logger_->log_debug("convert: parseProcessorNode: schedulingPeriod => [%" PRId64 "] ns", scheduling_period->count());
-        processor->setSchedulingPeriodNano(*scheduling_period);
-      }
-    } else {
-      processor->setCronPeriod(procCfg.schedulingPeriod);
-    }
-
-    if (auto penalization_period = utils::timeutils::StringToDuration<std::chrono::milliseconds>(procCfg.penalizationPeriod)) {
-      logger_->log_debug("convert: parseProcessorNode: penalizationPeriod => [%" PRId64 "] ms", penalization_period->count());
-      processor->setPenalizationPeriod(penalization_period.value());
-    }
-
-    if (auto yield_period = utils::timeutils::StringToDuration<std::chrono::milliseconds>(procCfg.yieldPeriod)) {
-      logger_->log_debug("convert: parseProcessorNode: yieldPeriod => [%" PRId64 "] ms", yield_period->count());
-      processor->setYieldPeriodMsec(yield_period.value());
-    }
-
-    // Default to running
-    processor->setScheduledState(core::RUNNING);
-
-    if (procCfg.schedulingStrategy == "TIMER_DRIVEN") {
-      processor->setSchedulingStrategy(core::TIMER_DRIVEN);
-      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
-    } else if (procCfg.schedulingStrategy == "EVENT_DRIVEN") {
-      processor->setSchedulingStrategy(core::EVENT_DRIVEN);
-      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
-    } else {
-      processor->setSchedulingStrategy(core::CRON_DRIVEN);
-      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
-    }
-
-    int32_t maxConcurrentTasks;
-    if (core::Property::StringToInt(procCfg.maxConcurrentTasks, maxConcurrentTasks)) {
-      logger_->log_debug("parseProcessorNode: maxConcurrentTasks => [%d]", maxConcurrentTasks);
-      processor->setMaxConcurrentTasks((uint8_t) maxConcurrentTasks);
-    }
-
-    if (core::Property::StringToInt(procCfg.runDurationNanos, runDurationNanos)) {
-      logger_->log_debug("parseProcessorNode: runDurationNanos => [%d]", runDurationNanos);
-      processor->setRunDurationNano(std::chrono::nanoseconds(runDurationNanos));
-    }
-
-    std::vector<core::Relationship> autoTerminatedRelationships;
-    for (auto &&relString : procCfg.autoTerminatedRelationships) {
-      core::Relationship relationship(relString, "");
-      logger_->log_debug("parseProcessorNode: autoTerminatedRelationship  => [%s]", relString);
-      autoTerminatedRelationships.push_back(relationship);
-    }
-
-    processor->setAutoTerminatedRelationships(autoTerminatedRelationships);
-
-    parentGroup->addProcessor(std::move(processor));
-  }
-}
-
-void YamlConfiguration::parseRemoteProcessGroupYaml(const YAML::Node& rpgNode, core::ProcessGroup* parentGroup) {
-  utils::Identifier uuid;
-  std::string id;
-
-  if (!parentGroup) {
-    logger_->log_error("parseRemoteProcessGroupYaml: no parent group exists");
-    return;
-  }
-
-  if (!rpgNode || !rpgNode.IsSequence()) {
-    return;
-  }
-  for (YAML::const_iterator iter = rpgNode.begin(); iter != rpgNode.end(); ++iter) {
-    auto currRpgNode = iter->as<YAML::Node>();
-
-    yaml::checkRequiredField(currRpgNode, "name", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
-    auto name = currRpgNode["name"].as<std::string>();
-    id = getOrGenerateId(currRpgNode);
-
-    logger_->log_debug("parseRemoteProcessGroupYaml: name => [%s], id => [%s]", name, id);
-
-    auto urlNode = getOptionalField(currRpgNode, "url", YAML::Node(""),
-    CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
-
-    auto url = urlNode.as<std::string>();
-    logger_->log_debug("parseRemoteProcessGroupYaml: url => [%s]", url);
-
-    uuid = id;
-    auto group = this->createRemoteProcessGroup(name, uuid);
-    group->setParent(parentGroup);
-
-    if (currRpgNode["yield period"]) {
-      auto yieldPeriod = currRpgNode["yield period"].as<std::string>();
-      logger_->log_debug("parseRemoteProcessGroupYaml: yield period => [%s]", yieldPeriod);
-
-      auto yield_period_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(yieldPeriod);
-      if (yield_period_value.has_value() && group) {
-        logger_->log_debug("parseRemoteProcessGroupYaml: yieldPeriod => [%" PRId64 "] ms", yield_period_value->count());
-        group->setYieldPeriodMsec(*yield_period_value);
-      }
-    }
-
-    if (currRpgNode["timeout"]) {
-      auto timeout = currRpgNode["timeout"].as<std::string>();
-      logger_->log_debug("parseRemoteProcessGroupYaml: timeout => [%s]", timeout);
-
-      auto timeout_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(timeout);
-      if (timeout_value.has_value() && group) {
-        logger_->log_debug("parseRemoteProcessGroupYaml: timeoutValue => [%" PRId64 "] ms", timeout_value->count());
-        group->setTimeout(timeout_value->count());
-      }
-    }
-
-    if (currRpgNode["local network interface"]) {
-      auto interface = currRpgNode["local network interface"].as<std::string>();
-      logger_->log_debug("parseRemoteProcessGroupYaml: local network interface => [%s]", interface);
-      group->setInterface(interface);
-    }
-
-    if (currRpgNode["transport protocol"]) {
-      auto transport_protocol = currRpgNode["transport protocol"].as<std::string>();
-      logger_->log_debug("parseRemoteProcessGroupYaml: transport protocol => [%s]", transport_protocol);
-      if (transport_protocol == "HTTP") {
-        group->setTransportProtocol(transport_protocol);
-        if (currRpgNode["proxy host"]) {
-          auto http_proxy_host = currRpgNode["proxy host"].as<std::string>();
-          logger_->log_debug("parseRemoteProcessGroupYaml: proxy host => [%s]", http_proxy_host);
-          group->setHttpProxyHost(http_proxy_host);
-          if (currRpgNode["proxy user"]) {
-            auto http_proxy_username = currRpgNode["proxy user"].as<std::string>();
-            logger_->log_debug("parseRemoteProcessGroupYaml: proxy user => [%s]", http_proxy_username);
-            group->setHttpProxyUserName(http_proxy_username);
-          }
-          if (currRpgNode["proxy password"]) {
-            auto http_proxy_password = currRpgNode["proxy password"].as<std::string>();
-            logger_->log_debug("parseRemoteProcessGroupYaml: proxy password => [%s]", http_proxy_password);
-            group->setHttpProxyPassWord(http_proxy_password);
+YamlConfiguration::YamlConfiguration(ConfigurationContext ctx)
+    : StructuredConfiguration(([&] {
+          if (!ctx.path) {
+            ctx.path = DEFAULT_NIFI_CONFIG_YML;
           }
-          if (currRpgNode["proxy port"]) {
-            auto http_proxy_port = currRpgNode["proxy port"].as<std::string>();
-            int32_t port;
-            if (core::Property::StringToInt(http_proxy_port, port)) {
-              logger_->log_debug("parseRemoteProcessGroupYaml: proxy port => [%d]", port);
-              group->setHttpProxyPort(port);
-            }
-          }
-        }
-      } else if (transport_protocol == "RAW") {
-        group->setTransportProtocol(transport_protocol);
-      } else {
-        std::stringstream stream;
-        stream << "Invalid transport protocol " << transport_protocol;
-        throw minifi::Exception(ExceptionType::SITE2SITE_EXCEPTION, stream.str().c_str());
-      }
-    }
-
-    group->setTransmitting(true);
-    group->setURL(url);
-
-    yaml::checkRequiredField(currRpgNode, "Input Ports", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
-    auto inputPorts = currRpgNode["Input Ports"].as<YAML::Node>();
-    if (inputPorts && inputPorts.IsSequence()) {
-      for (YAML::const_iterator portIter = inputPorts.begin(); portIter != inputPorts.end(); ++portIter) {
-        auto currPort = portIter->as<YAML::Node>();
-
-        this->parsePortYaml(currPort, group.get(), sitetosite::SEND);
-      }  // for node
-    }
-    auto outputPorts = currRpgNode["Output Ports"].as<YAML::Node>();
-    if (outputPorts && outputPorts.IsSequence()) {
-      for (YAML::const_iterator portIter = outputPorts.begin(); portIter != outputPorts.end(); ++portIter) {
-        logger_->log_debug("Got a current port, iterating...");
-
-        auto currPort = portIter->as<YAML::Node>();
-
-        this->parsePortYaml(currPort, group.get(), sitetosite::RECEIVE);
-      }  // for node
-    }
-    parentGroup->addProcessGroup(std::move(group));
-  }
-}
-
-void YamlConfiguration::parseProvenanceReportingYaml(const YAML::Node& reportNode, core::ProcessGroup* parentGroup) {
-  utils::Identifier port_uuid;
+          return std::move(ctx);
+        })(),
+        logging::LoggerFactory<YamlConfiguration>::getLogger()) {}
 
-  if (!parentGroup) {
-    logger_->log_error("parseProvenanceReportingYaml: no parent group exists");
-    return;
+std::unique_ptr<core::ProcessGroup> YamlConfiguration::getRoot() {
+  if (!config_path_) {
+    logger_->log_error("Cannot instantiate flow, no config file is set.");
+    throw Exception(ExceptionType::FLOW_EXCEPTION, "No config file specified");
   }
-
-  if (!reportNode || !reportNode.IsDefined() || reportNode.IsNull()) {
-    logger_->log_debug("no provenance reporting task specified");
-    return;
-  }
-
-  auto reportTask = createProvenanceReportTask();
-
-  const auto node = reportNode.as<YAML::Node>();
-
-  yaml::checkRequiredField(node, "scheduling strategy", CONFIG_YAML_PROVENANCE_REPORT_KEY);
-  auto schedulingStrategyStr = node["scheduling strategy"].as<std::string>();
-  yaml::checkRequiredField(node, "scheduling period", CONFIG_YAML_PROVENANCE_REPORT_KEY);
-  auto schedulingPeriodStr = node["scheduling period"].as<std::string>();
-
-  if (auto scheduling_period = utils::timeutils::StringToDuration<std::chrono::nanoseconds>(schedulingPeriodStr)) {
-    logger_->log_debug("ProvenanceReportingTask schedulingPeriod %" PRId64 " ns", scheduling_period->count());
-    reportTask->setSchedulingPeriodNano(*scheduling_period);
+  const auto configuration = filesystem_->read(config_path_.value());
+  if (!configuration) {
+  // non-existence of flow config file is not a dealbreaker, the caller might fetch it from network
+  return nullptr;
   }
-
-  if (schedulingStrategyStr == "TIMER_DRIVEN") {
-    reportTask->setSchedulingStrategy(core::TIMER_DRIVEN);
-    logger_->log_debug("ProvenanceReportingTask scheduling strategy %s", schedulingStrategyStr);
-  } else {
-    throw std::invalid_argument("Invalid scheduling strategy " + schedulingStrategyStr);
-  }
-
-  int64_t lvalue;
-  if (node["host"] && node["port"]) {
-    auto hostStr = node["host"].as<std::string>();
-
-    auto portStr = node["port"].as<std::string>();
-    if (core::Property::StringToInt(portStr, lvalue) && !hostStr.empty()) {
-      logger_->log_debug("ProvenanceReportingTask port %" PRId64, lvalue);
-      std::string url = hostStr + ":" + portStr;
-      reportTask->setURL(url);
-    }
-  }
-
-  if (node["url"]) {
-    auto urlStr = node["url"].as<std::string>();
-    if (!urlStr.empty()) {
-      reportTask->setURL(urlStr);
-      logger_->log_debug("ProvenanceReportingTask URL %s", urlStr);
-    }
-  }
-  yaml::checkRequiredField(node, "port uuid", CONFIG_YAML_PROVENANCE_REPORT_KEY);
-  auto portUUIDStr = node["port uuid"].as<std::string>();
-  yaml::checkRequiredField(node, "batch size", CONFIG_YAML_PROVENANCE_REPORT_KEY);
-  auto batchSizeStr = node["batch size"].as<std::string>();
-
-  logger_->log_debug("ProvenanceReportingTask port uuid %s", portUUIDStr);
-  port_uuid = portUUIDStr;
-  reportTask->setPortUUID(port_uuid);
-
-  if (core::Property::StringToInt(batchSizeStr, lvalue)) {
-    reportTask->setBatchSize(gsl::narrow<int>(lvalue));
-  }
-
-  reportTask->initialize();
-
-  // add processor to parent
-  reportTask->setScheduledState(core::RUNNING);
-  parentGroup->addProcessor(std::move(reportTask));
-}
-
-void YamlConfiguration::parseControllerServices(const YAML::Node& controllerServicesNode) {
-  if (!controllerServicesNode || !controllerServicesNode.IsSequence()) {
-    return;
-  }
-  for (const auto& iter : controllerServicesNode) {
-    const auto controllerServiceNode = iter.as<YAML::Node>();
-    try {
-      yaml::checkRequiredField(controllerServiceNode, "name", CONFIG_YAML_CONTROLLER_SERVICES_KEY);
-
-      auto type = yaml::getRequiredField(controllerServiceNode, std::vector<std::string>{"class", "type"}, CONFIG_YAML_CONTROLLER_SERVICES_KEY);
-      logger_->log_debug("Using type %s for controller service node", type);
-
-      std::string fullType = type;
-      auto lastOfIdx = type.find_last_of('.');
-      if (lastOfIdx != std::string::npos) {
-        lastOfIdx++;  // if a value is found, increment to move beyond the .
-        type = type.substr(lastOfIdx);
-      }
-
-      auto name = controllerServiceNode["name"].as<std::string>();
-      auto id = getRequiredIdField(controllerServiceNode, CONFIG_YAML_CONTROLLER_SERVICES_KEY);
-
-      utils::Identifier uuid;
-      uuid = id;
-      std::shared_ptr<core::controller::ControllerServiceNode> controller_service_node = createControllerService(type, fullType, name, uuid);
-      if (nullptr != controller_service_node) {
-        logger_->log_debug("Created Controller Service with UUID %s and name %s", id, name);
-        controller_service_node->initialize();
-        YAML::Node propertiesNode = controllerServiceNode["Properties"];
-        // we should propagate properties to the node and to the implementation
-        parsePropertiesNodeYaml(propertiesNode, *controller_service_node, name, CONFIG_YAML_CONTROLLER_SERVICES_KEY);
-        if (auto controllerServiceImpl = controller_service_node->getControllerServiceImplementation(); controllerServiceImpl) {
-          parsePropertiesNodeYaml(propertiesNode, *controllerServiceImpl, name, CONFIG_YAML_CONTROLLER_SERVICES_KEY);
-        }
-      } else {
-        logger_->log_debug("Could not locate %s", type);
-      }
-      controller_services_->put(id, controller_service_node);
-      controller_services_->put(name, controller_service_node);
-    } catch (YAML::InvalidNode &) {
-      throw Exception(ExceptionType::GENERAL_EXCEPTION, "Name, id, and class must be specified for controller services");
-    }
-  }
-}
-
-void YamlConfiguration::parseConnectionYaml(const YAML::Node& connectionsNode, core::ProcessGroup* parent) {
-  if (!parent) {
-    logger_->log_error("parseProcessNode: no parent group was provided");
-    return;
-  }
-  if (!connectionsNode || !connectionsNode.IsSequence()) {
-    return;
-  }
-
-  for (YAML::const_iterator iter = connectionsNode.begin(); iter != connectionsNode.end(); ++iter) {
-    const auto connectionNode = iter->as<YAML::Node>();
-
-    // Configure basic connection
-    const std::string id = getOrGenerateId(connectionNode);
-
-    // Default name to be same as ID
-    // If name is specified in configuration, use the value
-    const auto name = connectionNode["name"].as<std::string>(id);
-
-    const auto uuid = utils::Identifier::parse(id) | utils::orElse([this] {
-      logger_->log_debug("Incorrect connection UUID format.");
-      throw Exception(ExceptionType::GENERAL_EXCEPTION, "Incorrect connection UUID format.");
-    });
-
-    auto connection = createConnection(name, uuid.value());
-    logger_->log_debug("Created connection with UUID %s and name %s", id, name);
-    const yaml::YamlConnectionParser connectionParser(connectionNode, name, gsl::not_null<core::ProcessGroup*>{ parent }, logger_);
-    connectionParser.configureConnectionSourceRelationshipsFromYaml(*connection);
-    connection->setMaxQueueSize(connectionParser.getWorkQueueSizeFromYaml());
-    connection->setMaxQueueDataSize(connectionParser.getWorkQueueDataSizeFromYaml());
-    connection->setSwapThreshold(connectionParser.getSwapThresholdFromYaml());
-    connection->setSourceUUID(connectionParser.getSourceUUIDFromYaml());
-    connection->setDestinationUUID(connectionParser.getDestinationUUIDFromYaml());
-    connection->setFlowExpirationDuration(connectionParser.getFlowFileExpirationFromYaml());
-    connection->setDropEmptyFlowFiles(connectionParser.getDropEmptyFromYaml());
-
-    parent->addConnection(std::move(connection));
-  }
-}
-
-void YamlConfiguration::parsePortYaml(const YAML::Node& portNode, core::ProcessGroup* parent, sitetosite::TransferDirection direction) {
-  utils::Identifier uuid;
-
-  if (!parent) {
-    logger_->log_error("parseProcessNode: no parent group existed");
-    return;
-  }
-
-  const auto inputPortsObj = portNode.as<YAML::Node>();
-
-  // Check for required fields
-  yaml::checkRequiredField(inputPortsObj, "name", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
-  auto nameStr = inputPortsObj["name"].as<std::string>();
-  auto portId = getRequiredIdField(inputPortsObj, CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY,
-    "The field 'id' is required for "
-    "the port named '" + nameStr + "' in the YAML Config. If this port "
-    "is an input port for a NiFi Remote Process Group, the port "
-    "id should match the corresponding id specified in the NiFi configuration. "
-    "This is a UUID of the format XXXXXXXX-XXXX-XXXX-XXXX-XXXXXXXXXXXX.");
-  uuid = portId;
-
-  auto port = std::make_unique<minifi::RemoteProcessorGroupPort>(
-          stream_factory_, nameStr, parent->getURL(), this->configuration_, uuid);
-  port->setDirection(direction);
-  port->setTimeout(parent->getTimeout());
-  port->setTransmitting(true);
-  port->setYieldPeriodMsec(parent->getYieldPeriodMsec());
-  port->initialize();
-  if (!parent->getInterface().empty())
-    port->setInterface(parent->getInterface());
-  if (parent->getTransportProtocol() == "HTTP") {
-    port->enableHTTP();
-    if (!parent->getHttpProxyHost().empty())
-      port->setHTTPProxy(parent->getHTTPProxy());
-  }
-  // else defaults to RAW
-
-  // handle port properties
-  const auto nodeVal = portNode.as<YAML::Node>();
-  YAML::Node propertiesNode = nodeVal["Properties"];
-  parsePropertiesNodeYaml(propertiesNode, *port, nameStr, CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
-
-  // add processor to parent
-  auto& processor = *port;
-  parent->addProcessor(std::move(port));
-  processor.setScheduledState(core::RUNNING);
-
-  if (inputPortsObj["max concurrent tasks"]) {
-    auto rawMaxConcurrentTasks = inputPortsObj["max concurrent tasks"].as<std::string>();
-    int32_t maxConcurrentTasks;
-    if (core::Property::StringToInt(rawMaxConcurrentTasks, maxConcurrentTasks)) {
-      processor.setMaxConcurrentTasks(maxConcurrentTasks);
-    }
-    logger_->log_debug("parseProcessorNode: maxConcurrentTasks => [%d]", maxConcurrentTasks);
-    processor.setMaxConcurrentTasks(maxConcurrentTasks);
-  }
-}
-
-void YamlConfiguration::parsePropertyValueSequence(const std::string& propertyName, const YAML::Node& propertyValueNode, core::ConfigurableComponent& component) {
-  for (const auto& iter : propertyValueNode) {
-    if (iter.IsDefined()) {
-      const auto nodeVal = iter.as<YAML::Node>();
-      YAML::Node propertiesNode = nodeVal["value"];
-      // must insert the sequence in differently.
-      const auto rawValueString = propertiesNode.as<std::string>();
-      logger_->log_debug("Found %s=%s", propertyName, rawValueString);
-      if (!component.updateProperty(propertyName, rawValueString)) {
-        auto proc = dynamic_cast<core::Connectable*>(&component);
-        if (proc) {
-          logger_->log_warn("Received property %s with value %s but is not one of the properties for %s. Attempting to add as dynamic property.", propertyName, rawValueString, proc->getName());
-          if (!component.setDynamicProperty(propertyName, rawValueString)) {
-            logger_->log_warn("Unable to set the dynamic property %s with value %s", propertyName, rawValueString);
-          } else {
-            logger_->log_warn("Dynamic property %s with value %s set", propertyName, rawValueString);
-          }
-        }
-      }
-    }
-  }
-}
-
-PropertyValue YamlConfiguration::getValidatedProcessorPropertyForDefaultTypeInfo(const core::Property& propertyFromProcessor, const YAML::Node& propertyValueNode) {
-  PropertyValue defaultValue;
-  defaultValue = propertyFromProcessor.getDefaultValue();
-  const std::type_index defaultType = defaultValue.getTypeInfo();
   try {
-    PropertyValue coercedValue = defaultValue;
-    if (defaultType == typeid(int64_t)) {
-      coercedValue = propertyValueNode.as<int64_t>();
-    } else if (defaultType == typeid(uint64_t)) {
-      uint64_t integer_value;
-      if (YAML::convert<uint64_t>::decode(propertyValueNode, integer_value)) {
-        coercedValue = integer_value;
-      } else {
-        coercedValue = propertyValueNode.as<std::string>();
-      }
-    } else if (defaultType == typeid(int)) {
-      coercedValue = propertyValueNode.as<int>();
-    } else if (defaultType == typeid(bool)) {
-      coercedValue = propertyValueNode.as<bool>();
-    } else {
-      coercedValue = propertyValueNode.as<std::string>();
-    }
-    return coercedValue;
-  } catch (const std::exception& e) {
-    logger_->log_error("Fetching property failed with an exception of %s", e.what());
-    logger_->log_error("Invalid conversion for field %s. Value %s", propertyFromProcessor.getName(), propertyValueNode.as<std::string>());
-  } catch (...) {
-    logger_->log_error("Invalid conversion for field %s. Value %s", propertyFromProcessor.getName(), propertyValueNode.as<std::string>());
-  }
-  return defaultValue;
-}
-
-void YamlConfiguration::parseSingleProperty(const std::string& propertyName, const YAML::Node& propertyValueNode, core::ConfigurableComponent& processor) {
-  core::Property myProp(propertyName, "", "");
-  processor.getProperty(propertyName, myProp);
-  const PropertyValue coercedValue = getValidatedProcessorPropertyForDefaultTypeInfo(myProp, propertyValueNode);
-  bool property_set = false;
-  try {
-    property_set = processor.setProperty(myProp, coercedValue);
-  } catch(const utils::internal::InvalidValueException&) {
-    auto component = dynamic_cast<core::CoreComponent*>(&processor);
-    if (component == nullptr) {
-      logger_->log_error("processor was not a CoreComponent for property '%s'", propertyName);
-    } else {
-      logger_->log_error("Invalid value was set for property '%s' creating component '%s'", propertyName, component->getName());
-    }
+    YAML::Node rootYamlNode = YAML::Load(configuration.value());
+    flow::Node root{std::make_shared<YamlNode>(rootYamlNode)};
+    return getRootFrom(root);
+  } catch(...) {
+    logger_->log_error("Invalid yaml configuration file");
     throw;
   }
-  const auto rawValueString = propertyValueNode.as<std::string>();
-  if (!property_set) {
-    auto proc = dynamic_cast<core::Connectable*>(&processor);
-    if (proc) {
-      logger_->log_warn("Received property %s with value %s but is not one of the properties for %s. Attempting to add as dynamic property.", propertyName, rawValueString, proc->getName());
-      if (!processor.setDynamicProperty(propertyName, rawValueString)) {
-        logger_->log_warn("Unable to set the dynamic property %s with value %s", propertyName, rawValueString);
-      } else {
-        logger_->log_warn("Dynamic property %s with value %s set", propertyName, rawValueString);
-      }
-    }
-  } else {
-    logger_->log_debug("Property %s with value %s set", propertyName, rawValueString);
-  }
-}
-
-void YamlConfiguration::parsePropertyNodeElement(const std::string& propertyName, const YAML::Node& propertyValueNode, core::ConfigurableComponent& processor) {
-  logger_->log_trace("Encountered %s", propertyName);
-  if (propertyValueNode.IsNull() || !propertyValueNode.IsDefined()) {
-    return;
-  }
-  if (propertyValueNode.IsSequence()) {
-    parsePropertyValueSequence(propertyName, propertyValueNode, processor);
-  } else {
-    parseSingleProperty(propertyName, propertyValueNode, processor);
-  }
-}
-
-void YamlConfiguration::parsePropertiesNodeYaml(const YAML::Node& propertiesNode, core::ConfigurableComponent& component, const std::string& component_name,
-    const std::string& yaml_section) {
-  // Treat generically as a YAML node so we can perform inspection on entries to ensure they are populated
-  logger_->log_trace("Entered %s", component_name);
-  for (const auto& propertyElem : propertiesNode) {
-    const auto propertyName = propertyElem.first.as<std::string>();
-    const YAML::Node propertyValueNode = propertyElem.second;
-    parsePropertyNodeElement(propertyName, propertyValueNode, component);
-  }
-
-  validateComponentProperties(component, component_name, yaml_section);
 }
 
-void YamlConfiguration::parseFunnelsYaml(const YAML::Node& node, core::ProcessGroup* parent) {
-  if (!parent) {
-    logger_->log_error("parseFunnelsYaml: no parent group was provided");
-    return;
-  }
-  if (!node || !node.IsSequence()) {
-    return;
-  }
-
-  for (const auto& element : node) {
-    const auto funnel_node = element.as<YAML::Node>();
-
-    std::string id = getOrGenerateId(funnel_node);
-
-    // Default name to be same as ID
-    const auto name = funnel_node["name"].as<std::string>(id);
-
-    const auto uuid = utils::Identifier::parse(id) | utils::orElse([this] {
-      logger_->log_debug("Incorrect funnel UUID format.");
-      throw Exception(ExceptionType::GENERAL_EXCEPTION, "Incorrect funnel UUID format.");
-    });
-
-    auto funnel = std::make_unique<core::Funnel>(name, uuid.value());
-    logger_->log_debug("Created funnel with UUID %s and name %s", id, name);
-    funnel->setScheduledState(core::RUNNING);
-    funnel->setSchedulingStrategy(core::EVENT_DRIVEN);
-    parent->addProcessor(std::move(funnel));
-  }
-}
-
-void YamlConfiguration::validateComponentProperties(ConfigurableComponent& component, const std::string &component_name, const std::string &yaml_section) const {
-  const auto &component_properties = component.getProperties();
-
-  // Validate required properties
-  for (const auto &prop_pair : component_properties) {
-    if (prop_pair.second.getRequired()) {
-      if (prop_pair.second.getValue().to_string().empty()) {
-        std::string reason = utils::StringUtils::join_pack("required property '", prop_pair.second.getName(), "' is not set");
-        raiseComponentError(component_name, yaml_section, reason);
-      } else if (!prop_pair.second.getValue().validate(prop_pair.first).valid()) {
-        std::string reason = utils::StringUtils::join_pack("the value '", prop_pair.first, "' is not valid for property '", prop_pair.second.getName(), "'");
-        raiseComponentError(component_name, yaml_section, reason);
-      }
-    }
-  }
-
-  // Validate dependent properties
-  for (const auto &prop_pair : component_properties) {
-    const auto &dep_props = prop_pair.second.getDependentProperties();
-
-    if (prop_pair.second.getValue().to_string().empty()) {
-      continue;
-    }
-
-    for (const auto &dep_prop_key : dep_props) {
-      if (component_properties.at(dep_prop_key).getValue().to_string().empty()) {
-        std::string reason = utils::StringUtils::join_pack("property '", prop_pair.second.getName(),
-            "' depends on property '", dep_prop_key, "' which is not set");
-        raiseComponentError(component_name, yaml_section, reason);
-      }
-    }
-  }
-
-#ifdef YAML_CONFIGURATION_USE_REGEX
-  // Validate mutually-exclusive properties
-  for (const auto &prop_pair : component_properties) {
-    const auto &excl_props = prop_pair.second.getExclusiveOfProperties();
-
-    if (prop_pair.second.getValue().empty()) {
-      continue;
-    }
-
-    for (const auto &excl_pair : excl_props) {
-      utils::Regex excl_expr(excl_pair.second);
-      if (utils::regexMatch(component_properties.at(excl_pair.first).getValue().to_string(), excl_expr)) {
-        std::string reason = utils::StringUtils::join_pack("property '", prop_pair.second.getName(),
-            "' must not be set when the value of property '", excl_pair.first, "' matches '", excl_pair.second, "'");
-        raiseComponentError(component_name, yaml_section, reason);
-      }
-    }
-  }
-
-  // Validate regex properties
-  for (const auto &prop_pair : component_properties) {
-    const auto &prop_regex_str = prop_pair.second.getValidRegex();
-
-    if (!prop_regex_str.empty()) {
-      utils::Regex prop_regex(prop_regex_str);
-      if (!utils::regexMatch(prop_pair.second.getValue().to_string(), prop_regex)) {
-        std::string reason = utils::StringUtils::join_pack("property '", prop_pair.second.getName(), "' does not match validation pattern '", prop_regex_str, "'");
-        raiseComponentError(component_name, yaml_section, reason);
-      }
-    }
-  }
-#else
-  logging::LOG_INFO(logger_) << "Validation of mutally-exclusive properties is disabled in this build.";
-  logging::LOG_INFO(logger_) << "Regex validation of properties is not available in this build.";
-#endif  // YAML_CONFIGURATION_USE_REGEX
-}
-
-void YamlConfiguration::raiseComponentError(const std::string &component_name, const std::string &yaml_section, const std::string &reason) const {
-  std::string err_msg = "Unable to parse configuration file for component named '";
-  err_msg.append(component_name);
-  err_msg.append("' because " + reason);
-  if (!yaml_section.empty()) {
-    err_msg.append(" [in '" + yaml_section + "' section of configuration file]");
-  }
-
-  logging::LOG_ERROR(logger_) << err_msg;
-
-  throw std::invalid_argument(err_msg);
-}
-
-std::string YamlConfiguration::getOrGenerateId(const YAML::Node& yamlNode, const std::string& idField) {
-  std::string id;
-  auto node = yamlNode.as<YAML::Node>();
-
-  if (node[idField]) {
-    if (YAML::NodeType::Scalar == node[idField].Type()) {
-      id = node[idField].as<std::string>();
-      addNewId(id);
-      return id;
-    }
-    throw std::invalid_argument("getOrGenerateId: idField is expected to reference YAML::Node of YAML::NodeType::Scalar.");
-  }
-
-  id = id_generator_->generate().to_string();
-  logger_->log_debug("Generating random ID: id => [%s]", id);
-  return id;
-}
-
-std::string YamlConfiguration::getRequiredIdField(const YAML::Node& yaml_node, std::string_view yaml_section, std::string_view error_message) {
-  yaml::checkRequiredField(yaml_node, "id", yaml_section, error_message);
-  auto id = yaml_node["id"].as<std::string>();
-  addNewId(id);
-  return id;
-}
-
-YAML::Node YamlConfiguration::getOptionalField(const YAML::Node& yamlNode, const std::string& fieldName, const YAML::Node& defaultValue, const std::string& yamlSection,
-                                               const std::string& providedInfoMessage) {
-  std::string infoMessage = providedInfoMessage;
-  auto result = yamlNode.as<YAML::Node>()[fieldName];
-  if (!result) {
-    if (infoMessage.empty()) {
-      // Build a helpful info message for the user to inform them that a default is being used
-      infoMessage =
-          yamlNode.as<YAML::Node>()["name"] ?
-              "Using default value for optional field '" + fieldName + "' in component named '" + yamlNode.as<YAML::Node>()["name"].as<std::string>() + "'" :
-              "Using default value for optional field '" + fieldName + "' ";
-      if (!yamlSection.empty()) {
-        infoMessage += " [in '" + yamlSection + "' section of configuration file]: ";
-      }
-
-      infoMessage += defaultValue.as<std::string>();
-    }
-    logging::LOG_INFO(logger_) << infoMessage;
-    result = defaultValue;
+std::unique_ptr<core::ProcessGroup> YamlConfiguration::getYamlRoot(std::istream &yamlConfigStream) {
+  try {
+    YAML::Node rootYamlNode = YAML::Load(yamlConfigStream);
+    flow::Node root{std::make_shared<YamlNode>(rootYamlNode)};
+    return getRootFrom(root);
+  } catch (const YAML::ParserException &pe) {
+    logger_->log_error(pe.what());
+    std::rethrow_exception(std::current_exception());
   }
-
-  return result;
+  return nullptr;

Review Comment:
   I don't know if any compiler complains, removed it



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r1003006036


##########
libminifi/include/core/json/JsonNode.h:
##########
@@ -0,0 +1,248 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string>
+#include <utility>
+#include <memory>
+
+#include "core/flow/Node.h"
+#include "rapidjson/document.h"
+#include "utils/gsl.h"
+#include "utils/ValueCaster.h"
+
+namespace org::apache::nifi::minifi::core {
+
+class JsonNode : public flow::Node::NodeImpl {
+ public:
+  explicit JsonNode(const rapidjson::Value* node): node_(node) {}
+
+  explicit operator bool() const override {
+    return node_ != nullptr;
+  }
+  bool isSequence() const override {
+    return node_ ? node_->IsArray() : false;
+  }
+  bool isMap() const override {
+    return node_ ? node_->IsObject() : false;
+  }
+  bool isNull() const override {
+    return node_ ? node_->IsNull() : false;
+  }
+
+  nonstd::expected<std::string, std::exception_ptr> getString() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get string of invalid json value");
+      }
+      if (!node_->IsString()) {
+        throw std::runtime_error("Cannot get string of non-string json value");
+      }
+      return std::string{node_->GetString(), node_->GetStringLength()};
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());
+    }
+  }
+
+  nonstd::expected<int, std::exception_ptr> getInt() const override {
+    return getNumber<int>("int");
+  }
+  nonstd::expected<unsigned int, std::exception_ptr> getUInt() const override {
+    return getNumber<unsigned int>("unsigned int");
+  }
+  nonstd::expected<int64_t, std::exception_ptr> getInt64() const override {
+    return getNumber<int64_t>("int64_t");
+  }
+  nonstd::expected<uint64_t, std::exception_ptr> getUInt64() const override {
+    return getNumber<uint64_t>("uint64_t");
+  }
+
+  nonstd::expected<bool, std::exception_ptr> getBool() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get bool of invalid json value");
+      }
+      if (!node_->IsBool()) {
+        throw std::runtime_error("Cannot get bool of non-bool json value");
+      }
+      return node_->GetBool();
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());
+    }
+  }
+
+  std::string getDebugString() const override {
+    if (!node_) return "<invalid>";
+    if (node_->IsObject()) return "<Map>";
+    if (node_->IsArray()) return "<Array>";
+    if (node_->IsNull()) return "null";
+    if (node_->IsInt()) return std::to_string(node_->GetInt());
+    if (node_->IsUint()) return std::to_string(node_->GetUint());
+    if (node_->IsInt64()) return std::to_string(node_->GetInt64());
+    if (node_->IsUint64()) return std::to_string(node_->GetUint64());
+    if (node_->IsTrue()) return "true";
+    if (node_->IsFalse()) return "false";
+    if (node_->IsDouble()) return std::to_string(node_->GetDouble());
+    if (node_->IsString()) return '"' + std::string(node_->GetString(), node_->GetStringLength()) + '"';
+    return "<unknown>";
+  }
+
+  size_t size() const override {
+    if (!node_) {
+      throw std::runtime_error("Cannot get size of invalid json value");
+    }
+    if (!node_->IsArray()) {
+      throw std::runtime_error("Cannot get size of non-array json value");
+    }
+    return node_->Size();
+  }
+  flow::Node::Iterator begin() const override;
+  flow::Node::Iterator end() const override;
+
+  flow::Node operator[](std::string_view key) const override {
+    if (!node_) {
+      throw std::runtime_error("Cannot get member of invalid json value");
+    }
+    if (!node_->IsObject()) {
+      return flow::Node{std::make_shared<JsonNode>(nullptr)};

Review Comment:
   the current parsing logic assumes the yaml behavior, I don't think it is useful and IMO we should throw as early as possible, but I didn't want to make larger changes to the logic, implementing a "stricter" behavior for json while maintaining yaml backwards compatibility should come in part2



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] szaszm commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r989134695


##########
libminifi/include/core/json/JsonNode.h:
##########
@@ -0,0 +1,248 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string>
+#include <utility>
+#include <memory>
+
+#include "core/flow/Node.h"
+#include "rapidjson/document.h"
+#include "utils/gsl.h"
+#include "utils/ValueCaster.h"
+
+namespace org::apache::nifi::minifi::core {
+
+class JsonNode : public flow::Node::NodeImpl {

Review Comment:
   We should separate the roles of tree node and tagged union. This class should only be a tree node, and not handle multiple types. Just return whatever we get, and convert it to std::variant somewhere, so we don't need to reinvent the tagged union (again).



##########
libminifi/include/core/json/JsonNode.h:
##########
@@ -0,0 +1,248 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string>
+#include <utility>
+#include <memory>
+
+#include "core/flow/Node.h"
+#include "rapidjson/document.h"
+#include "utils/gsl.h"
+#include "utils/ValueCaster.h"
+
+namespace org::apache::nifi::minifi::core {
+
+class JsonNode : public flow::Node::NodeImpl {
+ public:
+  explicit JsonNode(const rapidjson::Value* node): node_(node) {}
+
+  explicit operator bool() const override {
+    return node_ != nullptr;
+  }
+  bool isSequence() const override {
+    return node_ ? node_->IsArray() : false;
+  }
+  bool isMap() const override {
+    return node_ ? node_->IsObject() : false;
+  }
+  bool isNull() const override {
+    return node_ ? node_->IsNull() : false;
+  }
+
+  nonstd::expected<std::string, std::exception_ptr> getString() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get string of invalid json value");
+      }
+      if (!node_->IsString()) {
+        throw std::runtime_error("Cannot get string of non-string json value");
+      }
+      return std::string{node_->GetString(), node_->GetStringLength()};
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());

Review Comment:
   Why not error codes instead? I don't see the point of abusing exceptions as error codes that are harder to use.



##########
libminifi/include/core/yaml/YamlNode.h:
##########
@@ -0,0 +1,176 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <memory>
+#include <string>
+#include <utility>
+
+#include "yaml-cpp/yaml.h"
+#include "core/flow/Node.h"
+#include "utils/gsl.h"
+
+
+namespace org::apache::nifi::minifi::core {
+
+class YamlNode : public flow::Node::NodeImpl {

Review Comment:
   Same here: separate tree node from tagged union. 
   https://en.wikipedia.org/wiki/Single-responsibility_principle



##########
libminifi/include/core/json/JsonNode.h:
##########
@@ -0,0 +1,248 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string>
+#include <utility>
+#include <memory>
+
+#include "core/flow/Node.h"
+#include "rapidjson/document.h"
+#include "utils/gsl.h"
+#include "utils/ValueCaster.h"
+
+namespace org::apache::nifi::minifi::core {
+
+class JsonNode : public flow::Node::NodeImpl {
+ public:
+  explicit JsonNode(const rapidjson::Value* node): node_(node) {}
+
+  explicit operator bool() const override {
+    return node_ != nullptr;
+  }
+  bool isSequence() const override {
+    return node_ ? node_->IsArray() : false;
+  }
+  bool isMap() const override {
+    return node_ ? node_->IsObject() : false;
+  }
+  bool isNull() const override {
+    return node_ ? node_->IsNull() : false;
+  }
+
+  nonstd::expected<std::string, std::exception_ptr> getString() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get string of invalid json value");
+      }
+      if (!node_->IsString()) {
+        throw std::runtime_error("Cannot get string of non-string json value");
+      }
+      return std::string{node_->GetString(), node_->GetStringLength()};
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());
+    }
+  }
+
+  nonstd::expected<int, std::exception_ptr> getInt() const override {
+    return getNumber<int>("int");
+  }
+  nonstd::expected<unsigned int, std::exception_ptr> getUInt() const override {
+    return getNumber<unsigned int>("unsigned int");
+  }
+  nonstd::expected<int64_t, std::exception_ptr> getInt64() const override {
+    return getNumber<int64_t>("int64_t");
+  }
+  nonstd::expected<uint64_t, std::exception_ptr> getUInt64() const override {
+    return getNumber<uint64_t>("uint64_t");
+  }
+
+  nonstd::expected<bool, std::exception_ptr> getBool() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get bool of invalid json value");
+      }
+      if (!node_->IsBool()) {
+        throw std::runtime_error("Cannot get bool of non-bool json value");
+      }
+      return node_->GetBool();
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());
+    }
+  }
+
+  std::string getDebugString() const override {
+    if (!node_) return "<invalid>";
+    if (node_->IsObject()) return "<Map>";
+    if (node_->IsArray()) return "<Array>";
+    if (node_->IsNull()) return "null";
+    if (node_->IsInt()) return std::to_string(node_->GetInt());
+    if (node_->IsUint()) return std::to_string(node_->GetUint());
+    if (node_->IsInt64()) return std::to_string(node_->GetInt64());
+    if (node_->IsUint64()) return std::to_string(node_->GetUint64());

Review Comment:
   I think we should represent these using a single type.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r1003026638


##########
libminifi/src/core/flow/StructuredConfiguration.cpp:
##########
@@ -0,0 +1,896 @@
+/**
+ *
+ * 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 <memory>
+#include <vector>
+#include <set>
+#include <cinttypes>
+
+#include "core/flow/StructuredConfiguration.h"
+#include "core/flow/CheckRequiredField.h"
+#include "core/flow/StructuredConnectionParser.h"
+#include "core/state/Value.h"
+#include "Defaults.h"
+#include "utils/TimeUtil.h"
+#include "utils/RegexUtils.h"
+
+namespace org::apache::nifi::minifi::core::flow {
+
+std::shared_ptr<utils::IdGenerator> StructuredConfiguration::id_generator_ = utils::IdGenerator::getIdGenerator();
+
+StructuredConfiguration::StructuredConfiguration(ConfigurationContext ctx, std::shared_ptr<logging::Logger> logger)
+    : FlowConfiguration(std::move(ctx)),
+      logger_(std::move(logger)) {}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::parseRootProcessGroup(const Node& root_flow_node) {
+  auto flow_controller_node = root_flow_node[CONFIG_FLOW_CONTROLLER_KEY];
+  auto root_group = parseProcessGroup(flow_controller_node, root_flow_node, true);
+  this->name_ = root_group->getName();
+  return root_group;
+}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::createProcessGroup(const Node& node, bool is_root) {
+  int version = 0;
+
+  checkRequiredField(node, "name", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+  auto flowName = node["name"].getString().value();
+
+  utils::Identifier uuid;
+  // assignment throws on invalid uuid
+  uuid = getOrGenerateId(node);
+
+  if (node["version"]) {
+    version = node["version"].getInt().value();
+  }
+
+  logger_->log_debug("parseRootProcessGroup: id => [%s], name => [%s]", uuid.to_string(), flowName);
+  std::unique_ptr<core::ProcessGroup> group;
+  if (is_root) {
+    group = FlowConfiguration::createRootProcessGroup(flowName, uuid, version);
+  } else {
+    group = FlowConfiguration::createSimpleProcessGroup(flowName, uuid, version);
+  }
+
+  if (node["onschedule retry interval"]) {
+    auto onScheduleRetryPeriod = node["onschedule retry interval"].getString().value();
+    logger_->log_debug("parseRootProcessGroup: onschedule retry period => [%s]", onScheduleRetryPeriod);
+
+    auto on_schedule_retry_period_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(onScheduleRetryPeriod);
+    if (on_schedule_retry_period_value.has_value() && group) {
+      logger_->log_debug("parseRootProcessGroup: onschedule retry => [%" PRId64 "] ms", on_schedule_retry_period_value->count());
+      group->setOnScheduleRetryPeriod(on_schedule_retry_period_value->count());
+    }
+  }
+
+  return group;
+}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::parseProcessGroup(const Node& headerNode, const Node& yamlNode, bool is_root) {
+  auto group = createProcessGroup(headerNode, is_root);
+  Node processorsNode = yamlNode[CONFIG_PROCESSORS_KEY];
+  Node connectionsNode = yamlNode[StructuredConnectionParser::CONFIG_CONNECTIONS_KEY];
+  Node funnelsNode = yamlNode[CONFIG_FUNNELS_KEY];
+  Node remoteProcessingGroupsNode = [&] {
+    // assignment is not supported on invalid Yaml nodes
+    Node candidate = yamlNode[CONFIG_REMOTE_PROCESS_GROUP_KEY];
+    if (candidate) {
+      return candidate;
+    }
+    return yamlNode[CONFIG_REMOTE_PROCESS_GROUP_KEY_V3];
+  }();
+  Node childProcessGroupNodeSeq = yamlNode["Process Groups"];
+
+  parseProcessorNode(processorsNode, group.get());
+  parseRemoteProcessGroup(remoteProcessingGroupsNode, group.get());
+  parseFunnels(funnelsNode, group.get());
+  // parse connections last to give feedback if the source and/or destination
+  // is not in the same process group
+  parseConnection(connectionsNode, group.get());
+
+  if (childProcessGroupNodeSeq && childProcessGroupNodeSeq.isSequence()) {
+    for (const auto childProcessGroupNode : childProcessGroupNodeSeq) {
+      group->addProcessGroup(parseProcessGroup(childProcessGroupNode, childProcessGroupNode));
+    }
+  }
+  return group;
+}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::getRootFrom(const Node& rootYamlNode) {
+  uuids_.clear();
+  Node controllerServiceNode = rootYamlNode[CONFIG_CONTROLLER_SERVICES_KEY];
+  Node provenanceReportNode = rootYamlNode[CONFIG_PROVENANCE_REPORT_KEY];
+
+  parseControllerServices(controllerServiceNode);
+  // Create the root process group
+  std::unique_ptr<core::ProcessGroup> root = parseRootProcessGroup(rootYamlNode);
+  parseProvenanceReporting(provenanceReportNode, root.get());
+
+  // set the controller services into the root group.
+  for (const auto& controller_service : controller_services_->getAllControllerServices()) {
+    root->addControllerService(controller_service->getName(), controller_service);
+    root->addControllerService(controller_service->getUUIDStr(), controller_service);
+  }
+
+  return root;
+}
+
+void StructuredConfiguration::parseProcessorNode(const Node& processors_node, core::ProcessGroup* parentGroup) {
+  int64_t runDurationNanos = -1;
+  utils::Identifier uuid;
+  std::unique_ptr<core::Processor> processor;
+
+  if (!parentGroup) {
+    logger_->log_error("parseProcessNodeYaml: no parent group exists");
+    return;
+  }
+
+  if (!processors_node) {
+    throw std::invalid_argument("Cannot instantiate a MiNiFi instance without a defined Processors configuration node.");
+  }
+  if (!processors_node.isSequence()) {
+    throw std::invalid_argument(
+        "Cannot instantiate a MiNiFi instance without a defined Processors configuration node.");
+  }
+  // Evaluate sequence of processors
+  for (const auto procNode : processors_node) {
+    core::ProcessorConfig procCfg;
+
+    checkRequiredField(procNode, "name", CONFIG_PROCESSORS_KEY);
+    procCfg.name = procNode["name"].getString().value();
+    procCfg.id = getOrGenerateId(procNode);
+
+    uuid = procCfg.id;
+    logger_->log_debug("parseProcessorNode: name => [%s] id => [%s]", procCfg.name, procCfg.id);
+    checkRequiredField(procNode, "class", CONFIG_PROCESSORS_KEY);
+    procCfg.javaClass = procNode["class"].getString().value();
+    logger_->log_debug("parseProcessorNode: class => [%s]", procCfg.javaClass);
+
+    // Determine the processor name only from the Java class
+    auto lastOfIdx = procCfg.javaClass.find_last_of('.');
+    if (lastOfIdx != std::string::npos) {
+      lastOfIdx++;  // if a value is found, increment to move beyond the .
+      std::string processorName = procCfg.javaClass.substr(lastOfIdx);
+      processor = this->createProcessor(processorName, procCfg.javaClass, uuid);
+    } else {
+      // Allow unqualified class names for core processors
+      processor = this->createProcessor(procCfg.javaClass, uuid);
+    }
+
+    if (!processor) {
+      logger_->log_error("Could not create a processor %s with id %s", procCfg.name, procCfg.id);
+      throw std::invalid_argument("Could not create processor " + procCfg.name);
+    }
+
+    processor->setName(procCfg.name);
+
+    processor->setFlowIdentifier(flow_version_->getFlowIdentifier());
+
+    procCfg.schedulingStrategy = getOptionalField(procNode, "scheduling strategy", DEFAULT_SCHEDULING_STRATEGY, CONFIG_PROCESSORS_KEY);
+    logger_->log_debug("parseProcessorNode: scheduling strategy => [%s]", procCfg.schedulingStrategy);
+
+    procCfg.schedulingPeriod = getOptionalField(procNode, "scheduling period", DEFAULT_SCHEDULING_PERIOD_STR, CONFIG_PROCESSORS_KEY);
+
+    logger_->log_debug("parseProcessorNode: scheduling period => [%s]", procCfg.schedulingPeriod);
+
+    if (auto tasksNode = procNode["max concurrent tasks"]) {
+      if (auto int_val = tasksNode.getUInt64()) {
+        procCfg.maxConcurrentTasks = std::to_string(int_val.value());
+      } else {
+        procCfg.maxConcurrentTasks = tasksNode.getString().value();
+      }
+      logger_->log_debug("parseProcessorNode: max concurrent tasks => [%s]", procCfg.maxConcurrentTasks);
+    }
+
+    if (procNode["penalization period"]) {
+      procCfg.penalizationPeriod = procNode["penalization period"].getString().value();
+      logger_->log_debug("parseProcessorNode: penalization period => [%s]", procCfg.penalizationPeriod);
+    }
+
+    if (procNode["yield period"]) {
+      procCfg.yieldPeriod = procNode["yield period"].getString().value();
+      logger_->log_debug("parseProcessorNode: yield period => [%s]", procCfg.yieldPeriod);
+    }
+
+    if (auto runNode = procNode["run duration nanos"]) {
+      if (auto int_val = runNode.getUInt64()) {
+        procCfg.runDurationNanos = std::to_string(int_val.value());
+      } else {
+        procCfg.runDurationNanos = runNode.getString().value();
+      }
+      logger_->log_debug("parseProcessorNode: run duration nanos => [%s]", procCfg.runDurationNanos);
+    }
+
+    // handle auto-terminated relationships
+    if (procNode["auto-terminated relationships list"]) {
+      Node autoTerminatedSequence = procNode["auto-terminated relationships list"];
+      std::vector<std::string> rawAutoTerminatedRelationshipValues;
+      if (autoTerminatedSequence.isSequence() && autoTerminatedSequence.size() > 0) {
+        for (const auto autoTerminatedRel : autoTerminatedSequence) {
+          rawAutoTerminatedRelationshipValues.push_back(autoTerminatedRel.getString().value());
+        }
+      }
+      procCfg.autoTerminatedRelationships = rawAutoTerminatedRelationshipValues;
+    }
+
+    // handle processor properties
+    if (procNode["Properties"]) {
+      Node propertiesNode = procNode["Properties"];
+      parsePropertiesNode(propertiesNode, *processor, procCfg.name, CONFIG_PROCESSORS_KEY);
+    }
+
+    // Take care of scheduling
+
+    if (procCfg.schedulingStrategy == "TIMER_DRIVEN" || procCfg.schedulingStrategy == "EVENT_DRIVEN") {
+      if (auto scheduling_period = utils::timeutils::StringToDuration<std::chrono::nanoseconds>(procCfg.schedulingPeriod)) {
+        logger_->log_debug("convert: parseProcessorNode: schedulingPeriod => [%" PRId64 "] ns", scheduling_period->count());
+        processor->setSchedulingPeriodNano(*scheduling_period);
+      }
+    } else {
+      processor->setCronPeriod(procCfg.schedulingPeriod);
+    }
+
+    if (auto penalization_period = utils::timeutils::StringToDuration<std::chrono::milliseconds>(procCfg.penalizationPeriod)) {
+      logger_->log_debug("convert: parseProcessorNode: penalizationPeriod => [%" PRId64 "] ms", penalization_period->count());
+      processor->setPenalizationPeriod(penalization_period.value());
+    }
+
+    if (auto yield_period = utils::timeutils::StringToDuration<std::chrono::milliseconds>(procCfg.yieldPeriod)) {
+      logger_->log_debug("convert: parseProcessorNode: yieldPeriod => [%" PRId64 "] ms", yield_period->count());
+      processor->setYieldPeriodMsec(yield_period.value());
+    }
+
+    // Default to running
+    processor->setScheduledState(core::RUNNING);
+
+    if (procCfg.schedulingStrategy == "TIMER_DRIVEN") {
+      processor->setSchedulingStrategy(core::TIMER_DRIVEN);
+      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
+    } else if (procCfg.schedulingStrategy == "EVENT_DRIVEN") {
+      processor->setSchedulingStrategy(core::EVENT_DRIVEN);
+      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
+    } else {
+      processor->setSchedulingStrategy(core::CRON_DRIVEN);
+      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
+    }
+
+    int32_t maxConcurrentTasks;
+    if (core::Property::StringToInt(procCfg.maxConcurrentTasks, maxConcurrentTasks)) {
+      logger_->log_debug("parseProcessorNode: maxConcurrentTasks => [%d]", maxConcurrentTasks);
+      processor->setMaxConcurrentTasks((uint8_t) maxConcurrentTasks);
+    }
+
+    if (core::Property::StringToInt(procCfg.runDurationNanos, runDurationNanos)) {
+      logger_->log_debug("parseProcessorNode: runDurationNanos => [%d]", runDurationNanos);
+      processor->setRunDurationNano(std::chrono::nanoseconds(runDurationNanos));
+    }
+
+    std::vector<core::Relationship> autoTerminatedRelationships;
+    for (auto &&relString : procCfg.autoTerminatedRelationships) {
+      core::Relationship relationship(relString, "");
+      logger_->log_debug("parseProcessorNode: autoTerminatedRelationship  => [%s]", relString);
+      autoTerminatedRelationships.push_back(relationship);
+    }
+
+    processor->setAutoTerminatedRelationships(autoTerminatedRelationships);
+
+    parentGroup->addProcessor(std::move(processor));
+  }
+}
+
+void StructuredConfiguration::parseRemoteProcessGroup(const Node& rpg_node_seq, core::ProcessGroup* parentGroup) {
+  utils::Identifier uuid;
+  std::string id;
+
+  if (!parentGroup) {
+    logger_->log_error("parseRemoteProcessGroupYaml: no parent group exists");
+    return;
+  }
+
+  if (!rpg_node_seq || !rpg_node_seq.isSequence()) {
+    return;
+  }
+  for (const auto currRpgNode : rpg_node_seq) {
+    checkRequiredField(currRpgNode, "name", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+    auto name = currRpgNode["name"].getString().value();
+    id = getOrGenerateId(currRpgNode);
+
+    logger_->log_debug("parseRemoteProcessGroupYaml: name => [%s], id => [%s]", name, id);
+
+    auto url = getOptionalField(currRpgNode, "url", "", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+
+    logger_->log_debug("parseRemoteProcessGroupYaml: url => [%s]", url);
+
+    uuid = id;
+    auto group = createRemoteProcessGroup(name, uuid);
+    group->setParent(parentGroup);
+
+    if (currRpgNode["yield period"]) {
+      auto yieldPeriod = currRpgNode["yield period"].getString().value();
+      logger_->log_debug("parseRemoteProcessGroupYaml: yield period => [%s]", yieldPeriod);
+
+      auto yield_period_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(yieldPeriod);
+      if (yield_period_value.has_value() && group) {
+        logger_->log_debug("parseRemoteProcessGroupYaml: yieldPeriod => [%" PRId64 "] ms", yield_period_value->count());
+        group->setYieldPeriodMsec(*yield_period_value);
+      }
+    }
+
+    if (currRpgNode["timeout"]) {
+      auto timeout = currRpgNode["timeout"].getString().value();
+      logger_->log_debug("parseRemoteProcessGroupYaml: timeout => [%s]", timeout);
+
+      auto timeout_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(timeout);
+      if (timeout_value.has_value() && group) {
+        logger_->log_debug("parseRemoteProcessGroupYaml: timeoutValue => [%" PRId64 "] ms", timeout_value->count());
+        group->setTimeout(timeout_value->count());
+      }
+    }
+
+    if (currRpgNode["local network interface"]) {
+      auto interface = currRpgNode["local network interface"].getString().value();
+      logger_->log_debug("parseRemoteProcessGroupYaml: local network interface => [%s]", interface);
+      group->setInterface(interface);
+    }
+
+    if (currRpgNode["transport protocol"]) {
+      auto transport_protocol = currRpgNode["transport protocol"].getString().value();
+      logger_->log_debug("parseRemoteProcessGroupYaml: transport protocol => [%s]", transport_protocol);
+      if (transport_protocol == "HTTP") {
+        group->setTransportProtocol(transport_protocol);
+        if (currRpgNode["proxy host"]) {
+          auto http_proxy_host = currRpgNode["proxy host"].getString().value();
+          logger_->log_debug("parseRemoteProcessGroupYaml: proxy host => [%s]", http_proxy_host);
+          group->setHttpProxyHost(http_proxy_host);
+          if (currRpgNode["proxy user"]) {
+            auto http_proxy_username = currRpgNode["proxy user"].getString().value();
+            logger_->log_debug("parseRemoteProcessGroupYaml: proxy user => [%s]", http_proxy_username);
+            group->setHttpProxyUserName(http_proxy_username);
+          }
+          if (currRpgNode["proxy password"]) {
+            auto http_proxy_password = currRpgNode["proxy password"].getString().value();
+            logger_->log_debug("parseRemoteProcessGroupYaml: proxy password => [%s]", http_proxy_password);
+            group->setHttpProxyPassWord(http_proxy_password);
+          }
+          if (currRpgNode["proxy port"]) {
+            auto http_proxy_port = currRpgNode["proxy port"].getString().value();
+            int32_t port;
+            if (core::Property::StringToInt(http_proxy_port, port)) {
+              logger_->log_debug("parseRemoteProcessGroupYaml: proxy port => [%d]", port);
+              group->setHttpProxyPort(port);
+            }
+          }
+        }
+      } else if (transport_protocol == "RAW") {
+        group->setTransportProtocol(transport_protocol);
+      } else {
+        std::stringstream stream;
+        stream << "Invalid transport protocol " << transport_protocol;
+        throw minifi::Exception(ExceptionType::SITE2SITE_EXCEPTION, stream.str().c_str());
+      }
+    }
+
+    group->setTransmitting(true);
+    group->setURL(url);
+
+    checkRequiredField(currRpgNode, "Input Ports", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+    auto inputPorts = currRpgNode["Input Ports"];
+    if (inputPorts && inputPorts.isSequence()) {
+      for (const auto currPort : inputPorts) {
+        parsePort(currPort, group.get(), sitetosite::SEND);
+      }  // for node
+    }
+    auto outputPorts = currRpgNode["Output Ports"];
+    if (outputPorts && outputPorts.isSequence()) {
+      for (const auto currPort : outputPorts) {
+        logger_->log_debug("Got a current port, iterating...");
+
+        parsePort(currPort, group.get(), sitetosite::RECEIVE);
+      }  // for node
+    }
+    parentGroup->addProcessGroup(std::move(group));
+  }
+}
+
+void StructuredConfiguration::parseProvenanceReporting(const Node& node, core::ProcessGroup* parent_group) {
+  utils::Identifier port_uuid;
+
+  if (!parent_group) {
+    logger_->log_error("parseProvenanceReportingYaml: no parent group exists");
+    return;
+  }
+
+  if (!node || node.isNull()) {
+    logger_->log_debug("no provenance reporting task specified");
+    return;
+  }
+
+  auto reportTask = createProvenanceReportTask();
+
+  checkRequiredField(node, "scheduling strategy", CONFIG_PROVENANCE_REPORT_KEY);
+  auto schedulingStrategyStr = node["scheduling strategy"].getString().value();
+  checkRequiredField(node, "scheduling period", CONFIG_PROVENANCE_REPORT_KEY);
+  auto schedulingPeriodStr = node["scheduling period"].getString().value();
+
+  if (auto scheduling_period = utils::timeutils::StringToDuration<std::chrono::nanoseconds>(schedulingPeriodStr)) {
+    logger_->log_debug("ProvenanceReportingTask schedulingPeriod %" PRId64 " ns", scheduling_period->count());
+    reportTask->setSchedulingPeriodNano(*scheduling_period);
+  }
+
+  if (schedulingStrategyStr == "TIMER_DRIVEN") {
+    reportTask->setSchedulingStrategy(core::TIMER_DRIVEN);
+    logger_->log_debug("ProvenanceReportingTask scheduling strategy %s", schedulingStrategyStr);
+  } else {
+    throw std::invalid_argument("Invalid scheduling strategy " + schedulingStrategyStr);
+  }
+
+  int64_t lvalue;
+  if (node["host"] && node["port"]) {
+    auto hostStr = node["host"].getString().value();
+
+    std::string portStr;
+    if (auto int_val = node["port"].getInt()) {
+      portStr = std::to_string(int_val.value());
+    } else {
+      portStr = node["port"].getString().value();
+    }
+    if (core::Property::StringToInt(portStr, lvalue) && !hostStr.empty()) {
+      logger_->log_debug("ProvenanceReportingTask port %" PRId64, lvalue);
+      std::string url = hostStr + ":" + portStr;
+      reportTask->setURL(url);
+    }
+  }
+
+  if (node["url"]) {
+    auto urlStr = node["url"].getString().value();
+    if (!urlStr.empty()) {
+      reportTask->setURL(urlStr);
+      logger_->log_debug("ProvenanceReportingTask URL %s", urlStr);
+    }
+  }
+  checkRequiredField(node, "port uuid", CONFIG_PROVENANCE_REPORT_KEY);
+  auto portUUIDStr = node["port uuid"].getString().value();
+  checkRequiredField(node, "batch size", CONFIG_PROVENANCE_REPORT_KEY);
+  auto batchSizeStr = node["batch size"].getString().value();
+
+  logger_->log_debug("ProvenanceReportingTask port uuid %s", portUUIDStr);
+  port_uuid = portUUIDStr;
+  reportTask->setPortUUID(port_uuid);
+
+  if (core::Property::StringToInt(batchSizeStr, lvalue)) {
+    reportTask->setBatchSize(gsl::narrow<int>(lvalue));
+  }
+
+  reportTask->initialize();
+
+  // add processor to parent
+  reportTask->setScheduledState(core::RUNNING);
+  parent_group->addProcessor(std::move(reportTask));
+}
+
+void StructuredConfiguration::parseControllerServices(const Node& controllerServicesNode) {
+  if (!controllerServicesNode || !controllerServicesNode.isSequence()) {
+    return;
+  }
+  for (const auto& controllerServiceNode : controllerServicesNode) {
+    checkRequiredField(controllerServiceNode, "name", CONFIG_CONTROLLER_SERVICES_KEY);
+
+    auto type = getRequiredField(controllerServiceNode, std::vector<std::string>{"class", "type"}, CONFIG_CONTROLLER_SERVICES_KEY);
+    logger_->log_debug("Using type %s for controller service node", type);
+
+    std::string fullType = type;
+    auto lastOfIdx = type.find_last_of('.');
+    if (lastOfIdx != std::string::npos) {
+      lastOfIdx++;  // if a value is found, increment to move beyond the .
+      type = type.substr(lastOfIdx);
+    }
+
+    auto name = controllerServiceNode["name"].getString().value();
+    auto id = getRequiredIdField(controllerServiceNode, CONFIG_CONTROLLER_SERVICES_KEY);
+
+    utils::Identifier uuid;
+    uuid = id;
+    std::shared_ptr<core::controller::ControllerServiceNode> controller_service_node = createControllerService(type, fullType, name, uuid);
+    if (nullptr != controller_service_node) {
+      logger_->log_debug("Created Controller Service with UUID %s and name %s", id, name);
+      controller_service_node->initialize();
+      if (Node propertiesNode = controllerServiceNode["Properties"]) {
+        // we should propagate properties to the node and to the implementation
+        parsePropertiesNode(propertiesNode, *controller_service_node, name, CONFIG_CONTROLLER_SERVICES_KEY);
+        if (auto controllerServiceImpl = controller_service_node->getControllerServiceImplementation(); controllerServiceImpl) {
+          parsePropertiesNode(propertiesNode, *controllerServiceImpl, name, CONFIG_CONTROLLER_SERVICES_KEY);
+        }
+      }
+    } else {
+      logger_->log_debug("Could not locate %s", type);
+    }
+    controller_services_->put(id, controller_service_node);
+    controller_services_->put(name, controller_service_node);
+  }
+}
+
+void StructuredConfiguration::parseConnection(const Node& connection_node_seq, core::ProcessGroup* parent) {
+  if (!parent) {
+    logger_->log_error("parseProcessNode: no parent group was provided");
+    return;
+  }
+  if (!connection_node_seq || !connection_node_seq.isSequence()) {
+    return;
+  }
+
+  for (const auto& connection_node : connection_node_seq) {
+    if (!connection_node || !connection_node.isMap()) {
+      logger_->log_error("Invalid connection node, ignoring");
+      continue;
+    }

Review Comment:
   added comment



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] szaszm commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r991595541


##########
libminifi/include/core/json/JsonNode.h:
##########
@@ -0,0 +1,248 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string>
+#include <utility>
+#include <memory>
+
+#include "core/flow/Node.h"
+#include "rapidjson/document.h"
+#include "utils/gsl.h"
+#include "utils/ValueCaster.h"
+
+namespace org::apache::nifi::minifi::core {
+
+class JsonNode : public flow::Node::NodeImpl {

Review Comment:
   All of the possible value types of a json node value. `std::variant<std::monostate /* for undefined or null */, bool, int64_t, std::string, std::pair<Iterator>>`, and visit with `std::visit`. I don't understand the code 100% yet, so I'm probably missing some details, but this is the gist of the idea.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] lordgamez commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r1043310888


##########
libminifi/include/core/flow/Node.h:
##########
@@ -0,0 +1,139 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string_view>
+#include <tuple>
+#include <optional>
+#include <string>
+#include <memory>
+#include <utility>
+#include "nonstd/expected.hpp"
+
+namespace org::apache::nifi::minifi::core::flow {
+
+class Node {
+ public:
+  struct Cursor {
+    int line;
+    int column;
+    int pos;

Review Comment:
   I think these should be initialized, just in case.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r997042389


##########
libminifi/include/core/json/JsonNode.h:
##########
@@ -0,0 +1,248 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string>
+#include <utility>
+#include <memory>
+
+#include "core/flow/Node.h"
+#include "rapidjson/document.h"
+#include "utils/gsl.h"
+#include "utils/ValueCaster.h"
+
+namespace org::apache::nifi::minifi::core {
+
+class JsonNode : public flow::Node::NodeImpl {

Review Comment:
   I wouldn't say that we have implemented a tagged union, both `rapjson::Value` and `YAML::Node` are already tagged unions, the way I think of it, is that `flow::Node` encapsulates the visitation of these tagged unions, having to return a single variant (with something like a `flow::Node::getValue` method as I understand, but correct me if I misunderstood) could be problematic as for `YAML::Node` each scalar is a string and conversions happen on the fly, and the logic already accesses the original string (even if it is an int64) so we would need a method like `flow::Node::getString`
   I'm also not sure how we would handle the conversion errors, we currently get a nice error message straight from the yaml parser with location info and such, either we would have to capture this (as we do now) or track it ourselves (seems to be a significant change currently)



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r979885096


##########
libminifi/include/core/flow/StructuredConfiguration.h:
##########
@@ -0,0 +1,234 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <memory>
+#include <optional>
+#include <string>
+#include <unordered_set>
+
+#include "core/FlowConfiguration.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "core/ProcessorConfig.h"
+#include "Exception.h"
+#include "io/StreamFactory.h"
+#include "io/validation.h"
+#include "sitetosite/SiteToSite.h"
+#include "utils/Id.h"
+#include "utils/StringUtils.h"
+#include "utils/file/FileSystem.h"
+#include "core/flow/Node.h"
+
+namespace org::apache::nifi::minifi::core {

Review Comment:
   makes sense, updated



##########
libminifi/include/core/json/JsonConfiguration.h:
##########
@@ -0,0 +1,71 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <memory>
+#include <optional>
+#include <string>
+#include <unordered_set>
+
+#include "core/FlowConfiguration.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "core/ProcessorConfig.h"
+#include "Exception.h"
+#include "io/StreamFactory.h"
+#include "io/validation.h"
+#include "sitetosite/SiteToSite.h"
+#include "utils/Id.h"
+#include "utils/StringUtils.h"
+#include "utils/file/FileSystem.h"
+#include "core/flow/StructuredConfiguration.h"
+
+class JsonConfigurationTestAccessor;
+
+namespace org::apache::nifi::minifi::core {
+
+class JsonConfiguration : public StructuredConfiguration {
+ public:
+  explicit JsonConfiguration(ConfigurationContext ctx);
+
+  ~JsonConfiguration() override = default;
+
+  /**
+   * Returns a shared pointer to a ProcessGroup object containing the

Review Comment:
   removed the comments



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r989737510


##########
libminifi/include/core/json/JsonNode.h:
##########
@@ -0,0 +1,248 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string>
+#include <utility>
+#include <memory>
+
+#include "core/flow/Node.h"
+#include "rapidjson/document.h"
+#include "utils/gsl.h"
+#include "utils/ValueCaster.h"
+
+namespace org::apache::nifi::minifi::core {
+
+class JsonNode : public flow::Node::NodeImpl {

Review Comment:
   how would this `std::variant` look like? (the template arguments), who would handle its visitation?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r989814853


##########
libminifi/src/core/yaml/YamlConfiguration.cpp:
##########
@@ -22,900 +22,66 @@
 #include <cinttypes>
 
 #include "core/yaml/YamlConfiguration.h"
-#include "core/yaml/CheckRequiredField.h"
-#include "core/yaml/YamlConnectionParser.h"
 #include "core/state/Value.h"
 #include "Defaults.h"
 #include "utils/TimeUtil.h"
-
-#ifdef YAML_CONFIGURATION_USE_REGEX
+#include "yaml-cpp/yaml.h"
+#include "core/yaml/YamlNode.h"
 #include "utils/RegexUtils.h"
-#endif  // YAML_CONFIGURATION_USE_REGEX
 
 namespace org::apache::nifi::minifi::core {
 
-std::shared_ptr<utils::IdGenerator> YamlConfiguration::id_generator_ = utils::IdGenerator::getIdGenerator();
-
-YamlConfiguration::YamlConfiguration(const std::shared_ptr<core::Repository>& repo, const std::shared_ptr<core::Repository>& flow_file_repo,
-                                     const std::shared_ptr<core::ContentRepository>& content_repo, const std::shared_ptr<io::StreamFactory>& stream_factory,
-                                     const std::shared_ptr<Configure>& configuration, const std::optional<std::string>& path,
-                                     const std::shared_ptr<utils::file::FileSystem>& filesystem)
-    : FlowConfiguration(repo, flow_file_repo, content_repo, stream_factory, configuration,
-                        path.value_or(DEFAULT_NIFI_CONFIG_YML), filesystem),
-      stream_factory_(stream_factory),
-      logger_(logging::LoggerFactory<YamlConfiguration>::getLogger()) {}
-
-std::unique_ptr<core::ProcessGroup> YamlConfiguration::parseRootProcessGroupYaml(const YAML::Node& rootFlowNode) {
-  auto flowControllerNode = rootFlowNode[CONFIG_YAML_FLOW_CONTROLLER_KEY];
-  auto rootGroup = parseProcessGroupYaml(flowControllerNode, rootFlowNode, true);
-  this->name_ = rootGroup->getName();
-  return rootGroup;
-}
-
-std::unique_ptr<core::ProcessGroup> YamlConfiguration::createProcessGroup(const YAML::Node& yamlNode, bool is_root) {
-  int version = 0;
-
-  yaml::checkRequiredField(yamlNode, "name", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
-  auto flowName = yamlNode["name"].as<std::string>();
-
-  utils::Identifier uuid;
-  // assignment throws on invalid uuid
-  uuid = getOrGenerateId(yamlNode);
-
-  if (yamlNode["version"]) {
-    version = yamlNode["version"].as<int>();
-  }
-
-  logger_->log_debug("parseRootProcessGroup: id => [%s], name => [%s]", uuid.to_string(), flowName);
-  std::unique_ptr<core::ProcessGroup> group;
-  if (is_root) {
-    group = FlowConfiguration::createRootProcessGroup(flowName, uuid, version);
-  } else {
-    group = FlowConfiguration::createSimpleProcessGroup(flowName, uuid, version);
-  }
-
-  if (yamlNode["onschedule retry interval"]) {
-    auto onScheduleRetryPeriod = yamlNode["onschedule retry interval"].as<std::string>();
-    logger_->log_debug("parseRootProcessGroup: onschedule retry period => [%s]", onScheduleRetryPeriod);
-
-    auto on_schedule_retry_period_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(onScheduleRetryPeriod);
-    if (on_schedule_retry_period_value.has_value() && group) {
-      logger_->log_debug("parseRootProcessGroup: onschedule retry => [%" PRId64 "] ms", on_schedule_retry_period_value->count());
-      group->setOnScheduleRetryPeriod(on_schedule_retry_period_value->count());
-    }
-  }
-
-  return group;
-}
-
-std::unique_ptr<core::ProcessGroup> YamlConfiguration::parseProcessGroupYaml(const YAML::Node& headerNode, const YAML::Node& yamlNode, bool is_root) {
-  auto group = createProcessGroup(headerNode, is_root);
-  YAML::Node processorsNode = yamlNode[CONFIG_YAML_PROCESSORS_KEY];
-  YAML::Node connectionsNode = yamlNode[yaml::YamlConnectionParser::CONFIG_YAML_CONNECTIONS_KEY];
-  YAML::Node funnelsNode = yamlNode[CONFIG_YAML_FUNNELS_KEY];
-  YAML::Node remoteProcessingGroupsNode = [&] {
-    // assignment is not supported on invalid Yaml nodes
-    YAML::Node candidate = yamlNode[CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY];
-    if (candidate) {
-      return candidate;
-    }
-    return yamlNode[CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY_V3];
-  }();
-  YAML::Node childProcessGroupNodeSeq = yamlNode["Process Groups"];
-
-  parseProcessorNodeYaml(processorsNode, group.get());
-  parseRemoteProcessGroupYaml(remoteProcessingGroupsNode, group.get());
-  parseFunnelsYaml(funnelsNode, group.get());
-  // parse connections last to give feedback if the source and/or destination
-  // is not in the same process group
-  parseConnectionYaml(connectionsNode, group.get());
-
-  if (childProcessGroupNodeSeq && childProcessGroupNodeSeq.IsSequence()) {
-    for (YAML::const_iterator it = childProcessGroupNodeSeq.begin(); it != childProcessGroupNodeSeq.end(); ++it) {
-      auto childProcessGroupNode = it->as<YAML::Node>();
-      group->addProcessGroup(parseProcessGroupYaml(childProcessGroupNode, childProcessGroupNode));
-    }
-  }
-  return group;
-}
-
-std::unique_ptr<core::ProcessGroup> YamlConfiguration::getYamlRoot(const YAML::Node& rootYamlNode) {
-  uuids_.clear();
-  YAML::Node controllerServiceNode = rootYamlNode[CONFIG_YAML_CONTROLLER_SERVICES_KEY];
-  YAML::Node provenanceReportNode = rootYamlNode[CONFIG_YAML_PROVENANCE_REPORT_KEY];
-
-  parseControllerServices(controllerServiceNode);
-  // Create the root process group
-  std::unique_ptr<core::ProcessGroup> root = parseRootProcessGroupYaml(rootYamlNode);
-  parseProvenanceReportingYaml(provenanceReportNode, root.get());
-
-  // set the controller services into the root group.
-  for (const auto& controller_service : controller_services_->getAllControllerServices()) {
-    root->addControllerService(controller_service->getName(), controller_service);
-    root->addControllerService(controller_service->getUUIDStr(), controller_service);
-  }
-
-  return root;
-}
-
-void YamlConfiguration::parseProcessorNodeYaml(const YAML::Node& processorsNode, core::ProcessGroup* parentGroup) {
-  int64_t runDurationNanos = -1;
-  utils::Identifier uuid;
-  std::unique_ptr<core::Processor> processor;
-
-  if (!parentGroup) {
-    logger_->log_error("parseProcessNodeYaml: no parent group exists");
-    return;
-  }
-
-  if (!processorsNode) {
-    throw std::invalid_argument("Cannot instantiate a MiNiFi instance without a defined Processors configuration node.");
-  }
-  if (!processorsNode.IsSequence()) {
-    throw std::invalid_argument(
-        "Cannot instantiate a MiNiFi instance without a defined Processors configuration node.");
-  }
-  // Evaluate sequence of processors
-  for (YAML::const_iterator iter = processorsNode.begin(); iter != processorsNode.end(); ++iter) {
-    core::ProcessorConfig procCfg;
-    const auto procNode = iter->as<YAML::Node>();
-
-    yaml::checkRequiredField(procNode, "name", CONFIG_YAML_PROCESSORS_KEY);
-    procCfg.name = procNode["name"].as<std::string>();
-    procCfg.id = getOrGenerateId(procNode);
-
-    uuid = procCfg.id;
-    logger_->log_debug("parseProcessorNode: name => [%s] id => [%s]", procCfg.name, procCfg.id);
-    yaml::checkRequiredField(procNode, "class", CONFIG_YAML_PROCESSORS_KEY);
-    procCfg.javaClass = procNode["class"].as<std::string>();
-    logger_->log_debug("parseProcessorNode: class => [%s]", procCfg.javaClass);
-
-    // Determine the processor name only from the Java class
-    auto lastOfIdx = procCfg.javaClass.find_last_of('.');
-    if (lastOfIdx != std::string::npos) {
-      lastOfIdx++;  // if a value is found, increment to move beyond the .
-      std::string processorName = procCfg.javaClass.substr(lastOfIdx);
-      processor = this->createProcessor(processorName, procCfg.javaClass, uuid);
-    } else {
-      // Allow unqualified class names for core processors
-      processor = this->createProcessor(procCfg.javaClass, uuid);
-    }
-
-    if (!processor) {
-      logger_->log_error("Could not create a processor %s with id %s", procCfg.name, procCfg.id);
-      throw std::invalid_argument("Could not create processor " + procCfg.name);
-    }
-
-    processor->setName(procCfg.name);
-
-    processor->setFlowIdentifier(flow_version_->getFlowIdentifier());
-
-    auto strategyNode = getOptionalField(procNode, "scheduling strategy", YAML::Node(DEFAULT_SCHEDULING_STRATEGY),
-    CONFIG_YAML_PROCESSORS_KEY);
-    procCfg.schedulingStrategy = strategyNode.as<std::string>();
-    logger_->log_debug("parseProcessorNode: scheduling strategy => [%s]", procCfg.schedulingStrategy);
-
-    auto periodNode = getOptionalField(procNode, "scheduling period", YAML::Node(DEFAULT_SCHEDULING_PERIOD_STR),
-    CONFIG_YAML_PROCESSORS_KEY);
-
-    procCfg.schedulingPeriod = periodNode.as<std::string>();
-    logger_->log_debug("parseProcessorNode: scheduling period => [%s]", procCfg.schedulingPeriod);
-
-    if (procNode["max concurrent tasks"]) {
-      procCfg.maxConcurrentTasks = procNode["max concurrent tasks"].as<std::string>();
-      logger_->log_debug("parseProcessorNode: max concurrent tasks => [%s]", procCfg.maxConcurrentTasks);
-    }
-
-    if (procNode["penalization period"]) {
-      procCfg.penalizationPeriod = procNode["penalization period"].as<std::string>();
-      logger_->log_debug("parseProcessorNode: penalization period => [%s]", procCfg.penalizationPeriod);
-    }
-
-    if (procNode["yield period"]) {
-      procCfg.yieldPeriod = procNode["yield period"].as<std::string>();
-      logger_->log_debug("parseProcessorNode: yield period => [%s]", procCfg.yieldPeriod);
-    }
-
-    if (procNode["run duration nanos"]) {
-      procCfg.runDurationNanos = procNode["run duration nanos"].as<std::string>();
-      logger_->log_debug("parseProcessorNode: run duration nanos => [%s]", procCfg.runDurationNanos);
-    }
-
-    // handle auto-terminated relationships
-    if (procNode["auto-terminated relationships list"]) {
-      YAML::Node autoTerminatedSequence = procNode["auto-terminated relationships list"];
-      std::vector<std::string> rawAutoTerminatedRelationshipValues;
-      if (autoTerminatedSequence.IsSequence() && !autoTerminatedSequence.IsNull() && autoTerminatedSequence.size() > 0) {
-        for (YAML::const_iterator relIter = autoTerminatedSequence.begin(); relIter != autoTerminatedSequence.end(); ++relIter) {
-          auto autoTerminatedRel = relIter->as<std::string>();
-          rawAutoTerminatedRelationshipValues.push_back(autoTerminatedRel);
-        }
-      }
-      procCfg.autoTerminatedRelationships = rawAutoTerminatedRelationshipValues;
-    }
-
-    // handle processor properties
-    if (procNode["Properties"]) {
-      YAML::Node propertiesNode = procNode["Properties"];
-      parsePropertiesNodeYaml(propertiesNode, *processor, procCfg.name, CONFIG_YAML_PROCESSORS_KEY);
-    }
-
-    // Take care of scheduling
-
-    if (procCfg.schedulingStrategy == "TIMER_DRIVEN" || procCfg.schedulingStrategy == "EVENT_DRIVEN") {
-      if (auto scheduling_period = utils::timeutils::StringToDuration<std::chrono::nanoseconds>(procCfg.schedulingPeriod)) {
-        logger_->log_debug("convert: parseProcessorNode: schedulingPeriod => [%" PRId64 "] ns", scheduling_period->count());
-        processor->setSchedulingPeriodNano(*scheduling_period);
-      }
-    } else {
-      processor->setCronPeriod(procCfg.schedulingPeriod);
-    }
-
-    if (auto penalization_period = utils::timeutils::StringToDuration<std::chrono::milliseconds>(procCfg.penalizationPeriod)) {
-      logger_->log_debug("convert: parseProcessorNode: penalizationPeriod => [%" PRId64 "] ms", penalization_period->count());
-      processor->setPenalizationPeriod(penalization_period.value());
-    }
-
-    if (auto yield_period = utils::timeutils::StringToDuration<std::chrono::milliseconds>(procCfg.yieldPeriod)) {
-      logger_->log_debug("convert: parseProcessorNode: yieldPeriod => [%" PRId64 "] ms", yield_period->count());
-      processor->setYieldPeriodMsec(yield_period.value());
-    }
-
-    // Default to running
-    processor->setScheduledState(core::RUNNING);
-
-    if (procCfg.schedulingStrategy == "TIMER_DRIVEN") {
-      processor->setSchedulingStrategy(core::TIMER_DRIVEN);
-      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
-    } else if (procCfg.schedulingStrategy == "EVENT_DRIVEN") {
-      processor->setSchedulingStrategy(core::EVENT_DRIVEN);
-      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
-    } else {
-      processor->setSchedulingStrategy(core::CRON_DRIVEN);
-      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
-    }
-
-    int32_t maxConcurrentTasks;
-    if (core::Property::StringToInt(procCfg.maxConcurrentTasks, maxConcurrentTasks)) {
-      logger_->log_debug("parseProcessorNode: maxConcurrentTasks => [%d]", maxConcurrentTasks);
-      processor->setMaxConcurrentTasks((uint8_t) maxConcurrentTasks);
-    }
-
-    if (core::Property::StringToInt(procCfg.runDurationNanos, runDurationNanos)) {
-      logger_->log_debug("parseProcessorNode: runDurationNanos => [%d]", runDurationNanos);
-      processor->setRunDurationNano(std::chrono::nanoseconds(runDurationNanos));
-    }
-
-    std::vector<core::Relationship> autoTerminatedRelationships;
-    for (auto &&relString : procCfg.autoTerminatedRelationships) {
-      core::Relationship relationship(relString, "");
-      logger_->log_debug("parseProcessorNode: autoTerminatedRelationship  => [%s]", relString);
-      autoTerminatedRelationships.push_back(relationship);
-    }
-
-    processor->setAutoTerminatedRelationships(autoTerminatedRelationships);
-
-    parentGroup->addProcessor(std::move(processor));
-  }
-}
-
-void YamlConfiguration::parseRemoteProcessGroupYaml(const YAML::Node& rpgNode, core::ProcessGroup* parentGroup) {
-  utils::Identifier uuid;
-  std::string id;
-
-  if (!parentGroup) {
-    logger_->log_error("parseRemoteProcessGroupYaml: no parent group exists");
-    return;
-  }
-
-  if (!rpgNode || !rpgNode.IsSequence()) {
-    return;
-  }
-  for (YAML::const_iterator iter = rpgNode.begin(); iter != rpgNode.end(); ++iter) {
-    auto currRpgNode = iter->as<YAML::Node>();
-
-    yaml::checkRequiredField(currRpgNode, "name", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
-    auto name = currRpgNode["name"].as<std::string>();
-    id = getOrGenerateId(currRpgNode);
-
-    logger_->log_debug("parseRemoteProcessGroupYaml: name => [%s], id => [%s]", name, id);
-
-    auto urlNode = getOptionalField(currRpgNode, "url", YAML::Node(""),
-    CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
-
-    auto url = urlNode.as<std::string>();
-    logger_->log_debug("parseRemoteProcessGroupYaml: url => [%s]", url);
-
-    uuid = id;
-    auto group = this->createRemoteProcessGroup(name, uuid);
-    group->setParent(parentGroup);
-
-    if (currRpgNode["yield period"]) {
-      auto yieldPeriod = currRpgNode["yield period"].as<std::string>();
-      logger_->log_debug("parseRemoteProcessGroupYaml: yield period => [%s]", yieldPeriod);
-
-      auto yield_period_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(yieldPeriod);
-      if (yield_period_value.has_value() && group) {
-        logger_->log_debug("parseRemoteProcessGroupYaml: yieldPeriod => [%" PRId64 "] ms", yield_period_value->count());
-        group->setYieldPeriodMsec(*yield_period_value);
-      }
-    }
-
-    if (currRpgNode["timeout"]) {
-      auto timeout = currRpgNode["timeout"].as<std::string>();
-      logger_->log_debug("parseRemoteProcessGroupYaml: timeout => [%s]", timeout);
-
-      auto timeout_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(timeout);
-      if (timeout_value.has_value() && group) {
-        logger_->log_debug("parseRemoteProcessGroupYaml: timeoutValue => [%" PRId64 "] ms", timeout_value->count());
-        group->setTimeout(timeout_value->count());
-      }
-    }
-
-    if (currRpgNode["local network interface"]) {
-      auto interface = currRpgNode["local network interface"].as<std::string>();
-      logger_->log_debug("parseRemoteProcessGroupYaml: local network interface => [%s]", interface);
-      group->setInterface(interface);
-    }
-
-    if (currRpgNode["transport protocol"]) {
-      auto transport_protocol = currRpgNode["transport protocol"].as<std::string>();
-      logger_->log_debug("parseRemoteProcessGroupYaml: transport protocol => [%s]", transport_protocol);
-      if (transport_protocol == "HTTP") {
-        group->setTransportProtocol(transport_protocol);
-        if (currRpgNode["proxy host"]) {
-          auto http_proxy_host = currRpgNode["proxy host"].as<std::string>();
-          logger_->log_debug("parseRemoteProcessGroupYaml: proxy host => [%s]", http_proxy_host);
-          group->setHttpProxyHost(http_proxy_host);
-          if (currRpgNode["proxy user"]) {
-            auto http_proxy_username = currRpgNode["proxy user"].as<std::string>();
-            logger_->log_debug("parseRemoteProcessGroupYaml: proxy user => [%s]", http_proxy_username);
-            group->setHttpProxyUserName(http_proxy_username);
-          }
-          if (currRpgNode["proxy password"]) {
-            auto http_proxy_password = currRpgNode["proxy password"].as<std::string>();
-            logger_->log_debug("parseRemoteProcessGroupYaml: proxy password => [%s]", http_proxy_password);
-            group->setHttpProxyPassWord(http_proxy_password);
+YamlConfiguration::YamlConfiguration(ConfigurationContext ctx)
+    : StructuredConfiguration(([&] {
+          if (!ctx.path) {
+            ctx.path = DEFAULT_NIFI_CONFIG_YML;
           }
-          if (currRpgNode["proxy port"]) {
-            auto http_proxy_port = currRpgNode["proxy port"].as<std::string>();
-            int32_t port;
-            if (core::Property::StringToInt(http_proxy_port, port)) {
-              logger_->log_debug("parseRemoteProcessGroupYaml: proxy port => [%d]", port);
-              group->setHttpProxyPort(port);
-            }
-          }
-        }
-      } else if (transport_protocol == "RAW") {
-        group->setTransportProtocol(transport_protocol);
-      } else {
-        std::stringstream stream;
-        stream << "Invalid transport protocol " << transport_protocol;
-        throw minifi::Exception(ExceptionType::SITE2SITE_EXCEPTION, stream.str().c_str());
-      }
-    }
-
-    group->setTransmitting(true);
-    group->setURL(url);
-
-    yaml::checkRequiredField(currRpgNode, "Input Ports", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
-    auto inputPorts = currRpgNode["Input Ports"].as<YAML::Node>();
-    if (inputPorts && inputPorts.IsSequence()) {
-      for (YAML::const_iterator portIter = inputPorts.begin(); portIter != inputPorts.end(); ++portIter) {
-        auto currPort = portIter->as<YAML::Node>();
-
-        this->parsePortYaml(currPort, group.get(), sitetosite::SEND);
-      }  // for node
-    }
-    auto outputPorts = currRpgNode["Output Ports"].as<YAML::Node>();
-    if (outputPorts && outputPorts.IsSequence()) {
-      for (YAML::const_iterator portIter = outputPorts.begin(); portIter != outputPorts.end(); ++portIter) {
-        logger_->log_debug("Got a current port, iterating...");
-
-        auto currPort = portIter->as<YAML::Node>();
-
-        this->parsePortYaml(currPort, group.get(), sitetosite::RECEIVE);
-      }  // for node
-    }
-    parentGroup->addProcessGroup(std::move(group));
-  }
-}
-
-void YamlConfiguration::parseProvenanceReportingYaml(const YAML::Node& reportNode, core::ProcessGroup* parentGroup) {
-  utils::Identifier port_uuid;
+          return std::move(ctx);
+        })(),
+        logging::LoggerFactory<YamlConfiguration>::getLogger()) {}
 
-  if (!parentGroup) {
-    logger_->log_error("parseProvenanceReportingYaml: no parent group exists");
-    return;
+std::unique_ptr<core::ProcessGroup> YamlConfiguration::getRoot() {
+  if (!config_path_) {
+    logger_->log_error("Cannot instantiate flow, no config file is set.");
+    throw Exception(ExceptionType::FLOW_EXCEPTION, "No config file specified");
   }
-
-  if (!reportNode || !reportNode.IsDefined() || reportNode.IsNull()) {
-    logger_->log_debug("no provenance reporting task specified");
-    return;
-  }
-
-  auto reportTask = createProvenanceReportTask();
-
-  const auto node = reportNode.as<YAML::Node>();
-
-  yaml::checkRequiredField(node, "scheduling strategy", CONFIG_YAML_PROVENANCE_REPORT_KEY);
-  auto schedulingStrategyStr = node["scheduling strategy"].as<std::string>();
-  yaml::checkRequiredField(node, "scheduling period", CONFIG_YAML_PROVENANCE_REPORT_KEY);
-  auto schedulingPeriodStr = node["scheduling period"].as<std::string>();
-
-  if (auto scheduling_period = utils::timeutils::StringToDuration<std::chrono::nanoseconds>(schedulingPeriodStr)) {
-    logger_->log_debug("ProvenanceReportingTask schedulingPeriod %" PRId64 " ns", scheduling_period->count());
-    reportTask->setSchedulingPeriodNano(*scheduling_period);
+  const auto configuration = filesystem_->read(config_path_.value());
+  if (!configuration) {
+  // non-existence of flow config file is not a dealbreaker, the caller might fetch it from network
+  return nullptr;
   }
-
-  if (schedulingStrategyStr == "TIMER_DRIVEN") {
-    reportTask->setSchedulingStrategy(core::TIMER_DRIVEN);
-    logger_->log_debug("ProvenanceReportingTask scheduling strategy %s", schedulingStrategyStr);
-  } else {
-    throw std::invalid_argument("Invalid scheduling strategy " + schedulingStrategyStr);
-  }
-
-  int64_t lvalue;
-  if (node["host"] && node["port"]) {
-    auto hostStr = node["host"].as<std::string>();
-
-    auto portStr = node["port"].as<std::string>();
-    if (core::Property::StringToInt(portStr, lvalue) && !hostStr.empty()) {
-      logger_->log_debug("ProvenanceReportingTask port %" PRId64, lvalue);
-      std::string url = hostStr + ":" + portStr;
-      reportTask->setURL(url);
-    }
-  }
-
-  if (node["url"]) {
-    auto urlStr = node["url"].as<std::string>();
-    if (!urlStr.empty()) {
-      reportTask->setURL(urlStr);
-      logger_->log_debug("ProvenanceReportingTask URL %s", urlStr);
-    }
-  }
-  yaml::checkRequiredField(node, "port uuid", CONFIG_YAML_PROVENANCE_REPORT_KEY);
-  auto portUUIDStr = node["port uuid"].as<std::string>();
-  yaml::checkRequiredField(node, "batch size", CONFIG_YAML_PROVENANCE_REPORT_KEY);
-  auto batchSizeStr = node["batch size"].as<std::string>();
-
-  logger_->log_debug("ProvenanceReportingTask port uuid %s", portUUIDStr);
-  port_uuid = portUUIDStr;
-  reportTask->setPortUUID(port_uuid);
-
-  if (core::Property::StringToInt(batchSizeStr, lvalue)) {
-    reportTask->setBatchSize(gsl::narrow<int>(lvalue));
-  }
-
-  reportTask->initialize();
-
-  // add processor to parent
-  reportTask->setScheduledState(core::RUNNING);
-  parentGroup->addProcessor(std::move(reportTask));
-}
-
-void YamlConfiguration::parseControllerServices(const YAML::Node& controllerServicesNode) {
-  if (!controllerServicesNode || !controllerServicesNode.IsSequence()) {
-    return;
-  }
-  for (const auto& iter : controllerServicesNode) {
-    const auto controllerServiceNode = iter.as<YAML::Node>();
-    try {
-      yaml::checkRequiredField(controllerServiceNode, "name", CONFIG_YAML_CONTROLLER_SERVICES_KEY);
-
-      auto type = yaml::getRequiredField(controllerServiceNode, std::vector<std::string>{"class", "type"}, CONFIG_YAML_CONTROLLER_SERVICES_KEY);
-      logger_->log_debug("Using type %s for controller service node", type);
-
-      std::string fullType = type;
-      auto lastOfIdx = type.find_last_of('.');
-      if (lastOfIdx != std::string::npos) {
-        lastOfIdx++;  // if a value is found, increment to move beyond the .
-        type = type.substr(lastOfIdx);
-      }
-
-      auto name = controllerServiceNode["name"].as<std::string>();
-      auto id = getRequiredIdField(controllerServiceNode, CONFIG_YAML_CONTROLLER_SERVICES_KEY);
-
-      utils::Identifier uuid;
-      uuid = id;
-      std::shared_ptr<core::controller::ControllerServiceNode> controller_service_node = createControllerService(type, fullType, name, uuid);
-      if (nullptr != controller_service_node) {
-        logger_->log_debug("Created Controller Service with UUID %s and name %s", id, name);
-        controller_service_node->initialize();
-        YAML::Node propertiesNode = controllerServiceNode["Properties"];
-        // we should propagate properties to the node and to the implementation
-        parsePropertiesNodeYaml(propertiesNode, *controller_service_node, name, CONFIG_YAML_CONTROLLER_SERVICES_KEY);
-        if (auto controllerServiceImpl = controller_service_node->getControllerServiceImplementation(); controllerServiceImpl) {
-          parsePropertiesNodeYaml(propertiesNode, *controllerServiceImpl, name, CONFIG_YAML_CONTROLLER_SERVICES_KEY);
-        }
-      } else {
-        logger_->log_debug("Could not locate %s", type);
-      }
-      controller_services_->put(id, controller_service_node);
-      controller_services_->put(name, controller_service_node);
-    } catch (YAML::InvalidNode &) {
-      throw Exception(ExceptionType::GENERAL_EXCEPTION, "Name, id, and class must be specified for controller services");
-    }
-  }
-}
-
-void YamlConfiguration::parseConnectionYaml(const YAML::Node& connectionsNode, core::ProcessGroup* parent) {
-  if (!parent) {
-    logger_->log_error("parseProcessNode: no parent group was provided");
-    return;
-  }
-  if (!connectionsNode || !connectionsNode.IsSequence()) {
-    return;
-  }
-
-  for (YAML::const_iterator iter = connectionsNode.begin(); iter != connectionsNode.end(); ++iter) {
-    const auto connectionNode = iter->as<YAML::Node>();
-
-    // Configure basic connection
-    const std::string id = getOrGenerateId(connectionNode);
-
-    // Default name to be same as ID
-    // If name is specified in configuration, use the value
-    const auto name = connectionNode["name"].as<std::string>(id);
-
-    const auto uuid = utils::Identifier::parse(id) | utils::orElse([this] {
-      logger_->log_debug("Incorrect connection UUID format.");
-      throw Exception(ExceptionType::GENERAL_EXCEPTION, "Incorrect connection UUID format.");
-    });
-
-    auto connection = createConnection(name, uuid.value());
-    logger_->log_debug("Created connection with UUID %s and name %s", id, name);
-    const yaml::YamlConnectionParser connectionParser(connectionNode, name, gsl::not_null<core::ProcessGroup*>{ parent }, logger_);
-    connectionParser.configureConnectionSourceRelationshipsFromYaml(*connection);
-    connection->setMaxQueueSize(connectionParser.getWorkQueueSizeFromYaml());
-    connection->setMaxQueueDataSize(connectionParser.getWorkQueueDataSizeFromYaml());
-    connection->setSwapThreshold(connectionParser.getSwapThresholdFromYaml());
-    connection->setSourceUUID(connectionParser.getSourceUUIDFromYaml());
-    connection->setDestinationUUID(connectionParser.getDestinationUUIDFromYaml());
-    connection->setFlowExpirationDuration(connectionParser.getFlowFileExpirationFromYaml());
-    connection->setDropEmptyFlowFiles(connectionParser.getDropEmptyFromYaml());
-
-    parent->addConnection(std::move(connection));
-  }
-}
-
-void YamlConfiguration::parsePortYaml(const YAML::Node& portNode, core::ProcessGroup* parent, sitetosite::TransferDirection direction) {
-  utils::Identifier uuid;
-
-  if (!parent) {
-    logger_->log_error("parseProcessNode: no parent group existed");
-    return;
-  }
-
-  const auto inputPortsObj = portNode.as<YAML::Node>();
-
-  // Check for required fields
-  yaml::checkRequiredField(inputPortsObj, "name", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
-  auto nameStr = inputPortsObj["name"].as<std::string>();
-  auto portId = getRequiredIdField(inputPortsObj, CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY,
-    "The field 'id' is required for "
-    "the port named '" + nameStr + "' in the YAML Config. If this port "
-    "is an input port for a NiFi Remote Process Group, the port "
-    "id should match the corresponding id specified in the NiFi configuration. "
-    "This is a UUID of the format XXXXXXXX-XXXX-XXXX-XXXX-XXXXXXXXXXXX.");
-  uuid = portId;
-
-  auto port = std::make_unique<minifi::RemoteProcessorGroupPort>(
-          stream_factory_, nameStr, parent->getURL(), this->configuration_, uuid);
-  port->setDirection(direction);
-  port->setTimeout(parent->getTimeout());
-  port->setTransmitting(true);
-  port->setYieldPeriodMsec(parent->getYieldPeriodMsec());
-  port->initialize();
-  if (!parent->getInterface().empty())
-    port->setInterface(parent->getInterface());
-  if (parent->getTransportProtocol() == "HTTP") {
-    port->enableHTTP();
-    if (!parent->getHttpProxyHost().empty())
-      port->setHTTPProxy(parent->getHTTPProxy());
-  }
-  // else defaults to RAW
-
-  // handle port properties
-  const auto nodeVal = portNode.as<YAML::Node>();
-  YAML::Node propertiesNode = nodeVal["Properties"];
-  parsePropertiesNodeYaml(propertiesNode, *port, nameStr, CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
-
-  // add processor to parent
-  auto& processor = *port;
-  parent->addProcessor(std::move(port));
-  processor.setScheduledState(core::RUNNING);
-
-  if (inputPortsObj["max concurrent tasks"]) {
-    auto rawMaxConcurrentTasks = inputPortsObj["max concurrent tasks"].as<std::string>();
-    int32_t maxConcurrentTasks;
-    if (core::Property::StringToInt(rawMaxConcurrentTasks, maxConcurrentTasks)) {
-      processor.setMaxConcurrentTasks(maxConcurrentTasks);
-    }
-    logger_->log_debug("parseProcessorNode: maxConcurrentTasks => [%d]", maxConcurrentTasks);
-    processor.setMaxConcurrentTasks(maxConcurrentTasks);
-  }
-}
-
-void YamlConfiguration::parsePropertyValueSequence(const std::string& propertyName, const YAML::Node& propertyValueNode, core::ConfigurableComponent& component) {
-  for (const auto& iter : propertyValueNode) {
-    if (iter.IsDefined()) {
-      const auto nodeVal = iter.as<YAML::Node>();
-      YAML::Node propertiesNode = nodeVal["value"];
-      // must insert the sequence in differently.
-      const auto rawValueString = propertiesNode.as<std::string>();
-      logger_->log_debug("Found %s=%s", propertyName, rawValueString);
-      if (!component.updateProperty(propertyName, rawValueString)) {
-        auto proc = dynamic_cast<core::Connectable*>(&component);
-        if (proc) {
-          logger_->log_warn("Received property %s with value %s but is not one of the properties for %s. Attempting to add as dynamic property.", propertyName, rawValueString, proc->getName());
-          if (!component.setDynamicProperty(propertyName, rawValueString)) {
-            logger_->log_warn("Unable to set the dynamic property %s with value %s", propertyName, rawValueString);
-          } else {
-            logger_->log_warn("Dynamic property %s with value %s set", propertyName, rawValueString);
-          }
-        }
-      }
-    }
-  }
-}
-
-PropertyValue YamlConfiguration::getValidatedProcessorPropertyForDefaultTypeInfo(const core::Property& propertyFromProcessor, const YAML::Node& propertyValueNode) {
-  PropertyValue defaultValue;
-  defaultValue = propertyFromProcessor.getDefaultValue();
-  const std::type_index defaultType = defaultValue.getTypeInfo();
   try {
-    PropertyValue coercedValue = defaultValue;
-    if (defaultType == typeid(int64_t)) {
-      coercedValue = propertyValueNode.as<int64_t>();
-    } else if (defaultType == typeid(uint64_t)) {
-      uint64_t integer_value;
-      if (YAML::convert<uint64_t>::decode(propertyValueNode, integer_value)) {
-        coercedValue = integer_value;
-      } else {
-        coercedValue = propertyValueNode.as<std::string>();
-      }
-    } else if (defaultType == typeid(int)) {
-      coercedValue = propertyValueNode.as<int>();
-    } else if (defaultType == typeid(bool)) {
-      coercedValue = propertyValueNode.as<bool>();
-    } else {
-      coercedValue = propertyValueNode.as<std::string>();
-    }
-    return coercedValue;
-  } catch (const std::exception& e) {
-    logger_->log_error("Fetching property failed with an exception of %s", e.what());
-    logger_->log_error("Invalid conversion for field %s. Value %s", propertyFromProcessor.getName(), propertyValueNode.as<std::string>());
-  } catch (...) {
-    logger_->log_error("Invalid conversion for field %s. Value %s", propertyFromProcessor.getName(), propertyValueNode.as<std::string>());
-  }
-  return defaultValue;
-}
-
-void YamlConfiguration::parseSingleProperty(const std::string& propertyName, const YAML::Node& propertyValueNode, core::ConfigurableComponent& processor) {
-  core::Property myProp(propertyName, "", "");
-  processor.getProperty(propertyName, myProp);
-  const PropertyValue coercedValue = getValidatedProcessorPropertyForDefaultTypeInfo(myProp, propertyValueNode);
-  bool property_set = false;
-  try {
-    property_set = processor.setProperty(myProp, coercedValue);
-  } catch(const utils::internal::InvalidValueException&) {
-    auto component = dynamic_cast<core::CoreComponent*>(&processor);
-    if (component == nullptr) {
-      logger_->log_error("processor was not a CoreComponent for property '%s'", propertyName);
-    } else {
-      logger_->log_error("Invalid value was set for property '%s' creating component '%s'", propertyName, component->getName());
-    }
+    YAML::Node rootYamlNode = YAML::Load(configuration.value());
+    flow::Node root{std::make_shared<YamlNode>(rootYamlNode)};
+    return getRootFrom(root);
+  } catch(...) {
+    logger_->log_error("Invalid yaml configuration file");
     throw;
   }
-  const auto rawValueString = propertyValueNode.as<std::string>();
-  if (!property_set) {
-    auto proc = dynamic_cast<core::Connectable*>(&processor);
-    if (proc) {
-      logger_->log_warn("Received property %s with value %s but is not one of the properties for %s. Attempting to add as dynamic property.", propertyName, rawValueString, proc->getName());
-      if (!processor.setDynamicProperty(propertyName, rawValueString)) {
-        logger_->log_warn("Unable to set the dynamic property %s with value %s", propertyName, rawValueString);
-      } else {
-        logger_->log_warn("Dynamic property %s with value %s set", propertyName, rawValueString);
-      }
-    }
-  } else {
-    logger_->log_debug("Property %s with value %s set", propertyName, rawValueString);
-  }
-}
-
-void YamlConfiguration::parsePropertyNodeElement(const std::string& propertyName, const YAML::Node& propertyValueNode, core::ConfigurableComponent& processor) {
-  logger_->log_trace("Encountered %s", propertyName);
-  if (propertyValueNode.IsNull() || !propertyValueNode.IsDefined()) {
-    return;
-  }
-  if (propertyValueNode.IsSequence()) {
-    parsePropertyValueSequence(propertyName, propertyValueNode, processor);
-  } else {
-    parseSingleProperty(propertyName, propertyValueNode, processor);
-  }
-}
-
-void YamlConfiguration::parsePropertiesNodeYaml(const YAML::Node& propertiesNode, core::ConfigurableComponent& component, const std::string& component_name,
-    const std::string& yaml_section) {
-  // Treat generically as a YAML node so we can perform inspection on entries to ensure they are populated
-  logger_->log_trace("Entered %s", component_name);
-  for (const auto& propertyElem : propertiesNode) {
-    const auto propertyName = propertyElem.first.as<std::string>();
-    const YAML::Node propertyValueNode = propertyElem.second;
-    parsePropertyNodeElement(propertyName, propertyValueNode, component);
-  }
-
-  validateComponentProperties(component, component_name, yaml_section);
 }
 
-void YamlConfiguration::parseFunnelsYaml(const YAML::Node& node, core::ProcessGroup* parent) {
-  if (!parent) {
-    logger_->log_error("parseFunnelsYaml: no parent group was provided");
-    return;
-  }
-  if (!node || !node.IsSequence()) {
-    return;
-  }
-
-  for (const auto& element : node) {
-    const auto funnel_node = element.as<YAML::Node>();
-
-    std::string id = getOrGenerateId(funnel_node);
-
-    // Default name to be same as ID
-    const auto name = funnel_node["name"].as<std::string>(id);
-
-    const auto uuid = utils::Identifier::parse(id) | utils::orElse([this] {
-      logger_->log_debug("Incorrect funnel UUID format.");
-      throw Exception(ExceptionType::GENERAL_EXCEPTION, "Incorrect funnel UUID format.");
-    });
-
-    auto funnel = std::make_unique<core::Funnel>(name, uuid.value());
-    logger_->log_debug("Created funnel with UUID %s and name %s", id, name);
-    funnel->setScheduledState(core::RUNNING);
-    funnel->setSchedulingStrategy(core::EVENT_DRIVEN);
-    parent->addProcessor(std::move(funnel));
-  }
-}
-
-void YamlConfiguration::validateComponentProperties(ConfigurableComponent& component, const std::string &component_name, const std::string &yaml_section) const {
-  const auto &component_properties = component.getProperties();
-
-  // Validate required properties
-  for (const auto &prop_pair : component_properties) {
-    if (prop_pair.second.getRequired()) {
-      if (prop_pair.second.getValue().to_string().empty()) {
-        std::string reason = utils::StringUtils::join_pack("required property '", prop_pair.second.getName(), "' is not set");
-        raiseComponentError(component_name, yaml_section, reason);
-      } else if (!prop_pair.second.getValue().validate(prop_pair.first).valid()) {
-        std::string reason = utils::StringUtils::join_pack("the value '", prop_pair.first, "' is not valid for property '", prop_pair.second.getName(), "'");
-        raiseComponentError(component_name, yaml_section, reason);
-      }
-    }
-  }
-
-  // Validate dependent properties
-  for (const auto &prop_pair : component_properties) {
-    const auto &dep_props = prop_pair.second.getDependentProperties();
-
-    if (prop_pair.second.getValue().to_string().empty()) {
-      continue;
-    }
-
-    for (const auto &dep_prop_key : dep_props) {
-      if (component_properties.at(dep_prop_key).getValue().to_string().empty()) {
-        std::string reason = utils::StringUtils::join_pack("property '", prop_pair.second.getName(),
-            "' depends on property '", dep_prop_key, "' which is not set");
-        raiseComponentError(component_name, yaml_section, reason);
-      }
-    }
-  }
-
-#ifdef YAML_CONFIGURATION_USE_REGEX
-  // Validate mutually-exclusive properties
-  for (const auto &prop_pair : component_properties) {
-    const auto &excl_props = prop_pair.second.getExclusiveOfProperties();
-
-    if (prop_pair.second.getValue().empty()) {
-      continue;
-    }
-
-    for (const auto &excl_pair : excl_props) {
-      utils::Regex excl_expr(excl_pair.second);
-      if (utils::regexMatch(component_properties.at(excl_pair.first).getValue().to_string(), excl_expr)) {
-        std::string reason = utils::StringUtils::join_pack("property '", prop_pair.second.getName(),
-            "' must not be set when the value of property '", excl_pair.first, "' matches '", excl_pair.second, "'");
-        raiseComponentError(component_name, yaml_section, reason);
-      }
-    }
-  }
-
-  // Validate regex properties
-  for (const auto &prop_pair : component_properties) {
-    const auto &prop_regex_str = prop_pair.second.getValidRegex();
-
-    if (!prop_regex_str.empty()) {
-      utils::Regex prop_regex(prop_regex_str);
-      if (!utils::regexMatch(prop_pair.second.getValue().to_string(), prop_regex)) {
-        std::string reason = utils::StringUtils::join_pack("property '", prop_pair.second.getName(), "' does not match validation pattern '", prop_regex_str, "'");
-        raiseComponentError(component_name, yaml_section, reason);
-      }
-    }
-  }
-#else
-  logging::LOG_INFO(logger_) << "Validation of mutally-exclusive properties is disabled in this build.";
-  logging::LOG_INFO(logger_) << "Regex validation of properties is not available in this build.";
-#endif  // YAML_CONFIGURATION_USE_REGEX
-}
-
-void YamlConfiguration::raiseComponentError(const std::string &component_name, const std::string &yaml_section, const std::string &reason) const {
-  std::string err_msg = "Unable to parse configuration file for component named '";
-  err_msg.append(component_name);
-  err_msg.append("' because " + reason);
-  if (!yaml_section.empty()) {
-    err_msg.append(" [in '" + yaml_section + "' section of configuration file]");
-  }
-
-  logging::LOG_ERROR(logger_) << err_msg;
-
-  throw std::invalid_argument(err_msg);
-}
-
-std::string YamlConfiguration::getOrGenerateId(const YAML::Node& yamlNode, const std::string& idField) {
-  std::string id;
-  auto node = yamlNode.as<YAML::Node>();
-
-  if (node[idField]) {
-    if (YAML::NodeType::Scalar == node[idField].Type()) {
-      id = node[idField].as<std::string>();
-      addNewId(id);
-      return id;
-    }
-    throw std::invalid_argument("getOrGenerateId: idField is expected to reference YAML::Node of YAML::NodeType::Scalar.");
-  }
-
-  id = id_generator_->generate().to_string();
-  logger_->log_debug("Generating random ID: id => [%s]", id);
-  return id;
-}
-
-std::string YamlConfiguration::getRequiredIdField(const YAML::Node& yaml_node, std::string_view yaml_section, std::string_view error_message) {
-  yaml::checkRequiredField(yaml_node, "id", yaml_section, error_message);
-  auto id = yaml_node["id"].as<std::string>();
-  addNewId(id);
-  return id;
-}
-
-YAML::Node YamlConfiguration::getOptionalField(const YAML::Node& yamlNode, const std::string& fieldName, const YAML::Node& defaultValue, const std::string& yamlSection,
-                                               const std::string& providedInfoMessage) {
-  std::string infoMessage = providedInfoMessage;
-  auto result = yamlNode.as<YAML::Node>()[fieldName];
-  if (!result) {
-    if (infoMessage.empty()) {
-      // Build a helpful info message for the user to inform them that a default is being used
-      infoMessage =
-          yamlNode.as<YAML::Node>()["name"] ?
-              "Using default value for optional field '" + fieldName + "' in component named '" + yamlNode.as<YAML::Node>()["name"].as<std::string>() + "'" :
-              "Using default value for optional field '" + fieldName + "' ";
-      if (!yamlSection.empty()) {
-        infoMessage += " [in '" + yamlSection + "' section of configuration file]: ";
-      }
-
-      infoMessage += defaultValue.as<std::string>();
-    }
-    logging::LOG_INFO(logger_) << infoMessage;
-    result = defaultValue;
+std::unique_ptr<core::ProcessGroup> YamlConfiguration::getYamlRoot(std::istream &yamlConfigStream) {
+  try {
+    YAML::Node rootYamlNode = YAML::Load(yamlConfigStream);
+    flow::Node root{std::make_shared<YamlNode>(rootYamlNode)};
+    return getRootFrom(root);
+  } catch (const YAML::ParserException &pe) {
+    logger_->log_error(pe.what());
+    std::rethrow_exception(std::current_exception());

Review Comment:
   I don't think they are different, replaced with `throw;`



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r989820156


##########
libminifi/src/core/flow/StructuredConfiguration.cpp:
##########
@@ -0,0 +1,896 @@
+/**
+ *
+ * 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 <memory>
+#include <vector>
+#include <set>
+#include <cinttypes>
+
+#include "core/flow/StructuredConfiguration.h"
+#include "core/flow/CheckRequiredField.h"
+#include "core/flow/StructuredConnectionParser.h"
+#include "core/state/Value.h"
+#include "Defaults.h"
+#include "utils/TimeUtil.h"
+#include "utils/RegexUtils.h"
+
+namespace org::apache::nifi::minifi::core::flow {
+
+std::shared_ptr<utils::IdGenerator> StructuredConfiguration::id_generator_ = utils::IdGenerator::getIdGenerator();
+
+StructuredConfiguration::StructuredConfiguration(ConfigurationContext ctx, std::shared_ptr<logging::Logger> logger)
+    : FlowConfiguration(std::move(ctx)),
+      logger_(std::move(logger)) {}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::parseRootProcessGroup(const Node& root_flow_node) {
+  auto flow_controller_node = root_flow_node[CONFIG_FLOW_CONTROLLER_KEY];
+  auto root_group = parseProcessGroup(flow_controller_node, root_flow_node, true);
+  this->name_ = root_group->getName();
+  return root_group;
+}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::createProcessGroup(const Node& node, bool is_root) {
+  int version = 0;
+
+  checkRequiredField(node, "name", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+  auto flowName = node["name"].getString().value();
+
+  utils::Identifier uuid;
+  // assignment throws on invalid uuid
+  uuid = getOrGenerateId(node);
+
+  if (node["version"]) {
+    version = node["version"].getInt().value();
+  }
+
+  logger_->log_debug("parseRootProcessGroup: id => [%s], name => [%s]", uuid.to_string(), flowName);
+  std::unique_ptr<core::ProcessGroup> group;
+  if (is_root) {
+    group = FlowConfiguration::createRootProcessGroup(flowName, uuid, version);
+  } else {
+    group = FlowConfiguration::createSimpleProcessGroup(flowName, uuid, version);
+  }
+
+  if (node["onschedule retry interval"]) {
+    auto onScheduleRetryPeriod = node["onschedule retry interval"].getString().value();
+    logger_->log_debug("parseRootProcessGroup: onschedule retry period => [%s]", onScheduleRetryPeriod);
+
+    auto on_schedule_retry_period_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(onScheduleRetryPeriod);
+    if (on_schedule_retry_period_value.has_value() && group) {
+      logger_->log_debug("parseRootProcessGroup: onschedule retry => [%" PRId64 "] ms", on_schedule_retry_period_value->count());
+      group->setOnScheduleRetryPeriod(on_schedule_retry_period_value->count());
+    }
+  }
+
+  return group;
+}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::parseProcessGroup(const Node& headerNode, const Node& yamlNode, bool is_root) {
+  auto group = createProcessGroup(headerNode, is_root);
+  Node processorsNode = yamlNode[CONFIG_PROCESSORS_KEY];
+  Node connectionsNode = yamlNode[StructuredConnectionParser::CONFIG_CONNECTIONS_KEY];
+  Node funnelsNode = yamlNode[CONFIG_FUNNELS_KEY];
+  Node remoteProcessingGroupsNode = [&] {
+    // assignment is not supported on invalid Yaml nodes
+    Node candidate = yamlNode[CONFIG_REMOTE_PROCESS_GROUP_KEY];
+    if (candidate) {
+      return candidate;
+    }
+    return yamlNode[CONFIG_REMOTE_PROCESS_GROUP_KEY_V3];
+  }();
+  Node childProcessGroupNodeSeq = yamlNode["Process Groups"];
+
+  parseProcessorNode(processorsNode, group.get());
+  parseRemoteProcessGroup(remoteProcessingGroupsNode, group.get());
+  parseFunnels(funnelsNode, group.get());
+  // parse connections last to give feedback if the source and/or destination
+  // is not in the same process group
+  parseConnection(connectionsNode, group.get());
+
+  if (childProcessGroupNodeSeq && childProcessGroupNodeSeq.isSequence()) {
+    for (const auto childProcessGroupNode : childProcessGroupNodeSeq) {
+      group->addProcessGroup(parseProcessGroup(childProcessGroupNode, childProcessGroupNode));
+    }
+  }
+  return group;
+}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::getRootFrom(const Node& rootYamlNode) {
+  uuids_.clear();
+  Node controllerServiceNode = rootYamlNode[CONFIG_CONTROLLER_SERVICES_KEY];
+  Node provenanceReportNode = rootYamlNode[CONFIG_PROVENANCE_REPORT_KEY];
+
+  parseControllerServices(controllerServiceNode);
+  // Create the root process group
+  std::unique_ptr<core::ProcessGroup> root = parseRootProcessGroup(rootYamlNode);
+  parseProvenanceReporting(provenanceReportNode, root.get());
+
+  // set the controller services into the root group.
+  for (const auto& controller_service : controller_services_->getAllControllerServices()) {
+    root->addControllerService(controller_service->getName(), controller_service);
+    root->addControllerService(controller_service->getUUIDStr(), controller_service);
+  }
+
+  return root;
+}
+
+void StructuredConfiguration::parseProcessorNode(const Node& processors_node, core::ProcessGroup* parentGroup) {
+  int64_t runDurationNanos = -1;
+  utils::Identifier uuid;
+  std::unique_ptr<core::Processor> processor;
+
+  if (!parentGroup) {
+    logger_->log_error("parseProcessNodeYaml: no parent group exists");
+    return;
+  }
+
+  if (!processors_node) {
+    throw std::invalid_argument("Cannot instantiate a MiNiFi instance without a defined Processors configuration node.");
+  }
+  if (!processors_node.isSequence()) {
+    throw std::invalid_argument(
+        "Cannot instantiate a MiNiFi instance without a defined Processors configuration node.");
+  }
+  // Evaluate sequence of processors
+  for (const auto procNode : processors_node) {
+    core::ProcessorConfig procCfg;
+
+    checkRequiredField(procNode, "name", CONFIG_PROCESSORS_KEY);
+    procCfg.name = procNode["name"].getString().value();
+    procCfg.id = getOrGenerateId(procNode);
+
+    uuid = procCfg.id;
+    logger_->log_debug("parseProcessorNode: name => [%s] id => [%s]", procCfg.name, procCfg.id);
+    checkRequiredField(procNode, "class", CONFIG_PROCESSORS_KEY);
+    procCfg.javaClass = procNode["class"].getString().value();
+    logger_->log_debug("parseProcessorNode: class => [%s]", procCfg.javaClass);
+
+    // Determine the processor name only from the Java class
+    auto lastOfIdx = procCfg.javaClass.find_last_of('.');
+    if (lastOfIdx != std::string::npos) {
+      lastOfIdx++;  // if a value is found, increment to move beyond the .
+      std::string processorName = procCfg.javaClass.substr(lastOfIdx);
+      processor = this->createProcessor(processorName, procCfg.javaClass, uuid);
+    } else {
+      // Allow unqualified class names for core processors
+      processor = this->createProcessor(procCfg.javaClass, uuid);
+    }
+
+    if (!processor) {
+      logger_->log_error("Could not create a processor %s with id %s", procCfg.name, procCfg.id);
+      throw std::invalid_argument("Could not create processor " + procCfg.name);
+    }
+
+    processor->setName(procCfg.name);
+
+    processor->setFlowIdentifier(flow_version_->getFlowIdentifier());
+
+    procCfg.schedulingStrategy = getOptionalField(procNode, "scheduling strategy", DEFAULT_SCHEDULING_STRATEGY, CONFIG_PROCESSORS_KEY);
+    logger_->log_debug("parseProcessorNode: scheduling strategy => [%s]", procCfg.schedulingStrategy);
+
+    procCfg.schedulingPeriod = getOptionalField(procNode, "scheduling period", DEFAULT_SCHEDULING_PERIOD_STR, CONFIG_PROCESSORS_KEY);
+
+    logger_->log_debug("parseProcessorNode: scheduling period => [%s]", procCfg.schedulingPeriod);
+
+    if (auto tasksNode = procNode["max concurrent tasks"]) {
+      if (auto int_val = tasksNode.getUInt64()) {
+        procCfg.maxConcurrentTasks = std::to_string(int_val.value());
+      } else {
+        procCfg.maxConcurrentTasks = tasksNode.getString().value();
+      }
+      logger_->log_debug("parseProcessorNode: max concurrent tasks => [%s]", procCfg.maxConcurrentTasks);
+    }
+
+    if (procNode["penalization period"]) {
+      procCfg.penalizationPeriod = procNode["penalization period"].getString().value();
+      logger_->log_debug("parseProcessorNode: penalization period => [%s]", procCfg.penalizationPeriod);
+    }
+
+    if (procNode["yield period"]) {
+      procCfg.yieldPeriod = procNode["yield period"].getString().value();
+      logger_->log_debug("parseProcessorNode: yield period => [%s]", procCfg.yieldPeriod);
+    }
+
+    if (auto runNode = procNode["run duration nanos"]) {
+      if (auto int_val = runNode.getUInt64()) {
+        procCfg.runDurationNanos = std::to_string(int_val.value());
+      } else {
+        procCfg.runDurationNanos = runNode.getString().value();
+      }
+      logger_->log_debug("parseProcessorNode: run duration nanos => [%s]", procCfg.runDurationNanos);
+    }
+
+    // handle auto-terminated relationships
+    if (procNode["auto-terminated relationships list"]) {
+      Node autoTerminatedSequence = procNode["auto-terminated relationships list"];
+      std::vector<std::string> rawAutoTerminatedRelationshipValues;
+      if (autoTerminatedSequence.isSequence() && autoTerminatedSequence.size() > 0) {
+        for (const auto autoTerminatedRel : autoTerminatedSequence) {
+          rawAutoTerminatedRelationshipValues.push_back(autoTerminatedRel.getString().value());
+        }
+      }
+      procCfg.autoTerminatedRelationships = rawAutoTerminatedRelationshipValues;
+    }
+
+    // handle processor properties
+    if (procNode["Properties"]) {
+      Node propertiesNode = procNode["Properties"];
+      parsePropertiesNode(propertiesNode, *processor, procCfg.name, CONFIG_PROCESSORS_KEY);
+    }
+
+    // Take care of scheduling
+
+    if (procCfg.schedulingStrategy == "TIMER_DRIVEN" || procCfg.schedulingStrategy == "EVENT_DRIVEN") {
+      if (auto scheduling_period = utils::timeutils::StringToDuration<std::chrono::nanoseconds>(procCfg.schedulingPeriod)) {
+        logger_->log_debug("convert: parseProcessorNode: schedulingPeriod => [%" PRId64 "] ns", scheduling_period->count());
+        processor->setSchedulingPeriodNano(*scheduling_period);
+      }
+    } else {
+      processor->setCronPeriod(procCfg.schedulingPeriod);
+    }
+
+    if (auto penalization_period = utils::timeutils::StringToDuration<std::chrono::milliseconds>(procCfg.penalizationPeriod)) {
+      logger_->log_debug("convert: parseProcessorNode: penalizationPeriod => [%" PRId64 "] ms", penalization_period->count());
+      processor->setPenalizationPeriod(penalization_period.value());
+    }
+
+    if (auto yield_period = utils::timeutils::StringToDuration<std::chrono::milliseconds>(procCfg.yieldPeriod)) {
+      logger_->log_debug("convert: parseProcessorNode: yieldPeriod => [%" PRId64 "] ms", yield_period->count());
+      processor->setYieldPeriodMsec(yield_period.value());
+    }
+
+    // Default to running
+    processor->setScheduledState(core::RUNNING);
+
+    if (procCfg.schedulingStrategy == "TIMER_DRIVEN") {
+      processor->setSchedulingStrategy(core::TIMER_DRIVEN);
+      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
+    } else if (procCfg.schedulingStrategy == "EVENT_DRIVEN") {
+      processor->setSchedulingStrategy(core::EVENT_DRIVEN);
+      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
+    } else {
+      processor->setSchedulingStrategy(core::CRON_DRIVEN);
+      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
+    }
+
+    int32_t maxConcurrentTasks;
+    if (core::Property::StringToInt(procCfg.maxConcurrentTasks, maxConcurrentTasks)) {
+      logger_->log_debug("parseProcessorNode: maxConcurrentTasks => [%d]", maxConcurrentTasks);
+      processor->setMaxConcurrentTasks((uint8_t) maxConcurrentTasks);
+    }
+
+    if (core::Property::StringToInt(procCfg.runDurationNanos, runDurationNanos)) {
+      logger_->log_debug("parseProcessorNode: runDurationNanos => [%d]", runDurationNanos);
+      processor->setRunDurationNano(std::chrono::nanoseconds(runDurationNanos));
+    }
+
+    std::vector<core::Relationship> autoTerminatedRelationships;
+    for (auto &&relString : procCfg.autoTerminatedRelationships) {
+      core::Relationship relationship(relString, "");
+      logger_->log_debug("parseProcessorNode: autoTerminatedRelationship  => [%s]", relString);
+      autoTerminatedRelationships.push_back(relationship);
+    }
+
+    processor->setAutoTerminatedRelationships(autoTerminatedRelationships);
+
+    parentGroup->addProcessor(std::move(processor));
+  }
+}
+
+void StructuredConfiguration::parseRemoteProcessGroup(const Node& rpg_node_seq, core::ProcessGroup* parentGroup) {
+  utils::Identifier uuid;
+  std::string id;
+
+  if (!parentGroup) {
+    logger_->log_error("parseRemoteProcessGroupYaml: no parent group exists");
+    return;
+  }
+
+  if (!rpg_node_seq || !rpg_node_seq.isSequence()) {
+    return;
+  }
+  for (const auto currRpgNode : rpg_node_seq) {
+    checkRequiredField(currRpgNode, "name", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+    auto name = currRpgNode["name"].getString().value();
+    id = getOrGenerateId(currRpgNode);
+
+    logger_->log_debug("parseRemoteProcessGroupYaml: name => [%s], id => [%s]", name, id);
+
+    auto url = getOptionalField(currRpgNode, "url", "", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+
+    logger_->log_debug("parseRemoteProcessGroupYaml: url => [%s]", url);
+
+    uuid = id;
+    auto group = createRemoteProcessGroup(name, uuid);
+    group->setParent(parentGroup);
+
+    if (currRpgNode["yield period"]) {
+      auto yieldPeriod = currRpgNode["yield period"].getString().value();
+      logger_->log_debug("parseRemoteProcessGroupYaml: yield period => [%s]", yieldPeriod);
+
+      auto yield_period_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(yieldPeriod);
+      if (yield_period_value.has_value() && group) {
+        logger_->log_debug("parseRemoteProcessGroupYaml: yieldPeriod => [%" PRId64 "] ms", yield_period_value->count());
+        group->setYieldPeriodMsec(*yield_period_value);
+      }
+    }
+
+    if (currRpgNode["timeout"]) {
+      auto timeout = currRpgNode["timeout"].getString().value();
+      logger_->log_debug("parseRemoteProcessGroupYaml: timeout => [%s]", timeout);
+
+      auto timeout_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(timeout);
+      if (timeout_value.has_value() && group) {
+        logger_->log_debug("parseRemoteProcessGroupYaml: timeoutValue => [%" PRId64 "] ms", timeout_value->count());
+        group->setTimeout(timeout_value->count());
+      }
+    }
+
+    if (currRpgNode["local network interface"]) {
+      auto interface = currRpgNode["local network interface"].getString().value();
+      logger_->log_debug("parseRemoteProcessGroupYaml: local network interface => [%s]", interface);
+      group->setInterface(interface);
+    }
+
+    if (currRpgNode["transport protocol"]) {
+      auto transport_protocol = currRpgNode["transport protocol"].getString().value();
+      logger_->log_debug("parseRemoteProcessGroupYaml: transport protocol => [%s]", transport_protocol);
+      if (transport_protocol == "HTTP") {
+        group->setTransportProtocol(transport_protocol);
+        if (currRpgNode["proxy host"]) {
+          auto http_proxy_host = currRpgNode["proxy host"].getString().value();
+          logger_->log_debug("parseRemoteProcessGroupYaml: proxy host => [%s]", http_proxy_host);
+          group->setHttpProxyHost(http_proxy_host);
+          if (currRpgNode["proxy user"]) {
+            auto http_proxy_username = currRpgNode["proxy user"].getString().value();
+            logger_->log_debug("parseRemoteProcessGroupYaml: proxy user => [%s]", http_proxy_username);
+            group->setHttpProxyUserName(http_proxy_username);
+          }
+          if (currRpgNode["proxy password"]) {
+            auto http_proxy_password = currRpgNode["proxy password"].getString().value();
+            logger_->log_debug("parseRemoteProcessGroupYaml: proxy password => [%s]", http_proxy_password);
+            group->setHttpProxyPassWord(http_proxy_password);
+          }
+          if (currRpgNode["proxy port"]) {
+            auto http_proxy_port = currRpgNode["proxy port"].getString().value();
+            int32_t port;
+            if (core::Property::StringToInt(http_proxy_port, port)) {
+              logger_->log_debug("parseRemoteProcessGroupYaml: proxy port => [%d]", port);
+              group->setHttpProxyPort(port);
+            }
+          }
+        }
+      } else if (transport_protocol == "RAW") {
+        group->setTransportProtocol(transport_protocol);
+      } else {
+        std::stringstream stream;
+        stream << "Invalid transport protocol " << transport_protocol;
+        throw minifi::Exception(ExceptionType::SITE2SITE_EXCEPTION, stream.str().c_str());
+      }
+    }
+
+    group->setTransmitting(true);
+    group->setURL(url);
+
+    checkRequiredField(currRpgNode, "Input Ports", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+    auto inputPorts = currRpgNode["Input Ports"];
+    if (inputPorts && inputPorts.isSequence()) {
+      for (const auto currPort : inputPorts) {
+        parsePort(currPort, group.get(), sitetosite::SEND);
+      }  // for node
+    }
+    auto outputPorts = currRpgNode["Output Ports"];
+    if (outputPorts && outputPorts.isSequence()) {
+      for (const auto currPort : outputPorts) {
+        logger_->log_debug("Got a current port, iterating...");
+
+        parsePort(currPort, group.get(), sitetosite::RECEIVE);
+      }  // for node
+    }
+    parentGroup->addProcessGroup(std::move(group));
+  }
+}
+
+void StructuredConfiguration::parseProvenanceReporting(const Node& node, core::ProcessGroup* parent_group) {
+  utils::Identifier port_uuid;
+
+  if (!parent_group) {
+    logger_->log_error("parseProvenanceReportingYaml: no parent group exists");
+    return;
+  }
+
+  if (!node || node.isNull()) {
+    logger_->log_debug("no provenance reporting task specified");
+    return;
+  }
+
+  auto reportTask = createProvenanceReportTask();
+
+  checkRequiredField(node, "scheduling strategy", CONFIG_PROVENANCE_REPORT_KEY);
+  auto schedulingStrategyStr = node["scheduling strategy"].getString().value();
+  checkRequiredField(node, "scheduling period", CONFIG_PROVENANCE_REPORT_KEY);
+  auto schedulingPeriodStr = node["scheduling period"].getString().value();
+
+  if (auto scheduling_period = utils::timeutils::StringToDuration<std::chrono::nanoseconds>(schedulingPeriodStr)) {
+    logger_->log_debug("ProvenanceReportingTask schedulingPeriod %" PRId64 " ns", scheduling_period->count());
+    reportTask->setSchedulingPeriodNano(*scheduling_period);
+  }
+
+  if (schedulingStrategyStr == "TIMER_DRIVEN") {
+    reportTask->setSchedulingStrategy(core::TIMER_DRIVEN);
+    logger_->log_debug("ProvenanceReportingTask scheduling strategy %s", schedulingStrategyStr);
+  } else {
+    throw std::invalid_argument("Invalid scheduling strategy " + schedulingStrategyStr);
+  }
+
+  int64_t lvalue;
+  if (node["host"] && node["port"]) {
+    auto hostStr = node["host"].getString().value();
+
+    std::string portStr;
+    if (auto int_val = node["port"].getInt()) {
+      portStr = std::to_string(int_val.value());
+    } else {
+      portStr = node["port"].getString().value();
+    }
+    if (core::Property::StringToInt(portStr, lvalue) && !hostStr.empty()) {
+      logger_->log_debug("ProvenanceReportingTask port %" PRId64, lvalue);
+      std::string url = hostStr + ":" + portStr;
+      reportTask->setURL(url);
+    }
+  }
+
+  if (node["url"]) {
+    auto urlStr = node["url"].getString().value();
+    if (!urlStr.empty()) {
+      reportTask->setURL(urlStr);
+      logger_->log_debug("ProvenanceReportingTask URL %s", urlStr);
+    }
+  }
+  checkRequiredField(node, "port uuid", CONFIG_PROVENANCE_REPORT_KEY);
+  auto portUUIDStr = node["port uuid"].getString().value();
+  checkRequiredField(node, "batch size", CONFIG_PROVENANCE_REPORT_KEY);
+  auto batchSizeStr = node["batch size"].getString().value();
+
+  logger_->log_debug("ProvenanceReportingTask port uuid %s", portUUIDStr);
+  port_uuid = portUUIDStr;
+  reportTask->setPortUUID(port_uuid);
+
+  if (core::Property::StringToInt(batchSizeStr, lvalue)) {
+    reportTask->setBatchSize(gsl::narrow<int>(lvalue));
+  }
+
+  reportTask->initialize();
+
+  // add processor to parent
+  reportTask->setScheduledState(core::RUNNING);
+  parent_group->addProcessor(std::move(reportTask));
+}
+
+void StructuredConfiguration::parseControllerServices(const Node& controllerServicesNode) {
+  if (!controllerServicesNode || !controllerServicesNode.isSequence()) {
+    return;
+  }
+  for (const auto& controllerServiceNode : controllerServicesNode) {
+    checkRequiredField(controllerServiceNode, "name", CONFIG_CONTROLLER_SERVICES_KEY);
+
+    auto type = getRequiredField(controllerServiceNode, std::vector<std::string>{"class", "type"}, CONFIG_CONTROLLER_SERVICES_KEY);
+    logger_->log_debug("Using type %s for controller service node", type);
+
+    std::string fullType = type;
+    auto lastOfIdx = type.find_last_of('.');
+    if (lastOfIdx != std::string::npos) {
+      lastOfIdx++;  // if a value is found, increment to move beyond the .
+      type = type.substr(lastOfIdx);
+    }
+
+    auto name = controllerServiceNode["name"].getString().value();
+    auto id = getRequiredIdField(controllerServiceNode, CONFIG_CONTROLLER_SERVICES_KEY);
+
+    utils::Identifier uuid;
+    uuid = id;
+    std::shared_ptr<core::controller::ControllerServiceNode> controller_service_node = createControllerService(type, fullType, name, uuid);
+    if (nullptr != controller_service_node) {
+      logger_->log_debug("Created Controller Service with UUID %s and name %s", id, name);
+      controller_service_node->initialize();
+      if (Node propertiesNode = controllerServiceNode["Properties"]) {
+        // we should propagate properties to the node and to the implementation
+        parsePropertiesNode(propertiesNode, *controller_service_node, name, CONFIG_CONTROLLER_SERVICES_KEY);
+        if (auto controllerServiceImpl = controller_service_node->getControllerServiceImplementation(); controllerServiceImpl) {
+          parsePropertiesNode(propertiesNode, *controllerServiceImpl, name, CONFIG_CONTROLLER_SERVICES_KEY);
+        }
+      }
+    } else {
+      logger_->log_debug("Could not locate %s", type);
+    }
+    controller_services_->put(id, controller_service_node);
+    controller_services_->put(name, controller_service_node);
+  }
+}
+
+void StructuredConfiguration::parseConnection(const Node& connection_node_seq, core::ProcessGroup* parent) {
+  if (!parent) {
+    logger_->log_error("parseProcessNode: no parent group was provided");
+    return;
+  }
+  if (!connection_node_seq || !connection_node_seq.isSequence()) {
+    return;
+  }
+
+  for (const auto& connection_node : connection_node_seq) {
+    if (!connection_node || !connection_node.isMap()) {
+      logger_->log_error("Invalid connection node, ignoring");
+      continue;
+    }

Review Comment:
   previously when a connection node was invalid the ConnectionParser did not complain, and an unreachable connection was created, ConnectionParser now needs a valid connection node, but I tried to preserve the original behavior 



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
fgerlits commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r980160534


##########
libminifi/include/core/flow/StructuredConfiguration.h:
##########
@@ -0,0 +1,234 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <memory>
+#include <optional>
+#include <string>
+#include <unordered_set>
+
+#include "core/FlowConfiguration.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "core/ProcessorConfig.h"
+#include "Exception.h"
+#include "io/StreamFactory.h"
+#include "io/validation.h"
+#include "sitetosite/SiteToSite.h"
+#include "utils/Id.h"
+#include "utils/StringUtils.h"
+#include "utils/file/FileSystem.h"
+#include "core/flow/Node.h"
+
+namespace org::apache::nifi::minifi::core {
+
+static constexpr char const* CONFIG_FLOW_CONTROLLER_KEY = "Flow Controller";
+static constexpr char const* CONFIG_PROCESSORS_KEY = "Processors";
+static constexpr char const* CONFIG_CONTROLLER_SERVICES_KEY = "Controller Services";
+static constexpr char const* CONFIG_REMOTE_PROCESS_GROUP_KEY = "Remote Processing Groups";
+static constexpr char const* CONFIG_REMOTE_PROCESS_GROUP_KEY_V3 = "Remote Process Groups";
+static constexpr char const* CONFIG_PROVENANCE_REPORT_KEY = "Provenance Reporting";
+static constexpr char const* CONFIG_FUNNELS_KEY = "Funnels";
+
+#define CONFIGURATION_USE_REGEX
+
+// Disable regex in EL for incompatible compilers
+#if __GNUC__ < 4 || (__GNUC__ == 4 && __GNUC_MINOR__ < 9)
+#undef CONFIGURATION_USE_REGEX
+#endif

Review Comment:
   can you remove it from `YamlConfiguration.cpp` and `YamlConfigurationTests.cpp`, too, please?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r979899914


##########
libminifi/include/core/flow/Node.h:
##########
@@ -0,0 +1,143 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string_view>
+#include <tuple>
+#include <optional>
+#include "nonstd/expected.hpp"
+
+namespace org::apache::nifi::minifi::core::flow {
+
+class Node {
+ public:
+  struct Cursor {
+    int line;
+    int column;
+    int pos;
+  };
+
+  class Impl;
+  class Iterator {
+   public:
+    class Value;
+
+    class Impl {
+     public:
+      virtual Impl& operator++() = 0;
+      virtual bool operator==(const Impl& other) const = 0;
+      virtual Value operator*() const = 0;
+      bool operator!=(const Impl& other) const {return !(*this == other);}
+
+      virtual std::unique_ptr<Impl> clone() const = 0;
+      virtual ~Impl() = default;
+    };
+
+    Iterator& operator++() {
+      impl_->operator++();
+      return *this;
+    }
+
+    explicit Iterator(std::unique_ptr<Impl> impl) : impl_(std::move(impl)) {}
+    Iterator(const Iterator& other): impl_(other.impl_->clone()) {}
+    Iterator(Iterator&&) = default;
+    Iterator& operator=(const Iterator& other) {
+      if (this == &other) {
+        return *this;
+      }
+      impl_ = other.impl_->clone();
+      return *this;
+    }
+    Iterator& operator=(Iterator&&) = default;
+
+    bool operator==(const Iterator& other) const {return impl_->operator==(*other.impl_);}
+    bool operator!=(const Iterator& other) const {return !(*this == other);}
+
+    Value operator*() const;
+
+   private:
+    std::unique_ptr<Impl> impl_;
+  };
+
+  class Impl {
+   public:
+    virtual explicit operator bool() const = 0;
+    virtual bool isSequence() const = 0;
+    virtual bool isMap() const = 0;
+    virtual bool isNull() const = 0;
+    virtual bool isScalar() const = 0;
+
+    virtual nonstd::expected<std::string, std::exception_ptr> getString() const = 0;
+    virtual nonstd::expected<int, std::exception_ptr> getInt() const = 0;
+    virtual nonstd::expected<unsigned int, std::exception_ptr> getUInt() const = 0;
+    virtual nonstd::expected<bool, std::exception_ptr> getBool() const = 0;
+    virtual nonstd::expected<int64_t, std::exception_ptr> getInt64() const = 0;
+    virtual nonstd::expected<uint64_t, std::exception_ptr> getUInt64() const = 0;
+
+    virtual std::string getDebugString() const = 0;
+
+    virtual size_t size() const = 0;
+    virtual Iterator begin() const = 0;
+    virtual Iterator end() const = 0;
+    virtual Node operator[](std::string_view key) const = 0;
+
+    virtual std::optional<Cursor> getCursor() const {return std::nullopt;}
+
+    virtual ~Impl() = default;
+  };
+
+  Node() = default;
+  explicit Node(std::shared_ptr<Impl> impl): impl_(std::move(impl)) {}
+
+  explicit operator bool() const {return impl_->operator bool();}
+  bool isSequence() const {return impl_->isSequence();}
+  bool isMap() const {return impl_->isMap();}
+  bool isNull() const {return impl_->isNull();}
+  bool isScalar() const {return impl_->isScalar();}
+
+  nonstd::expected<std::string, std::exception_ptr> getString() const {return impl_->getString();}
+  nonstd::expected<bool, std::exception_ptr> getBool() const {return impl_->getBool();}
+  nonstd::expected<int, std::exception_ptr> getInt() const {return impl_->getInt();}
+  nonstd::expected<unsigned int, std::exception_ptr> getUInt() const {return impl_->getUInt();}
+  nonstd::expected<int64_t, std::exception_ptr> getInt64() const {return impl_->getInt64();}
+  nonstd::expected<uint64_t, std::exception_ptr> getUInt64() const {return impl_->getUInt64();}
+
+  // return a string representation of the node (need not to be deserializable)
+  std::string getDebugString() const {return impl_->getDebugString();}
+
+  size_t size() const {return impl_->size();}
+  size_t empty() const {
+    return size() == 0;
+  }
+  Iterator begin() const {return impl_->begin();}
+  Iterator end() const {return impl_->end();}
+  Node operator[](std::string_view key) const {return impl_->operator[](key);}
+
+  std::optional<Cursor> getCursor() const {return impl_->getCursor();}
+
+ private:
+  std::shared_ptr<Impl> impl_;
+};
+
+class Node::Iterator::Value : public Node, public std::pair<Node, Node> {
+ public:
+  Value(Node node, Node key, Node value): Node{std::move(node)}, std::pair<Node, Node>{std::move(key), std::move(value)} {}
+};

Review Comment:
   I wanted to mimic the `yaml-cpp`'s implementation allowing uniform iteration on both maps and sequences without having to write forwarding methods if the `Node`-case was a member



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
fgerlits commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r976409031


##########
libminifi/include/Defaults.h:
##########
@@ -19,12 +19,14 @@
 
 #ifdef WIN32
 #define DEFAULT_NIFI_CONFIG_YML "\\conf\\config.yml"
+#define DEFAULT_NIFI_CONFIG_JSON "\\conf\\config.json"
 #define DEFAULT_NIFI_PROPERTIES_FILE "\\conf\\minifi.properties"
 #define DEFAULT_LOG_PROPERTIES_FILE "\\conf\\minifi-log.properties"
 #define DEFAULT_UID_PROPERTIES_FILE "\\conf\\minifi-uid.properties"
 #define DEFAULT_BOOTSTRAP_FILE "\\conf\\bootstrap.conf"
 #else
 #define DEFAULT_NIFI_CONFIG_YML "./conf/config.yml"
+#define DEFAULT_NIFI_CONFIG_JSON "./conf/config.json"
 #define DEFAULT_NIFI_PROPERTIES_FILE "./conf/minifi.properties"
 #define DEFAULT_LOG_PROPERTIES_FILE "./conf/minifi-log.properties"
 #define DEFAULT_UID_PROPERTIES_FILE "./conf/minifi-uid.properties"

Review Comment:
   Not a problem, just a note: if #1409 gets merged, then these will need to be changed to relative paths `conf\\config.json` and `conf/config.json`.  I hope git is smart enough to flag this as a conflict, but it may not be.



##########
libminifi/include/core/flow/Node.h:
##########
@@ -0,0 +1,143 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string_view>
+#include <tuple>
+#include <optional>
+#include "nonstd/expected.hpp"
+
+namespace org::apache::nifi::minifi::core::flow {
+
+class Node {
+ public:
+  struct Cursor {
+    int line;
+    int column;
+    int pos;
+  };
+
+  class Impl;
+  class Iterator {
+   public:
+    class Value;
+
+    class Impl {
+     public:
+      virtual Impl& operator++() = 0;
+      virtual bool operator==(const Impl& other) const = 0;
+      virtual Value operator*() const = 0;
+      bool operator!=(const Impl& other) const {return !(*this == other);}
+
+      virtual std::unique_ptr<Impl> clone() const = 0;
+      virtual ~Impl() = default;
+    };
+
+    Iterator& operator++() {
+      impl_->operator++();
+      return *this;
+    }
+
+    explicit Iterator(std::unique_ptr<Impl> impl) : impl_(std::move(impl)) {}
+    Iterator(const Iterator& other): impl_(other.impl_->clone()) {}
+    Iterator(Iterator&&) = default;
+    Iterator& operator=(const Iterator& other) {
+      if (this == &other) {
+        return *this;
+      }
+      impl_ = other.impl_->clone();
+      return *this;
+    }
+    Iterator& operator=(Iterator&&) = default;
+
+    bool operator==(const Iterator& other) const {return impl_->operator==(*other.impl_);}
+    bool operator!=(const Iterator& other) const {return !(*this == other);}
+
+    Value operator*() const;
+
+   private:
+    std::unique_ptr<Impl> impl_;
+  };
+
+  class Impl {
+   public:
+    virtual explicit operator bool() const = 0;
+    virtual bool isSequence() const = 0;
+    virtual bool isMap() const = 0;
+    virtual bool isNull() const = 0;
+    virtual bool isScalar() const = 0;
+
+    virtual nonstd::expected<std::string, std::exception_ptr> getString() const = 0;
+    virtual nonstd::expected<int, std::exception_ptr> getInt() const = 0;
+    virtual nonstd::expected<unsigned int, std::exception_ptr> getUInt() const = 0;
+    virtual nonstd::expected<bool, std::exception_ptr> getBool() const = 0;
+    virtual nonstd::expected<int64_t, std::exception_ptr> getInt64() const = 0;
+    virtual nonstd::expected<uint64_t, std::exception_ptr> getUInt64() const = 0;
+
+    virtual std::string getDebugString() const = 0;
+
+    virtual size_t size() const = 0;
+    virtual Iterator begin() const = 0;
+    virtual Iterator end() const = 0;
+    virtual Node operator[](std::string_view key) const = 0;
+
+    virtual std::optional<Cursor> getCursor() const {return std::nullopt;}

Review Comment:
   I think it would be nicer to make `getCursor` pure virtual, too, and move this dummy implementation to `JsonNode`.



##########
libminifi/include/core/flow/README.md:
##########
@@ -0,0 +1,57 @@
+## Differences between JSON and YAML implementation
+
+### YAML
+
+The possible types of a `YAML::Node` are:
+* Undefined
+* Null
+* Map
+* Sequence
+* Scalar
+
+#### Undefined
+
+The result of querying any member of `Null`, querying non-existing members of `Map`,
+or non-existing indices of `Sequence`.
+
+Note that for `Map`s string conversion applies `map[0]` could be valid, given a key `"0"`,
+while for `Sequence`s string index parsing does NOT happen `seq["0"]`
+will return `Undefined` even if the sequence is non-empty.
+
+Querying or otherwise accessing an `Undefined` (other than `operator bool` or `IsDefined`) usually throws.

Review Comment:
   Do you want to add `IsDefined` to the `flow::Node` interface?  Or can we still access the underlying `YAML::Node`?



##########
libminifi/src/core/flow/StructuredConnectionParser.cpp:
##########
@@ -108,20 +117,20 @@ uint64_t YamlConnectionParser::getSwapThresholdFromYaml() const {
   return 0;

Review Comment:
   I can't comment at line 106 because it hasn't changed, but I suspect it should have changed: `YamlConnectionParser` -> `StructuredConnectionParser`.



##########
extensions/standard-processors/tests/unit/YamlConnectionParserTest.cpp:
##########
@@ -53,26 +56,21 @@ TEST_CASE("Connections components are parsed from yaml", "[YamlConfiguration]")
           "- something_else\n" };
       expectations = { { "success", "" }, { "failure", "" }, { "something_else", "" } };
     }
-    YAML::Node connection_node = YAML::Load(serialized_yaml);
-    YamlConnectionParser yaml_connection_parser(connection_node, "test_node", parent_ptr, logger);
-    yaml_connection_parser.configureConnectionSourceRelationshipsFromYaml(*connection);
+    YAML::Node yaml_node = YAML::Load(serialized_yaml);
+    flow::Node connection_node{std::make_shared<YamlNode>(yaml_node)};
+    StructuredConnectionParser yaml_connection_parser(connection_node, "test_node", parent_ptr, logger);
+    yaml_connection_parser.configureConnectionSourceRelationships(*connection);
     const std::set<core::Relationship>& relationships = connection->getRelationships();
     REQUIRE(expectations == relationships);
   }
   SECTION("Queue size limits are read") {
-    YAML::Node connection_node = YAML::Load(std::string {
+    YAML::Node yaml_node = YAML::Load(std::string {
         "max work queue size: 231\n"
         "max work queue data size: 12 MB\n" });
     YamlConnectionParser yaml_connection_parser(connection_node, "test_node", parent_ptr, logger);
     REQUIRE(231 == yaml_connection_parser.getWorkQueueSizeFromYaml());
     REQUIRE(12582912 == yaml_connection_parser.getWorkQueueDataSizeFromYaml());  // 12 * 1024 * 1024 B
   }
-  SECTION("Queue swap threshold is read") {
-    YAML::Node connection_node = YAML::Load(std::string {
-        "swap threshold: 231\n" });
-    YamlConnectionParser yaml_connection_parser(connection_node, "test_node", parent_ptr, logger);
-    REQUIRE(231 == yaml_connection_parser.getSwapThresholdFromYaml());
-  }

Review Comment:
   why was this test removed?



##########
libminifi/include/core/flow/Node.h:
##########
@@ -0,0 +1,143 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string_view>
+#include <tuple>
+#include <optional>
+#include "nonstd/expected.hpp"
+
+namespace org::apache::nifi::minifi::core::flow {
+
+class Node {
+ public:
+  struct Cursor {
+    int line;
+    int column;
+    int pos;
+  };
+
+  class Impl;
+  class Iterator {
+   public:
+    class Value;
+
+    class Impl {
+     public:
+      virtual Impl& operator++() = 0;
+      virtual bool operator==(const Impl& other) const = 0;
+      virtual Value operator*() const = 0;
+      bool operator!=(const Impl& other) const {return !(*this == other);}
+
+      virtual std::unique_ptr<Impl> clone() const = 0;
+      virtual ~Impl() = default;
+    };
+
+    Iterator& operator++() {
+      impl_->operator++();
+      return *this;
+    }
+
+    explicit Iterator(std::unique_ptr<Impl> impl) : impl_(std::move(impl)) {}
+    Iterator(const Iterator& other): impl_(other.impl_->clone()) {}
+    Iterator(Iterator&&) = default;
+    Iterator& operator=(const Iterator& other) {
+      if (this == &other) {
+        return *this;
+      }
+      impl_ = other.impl_->clone();
+      return *this;
+    }
+    Iterator& operator=(Iterator&&) = default;
+
+    bool operator==(const Iterator& other) const {return impl_->operator==(*other.impl_);}
+    bool operator!=(const Iterator& other) const {return !(*this == other);}
+
+    Value operator*() const;
+
+   private:
+    std::unique_ptr<Impl> impl_;
+  };
+
+  class Impl {
+   public:
+    virtual explicit operator bool() const = 0;
+    virtual bool isSequence() const = 0;
+    virtual bool isMap() const = 0;
+    virtual bool isNull() const = 0;
+    virtual bool isScalar() const = 0;
+
+    virtual nonstd::expected<std::string, std::exception_ptr> getString() const = 0;
+    virtual nonstd::expected<int, std::exception_ptr> getInt() const = 0;
+    virtual nonstd::expected<unsigned int, std::exception_ptr> getUInt() const = 0;
+    virtual nonstd::expected<bool, std::exception_ptr> getBool() const = 0;
+    virtual nonstd::expected<int64_t, std::exception_ptr> getInt64() const = 0;
+    virtual nonstd::expected<uint64_t, std::exception_ptr> getUInt64() const = 0;
+
+    virtual std::string getDebugString() const = 0;
+
+    virtual size_t size() const = 0;
+    virtual Iterator begin() const = 0;
+    virtual Iterator end() const = 0;
+    virtual Node operator[](std::string_view key) const = 0;
+
+    virtual std::optional<Cursor> getCursor() const {return std::nullopt;}
+
+    virtual ~Impl() = default;
+  };
+
+  Node() = default;
+  explicit Node(std::shared_ptr<Impl> impl): impl_(std::move(impl)) {}
+
+  explicit operator bool() const {return impl_->operator bool();}
+  bool isSequence() const {return impl_->isSequence();}
+  bool isMap() const {return impl_->isMap();}
+  bool isNull() const {return impl_->isNull();}
+  bool isScalar() const {return impl_->isScalar();}
+
+  nonstd::expected<std::string, std::exception_ptr> getString() const {return impl_->getString();}
+  nonstd::expected<bool, std::exception_ptr> getBool() const {return impl_->getBool();}
+  nonstd::expected<int, std::exception_ptr> getInt() const {return impl_->getInt();}
+  nonstd::expected<unsigned int, std::exception_ptr> getUInt() const {return impl_->getUInt();}
+  nonstd::expected<int64_t, std::exception_ptr> getInt64() const {return impl_->getInt64();}
+  nonstd::expected<uint64_t, std::exception_ptr> getUInt64() const {return impl_->getUInt64();}
+
+  // return a string representation of the node (need not to be deserializable)
+  std::string getDebugString() const {return impl_->getDebugString();}
+
+  size_t size() const {return impl_->size();}
+  size_t empty() const {
+    return size() == 0;
+  }
+  Iterator begin() const {return impl_->begin();}
+  Iterator end() const {return impl_->end();}
+  Node operator[](std::string_view key) const {return impl_->operator[](key);}
+
+  std::optional<Cursor> getCursor() const {return impl_->getCursor();}
+
+ private:
+  std::shared_ptr<Impl> impl_;
+};
+
+class Node::Iterator::Value : public Node, public std::pair<Node, Node> {
+ public:
+  Value(Node node, Node key, Node value): Node{std::move(node)}, std::pair<Node, Node>{std::move(key), std::move(value)} {}
+};

Review Comment:
   Why do we use inheritance here?  I seems to me that a `Value` contains a `Node` and a key-value pair, it isn't something that is both a `Node` and a key-value pair.



##########
libminifi/include/core/flow/README.md:
##########
@@ -0,0 +1,57 @@
+## Differences between JSON and YAML implementation
+
+### YAML
+
+The possible types of a `YAML::Node` are:
+* Undefined
+* Null
+* Map
+* Sequence
+* Scalar
+
+#### Undefined
+
+The result of querying any member of `Null`, querying non-existing members of `Map`,
+or non-existing indices of `Sequence`.
+
+Note that for `Map`s string conversion applies `map[0]` could be valid, given a key `"0"`,
+while for `Sequence`s string index parsing does NOT happen `seq["0"]`
+will return `Undefined` even if the sequence is non-empty.
+
+Querying or otherwise accessing an `Undefined` (other than `operator bool` or `IsDefined`) usually throws.
+
+#### Null
+
+The value of parsing an empty document, the value of a `Map` item with empty value,
+the value of an omitted `Sequence` item.
+
+```
+key1:      # this is a Null
+key2: ''   # this is a Scalar, the empty string
+arr:
+  - one
+  -        # Null as well
+  - three
+```
+
+#### Scalar
+
+A string value, all conversions to numbers happen on the fly.
+
+### Conversions
+
+#### 1. `::as&lt;std::string&gt;`
+
+* `Null` --> `"null"`
+* `Scalar` --> the string value
+* others --> throws
+
+#### 2. `::as&lt;YAML::Node&gt;`

Review Comment:
   these html escapes are not resolved by either Github or Clion: https://github.com/adamdebreceni/nifi-minifi-cpp/blob/MINIFICPP-1846/libminifi/include/core/flow/README.md
   
   writing `<` and `>` works, at least in Clion



##########
libminifi/include/core/flow/StructuredConfiguration.h:
##########
@@ -0,0 +1,234 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <memory>
+#include <optional>
+#include <string>
+#include <unordered_set>
+
+#include "core/FlowConfiguration.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "core/ProcessorConfig.h"
+#include "Exception.h"
+#include "io/StreamFactory.h"
+#include "io/validation.h"
+#include "sitetosite/SiteToSite.h"
+#include "utils/Id.h"
+#include "utils/StringUtils.h"
+#include "utils/file/FileSystem.h"
+#include "core/flow/Node.h"
+
+namespace org::apache::nifi::minifi::core {
+
+static constexpr char const* CONFIG_FLOW_CONTROLLER_KEY = "Flow Controller";
+static constexpr char const* CONFIG_PROCESSORS_KEY = "Processors";
+static constexpr char const* CONFIG_CONTROLLER_SERVICES_KEY = "Controller Services";
+static constexpr char const* CONFIG_REMOTE_PROCESS_GROUP_KEY = "Remote Processing Groups";
+static constexpr char const* CONFIG_REMOTE_PROCESS_GROUP_KEY_V3 = "Remote Process Groups";
+static constexpr char const* CONFIG_PROVENANCE_REPORT_KEY = "Provenance Reporting";
+static constexpr char const* CONFIG_FUNNELS_KEY = "Funnels";
+
+#define CONFIGURATION_USE_REGEX
+
+// Disable regex in EL for incompatible compilers
+#if __GNUC__ < 4 || (__GNUC__ == 4 && __GNUC_MINOR__ < 9)
+#undef CONFIGURATION_USE_REGEX
+#endif

Review Comment:
   Can we get rid of this?  We don't support gcc 4.8 any more.



##########
libminifi/include/core/flow/StructuredConfiguration.h:
##########
@@ -0,0 +1,234 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <memory>
+#include <optional>
+#include <string>
+#include <unordered_set>
+
+#include "core/FlowConfiguration.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "core/ProcessorConfig.h"
+#include "Exception.h"
+#include "io/StreamFactory.h"
+#include "io/validation.h"
+#include "sitetosite/SiteToSite.h"
+#include "utils/Id.h"
+#include "utils/StringUtils.h"
+#include "utils/file/FileSystem.h"
+#include "core/flow/Node.h"
+
+namespace org::apache::nifi::minifi::core {
+
+static constexpr char const* CONFIG_FLOW_CONTROLLER_KEY = "Flow Controller";
+static constexpr char const* CONFIG_PROCESSORS_KEY = "Processors";
+static constexpr char const* CONFIG_CONTROLLER_SERVICES_KEY = "Controller Services";
+static constexpr char const* CONFIG_REMOTE_PROCESS_GROUP_KEY = "Remote Processing Groups";
+static constexpr char const* CONFIG_REMOTE_PROCESS_GROUP_KEY_V3 = "Remote Process Groups";
+static constexpr char const* CONFIG_PROVENANCE_REPORT_KEY = "Provenance Reporting";
+static constexpr char const* CONFIG_FUNNELS_KEY = "Funnels";
+
+#define CONFIGURATION_USE_REGEX
+
+// Disable regex in EL for incompatible compilers
+#if __GNUC__ < 4 || (__GNUC__ == 4 && __GNUC_MINOR__ < 9)
+#undef CONFIGURATION_USE_REGEX
+#endif
+
+class StructuredConfiguration : public FlowConfiguration {
+ public:
+  StructuredConfiguration(ConfigurationContext ctx, std::shared_ptr<logging::Logger> logger);
+
+  /**
+   * Iterates all component property validation rules and checks that configured state
+   * is valid. If state is determined to be invalid, conf parsing ends and an error is raised.
+   *
+   * @param component
+   * @param component_name
+   * @param section
+   */
+  void validateComponentProperties(ConfigurableComponent& component, const std::string &component_name, const std::string &section) const;
+
+ protected:
+  /**
+   * Returns a shared pointer to a ProcessGroup object containing the
+   * flow configuration.
+   *
+   * @param root_node a pointer to a flow::Node object containing the root
+   *                       node of the parsed document
+   * @return             the root ProcessGroup node of the flow
+   *                       configuration tree
+   */
+  std::unique_ptr<core::ProcessGroup> getRootFrom(const flow::Node& root_node);
+
+  std::unique_ptr<core::ProcessGroup> createProcessGroup(const flow::Node& node, bool is_root = false);
+
+  std::unique_ptr<core::ProcessGroup> parseProcessGroup(const flow::Node& header_node, const flow::Node& node, bool is_root = false);
+  /**
+   * Parses processors from its corresponding config node and adds
+   * them to a parent ProcessGroup. The processors_node argument must point
+   * to a flow::Node containing the processors configuration. Processor
+   * objects will be created and added to the parent ProcessGroup specified
+   * by the parent argument.
+   *
+   * @param processor_node_seq  the flow::Node containing the processor configuration
+   * @param parent              the parent ProcessGroup to which the the created
+   *                            Processor should be added
+   */
+  void parseProcessorNode(const flow::Node& processor_node_seq, core::ProcessGroup* parent);
+
+  /**
+   * Parses a port from its corresponding config node and adds
+   * it to a parent ProcessGroup. The port_node argument must point
+   * to a flow::Node containing the port configuration. A RemoteProcessorGroupPort
+   * object will be created a added to the parent ProcessGroup specified
+   * by the parent argument.
+   *
+   * @param port_node  the flow::Node containing the port configuration
+   * @param parent    the parent ProcessGroup for the port
+   * @param direction the TransferDirection of the port
+   */
+  void parsePort(const flow::Node& port_node, core::ProcessGroup* parent, sitetosite::TransferDirection direction);
+
+  /**
+   * Parses the root level node for the flow configuration and
+   * returns a ProcessGroup containing the tree of flow configuration
+   * objects.
+   *
+   * @param root_flow_node
+   * @return
+   */
+  std::unique_ptr<core::ProcessGroup> parseRootProcessGroup(const flow::Node& root_flow_node);
+
+  void parseProcessorProperty(const flow::Node& doc, const flow::Node& node, std::shared_ptr<core::Processor> processor);
+
+  void parseControllerServices(const flow::Node& controller_services_node);
+
+  /**
+   * Parses the Connections section of a configuration.
+   * The resulting Connections are added to the parent ProcessGroup.
+   *
+   * @param connection_node_seq   the Node containing the Connections section
+   *                              of the configuration
+   * @param parent                the root node of flow configuration to which
+   *                              to add the connections that are parsed
+   */
+  void parseConnection(const flow::Node& connection_node_seq, core::ProcessGroup* parent);
+
+  /**
+   * Parses the Remote Process Group section of a configuration.
+   * The resulting Process Group is added to the parent ProcessGroup.
+   *
+   * @param rpg_node_seq  the flow::Node containing the Remote Process Group
+   *                      section of the configuration YAML

Review Comment:
   there is a leftover YAML here



##########
libminifi/include/core/json/JsonConfiguration.h:
##########
@@ -0,0 +1,71 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <memory>
+#include <optional>
+#include <string>
+#include <unordered_set>
+
+#include "core/FlowConfiguration.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "core/ProcessorConfig.h"
+#include "Exception.h"
+#include "io/StreamFactory.h"
+#include "io/validation.h"
+#include "sitetosite/SiteToSite.h"
+#include "utils/Id.h"
+#include "utils/StringUtils.h"
+#include "utils/file/FileSystem.h"
+#include "core/flow/StructuredConfiguration.h"
+
+class JsonConfigurationTestAccessor;
+
+namespace org::apache::nifi::minifi::core {
+
+class JsonConfiguration : public StructuredConfiguration {
+ public:
+  explicit JsonConfiguration(ConfigurationContext ctx);
+
+  ~JsonConfiguration() override = default;
+
+  /**
+   * Returns a shared pointer to a ProcessGroup object containing the

Review Comment:
   typo: "shared pointer" -> "unique pointer" or "smart pointer" (also at line 58)
   
   another option is to remove the comments entirely, as I don't think they add much information



##########
libminifi/include/core/flow/Node.h:
##########
@@ -0,0 +1,143 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string_view>
+#include <tuple>
+#include <optional>
+#include "nonstd/expected.hpp"
+
+namespace org::apache::nifi::minifi::core::flow {
+
+class Node {
+ public:
+  struct Cursor {
+    int line;
+    int column;
+    int pos;
+  };
+
+  class Impl;
+  class Iterator {
+   public:
+    class Value;
+
+    class Impl {
+     public:
+      virtual Impl& operator++() = 0;
+      virtual bool operator==(const Impl& other) const = 0;
+      virtual Value operator*() const = 0;
+      bool operator!=(const Impl& other) const {return !(*this == other);}
+
+      virtual std::unique_ptr<Impl> clone() const = 0;
+      virtual ~Impl() = default;
+    };
+
+    Iterator& operator++() {
+      impl_->operator++();
+      return *this;
+    }
+
+    explicit Iterator(std::unique_ptr<Impl> impl) : impl_(std::move(impl)) {}
+    Iterator(const Iterator& other): impl_(other.impl_->clone()) {}
+    Iterator(Iterator&&) = default;
+    Iterator& operator=(const Iterator& other) {
+      if (this == &other) {
+        return *this;
+      }
+      impl_ = other.impl_->clone();
+      return *this;
+    }
+    Iterator& operator=(Iterator&&) = default;
+
+    bool operator==(const Iterator& other) const {return impl_->operator==(*other.impl_);}
+    bool operator!=(const Iterator& other) const {return !(*this == other);}
+
+    Value operator*() const;
+
+   private:
+    std::unique_ptr<Impl> impl_;
+  };
+
+  class Impl {

Review Comment:
   it would help readability for me if these two `Impl`s had different names (eg, `IteratorImpl` and `NodeImpl`)



##########
libminifi/include/core/json/JsonNode.h:
##########
@@ -0,0 +1,255 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include "core/flow/Node.h"
+#include "rapidjson/document.h"
+#include "utils/gsl.h"
+
+namespace org::apache::nifi::minifi::core {
+
+class JsonNode : public flow::Node::Impl {
+ public:
+  explicit JsonNode(const rapidjson::Value* node): node_(node) {}
+
+  explicit operator bool() const override {
+    return node_ != nullptr;
+  }
+  bool isSequence() const override {
+    return node_ ? node_->IsArray() : false;
+  }
+  bool isMap() const override {
+    return node_ ? node_->IsObject() : false;
+  }
+  bool isNull() const override {
+    return node_ ? node_->IsNull() : false;
+  }
+  bool isScalar() const override {
+    // TODO:

Review Comment:
   Is this left for part 2?



##########
libminifi/include/core/flow/StructuredConfiguration.h:
##########
@@ -0,0 +1,234 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <memory>
+#include <optional>
+#include <string>
+#include <unordered_set>
+
+#include "core/FlowConfiguration.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "core/ProcessorConfig.h"
+#include "Exception.h"
+#include "io/StreamFactory.h"
+#include "io/validation.h"
+#include "sitetosite/SiteToSite.h"
+#include "utils/Id.h"
+#include "utils/StringUtils.h"
+#include "utils/file/FileSystem.h"
+#include "core/flow/Node.h"
+
+namespace org::apache::nifi::minifi::core {
+
+static constexpr char const* CONFIG_FLOW_CONTROLLER_KEY = "Flow Controller";
+static constexpr char const* CONFIG_PROCESSORS_KEY = "Processors";
+static constexpr char const* CONFIG_CONTROLLER_SERVICES_KEY = "Controller Services";
+static constexpr char const* CONFIG_REMOTE_PROCESS_GROUP_KEY = "Remote Processing Groups";
+static constexpr char const* CONFIG_REMOTE_PROCESS_GROUP_KEY_V3 = "Remote Process Groups";
+static constexpr char const* CONFIG_PROVENANCE_REPORT_KEY = "Provenance Reporting";
+static constexpr char const* CONFIG_FUNNELS_KEY = "Funnels";
+
+#define CONFIGURATION_USE_REGEX
+
+// Disable regex in EL for incompatible compilers
+#if __GNUC__ < 4 || (__GNUC__ == 4 && __GNUC_MINOR__ < 9)
+#undef CONFIGURATION_USE_REGEX
+#endif
+
+class StructuredConfiguration : public FlowConfiguration {
+ public:
+  StructuredConfiguration(ConfigurationContext ctx, std::shared_ptr<logging::Logger> logger);
+
+  /**
+   * Iterates all component property validation rules and checks that configured state
+   * is valid. If state is determined to be invalid, conf parsing ends and an error is raised.
+   *
+   * @param component
+   * @param component_name
+   * @param section
+   */
+  void validateComponentProperties(ConfigurableComponent& component, const std::string &component_name, const std::string &section) const;
+
+ protected:
+  /**
+   * Returns a shared pointer to a ProcessGroup object containing the
+   * flow configuration.
+   *
+   * @param root_node a pointer to a flow::Node object containing the root
+   *                       node of the parsed document
+   * @return             the root ProcessGroup node of the flow
+   *                       configuration tree
+   */
+  std::unique_ptr<core::ProcessGroup> getRootFrom(const flow::Node& root_node);
+
+  std::unique_ptr<core::ProcessGroup> createProcessGroup(const flow::Node& node, bool is_root = false);
+
+  std::unique_ptr<core::ProcessGroup> parseProcessGroup(const flow::Node& header_node, const flow::Node& node, bool is_root = false);
+  /**
+   * Parses processors from its corresponding config node and adds
+   * them to a parent ProcessGroup. The processors_node argument must point

Review Comment:
   I like the argument name `processors_node` better than `processor_node_seq`, but either way, it should match the actual argument name.



##########
libminifi/include/core/flow/StructuredConfiguration.h:
##########
@@ -0,0 +1,234 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <memory>
+#include <optional>
+#include <string>
+#include <unordered_set>
+
+#include "core/FlowConfiguration.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "core/ProcessorConfig.h"
+#include "Exception.h"
+#include "io/StreamFactory.h"
+#include "io/validation.h"
+#include "sitetosite/SiteToSite.h"
+#include "utils/Id.h"
+#include "utils/StringUtils.h"
+#include "utils/file/FileSystem.h"
+#include "core/flow/Node.h"
+
+namespace org::apache::nifi::minifi::core {

Review Comment:
   Should this be `...::core::flow`?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r989810228


##########
libminifi/include/core/json/JsonNode.h:
##########
@@ -0,0 +1,248 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string>
+#include <utility>
+#include <memory>
+
+#include "core/flow/Node.h"
+#include "rapidjson/document.h"
+#include "utils/gsl.h"
+#include "utils/ValueCaster.h"
+
+namespace org::apache::nifi::minifi::core {
+
+class JsonNode : public flow::Node::NodeImpl {
+ public:
+  explicit JsonNode(const rapidjson::Value* node): node_(node) {}
+
+  explicit operator bool() const override {
+    return node_ != nullptr;
+  }
+  bool isSequence() const override {
+    return node_ ? node_->IsArray() : false;
+  }
+  bool isMap() const override {
+    return node_ ? node_->IsObject() : false;
+  }
+  bool isNull() const override {
+    return node_ ? node_->IsNull() : false;
+  }
+
+  nonstd::expected<std::string, std::exception_ptr> getString() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get string of invalid json value");
+      }
+      if (!node_->IsString()) {
+        throw std::runtime_error("Cannot get string of non-string json value");
+      }
+      return std::string{node_->GetString(), node_->GetStringLength()};
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());
+    }
+  }
+
+  nonstd::expected<int, std::exception_ptr> getInt() const override {
+    return getNumber<int>("int");
+  }
+  nonstd::expected<unsigned int, std::exception_ptr> getUInt() const override {
+    return getNumber<unsigned int>("unsigned int");
+  }
+  nonstd::expected<int64_t, std::exception_ptr> getInt64() const override {
+    return getNumber<int64_t>("int64_t");
+  }
+  nonstd::expected<uint64_t, std::exception_ptr> getUInt64() const override {
+    return getNumber<uint64_t>("uint64_t");
+  }
+
+  nonstd::expected<bool, std::exception_ptr> getBool() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get bool of invalid json value");
+      }
+      if (!node_->IsBool()) {
+        throw std::runtime_error("Cannot get bool of non-bool json value");
+      }
+      return node_->GetBool();
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());
+    }
+  }
+
+  std::string getDebugString() const override {
+    if (!node_) return "<invalid>";
+    if (node_->IsObject()) return "<Map>";
+    if (node_->IsArray()) return "<Array>";
+    if (node_->IsNull()) return "null";
+    if (node_->IsInt()) return std::to_string(node_->GetInt());
+    if (node_->IsUint()) return std::to_string(node_->GetUint());
+    if (node_->IsInt64()) return std::to_string(node_->GetInt64());
+    if (node_->IsUint64()) return std::to_string(node_->GetUint64());
+    if (node_->IsTrue()) return "true";
+    if (node_->IsFalse()) return "false";
+    if (node_->IsDouble()) return std::to_string(node_->GetDouble());
+    if (node_->IsString()) return '"' + std::string(node_->GetString(), node_->GetStringLength()) + '"';
+    return "<unknown>";
+  }
+
+  size_t size() const override {
+    if (!node_) {
+      throw std::runtime_error("Cannot get size of invalid json value");
+    }
+    if (!node_->IsArray()) {
+      throw std::runtime_error("Cannot get size of non-array json value");
+    }
+    return node_->Size();
+  }
+  flow::Node::Iterator begin() const override;
+  flow::Node::Iterator end() const override;
+
+  flow::Node operator[](std::string_view key) const override {
+    if (!node_) {
+      throw std::runtime_error("Cannot get member of invalid json value");
+    }
+    if (!node_->IsObject()) {
+      return flow::Node{std::make_shared<JsonNode>(nullptr)};

Review Comment:
   changed the implementation to be in-sync with the behavior of `YAML::Node::operator[]`



##########
libminifi/include/core/json/JsonNode.h:
##########
@@ -0,0 +1,248 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string>
+#include <utility>
+#include <memory>
+
+#include "core/flow/Node.h"
+#include "rapidjson/document.h"
+#include "utils/gsl.h"
+#include "utils/ValueCaster.h"
+
+namespace org::apache::nifi::minifi::core {
+
+class JsonNode : public flow::Node::NodeImpl {
+ public:
+  explicit JsonNode(const rapidjson::Value* node): node_(node) {}
+
+  explicit operator bool() const override {
+    return node_ != nullptr;
+  }
+  bool isSequence() const override {
+    return node_ ? node_->IsArray() : false;
+  }
+  bool isMap() const override {
+    return node_ ? node_->IsObject() : false;
+  }
+  bool isNull() const override {
+    return node_ ? node_->IsNull() : false;
+  }
+
+  nonstd::expected<std::string, std::exception_ptr> getString() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get string of invalid json value");
+      }
+      if (!node_->IsString()) {
+        throw std::runtime_error("Cannot get string of non-string json value");
+      }
+      return std::string{node_->GetString(), node_->GetStringLength()};
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());
+    }
+  }
+
+  nonstd::expected<int, std::exception_ptr> getInt() const override {
+    return getNumber<int>("int");
+  }
+  nonstd::expected<unsigned int, std::exception_ptr> getUInt() const override {
+    return getNumber<unsigned int>("unsigned int");
+  }
+  nonstd::expected<int64_t, std::exception_ptr> getInt64() const override {
+    return getNumber<int64_t>("int64_t");
+  }
+  nonstd::expected<uint64_t, std::exception_ptr> getUInt64() const override {
+    return getNumber<uint64_t>("uint64_t");
+  }
+
+  nonstd::expected<bool, std::exception_ptr> getBool() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get bool of invalid json value");
+      }
+      if (!node_->IsBool()) {
+        throw std::runtime_error("Cannot get bool of non-bool json value");
+      }
+      return node_->GetBool();
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());
+    }
+  }
+
+  std::string getDebugString() const override {
+    if (!node_) return "<invalid>";
+    if (node_->IsObject()) return "<Map>";
+    if (node_->IsArray()) return "<Array>";
+    if (node_->IsNull()) return "null";
+    if (node_->IsInt()) return std::to_string(node_->GetInt());
+    if (node_->IsUint()) return std::to_string(node_->GetUint());
+    if (node_->IsInt64()) return std::to_string(node_->GetInt64());
+    if (node_->IsUint64()) return std::to_string(node_->GetUint64());
+    if (node_->IsTrue()) return "true";
+    if (node_->IsFalse()) return "false";
+    if (node_->IsDouble()) return std::to_string(node_->GetDouble());
+    if (node_->IsString()) return '"' + std::string(node_->GetString(), node_->GetStringLength()) + '"';
+    return "<unknown>";
+  }
+
+  size_t size() const override {
+    if (!node_) {
+      throw std::runtime_error("Cannot get size of invalid json value");
+    }
+    if (!node_->IsArray()) {
+      throw std::runtime_error("Cannot get size of non-array json value");
+    }
+    return node_->Size();
+  }
+  flow::Node::Iterator begin() const override;
+  flow::Node::Iterator end() const override;
+
+  flow::Node operator[](std::string_view key) const override {
+    if (!node_) {
+      throw std::runtime_error("Cannot get member of invalid json value");
+    }
+    if (!node_->IsObject()) {
+      return flow::Node{std::make_shared<JsonNode>(nullptr)};
+    }
+    auto it = node_->FindMember(rapidjson::Value(rapidjson::StringRef(key.data(), key.length())));
+    if (it == node_->MemberEnd()) {
+      return flow::Node{std::make_shared<JsonNode>(nullptr)};
+    }
+    return flow::Node{std::make_shared<JsonNode>(&it->value)};
+  }
+
+  std::optional<flow::Node::Cursor> getCursor() const override {
+    return std::nullopt;
+  }
+
+ private:
+  template<typename T, typename U>
+  std::optional<T> narrow(const U& value) {
+    T res = static_cast<T>(value);
+    if (static_cast<U>(res) != value) {
+      return std::nullopt;
+    }
+    return res;
+  }

Review Comment:
   removed



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r1003037571


##########
libminifi/src/core/flow/StructuredConnectionParser.cpp:
##########
@@ -46,18 +46,18 @@ void YamlConnectionParser::addFunnelRelationshipToConnection(minifi::Connection&
   }
 }
 
-void YamlConnectionParser::configureConnectionSourceRelationshipsFromYaml(minifi::Connection& connection) const {
+void StructuredConnectionParser::configureConnectionSourceRelationships(minifi::Connection& connection) const {
   // Configure connection source
-  if (connectionNode_.as<YAML::Node>()["source relationship name"] && !connectionNode_["source relationship name"].as<std::string>().empty()) {
-    addNewRelationshipToConnection(connectionNode_["source relationship name"].as<std::string>(), connection);
-  } else if (connectionNode_.as<YAML::Node>()["source relationship names"]) {
+  if (connectionNode_["source relationship name"] && !connectionNode_["source relationship name"].getString().value().empty()) {
+    addNewRelationshipToConnection(connectionNode_["source relationship name"].getString().value(), connection);
+  } else if (connectionNode_["source relationship names"]) {
     auto relList = connectionNode_["source relationship names"];

Review Comment:
   as I mentioned in another comment (which I cannot find), I am not a fan of lifetime extensions (since `flow::Node` is just a proxy we return by value), but as part2, I would like to do some cleanup in this files, there we could rewrite all such accesses if we decide 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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r991945405


##########
libminifi/include/core/json/JsonNode.h:
##########
@@ -0,0 +1,248 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string>
+#include <utility>
+#include <memory>
+
+#include "core/flow/Node.h"
+#include "rapidjson/document.h"
+#include "utils/gsl.h"
+#include "utils/ValueCaster.h"
+
+namespace org::apache::nifi::minifi::core {
+
+class JsonNode : public flow::Node::NodeImpl {
+ public:
+  explicit JsonNode(const rapidjson::Value* node): node_(node) {}
+
+  explicit operator bool() const override {
+    return node_ != nullptr;
+  }
+  bool isSequence() const override {
+    return node_ ? node_->IsArray() : false;
+  }
+  bool isMap() const override {
+    return node_ ? node_->IsObject() : false;
+  }
+  bool isNull() const override {
+    return node_ ? node_->IsNull() : false;
+  }
+
+  nonstd::expected<std::string, std::exception_ptr> getString() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get string of invalid json value");
+      }
+      if (!node_->IsString()) {
+        throw std::runtime_error("Cannot get string of non-string json value");
+      }
+      return std::string{node_->GetString(), node_->GetStringLength()};
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());
+    }
+  }
+
+  nonstd::expected<int, std::exception_ptr> getInt() const override {
+    return getNumber<int>("int");
+  }
+  nonstd::expected<unsigned int, std::exception_ptr> getUInt() const override {
+    return getNumber<unsigned int>("unsigned int");
+  }
+  nonstd::expected<int64_t, std::exception_ptr> getInt64() const override {
+    return getNumber<int64_t>("int64_t");
+  }
+  nonstd::expected<uint64_t, std::exception_ptr> getUInt64() const override {
+    return getNumber<uint64_t>("uint64_t");
+  }
+
+  nonstd::expected<bool, std::exception_ptr> getBool() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get bool of invalid json value");
+      }
+      if (!node_->IsBool()) {
+        throw std::runtime_error("Cannot get bool of non-bool json value");
+      }
+      return node_->GetBool();
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());
+    }
+  }
+
+  std::string getDebugString() const override {
+    if (!node_) return "<invalid>";
+    if (node_->IsObject()) return "<Map>";
+    if (node_->IsArray()) return "<Array>";
+    if (node_->IsNull()) return "null";
+    if (node_->IsInt()) return std::to_string(node_->GetInt());
+    if (node_->IsUint()) return std::to_string(node_->GetUint());
+    if (node_->IsInt64()) return std::to_string(node_->GetInt64());
+    if (node_->IsUint64()) return std::to_string(node_->GetUint64());

Review Comment:
   for `float`/`double` `rapidjson` does not try to convert to integers, so I would keep the separate `getDouble` method



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] martinzink closed pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
martinzink closed pull request #1391: MINIFICPP-1846 - Json configuration support part 1
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] szaszm commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r1049706284


##########
libminifi/include/core/json/JsonNode.h:
##########
@@ -0,0 +1,248 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string>
+#include <utility>
+#include <memory>
+
+#include "core/flow/Node.h"
+#include "rapidjson/document.h"
+#include "utils/gsl.h"
+#include "utils/ValueCaster.h"
+
+namespace org::apache::nifi::minifi::core {
+
+class JsonNode : public flow::Node::NodeImpl {

Review Comment:
   Sorry for the delayed answer. I didn't quite understand the situation with the late conversion and error messages, which would be difficult to handle with std::variant and related facilities. If we don't have the type available at parsing, it does indeed complicate things.
   We can keep it as is if you think this is the best data structure to solve the problem.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r979885664


##########
libminifi/include/core/json/JsonNode.h:
##########
@@ -0,0 +1,255 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include "core/flow/Node.h"
+#include "rapidjson/document.h"
+#include "utils/gsl.h"
+
+namespace org::apache::nifi::minifi::core {
+
+class JsonNode : public flow::Node::Impl {
+ public:
+  explicit JsonNode(const rapidjson::Value* node): node_(node) {}
+
+  explicit operator bool() const override {
+    return node_ != nullptr;
+  }
+  bool isSequence() const override {
+    return node_ ? node_->IsArray() : false;
+  }
+  bool isMap() const override {
+    return node_ ? node_->IsObject() : false;
+  }
+  bool isNull() const override {
+    return node_ ? node_->IsNull() : false;
+  }
+  bool isScalar() const override {
+    // TODO:

Review Comment:
   I was only used in a single place, so I removed this method



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r989825552


##########
libminifi/include/core/flow/StructuredConfiguration.h:
##########
@@ -0,0 +1,227 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <memory>
+#include <optional>
+#include <string>
+#include <unordered_set>
+
+#include "core/FlowConfiguration.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "core/ProcessorConfig.h"
+#include "Exception.h"
+#include "io/StreamFactory.h"
+#include "io/validation.h"
+#include "sitetosite/SiteToSite.h"
+#include "utils/Id.h"
+#include "utils/StringUtils.h"
+#include "utils/file/FileSystem.h"
+#include "core/flow/Node.h"
+
+namespace org::apache::nifi::minifi::core::flow {
+
+static constexpr char const* CONFIG_FLOW_CONTROLLER_KEY = "Flow Controller";
+static constexpr char const* CONFIG_PROCESSORS_KEY = "Processors";
+static constexpr char const* CONFIG_CONTROLLER_SERVICES_KEY = "Controller Services";
+static constexpr char const* CONFIG_REMOTE_PROCESS_GROUP_KEY = "Remote Processing Groups";
+static constexpr char const* CONFIG_REMOTE_PROCESS_GROUP_KEY_V3 = "Remote Process Groups";
+static constexpr char const* CONFIG_PROVENANCE_REPORT_KEY = "Provenance Reporting";
+static constexpr char const* CONFIG_FUNNELS_KEY = "Funnels";
+
+class StructuredConfiguration : public FlowConfiguration {
+ public:
+  StructuredConfiguration(ConfigurationContext ctx, std::shared_ptr<logging::Logger> logger);
+
+  /**
+   * Iterates all component property validation rules and checks that configured state
+   * is valid. If state is determined to be invalid, conf parsing ends and an error is raised.
+   *
+   * @param component
+   * @param component_name
+   * @param section
+   */
+  void validateComponentProperties(ConfigurableComponent& component, const std::string &component_name, const std::string &section) const;
+
+ protected:
+  /**
+   * Returns a shared pointer to a ProcessGroup object containing the
+   * flow configuration.
+   *
+   * @param root_node a pointer to a Node object containing the root
+   *                       node of the parsed document
+   * @return             the root ProcessGroup node of the flow
+   *                       configuration tree
+   */
+  std::unique_ptr<core::ProcessGroup> getRootFrom(const Node& root_node);
+
+  std::unique_ptr<core::ProcessGroup> createProcessGroup(const Node& node, bool is_root = false);
+
+  std::unique_ptr<core::ProcessGroup> parseProcessGroup(const Node& header_node, const Node& node, bool is_root = false);
+  /**
+   * Parses processors from its corresponding config node and adds
+   * them to a parent ProcessGroup. The processors_node argument must point
+   * to a Node containing the processors configuration. Processor
+   * objects will be created and added to the parent ProcessGroup specified
+   * by the parent argument.
+   *
+   * @param processors_node  the Node containing the processor configuration
+   * @param parent              the parent ProcessGroup to which the the created
+   *                            Processor should be added
+   */
+  void parseProcessorNode(const Node& processors_node, core::ProcessGroup* parent);
+
+  /**
+   * Parses a port from its corresponding config node and adds
+   * it to a parent ProcessGroup. The port_node argument must point
+   * to a Node containing the port configuration. A RemoteProcessorGroupPort
+   * object will be created a added to the parent ProcessGroup specified
+   * by the parent argument.
+   *
+   * @param port_node  the Node containing the port configuration
+   * @param parent    the parent ProcessGroup for the port
+   * @param direction the TransferDirection of the port
+   */
+  void parsePort(const Node& port_node, core::ProcessGroup* parent, sitetosite::TransferDirection direction);
+
+  /**
+   * Parses the root level node for the flow configuration and
+   * returns a ProcessGroup containing the tree of flow configuration
+   * objects.
+   *
+   * @param root_flow_node
+   * @return
+   */
+  std::unique_ptr<core::ProcessGroup> parseRootProcessGroup(const Node& root_flow_node);
+
+  void parseProcessorProperty(const Node& doc, const Node& node, std::shared_ptr<core::Processor> processor);
+
+  void parseControllerServices(const Node& controller_services_node);
+
+  /**
+   * Parses the Connections section of a configuration.
+   * The resulting Connections are added to the parent ProcessGroup.
+   *
+   * @param connection_node_seq   the Node containing the Connections section
+   *                              of the configuration
+   * @param parent                the root node of flow configuration to which
+   *                              to add the connections that are parsed
+   */
+  void parseConnection(const Node& connection_node_seq, core::ProcessGroup* parent);
+
+  /**
+   * Parses the Remote Process Group section of a configuration.
+   * The resulting Process Group is added to the parent ProcessGroup.
+   *
+   * @param rpg_node_seq  the Node containing the Remote Process Group
+   *                      section of the configuration
+   * @param parent        the root node of flow configuration to which
+   *                      to add the process groups that are parsed
+   */
+  void parseRemoteProcessGroup(const Node& rpg_node_seq, core::ProcessGroup* parent);
+
+  /**
+   * Parses the Provenance Reporting section of a configuration.
+   * The resulting Provenance Reporting processor is added to the
+   * parent ProcessGroup.
+   *
+   * @param report_node  the Node containing the provenance
+   *                      reporting configuration
+   * @param parent_group the root node of flow configuration to which
+   *                      to add the provenance reporting config
+   */
+  void parseProvenanceReporting(const Node& report_node, core::ProcessGroup* parent_group);
+
+  /**
+   * A helper function to parse the Properties Node for a processor.
+   *
+   * @param properties_node the Node containing the properties
+   * @param processor      the Processor to which to add the resulting properties
+   */
+  void parsePropertiesNode(const Node& properties_node, core::ConfigurableComponent& component, const std::string& component_name, const std::string& section);
+
+  /**
+   * Parses the Funnels section of a configuration.
+   * The resulting Funnels are added to the parent ProcessGroup.
+   *
+   * @param node   the Node containing the Funnels section
+   *                 of the configuration
+   * @param parent the root node of flow configuration to which
+   *                 to add the funnels that are parsed
+   */
+  void parseFunnels(const Node& node, core::ProcessGroup* parent);
+
+  /**
+   * A helper function for parsing or generating optional id fields.
+   *
+   * In parsing flow configurations for config schema v1, the
+   * 'id' field of most component types that contains a UUID is optional.
+   * This function will check for the existence of the specified
+   * idField in the specified node. If present, the field will be parsed
+   * as a UUID and the UUID string will be returned. If not present, a
+   * random UUID string will be generated and returned.
+   *
+   * @param node     a pointer to the Node that will be checked for the
+   *                   presence of an idField
+   * @param id_field  the string of the name of the idField to check for. This
+   *                   is optional and defaults to 'id'
+   * @return         the parsed or generated UUID string
+   */
+  std::string getOrGenerateId(const Node& node, const std::string& id_field = "id");
+
+  std::string getRequiredIdField(const Node& node, std::string_view section = "", std::string_view error_message = "");
+
+  /**
+   * This is a helper function for getting an optional value, if it exists.
+   * If it does not exist, returns the provided default value.
+   *
+   * @param node         the flow node to check
+   * @param field_name    the optional field key
+   * @param default_value the default value to use if field is not set
+   * @param section  [optional] the top level section of the config
+   *                       for the node. This is used fpr generating a
+   *                       useful info message for troubleshooting.
+   * @param info_message  [optional] the info message string to use if
+   *                       the optional field is missing. If not provided,
+   *                       a default info message will be generated.
+   */
+  std::string getOptionalField(const Node& node, const std::string& field_name, const std::string& default_value, const std::string& section = "", const std::string& info_message = "");
+
+  static std::shared_ptr<utils::IdGenerator> id_generator_;
+  std::unordered_set<std::string> uuids_;
+  std::shared_ptr<logging::Logger> logger_;

Review Comment:
   I would argue that `JsonConfiguration` having a `StructuredConfiguration` base class is an implementation detail and all log messages should "originate" from the most derived class, e.g. having an error from `TailFile`'s base class `Processor`, through `TailFile`'s logger is easy to track, while having multiple messages from `Processor::logger_` we cannot be sure which processor it comes from (e.g. if we have both a `TailFile` and a `MergeContent`)



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r989826930


##########
libminifi/include/core/flow/StructuredConfiguration.h:
##########
@@ -0,0 +1,227 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <memory>
+#include <optional>
+#include <string>
+#include <unordered_set>
+
+#include "core/FlowConfiguration.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "core/ProcessorConfig.h"
+#include "Exception.h"
+#include "io/StreamFactory.h"
+#include "io/validation.h"
+#include "sitetosite/SiteToSite.h"
+#include "utils/Id.h"
+#include "utils/StringUtils.h"
+#include "utils/file/FileSystem.h"
+#include "core/flow/Node.h"
+
+namespace org::apache::nifi::minifi::core::flow {
+
+static constexpr char const* CONFIG_FLOW_CONTROLLER_KEY = "Flow Controller";
+static constexpr char const* CONFIG_PROCESSORS_KEY = "Processors";
+static constexpr char const* CONFIG_CONTROLLER_SERVICES_KEY = "Controller Services";
+static constexpr char const* CONFIG_REMOTE_PROCESS_GROUP_KEY = "Remote Processing Groups";
+static constexpr char const* CONFIG_REMOTE_PROCESS_GROUP_KEY_V3 = "Remote Process Groups";
+static constexpr char const* CONFIG_PROVENANCE_REPORT_KEY = "Provenance Reporting";
+static constexpr char const* CONFIG_FUNNELS_KEY = "Funnels";
+
+class StructuredConfiguration : public FlowConfiguration {
+ public:
+  StructuredConfiguration(ConfigurationContext ctx, std::shared_ptr<logging::Logger> logger);
+
+  /**
+   * Iterates all component property validation rules and checks that configured state
+   * is valid. If state is determined to be invalid, conf parsing ends and an error is raised.
+   *
+   * @param component
+   * @param component_name
+   * @param section
+   */
+  void validateComponentProperties(ConfigurableComponent& component, const std::string &component_name, const std::string &section) const;
+
+ protected:
+  /**
+   * Returns a shared pointer to a ProcessGroup object containing the
+   * flow configuration.
+   *
+   * @param root_node a pointer to a Node object containing the root
+   *                       node of the parsed document
+   * @return             the root ProcessGroup node of the flow
+   *                       configuration tree
+   */
+  std::unique_ptr<core::ProcessGroup> getRootFrom(const Node& root_node);
+
+  std::unique_ptr<core::ProcessGroup> createProcessGroup(const Node& node, bool is_root = false);
+
+  std::unique_ptr<core::ProcessGroup> parseProcessGroup(const Node& header_node, const Node& node, bool is_root = false);
+  /**
+   * Parses processors from its corresponding config node and adds
+   * them to a parent ProcessGroup. The processors_node argument must point
+   * to a Node containing the processors configuration. Processor
+   * objects will be created and added to the parent ProcessGroup specified
+   * by the parent argument.
+   *
+   * @param processors_node  the Node containing the processor configuration
+   * @param parent              the parent ProcessGroup to which the the created
+   *                            Processor should be added
+   */
+  void parseProcessorNode(const Node& processors_node, core::ProcessGroup* parent);
+
+  /**
+   * Parses a port from its corresponding config node and adds
+   * it to a parent ProcessGroup. The port_node argument must point
+   * to a Node containing the port configuration. A RemoteProcessorGroupPort
+   * object will be created a added to the parent ProcessGroup specified
+   * by the parent argument.
+   *
+   * @param port_node  the Node containing the port configuration
+   * @param parent    the parent ProcessGroup for the port
+   * @param direction the TransferDirection of the port
+   */
+  void parsePort(const Node& port_node, core::ProcessGroup* parent, sitetosite::TransferDirection direction);
+
+  /**
+   * Parses the root level node for the flow configuration and
+   * returns a ProcessGroup containing the tree of flow configuration
+   * objects.
+   *
+   * @param root_flow_node
+   * @return
+   */
+  std::unique_ptr<core::ProcessGroup> parseRootProcessGroup(const Node& root_flow_node);
+
+  void parseProcessorProperty(const Node& doc, const Node& node, std::shared_ptr<core::Processor> processor);
+
+  void parseControllerServices(const Node& controller_services_node);
+
+  /**
+   * Parses the Connections section of a configuration.
+   * The resulting Connections are added to the parent ProcessGroup.
+   *
+   * @param connection_node_seq   the Node containing the Connections section
+   *                              of the configuration
+   * @param parent                the root node of flow configuration to which
+   *                              to add the connections that are parsed
+   */
+  void parseConnection(const Node& connection_node_seq, core::ProcessGroup* parent);
+
+  /**
+   * Parses the Remote Process Group section of a configuration.
+   * The resulting Process Group is added to the parent ProcessGroup.
+   *
+   * @param rpg_node_seq  the Node containing the Remote Process Group
+   *                      section of the configuration
+   * @param parent        the root node of flow configuration to which
+   *                      to add the process groups that are parsed
+   */
+  void parseRemoteProcessGroup(const Node& rpg_node_seq, core::ProcessGroup* parent);
+
+  /**
+   * Parses the Provenance Reporting section of a configuration.
+   * The resulting Provenance Reporting processor is added to the
+   * parent ProcessGroup.
+   *
+   * @param report_node  the Node containing the provenance
+   *                      reporting configuration
+   * @param parent_group the root node of flow configuration to which
+   *                      to add the provenance reporting config
+   */
+  void parseProvenanceReporting(const Node& report_node, core::ProcessGroup* parent_group);
+
+  /**
+   * A helper function to parse the Properties Node for a processor.
+   *
+   * @param properties_node the Node containing the properties
+   * @param processor      the Processor to which to add the resulting properties
+   */
+  void parsePropertiesNode(const Node& properties_node, core::ConfigurableComponent& component, const std::string& component_name, const std::string& section);
+
+  /**
+   * Parses the Funnels section of a configuration.
+   * The resulting Funnels are added to the parent ProcessGroup.
+   *
+   * @param node   the Node containing the Funnels section
+   *                 of the configuration
+   * @param parent the root node of flow configuration to which
+   *                 to add the funnels that are parsed
+   */
+  void parseFunnels(const Node& node, core::ProcessGroup* parent);
+
+  /**
+   * A helper function for parsing or generating optional id fields.
+   *
+   * In parsing flow configurations for config schema v1, the
+   * 'id' field of most component types that contains a UUID is optional.
+   * This function will check for the existence of the specified
+   * idField in the specified node. If present, the field will be parsed
+   * as a UUID and the UUID string will be returned. If not present, a
+   * random UUID string will be generated and returned.
+   *
+   * @param node     a pointer to the Node that will be checked for the
+   *                   presence of an idField
+   * @param id_field  the string of the name of the idField to check for. This
+   *                   is optional and defaults to 'id'
+   * @return         the parsed or generated UUID string
+   */
+  std::string getOrGenerateId(const Node& node, const std::string& id_field = "id");
+
+  std::string getRequiredIdField(const Node& node, std::string_view section = "", std::string_view error_message = "");
+
+  /**
+   * This is a helper function for getting an optional value, if it exists.
+   * If it does not exist, returns the provided default value.
+   *
+   * @param node         the flow node to check
+   * @param field_name    the optional field key
+   * @param default_value the default value to use if field is not set
+   * @param section  [optional] the top level section of the config
+   *                       for the node. This is used fpr generating a
+   *                       useful info message for troubleshooting.
+   * @param info_message  [optional] the info message string to use if
+   *                       the optional field is missing. If not provided,
+   *                       a default info message will be generated.
+   */
+  std::string getOptionalField(const Node& node, const std::string& field_name, const std::string& default_value, const std::string& section = "", const std::string& info_message = "");
+
+  static std::shared_ptr<utils::IdGenerator> id_generator_;
+  std::unordered_set<std::string> uuids_;
+  std::shared_ptr<logging::Logger> logger_;

Review Comment:
   this is kind of relevant, users ask for more specificity: https://issues.apache.org/jira/browse/MINIFICPP-1948



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r989733092


##########
libminifi/include/core/json/JsonNode.h:
##########
@@ -0,0 +1,248 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string>
+#include <utility>
+#include <memory>
+
+#include "core/flow/Node.h"
+#include "rapidjson/document.h"
+#include "utils/gsl.h"
+#include "utils/ValueCaster.h"
+
+namespace org::apache::nifi::minifi::core {
+
+class JsonNode : public flow::Node::NodeImpl {
+ public:
+  explicit JsonNode(const rapidjson::Value* node): node_(node) {}
+
+  explicit operator bool() const override {
+    return node_ != nullptr;
+  }
+  bool isSequence() const override {
+    return node_ ? node_->IsArray() : false;
+  }
+  bool isMap() const override {
+    return node_ ? node_->IsObject() : false;
+  }
+  bool isNull() const override {
+    return node_ ? node_->IsNull() : false;
+  }
+
+  nonstd::expected<std::string, std::exception_ptr> getString() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get string of invalid json value");
+      }
+      if (!node_->IsString()) {
+        throw std::runtime_error("Cannot get string of non-string json value");
+      }
+      return std::string{node_->GetString(), node_->GetStringLength()};
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());
+    }
+  }
+
+  nonstd::expected<int, std::exception_ptr> getInt() const override {
+    return getNumber<int>("int");
+  }
+  nonstd::expected<unsigned int, std::exception_ptr> getUInt() const override {
+    return getNumber<unsigned int>("unsigned int");
+  }
+  nonstd::expected<int64_t, std::exception_ptr> getInt64() const override {
+    return getNumber<int64_t>("int64_t");
+  }
+  nonstd::expected<uint64_t, std::exception_ptr> getUInt64() const override {
+    return getNumber<uint64_t>("uint64_t");
+  }
+
+  nonstd::expected<bool, std::exception_ptr> getBool() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get bool of invalid json value");
+      }
+      if (!node_->IsBool()) {
+        throw std::runtime_error("Cannot get bool of non-bool json value");
+      }
+      return node_->GetBool();
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());
+    }
+  }
+
+  std::string getDebugString() const override {
+    if (!node_) return "<invalid>";
+    if (node_->IsObject()) return "<Map>";
+    if (node_->IsArray()) return "<Array>";
+    if (node_->IsNull()) return "null";
+    if (node_->IsInt()) return std::to_string(node_->GetInt());
+    if (node_->IsUint()) return std::to_string(node_->GetUint());
+    if (node_->IsInt64()) return std::to_string(node_->GetInt64());
+    if (node_->IsUint64()) return std::to_string(node_->GetUint64());

Review Comment:
   could you elaborate on this a bit more?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
fgerlits commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r997061283


##########
libminifi/include/core/flow/StructuredConfiguration.h:
##########
@@ -0,0 +1,227 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <memory>
+#include <optional>
+#include <string>
+#include <unordered_set>
+
+#include "core/FlowConfiguration.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "core/ProcessorConfig.h"
+#include "Exception.h"
+#include "io/StreamFactory.h"
+#include "io/validation.h"
+#include "sitetosite/SiteToSite.h"
+#include "utils/Id.h"
+#include "utils/StringUtils.h"
+#include "utils/file/FileSystem.h"
+#include "core/flow/Node.h"
+
+namespace org::apache::nifi::minifi::core::flow {
+
+static constexpr char const* CONFIG_FLOW_CONTROLLER_KEY = "Flow Controller";
+static constexpr char const* CONFIG_PROCESSORS_KEY = "Processors";
+static constexpr char const* CONFIG_CONTROLLER_SERVICES_KEY = "Controller Services";
+static constexpr char const* CONFIG_REMOTE_PROCESS_GROUP_KEY = "Remote Processing Groups";
+static constexpr char const* CONFIG_REMOTE_PROCESS_GROUP_KEY_V3 = "Remote Process Groups";
+static constexpr char const* CONFIG_PROVENANCE_REPORT_KEY = "Provenance Reporting";
+static constexpr char const* CONFIG_FUNNELS_KEY = "Funnels";
+
+class StructuredConfiguration : public FlowConfiguration {
+ public:
+  StructuredConfiguration(ConfigurationContext ctx, std::shared_ptr<logging::Logger> logger);
+
+  /**
+   * Iterates all component property validation rules and checks that configured state
+   * is valid. If state is determined to be invalid, conf parsing ends and an error is raised.
+   *
+   * @param component
+   * @param component_name
+   * @param section
+   */
+  void validateComponentProperties(ConfigurableComponent& component, const std::string &component_name, const std::string &section) const;
+
+ protected:
+  /**
+   * Returns a shared pointer to a ProcessGroup object containing the
+   * flow configuration.
+   *
+   * @param root_node a pointer to a Node object containing the root
+   *                       node of the parsed document
+   * @return             the root ProcessGroup node of the flow
+   *                       configuration tree
+   */
+  std::unique_ptr<core::ProcessGroup> getRootFrom(const Node& root_node);
+
+  std::unique_ptr<core::ProcessGroup> createProcessGroup(const Node& node, bool is_root = false);
+
+  std::unique_ptr<core::ProcessGroup> parseProcessGroup(const Node& header_node, const Node& node, bool is_root = false);
+  /**
+   * Parses processors from its corresponding config node and adds
+   * them to a parent ProcessGroup. The processors_node argument must point
+   * to a Node containing the processors configuration. Processor
+   * objects will be created and added to the parent ProcessGroup specified
+   * by the parent argument.
+   *
+   * @param processors_node  the Node containing the processor configuration
+   * @param parent              the parent ProcessGroup to which the the created
+   *                            Processor should be added
+   */
+  void parseProcessorNode(const Node& processors_node, core::ProcessGroup* parent);
+
+  /**
+   * Parses a port from its corresponding config node and adds
+   * it to a parent ProcessGroup. The port_node argument must point
+   * to a Node containing the port configuration. A RemoteProcessorGroupPort
+   * object will be created a added to the parent ProcessGroup specified
+   * by the parent argument.
+   *
+   * @param port_node  the Node containing the port configuration
+   * @param parent    the parent ProcessGroup for the port
+   * @param direction the TransferDirection of the port
+   */
+  void parsePort(const Node& port_node, core::ProcessGroup* parent, sitetosite::TransferDirection direction);
+
+  /**
+   * Parses the root level node for the flow configuration and
+   * returns a ProcessGroup containing the tree of flow configuration
+   * objects.
+   *
+   * @param root_flow_node
+   * @return
+   */
+  std::unique_ptr<core::ProcessGroup> parseRootProcessGroup(const Node& root_flow_node);
+
+  void parseProcessorProperty(const Node& doc, const Node& node, std::shared_ptr<core::Processor> processor);
+
+  void parseControllerServices(const Node& controller_services_node);
+
+  /**
+   * Parses the Connections section of a configuration.
+   * The resulting Connections are added to the parent ProcessGroup.
+   *
+   * @param connection_node_seq   the Node containing the Connections section
+   *                              of the configuration
+   * @param parent                the root node of flow configuration to which
+   *                              to add the connections that are parsed
+   */
+  void parseConnection(const Node& connection_node_seq, core::ProcessGroup* parent);
+
+  /**
+   * Parses the Remote Process Group section of a configuration.
+   * The resulting Process Group is added to the parent ProcessGroup.
+   *
+   * @param rpg_node_seq  the Node containing the Remote Process Group
+   *                      section of the configuration
+   * @param parent        the root node of flow configuration to which
+   *                      to add the process groups that are parsed
+   */
+  void parseRemoteProcessGroup(const Node& rpg_node_seq, core::ProcessGroup* parent);
+
+  /**
+   * Parses the Provenance Reporting section of a configuration.
+   * The resulting Provenance Reporting processor is added to the
+   * parent ProcessGroup.
+   *
+   * @param report_node  the Node containing the provenance
+   *                      reporting configuration
+   * @param parent_group the root node of flow configuration to which
+   *                      to add the provenance reporting config
+   */
+  void parseProvenanceReporting(const Node& report_node, core::ProcessGroup* parent_group);
+
+  /**
+   * A helper function to parse the Properties Node for a processor.
+   *
+   * @param properties_node the Node containing the properties
+   * @param processor      the Processor to which to add the resulting properties
+   */
+  void parsePropertiesNode(const Node& properties_node, core::ConfigurableComponent& component, const std::string& component_name, const std::string& section);
+
+  /**
+   * Parses the Funnels section of a configuration.
+   * The resulting Funnels are added to the parent ProcessGroup.
+   *
+   * @param node   the Node containing the Funnels section
+   *                 of the configuration
+   * @param parent the root node of flow configuration to which
+   *                 to add the funnels that are parsed
+   */
+  void parseFunnels(const Node& node, core::ProcessGroup* parent);
+
+  /**
+   * A helper function for parsing or generating optional id fields.
+   *
+   * In parsing flow configurations for config schema v1, the
+   * 'id' field of most component types that contains a UUID is optional.
+   * This function will check for the existence of the specified
+   * idField in the specified node. If present, the field will be parsed
+   * as a UUID and the UUID string will be returned. If not present, a
+   * random UUID string will be generated and returned.
+   *
+   * @param node     a pointer to the Node that will be checked for the
+   *                   presence of an idField
+   * @param id_field  the string of the name of the idField to check for. This
+   *                   is optional and defaults to 'id'
+   * @return         the parsed or generated UUID string
+   */
+  std::string getOrGenerateId(const Node& node, const std::string& id_field = "id");
+
+  std::string getRequiredIdField(const Node& node, std::string_view section = "", std::string_view error_message = "");
+
+  /**
+   * This is a helper function for getting an optional value, if it exists.
+   * If it does not exist, returns the provided default value.
+   *
+   * @param node         the flow node to check
+   * @param field_name    the optional field key
+   * @param default_value the default value to use if field is not set
+   * @param section  [optional] the top level section of the config
+   *                       for the node. This is used fpr generating a
+   *                       useful info message for troubleshooting.
+   * @param info_message  [optional] the info message string to use if
+   *                       the optional field is missing. If not provided,
+   *                       a default info message will be generated.
+   */
+  std::string getOptionalField(const Node& node, const std::string& field_name, const std::string& default_value, const std::string& section = "", const std::string& info_message = "");
+
+  static std::shared_ptr<utils::IdGenerator> id_generator_;
+  std::unordered_set<std::string> uuids_;
+  std::shared_ptr<logging::Logger> logger_;

Review Comment:
   I don't think this will give much extra info to users, since each minifi will either have a yaml or a json configuration but not both.  On the other hand, it will make our future selves' lives slightly harder.  But if you think it's worth it, I don't mind.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r979884804


##########
libminifi/include/core/flow/StructuredConfiguration.h:
##########
@@ -0,0 +1,234 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <memory>
+#include <optional>
+#include <string>
+#include <unordered_set>
+
+#include "core/FlowConfiguration.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "core/ProcessorConfig.h"
+#include "Exception.h"
+#include "io/StreamFactory.h"
+#include "io/validation.h"
+#include "sitetosite/SiteToSite.h"
+#include "utils/Id.h"
+#include "utils/StringUtils.h"
+#include "utils/file/FileSystem.h"
+#include "core/flow/Node.h"
+
+namespace org::apache::nifi::minifi::core {
+
+static constexpr char const* CONFIG_FLOW_CONTROLLER_KEY = "Flow Controller";
+static constexpr char const* CONFIG_PROCESSORS_KEY = "Processors";
+static constexpr char const* CONFIG_CONTROLLER_SERVICES_KEY = "Controller Services";
+static constexpr char const* CONFIG_REMOTE_PROCESS_GROUP_KEY = "Remote Processing Groups";
+static constexpr char const* CONFIG_REMOTE_PROCESS_GROUP_KEY_V3 = "Remote Process Groups";
+static constexpr char const* CONFIG_PROVENANCE_REPORT_KEY = "Provenance Reporting";
+static constexpr char const* CONFIG_FUNNELS_KEY = "Funnels";
+
+#define CONFIGURATION_USE_REGEX
+
+// Disable regex in EL for incompatible compilers
+#if __GNUC__ < 4 || (__GNUC__ == 4 && __GNUC_MINOR__ < 9)
+#undef CONFIGURATION_USE_REGEX
+#endif

Review Comment:
   removed



##########
libminifi/include/core/flow/StructuredConfiguration.h:
##########
@@ -0,0 +1,234 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <memory>
+#include <optional>
+#include <string>
+#include <unordered_set>
+
+#include "core/FlowConfiguration.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "core/ProcessorConfig.h"
+#include "Exception.h"
+#include "io/StreamFactory.h"
+#include "io/validation.h"
+#include "sitetosite/SiteToSite.h"
+#include "utils/Id.h"
+#include "utils/StringUtils.h"
+#include "utils/file/FileSystem.h"
+#include "core/flow/Node.h"
+
+namespace org::apache::nifi::minifi::core {
+
+static constexpr char const* CONFIG_FLOW_CONTROLLER_KEY = "Flow Controller";
+static constexpr char const* CONFIG_PROCESSORS_KEY = "Processors";
+static constexpr char const* CONFIG_CONTROLLER_SERVICES_KEY = "Controller Services";
+static constexpr char const* CONFIG_REMOTE_PROCESS_GROUP_KEY = "Remote Processing Groups";
+static constexpr char const* CONFIG_REMOTE_PROCESS_GROUP_KEY_V3 = "Remote Process Groups";
+static constexpr char const* CONFIG_PROVENANCE_REPORT_KEY = "Provenance Reporting";
+static constexpr char const* CONFIG_FUNNELS_KEY = "Funnels";
+
+#define CONFIGURATION_USE_REGEX
+
+// Disable regex in EL for incompatible compilers
+#if __GNUC__ < 4 || (__GNUC__ == 4 && __GNUC_MINOR__ < 9)
+#undef CONFIGURATION_USE_REGEX
+#endif
+
+class StructuredConfiguration : public FlowConfiguration {
+ public:
+  StructuredConfiguration(ConfigurationContext ctx, std::shared_ptr<logging::Logger> logger);
+
+  /**
+   * Iterates all component property validation rules and checks that configured state
+   * is valid. If state is determined to be invalid, conf parsing ends and an error is raised.
+   *
+   * @param component
+   * @param component_name
+   * @param section
+   */
+  void validateComponentProperties(ConfigurableComponent& component, const std::string &component_name, const std::string &section) const;
+
+ protected:
+  /**
+   * Returns a shared pointer to a ProcessGroup object containing the
+   * flow configuration.
+   *
+   * @param root_node a pointer to a flow::Node object containing the root
+   *                       node of the parsed document
+   * @return             the root ProcessGroup node of the flow
+   *                       configuration tree
+   */
+  std::unique_ptr<core::ProcessGroup> getRootFrom(const flow::Node& root_node);
+
+  std::unique_ptr<core::ProcessGroup> createProcessGroup(const flow::Node& node, bool is_root = false);
+
+  std::unique_ptr<core::ProcessGroup> parseProcessGroup(const flow::Node& header_node, const flow::Node& node, bool is_root = false);
+  /**
+   * Parses processors from its corresponding config node and adds
+   * them to a parent ProcessGroup. The processors_node argument must point

Review Comment:
   renamed



##########
libminifi/include/core/flow/StructuredConfiguration.h:
##########
@@ -0,0 +1,234 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <memory>
+#include <optional>
+#include <string>
+#include <unordered_set>
+
+#include "core/FlowConfiguration.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "core/ProcessorConfig.h"
+#include "Exception.h"
+#include "io/StreamFactory.h"
+#include "io/validation.h"
+#include "sitetosite/SiteToSite.h"
+#include "utils/Id.h"
+#include "utils/StringUtils.h"
+#include "utils/file/FileSystem.h"
+#include "core/flow/Node.h"
+
+namespace org::apache::nifi::minifi::core {
+
+static constexpr char const* CONFIG_FLOW_CONTROLLER_KEY = "Flow Controller";
+static constexpr char const* CONFIG_PROCESSORS_KEY = "Processors";
+static constexpr char const* CONFIG_CONTROLLER_SERVICES_KEY = "Controller Services";
+static constexpr char const* CONFIG_REMOTE_PROCESS_GROUP_KEY = "Remote Processing Groups";
+static constexpr char const* CONFIG_REMOTE_PROCESS_GROUP_KEY_V3 = "Remote Process Groups";
+static constexpr char const* CONFIG_PROVENANCE_REPORT_KEY = "Provenance Reporting";
+static constexpr char const* CONFIG_FUNNELS_KEY = "Funnels";
+
+#define CONFIGURATION_USE_REGEX
+
+// Disable regex in EL for incompatible compilers
+#if __GNUC__ < 4 || (__GNUC__ == 4 && __GNUC_MINOR__ < 9)
+#undef CONFIGURATION_USE_REGEX
+#endif
+
+class StructuredConfiguration : public FlowConfiguration {
+ public:
+  StructuredConfiguration(ConfigurationContext ctx, std::shared_ptr<logging::Logger> logger);
+
+  /**
+   * Iterates all component property validation rules and checks that configured state
+   * is valid. If state is determined to be invalid, conf parsing ends and an error is raised.
+   *
+   * @param component
+   * @param component_name
+   * @param section
+   */
+  void validateComponentProperties(ConfigurableComponent& component, const std::string &component_name, const std::string &section) const;
+
+ protected:
+  /**
+   * Returns a shared pointer to a ProcessGroup object containing the
+   * flow configuration.
+   *
+   * @param root_node a pointer to a flow::Node object containing the root
+   *                       node of the parsed document
+   * @return             the root ProcessGroup node of the flow
+   *                       configuration tree
+   */
+  std::unique_ptr<core::ProcessGroup> getRootFrom(const flow::Node& root_node);
+
+  std::unique_ptr<core::ProcessGroup> createProcessGroup(const flow::Node& node, bool is_root = false);
+
+  std::unique_ptr<core::ProcessGroup> parseProcessGroup(const flow::Node& header_node, const flow::Node& node, bool is_root = false);
+  /**
+   * Parses processors from its corresponding config node and adds
+   * them to a parent ProcessGroup. The processors_node argument must point
+   * to a flow::Node containing the processors configuration. Processor
+   * objects will be created and added to the parent ProcessGroup specified
+   * by the parent argument.
+   *
+   * @param processor_node_seq  the flow::Node containing the processor configuration
+   * @param parent              the parent ProcessGroup to which the the created
+   *                            Processor should be added
+   */
+  void parseProcessorNode(const flow::Node& processor_node_seq, core::ProcessGroup* parent);
+
+  /**
+   * Parses a port from its corresponding config node and adds
+   * it to a parent ProcessGroup. The port_node argument must point
+   * to a flow::Node containing the port configuration. A RemoteProcessorGroupPort
+   * object will be created a added to the parent ProcessGroup specified
+   * by the parent argument.
+   *
+   * @param port_node  the flow::Node containing the port configuration
+   * @param parent    the parent ProcessGroup for the port
+   * @param direction the TransferDirection of the port
+   */
+  void parsePort(const flow::Node& port_node, core::ProcessGroup* parent, sitetosite::TransferDirection direction);
+
+  /**
+   * Parses the root level node for the flow configuration and
+   * returns a ProcessGroup containing the tree of flow configuration
+   * objects.
+   *
+   * @param root_flow_node
+   * @return
+   */
+  std::unique_ptr<core::ProcessGroup> parseRootProcessGroup(const flow::Node& root_flow_node);
+
+  void parseProcessorProperty(const flow::Node& doc, const flow::Node& node, std::shared_ptr<core::Processor> processor);
+
+  void parseControllerServices(const flow::Node& controller_services_node);
+
+  /**
+   * Parses the Connections section of a configuration.
+   * The resulting Connections are added to the parent ProcessGroup.
+   *
+   * @param connection_node_seq   the Node containing the Connections section
+   *                              of the configuration
+   * @param parent                the root node of flow configuration to which
+   *                              to add the connections that are parsed
+   */
+  void parseConnection(const flow::Node& connection_node_seq, core::ProcessGroup* parent);
+
+  /**
+   * Parses the Remote Process Group section of a configuration.
+   * The resulting Process Group is added to the parent ProcessGroup.
+   *
+   * @param rpg_node_seq  the flow::Node containing the Remote Process Group
+   *                      section of the configuration YAML

Review Comment:
   removed



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r982151109


##########
libminifi/include/core/json/JsonConfiguration.h:
##########
@@ -0,0 +1,71 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <memory>
+#include <optional>
+#include <string>
+#include <unordered_set>
+
+#include "core/FlowConfiguration.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "core/ProcessorConfig.h"
+#include "Exception.h"
+#include "io/StreamFactory.h"
+#include "io/validation.h"
+#include "sitetosite/SiteToSite.h"
+#include "utils/Id.h"
+#include "utils/StringUtils.h"
+#include "utils/file/FileSystem.h"
+#include "core/flow/StructuredConfiguration.h"
+
+class JsonConfigurationTestAccessor;
+
+namespace org::apache::nifi::minifi::core {
+
+class JsonConfiguration : public StructuredConfiguration {
+ public:
+  explicit JsonConfiguration(ConfigurationContext ctx);
+
+  ~JsonConfiguration() override = default;
+
+  /**
+   * Returns a shared pointer to a ProcessGroup object containing the

Review Comment:
   removed the comment



##########
libminifi/include/core/flow/StructuredConfiguration.h:
##########
@@ -0,0 +1,234 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <memory>
+#include <optional>
+#include <string>
+#include <unordered_set>
+
+#include "core/FlowConfiguration.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "core/ProcessorConfig.h"
+#include "Exception.h"
+#include "io/StreamFactory.h"
+#include "io/validation.h"
+#include "sitetosite/SiteToSite.h"
+#include "utils/Id.h"
+#include "utils/StringUtils.h"
+#include "utils/file/FileSystem.h"
+#include "core/flow/Node.h"
+
+namespace org::apache::nifi::minifi::core {
+
+static constexpr char const* CONFIG_FLOW_CONTROLLER_KEY = "Flow Controller";
+static constexpr char const* CONFIG_PROCESSORS_KEY = "Processors";
+static constexpr char const* CONFIG_CONTROLLER_SERVICES_KEY = "Controller Services";
+static constexpr char const* CONFIG_REMOTE_PROCESS_GROUP_KEY = "Remote Processing Groups";
+static constexpr char const* CONFIG_REMOTE_PROCESS_GROUP_KEY_V3 = "Remote Process Groups";
+static constexpr char const* CONFIG_PROVENANCE_REPORT_KEY = "Provenance Reporting";
+static constexpr char const* CONFIG_FUNNELS_KEY = "Funnels";
+
+#define CONFIGURATION_USE_REGEX
+
+// Disable regex in EL for incompatible compilers
+#if __GNUC__ < 4 || (__GNUC__ == 4 && __GNUC_MINOR__ < 9)
+#undef CONFIGURATION_USE_REGEX
+#endif

Review Comment:
   removed



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r989813669


##########
libminifi/src/core/flow/StructuredConfiguration.cpp:
##########
@@ -0,0 +1,896 @@
+/**
+ *
+ * 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 <memory>
+#include <vector>
+#include <set>
+#include <cinttypes>
+
+#include "core/flow/StructuredConfiguration.h"
+#include "core/flow/CheckRequiredField.h"
+#include "core/flow/StructuredConnectionParser.h"
+#include "core/state/Value.h"
+#include "Defaults.h"
+#include "utils/TimeUtil.h"
+#include "utils/RegexUtils.h"
+
+namespace org::apache::nifi::minifi::core::flow {
+
+std::shared_ptr<utils::IdGenerator> StructuredConfiguration::id_generator_ = utils::IdGenerator::getIdGenerator();
+
+StructuredConfiguration::StructuredConfiguration(ConfigurationContext ctx, std::shared_ptr<logging::Logger> logger)
+    : FlowConfiguration(std::move(ctx)),
+      logger_(std::move(logger)) {}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::parseRootProcessGroup(const Node& root_flow_node) {
+  auto flow_controller_node = root_flow_node[CONFIG_FLOW_CONTROLLER_KEY];
+  auto root_group = parseProcessGroup(flow_controller_node, root_flow_node, true);
+  this->name_ = root_group->getName();
+  return root_group;
+}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::createProcessGroup(const Node& node, bool is_root) {
+  int version = 0;
+
+  checkRequiredField(node, "name", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+  auto flowName = node["name"].getString().value();
+
+  utils::Identifier uuid;
+  // assignment throws on invalid uuid
+  uuid = getOrGenerateId(node);
+
+  if (node["version"]) {
+    version = node["version"].getInt().value();
+  }
+
+  logger_->log_debug("parseRootProcessGroup: id => [%s], name => [%s]", uuid.to_string(), flowName);
+  std::unique_ptr<core::ProcessGroup> group;
+  if (is_root) {
+    group = FlowConfiguration::createRootProcessGroup(flowName, uuid, version);
+  } else {
+    group = FlowConfiguration::createSimpleProcessGroup(flowName, uuid, version);
+  }
+
+  if (node["onschedule retry interval"]) {
+    auto onScheduleRetryPeriod = node["onschedule retry interval"].getString().value();
+    logger_->log_debug("parseRootProcessGroup: onschedule retry period => [%s]", onScheduleRetryPeriod);
+
+    auto on_schedule_retry_period_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(onScheduleRetryPeriod);
+    if (on_schedule_retry_period_value.has_value() && group) {
+      logger_->log_debug("parseRootProcessGroup: onschedule retry => [%" PRId64 "] ms", on_schedule_retry_period_value->count());
+      group->setOnScheduleRetryPeriod(on_schedule_retry_period_value->count());
+    }
+  }
+
+  return group;
+}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::parseProcessGroup(const Node& headerNode, const Node& yamlNode, bool is_root) {
+  auto group = createProcessGroup(headerNode, is_root);
+  Node processorsNode = yamlNode[CONFIG_PROCESSORS_KEY];
+  Node connectionsNode = yamlNode[StructuredConnectionParser::CONFIG_CONNECTIONS_KEY];
+  Node funnelsNode = yamlNode[CONFIG_FUNNELS_KEY];
+  Node remoteProcessingGroupsNode = [&] {
+    // assignment is not supported on invalid Yaml nodes
+    Node candidate = yamlNode[CONFIG_REMOTE_PROCESS_GROUP_KEY];
+    if (candidate) {
+      return candidate;
+    }
+    return yamlNode[CONFIG_REMOTE_PROCESS_GROUP_KEY_V3];
+  }();
+  Node childProcessGroupNodeSeq = yamlNode["Process Groups"];
+
+  parseProcessorNode(processorsNode, group.get());
+  parseRemoteProcessGroup(remoteProcessingGroupsNode, group.get());
+  parseFunnels(funnelsNode, group.get());
+  // parse connections last to give feedback if the source and/or destination
+  // is not in the same process group
+  parseConnection(connectionsNode, group.get());
+
+  if (childProcessGroupNodeSeq && childProcessGroupNodeSeq.isSequence()) {
+    for (const auto childProcessGroupNode : childProcessGroupNodeSeq) {

Review Comment:
   they are used inconsistently, but since we usually return a wrapper `flow::Node` by value, it makes little difference, changed the loop variables to `const auto&` in this file, for simple variables I would vote for `const auto`, but I could change those as well



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r989731468


##########
libminifi/include/core/json/JsonNode.h:
##########
@@ -0,0 +1,248 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string>
+#include <utility>
+#include <memory>
+
+#include "core/flow/Node.h"
+#include "rapidjson/document.h"
+#include "utils/gsl.h"
+#include "utils/ValueCaster.h"
+
+namespace org::apache::nifi::minifi::core {
+
+class JsonNode : public flow::Node::NodeImpl {
+ public:
+  explicit JsonNode(const rapidjson::Value* node): node_(node) {}
+
+  explicit operator bool() const override {
+    return node_ != nullptr;
+  }
+  bool isSequence() const override {
+    return node_ ? node_->IsArray() : false;
+  }
+  bool isMap() const override {
+    return node_ ? node_->IsObject() : false;
+  }
+  bool isNull() const override {
+    return node_ ? node_->IsNull() : false;
+  }
+
+  nonstd::expected<std::string, std::exception_ptr> getString() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get string of invalid json value");
+      }
+      if (!node_->IsString()) {
+        throw std::runtime_error("Cannot get string of non-string json value");
+      }
+      return std::string{node_->GetString(), node_->GetStringLength()};
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());

Review Comment:
   `nonstd::expected` is specialized on the `std::exception_ptr` so we get the original information instead of having to define custom error codes and their to/from converters



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r1043458866


##########
libminifi/include/core/flow/Node.h:
##########
@@ -0,0 +1,139 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string_view>
+#include <tuple>
+#include <optional>
+#include <string>
+#include <memory>
+#include <utility>
+#include "nonstd/expected.hpp"
+
+namespace org::apache::nifi::minifi::core::flow {
+
+class Node {
+ public:
+  struct Cursor {
+    int line;
+    int column;
+    int pos;

Review Comment:
   added initializers



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] szaszm commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r991592543


##########
libminifi/include/core/json/JsonNode.h:
##########
@@ -0,0 +1,248 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string>
+#include <utility>
+#include <memory>
+
+#include "core/flow/Node.h"
+#include "rapidjson/document.h"
+#include "utils/gsl.h"
+#include "utils/ValueCaster.h"
+
+namespace org::apache::nifi::minifi::core {
+
+class JsonNode : public flow::Node::NodeImpl {
+ public:
+  explicit JsonNode(const rapidjson::Value* node): node_(node) {}
+
+  explicit operator bool() const override {
+    return node_ != nullptr;
+  }
+  bool isSequence() const override {
+    return node_ ? node_->IsArray() : false;
+  }
+  bool isMap() const override {
+    return node_ ? node_->IsObject() : false;
+  }
+  bool isNull() const override {
+    return node_ ? node_->IsNull() : false;
+  }
+
+  nonstd::expected<std::string, std::exception_ptr> getString() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get string of invalid json value");
+      }
+      if (!node_->IsString()) {
+        throw std::runtime_error("Cannot get string of non-string json value");
+      }
+      return std::string{node_->GetString(), node_->GetStringLength()};
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());
+    }
+  }
+
+  nonstd::expected<int, std::exception_ptr> getInt() const override {
+    return getNumber<int>("int");
+  }
+  nonstd::expected<unsigned int, std::exception_ptr> getUInt() const override {
+    return getNumber<unsigned int>("unsigned int");
+  }
+  nonstd::expected<int64_t, std::exception_ptr> getInt64() const override {
+    return getNumber<int64_t>("int64_t");
+  }
+  nonstd::expected<uint64_t, std::exception_ptr> getUInt64() const override {
+    return getNumber<uint64_t>("uint64_t");
+  }
+
+  nonstd::expected<bool, std::exception_ptr> getBool() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get bool of invalid json value");
+      }
+      if (!node_->IsBool()) {
+        throw std::runtime_error("Cannot get bool of non-bool json value");
+      }
+      return node_->GetBool();
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());
+    }
+  }
+
+  std::string getDebugString() const override {
+    if (!node_) return "<invalid>";
+    if (node_->IsObject()) return "<Map>";
+    if (node_->IsArray()) return "<Array>";
+    if (node_->IsNull()) return "null";
+    if (node_->IsInt()) return std::to_string(node_->GetInt());
+    if (node_->IsUint()) return std::to_string(node_->GetUint());
+    if (node_->IsInt64()) return std::to_string(node_->GetInt64());
+    if (node_->IsUint64()) return std::to_string(node_->GetUint64());

Review Comment:
   Just use int64_t to represent all integers in the config. I think JSON only has one number type, and that's floating point, but maybe having some special handling for integer types makes sense. But supporting 4 separate integer types probably adds more complexity than value.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
fgerlits commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r980174559


##########
libminifi/include/core/json/JsonConfiguration.h:
##########
@@ -0,0 +1,71 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <memory>
+#include <optional>
+#include <string>
+#include <unordered_set>
+
+#include "core/FlowConfiguration.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "core/ProcessorConfig.h"
+#include "Exception.h"
+#include "io/StreamFactory.h"
+#include "io/validation.h"
+#include "sitetosite/SiteToSite.h"
+#include "utils/Id.h"
+#include "utils/StringUtils.h"
+#include "utils/file/FileSystem.h"
+#include "core/flow/StructuredConfiguration.h"
+
+class JsonConfigurationTestAccessor;
+
+namespace org::apache::nifi::minifi::core {
+
+class JsonConfiguration : public StructuredConfiguration {
+ public:
+  explicit JsonConfiguration(ConfigurationContext ctx);
+
+  ~JsonConfiguration() override = default;
+
+  /**
+   * Returns a shared pointer to a ProcessGroup object containing the

Review Comment:
   I would have removed the whole comment blocks, as they look a bit weird now.  But if you think the remaining parts of the comments are useful, I don't mind keeping them.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r979881572


##########
extensions/standard-processors/tests/unit/YamlConnectionParserTest.cpp:
##########
@@ -53,26 +56,21 @@ TEST_CASE("Connections components are parsed from yaml", "[YamlConfiguration]")
           "- something_else\n" };
       expectations = { { "success", "" }, { "failure", "" }, { "something_else", "" } };
     }
-    YAML::Node connection_node = YAML::Load(serialized_yaml);
-    YamlConnectionParser yaml_connection_parser(connection_node, "test_node", parent_ptr, logger);
-    yaml_connection_parser.configureConnectionSourceRelationshipsFromYaml(*connection);
+    YAML::Node yaml_node = YAML::Load(serialized_yaml);
+    flow::Node connection_node{std::make_shared<YamlNode>(yaml_node)};
+    StructuredConnectionParser yaml_connection_parser(connection_node, "test_node", parent_ptr, logger);
+    yaml_connection_parser.configureConnectionSourceRelationships(*connection);
     const std::set<core::Relationship>& relationships = connection->getRelationships();
     REQUIRE(expectations == relationships);
   }
   SECTION("Queue size limits are read") {
-    YAML::Node connection_node = YAML::Load(std::string {
+    YAML::Node yaml_node = YAML::Load(std::string {
         "max work queue size: 231\n"
         "max work queue data size: 12 MB\n" });
     YamlConnectionParser yaml_connection_parser(connection_node, "test_node", parent_ptr, logger);
     REQUIRE(231 == yaml_connection_parser.getWorkQueueSizeFromYaml());
     REQUIRE(12582912 == yaml_connection_parser.getWorkQueueDataSizeFromYaml());  // 12 * 1024 * 1024 B
   }
-  SECTION("Queue swap threshold is read") {
-    YAML::Node connection_node = YAML::Load(std::string {
-        "swap threshold: 231\n" });
-    YamlConnectionParser yaml_connection_parser(connection_node, "test_node", parent_ptr, logger);
-    REQUIRE(231 == yaml_connection_parser.getSwapThresholdFromYaml());
-  }

Review Comment:
   rebase accident, readded it



##########
libminifi/src/core/flow/StructuredConnectionParser.cpp:
##########
@@ -108,20 +117,20 @@ uint64_t YamlConnectionParser::getSwapThresholdFromYaml() const {
   return 0;

Review Comment:
   changed it



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
fgerlits commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r983452582


##########
libminifi/include/core/json/JsonNode.h:
##########
@@ -0,0 +1,248 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string>
+#include <utility>
+#include <memory>
+
+#include "core/flow/Node.h"
+#include "rapidjson/document.h"
+#include "utils/gsl.h"
+#include "utils/ValueCaster.h"
+
+namespace org::apache::nifi::minifi::core {
+
+class JsonNode : public flow::Node::NodeImpl {
+ public:
+  explicit JsonNode(const rapidjson::Value* node): node_(node) {}
+
+  explicit operator bool() const override {
+    return node_ != nullptr;
+  }
+  bool isSequence() const override {
+    return node_ ? node_->IsArray() : false;
+  }
+  bool isMap() const override {
+    return node_ ? node_->IsObject() : false;
+  }
+  bool isNull() const override {
+    return node_ ? node_->IsNull() : false;
+  }
+
+  nonstd::expected<std::string, std::exception_ptr> getString() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get string of invalid json value");
+      }
+      if (!node_->IsString()) {
+        throw std::runtime_error("Cannot get string of non-string json value");
+      }
+      return std::string{node_->GetString(), node_->GetStringLength()};
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());
+    }
+  }
+
+  nonstd::expected<int, std::exception_ptr> getInt() const override {
+    return getNumber<int>("int");
+  }
+  nonstd::expected<unsigned int, std::exception_ptr> getUInt() const override {
+    return getNumber<unsigned int>("unsigned int");
+  }
+  nonstd::expected<int64_t, std::exception_ptr> getInt64() const override {
+    return getNumber<int64_t>("int64_t");
+  }
+  nonstd::expected<uint64_t, std::exception_ptr> getUInt64() const override {
+    return getNumber<uint64_t>("uint64_t");
+  }
+
+  nonstd::expected<bool, std::exception_ptr> getBool() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get bool of invalid json value");
+      }
+      if (!node_->IsBool()) {
+        throw std::runtime_error("Cannot get bool of non-bool json value");
+      }
+      return node_->GetBool();
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());
+    }
+  }
+
+  std::string getDebugString() const override {
+    if (!node_) return "<invalid>";
+    if (node_->IsObject()) return "<Map>";
+    if (node_->IsArray()) return "<Array>";
+    if (node_->IsNull()) return "null";
+    if (node_->IsInt()) return std::to_string(node_->GetInt());
+    if (node_->IsUint()) return std::to_string(node_->GetUint());
+    if (node_->IsInt64()) return std::to_string(node_->GetInt64());
+    if (node_->IsUint64()) return std::to_string(node_->GetUint64());
+    if (node_->IsTrue()) return "true";
+    if (node_->IsFalse()) return "false";
+    if (node_->IsDouble()) return std::to_string(node_->GetDouble());
+    if (node_->IsString()) return '"' + std::string(node_->GetString(), node_->GetStringLength()) + '"';
+    return "<unknown>";
+  }
+
+  size_t size() const override {
+    if (!node_) {
+      throw std::runtime_error("Cannot get size of invalid json value");
+    }
+    if (!node_->IsArray()) {
+      throw std::runtime_error("Cannot get size of non-array json value");
+    }
+    return node_->Size();
+  }
+  flow::Node::Iterator begin() const override;
+  flow::Node::Iterator end() const override;
+
+  flow::Node operator[](std::string_view key) const override {
+    if (!node_) {
+      throw std::runtime_error("Cannot get member of invalid json value");
+    }
+    if (!node_->IsObject()) {
+      return flow::Node{std::make_shared<JsonNode>(nullptr)};

Review Comment:
   we throw at other places which look similar to this; why not here?



##########
libminifi/include/core/json/JsonNode.h:
##########
@@ -0,0 +1,248 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string>
+#include <utility>
+#include <memory>
+
+#include "core/flow/Node.h"
+#include "rapidjson/document.h"
+#include "utils/gsl.h"
+#include "utils/ValueCaster.h"
+
+namespace org::apache::nifi::minifi::core {
+
+class JsonNode : public flow::Node::NodeImpl {
+ public:
+  explicit JsonNode(const rapidjson::Value* node): node_(node) {}
+
+  explicit operator bool() const override {
+    return node_ != nullptr;
+  }
+  bool isSequence() const override {
+    return node_ ? node_->IsArray() : false;
+  }
+  bool isMap() const override {
+    return node_ ? node_->IsObject() : false;
+  }
+  bool isNull() const override {
+    return node_ ? node_->IsNull() : false;
+  }
+
+  nonstd::expected<std::string, std::exception_ptr> getString() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get string of invalid json value");
+      }
+      if (!node_->IsString()) {
+        throw std::runtime_error("Cannot get string of non-string json value");
+      }
+      return std::string{node_->GetString(), node_->GetStringLength()};
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());
+    }
+  }
+
+  nonstd::expected<int, std::exception_ptr> getInt() const override {
+    return getNumber<int>("int");
+  }
+  nonstd::expected<unsigned int, std::exception_ptr> getUInt() const override {
+    return getNumber<unsigned int>("unsigned int");
+  }
+  nonstd::expected<int64_t, std::exception_ptr> getInt64() const override {
+    return getNumber<int64_t>("int64_t");
+  }
+  nonstd::expected<uint64_t, std::exception_ptr> getUInt64() const override {
+    return getNumber<uint64_t>("uint64_t");
+  }
+
+  nonstd::expected<bool, std::exception_ptr> getBool() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get bool of invalid json value");
+      }
+      if (!node_->IsBool()) {
+        throw std::runtime_error("Cannot get bool of non-bool json value");
+      }
+      return node_->GetBool();
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());
+    }
+  }
+
+  std::string getDebugString() const override {
+    if (!node_) return "<invalid>";
+    if (node_->IsObject()) return "<Map>";
+    if (node_->IsArray()) return "<Array>";
+    if (node_->IsNull()) return "null";
+    if (node_->IsInt()) return std::to_string(node_->GetInt());
+    if (node_->IsUint()) return std::to_string(node_->GetUint());
+    if (node_->IsInt64()) return std::to_string(node_->GetInt64());
+    if (node_->IsUint64()) return std::to_string(node_->GetUint64());
+    if (node_->IsTrue()) return "true";
+    if (node_->IsFalse()) return "false";
+    if (node_->IsDouble()) return std::to_string(node_->GetDouble());
+    if (node_->IsString()) return '"' + std::string(node_->GetString(), node_->GetStringLength()) + '"';
+    return "<unknown>";
+  }
+
+  size_t size() const override {
+    if (!node_) {
+      throw std::runtime_error("Cannot get size of invalid json value");
+    }
+    if (!node_->IsArray()) {
+      throw std::runtime_error("Cannot get size of non-array json value");
+    }
+    return node_->Size();
+  }
+  flow::Node::Iterator begin() const override;
+  flow::Node::Iterator end() const override;
+
+  flow::Node operator[](std::string_view key) const override {
+    if (!node_) {
+      throw std::runtime_error("Cannot get member of invalid json value");
+    }
+    if (!node_->IsObject()) {
+      return flow::Node{std::make_shared<JsonNode>(nullptr)};
+    }
+    auto it = node_->FindMember(rapidjson::Value(rapidjson::StringRef(key.data(), key.length())));
+    if (it == node_->MemberEnd()) {
+      return flow::Node{std::make_shared<JsonNode>(nullptr)};
+    }
+    return flow::Node{std::make_shared<JsonNode>(&it->value)};
+  }
+
+  std::optional<flow::Node::Cursor> getCursor() const override {
+    return std::nullopt;
+  }
+
+ private:
+  template<typename T, typename U>
+  std::optional<T> narrow(const U& value) {
+    T res = static_cast<T>(value);
+    if (static_cast<U>(res) != value) {
+      return std::nullopt;
+    }
+    return res;
+  }
+
+  template<typename T>
+  nonstd::expected<T, std::exception_ptr> getNumber(const char* type_name) const {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get " + std::string(type_name) + " of invalid json value");
+      }
+      T result;
+      if (node_->IsInt() && utils::internal::cast_if_in_range(node_->GetInt(), result)) return result;
+      if (node_->IsUint() && utils::internal::cast_if_in_range(node_->GetUint(), result)) return result;
+      if (node_->IsInt64() && utils::internal::cast_if_in_range(node_->GetInt64(), result)) return result;
+      if (node_->IsUint64() && utils::internal::cast_if_in_range(node_->GetUint64(), result)) return result;
+      throw std::runtime_error("Cannot get " + std::string(type_name) + " of non-numeric json value");

Review Comment:
   ```suggestion
         throw std::runtime_error("Cannot get " + std::string(type_name) + " from json value; it is either non-numeric or out of range");
   ```



##########
libminifi/include/core/flow/StructuredConfiguration.h:
##########
@@ -0,0 +1,227 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <memory>
+#include <optional>
+#include <string>
+#include <unordered_set>
+
+#include "core/FlowConfiguration.h"
+#include "core/logging/LoggerConfiguration.h"
+#include "core/ProcessorConfig.h"
+#include "Exception.h"
+#include "io/StreamFactory.h"
+#include "io/validation.h"
+#include "sitetosite/SiteToSite.h"
+#include "utils/Id.h"
+#include "utils/StringUtils.h"
+#include "utils/file/FileSystem.h"
+#include "core/flow/Node.h"
+
+namespace org::apache::nifi::minifi::core::flow {
+
+static constexpr char const* CONFIG_FLOW_CONTROLLER_KEY = "Flow Controller";
+static constexpr char const* CONFIG_PROCESSORS_KEY = "Processors";
+static constexpr char const* CONFIG_CONTROLLER_SERVICES_KEY = "Controller Services";
+static constexpr char const* CONFIG_REMOTE_PROCESS_GROUP_KEY = "Remote Processing Groups";
+static constexpr char const* CONFIG_REMOTE_PROCESS_GROUP_KEY_V3 = "Remote Process Groups";
+static constexpr char const* CONFIG_PROVENANCE_REPORT_KEY = "Provenance Reporting";
+static constexpr char const* CONFIG_FUNNELS_KEY = "Funnels";
+
+class StructuredConfiguration : public FlowConfiguration {
+ public:
+  StructuredConfiguration(ConfigurationContext ctx, std::shared_ptr<logging::Logger> logger);
+
+  /**
+   * Iterates all component property validation rules and checks that configured state
+   * is valid. If state is determined to be invalid, conf parsing ends and an error is raised.
+   *
+   * @param component
+   * @param component_name
+   * @param section
+   */
+  void validateComponentProperties(ConfigurableComponent& component, const std::string &component_name, const std::string &section) const;
+
+ protected:
+  /**
+   * Returns a shared pointer to a ProcessGroup object containing the
+   * flow configuration.
+   *
+   * @param root_node a pointer to a Node object containing the root
+   *                       node of the parsed document
+   * @return             the root ProcessGroup node of the flow
+   *                       configuration tree
+   */
+  std::unique_ptr<core::ProcessGroup> getRootFrom(const Node& root_node);
+
+  std::unique_ptr<core::ProcessGroup> createProcessGroup(const Node& node, bool is_root = false);
+
+  std::unique_ptr<core::ProcessGroup> parseProcessGroup(const Node& header_node, const Node& node, bool is_root = false);
+  /**
+   * Parses processors from its corresponding config node and adds
+   * them to a parent ProcessGroup. The processors_node argument must point
+   * to a Node containing the processors configuration. Processor
+   * objects will be created and added to the parent ProcessGroup specified
+   * by the parent argument.
+   *
+   * @param processors_node  the Node containing the processor configuration
+   * @param parent              the parent ProcessGroup to which the the created
+   *                            Processor should be added
+   */
+  void parseProcessorNode(const Node& processors_node, core::ProcessGroup* parent);
+
+  /**
+   * Parses a port from its corresponding config node and adds
+   * it to a parent ProcessGroup. The port_node argument must point
+   * to a Node containing the port configuration. A RemoteProcessorGroupPort
+   * object will be created a added to the parent ProcessGroup specified
+   * by the parent argument.
+   *
+   * @param port_node  the Node containing the port configuration
+   * @param parent    the parent ProcessGroup for the port
+   * @param direction the TransferDirection of the port
+   */
+  void parsePort(const Node& port_node, core::ProcessGroup* parent, sitetosite::TransferDirection direction);
+
+  /**
+   * Parses the root level node for the flow configuration and
+   * returns a ProcessGroup containing the tree of flow configuration
+   * objects.
+   *
+   * @param root_flow_node
+   * @return
+   */
+  std::unique_ptr<core::ProcessGroup> parseRootProcessGroup(const Node& root_flow_node);
+
+  void parseProcessorProperty(const Node& doc, const Node& node, std::shared_ptr<core::Processor> processor);
+
+  void parseControllerServices(const Node& controller_services_node);
+
+  /**
+   * Parses the Connections section of a configuration.
+   * The resulting Connections are added to the parent ProcessGroup.
+   *
+   * @param connection_node_seq   the Node containing the Connections section
+   *                              of the configuration
+   * @param parent                the root node of flow configuration to which
+   *                              to add the connections that are parsed
+   */
+  void parseConnection(const Node& connection_node_seq, core::ProcessGroup* parent);
+
+  /**
+   * Parses the Remote Process Group section of a configuration.
+   * The resulting Process Group is added to the parent ProcessGroup.
+   *
+   * @param rpg_node_seq  the Node containing the Remote Process Group
+   *                      section of the configuration
+   * @param parent        the root node of flow configuration to which
+   *                      to add the process groups that are parsed
+   */
+  void parseRemoteProcessGroup(const Node& rpg_node_seq, core::ProcessGroup* parent);
+
+  /**
+   * Parses the Provenance Reporting section of a configuration.
+   * The resulting Provenance Reporting processor is added to the
+   * parent ProcessGroup.
+   *
+   * @param report_node  the Node containing the provenance
+   *                      reporting configuration
+   * @param parent_group the root node of flow configuration to which
+   *                      to add the provenance reporting config
+   */
+  void parseProvenanceReporting(const Node& report_node, core::ProcessGroup* parent_group);
+
+  /**
+   * A helper function to parse the Properties Node for a processor.
+   *
+   * @param properties_node the Node containing the properties
+   * @param processor      the Processor to which to add the resulting properties
+   */
+  void parsePropertiesNode(const Node& properties_node, core::ConfigurableComponent& component, const std::string& component_name, const std::string& section);
+
+  /**
+   * Parses the Funnels section of a configuration.
+   * The resulting Funnels are added to the parent ProcessGroup.
+   *
+   * @param node   the Node containing the Funnels section
+   *                 of the configuration
+   * @param parent the root node of flow configuration to which
+   *                 to add the funnels that are parsed
+   */
+  void parseFunnels(const Node& node, core::ProcessGroup* parent);
+
+  /**
+   * A helper function for parsing or generating optional id fields.
+   *
+   * In parsing flow configurations for config schema v1, the
+   * 'id' field of most component types that contains a UUID is optional.
+   * This function will check for the existence of the specified
+   * idField in the specified node. If present, the field will be parsed
+   * as a UUID and the UUID string will be returned. If not present, a
+   * random UUID string will be generated and returned.
+   *
+   * @param node     a pointer to the Node that will be checked for the
+   *                   presence of an idField
+   * @param id_field  the string of the name of the idField to check for. This
+   *                   is optional and defaults to 'id'
+   * @return         the parsed or generated UUID string
+   */
+  std::string getOrGenerateId(const Node& node, const std::string& id_field = "id");
+
+  std::string getRequiredIdField(const Node& node, std::string_view section = "", std::string_view error_message = "");
+
+  /**
+   * This is a helper function for getting an optional value, if it exists.
+   * If it does not exist, returns the provided default value.
+   *
+   * @param node         the flow node to check
+   * @param field_name    the optional field key
+   * @param default_value the default value to use if field is not set
+   * @param section  [optional] the top level section of the config
+   *                       for the node. This is used fpr generating a
+   *                       useful info message for troubleshooting.
+   * @param info_message  [optional] the info message string to use if
+   *                       the optional field is missing. If not provided,
+   *                       a default info message will be generated.
+   */
+  std::string getOptionalField(const Node& node, const std::string& field_name, const std::string& default_value, const std::string& section = "", const std::string& info_message = "");
+
+  static std::shared_ptr<utils::IdGenerator> id_generator_;
+  std::unordered_set<std::string> uuids_;
+  std::shared_ptr<logging::Logger> logger_;

Review Comment:
   I don't understand the rationale of the derived classes passing in their loggers as constructor parameters and then storing the logger here in the parent class.  It is going to be more difficult to find where the log is coming from: if it says it's from `JsonConfiguration`, then it may be from either `JsonConfiguration` or `StructuredConfiguration`.  Why not have three separate loggers in the three classes?



##########
libminifi/src/core/json/JsonConfiguration.cpp:
##########
@@ -0,0 +1,89 @@
+/**
+ *
+ * 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 <memory>
+#include <vector>
+#include <set>
+#include <cinttypes>
+#include <variant>
+
+#include "core/json/JsonConfiguration.h"
+#include "core/json/JsonNode.h"
+#include "core/state/Value.h"
+#include "Defaults.h"
+#include "utils/TimeUtil.h"
+
+#include "rapidjson/rapidjson.h"
+#include "rapidjson/document.h"
+
+namespace org::apache::nifi::minifi::core {
+
+namespace {
+
+}  // namespace
+
+
+JsonConfiguration::JsonConfiguration(ConfigurationContext ctx)
+    : StructuredConfiguration(([&] {
+                                if (!ctx.path) {
+                                  ctx.path = DEFAULT_NIFI_CONFIG_JSON;
+                                }
+                                return std::move(ctx);
+                              })(),
+                              logging::LoggerFactory<JsonConfiguration>::getLogger()) {}
+
+std::unique_ptr<core::ProcessGroup> JsonConfiguration::getRoot() {
+  if (!config_path_) {
+    logger_->log_error("Cannot instantiate flow, no config file is set.");
+    throw Exception(ExceptionType::FLOW_EXCEPTION, "No config file specified");
+  }
+  const auto configuration = filesystem_->read(config_path_.value());
+  if (!configuration) {
+    // non-existence of flow config file is not a dealbreaker, the caller might fetch it from network
+    return nullptr;
+  }
+  try {
+    rapidjson::Document doc;
+    rapidjson::ParseResult res = doc.Parse(configuration->c_str(), configuration->length());
+    if (!res) {
+      throw std::runtime_error("Could not parse json file");
+    }
+    flow::Node root{std::make_shared<JsonNode>(&doc)};
+    return getRootFrom(root);
+  } catch(...) {
+    logger_->log_error("Invalid json configuration file");
+    throw;
+  }

Review Comment:
   This looks almost identical to `getRootFromPayload()` except for the `...` in the `catch`.  Can this block be replaced by `getRootFromPayload(*configuration)`?
   
   edit: same comment at `YamlConfiguration.cpp::getRoot()`



##########
libminifi/src/core/flow/StructuredConfiguration.cpp:
##########
@@ -0,0 +1,896 @@
+/**
+ *
+ * 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 <memory>
+#include <vector>
+#include <set>
+#include <cinttypes>
+
+#include "core/flow/StructuredConfiguration.h"
+#include "core/flow/CheckRequiredField.h"
+#include "core/flow/StructuredConnectionParser.h"
+#include "core/state/Value.h"
+#include "Defaults.h"
+#include "utils/TimeUtil.h"
+#include "utils/RegexUtils.h"
+
+namespace org::apache::nifi::minifi::core::flow {
+
+std::shared_ptr<utils::IdGenerator> StructuredConfiguration::id_generator_ = utils::IdGenerator::getIdGenerator();
+
+StructuredConfiguration::StructuredConfiguration(ConfigurationContext ctx, std::shared_ptr<logging::Logger> logger)
+    : FlowConfiguration(std::move(ctx)),
+      logger_(std::move(logger)) {}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::parseRootProcessGroup(const Node& root_flow_node) {
+  auto flow_controller_node = root_flow_node[CONFIG_FLOW_CONTROLLER_KEY];
+  auto root_group = parseProcessGroup(flow_controller_node, root_flow_node, true);
+  this->name_ = root_group->getName();
+  return root_group;
+}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::createProcessGroup(const Node& node, bool is_root) {
+  int version = 0;
+
+  checkRequiredField(node, "name", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+  auto flowName = node["name"].getString().value();
+
+  utils::Identifier uuid;
+  // assignment throws on invalid uuid
+  uuid = getOrGenerateId(node);
+
+  if (node["version"]) {
+    version = node["version"].getInt().value();
+  }
+
+  logger_->log_debug("parseRootProcessGroup: id => [%s], name => [%s]", uuid.to_string(), flowName);
+  std::unique_ptr<core::ProcessGroup> group;
+  if (is_root) {
+    group = FlowConfiguration::createRootProcessGroup(flowName, uuid, version);
+  } else {
+    group = FlowConfiguration::createSimpleProcessGroup(flowName, uuid, version);
+  }
+
+  if (node["onschedule retry interval"]) {
+    auto onScheduleRetryPeriod = node["onschedule retry interval"].getString().value();
+    logger_->log_debug("parseRootProcessGroup: onschedule retry period => [%s]", onScheduleRetryPeriod);
+
+    auto on_schedule_retry_period_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(onScheduleRetryPeriod);
+    if (on_schedule_retry_period_value.has_value() && group) {
+      logger_->log_debug("parseRootProcessGroup: onschedule retry => [%" PRId64 "] ms", on_schedule_retry_period_value->count());
+      group->setOnScheduleRetryPeriod(on_schedule_retry_period_value->count());
+    }
+  }
+
+  return group;
+}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::parseProcessGroup(const Node& headerNode, const Node& yamlNode, bool is_root) {
+  auto group = createProcessGroup(headerNode, is_root);
+  Node processorsNode = yamlNode[CONFIG_PROCESSORS_KEY];
+  Node connectionsNode = yamlNode[StructuredConnectionParser::CONFIG_CONNECTIONS_KEY];
+  Node funnelsNode = yamlNode[CONFIG_FUNNELS_KEY];
+  Node remoteProcessingGroupsNode = [&] {
+    // assignment is not supported on invalid Yaml nodes
+    Node candidate = yamlNode[CONFIG_REMOTE_PROCESS_GROUP_KEY];
+    if (candidate) {
+      return candidate;
+    }
+    return yamlNode[CONFIG_REMOTE_PROCESS_GROUP_KEY_V3];
+  }();
+  Node childProcessGroupNodeSeq = yamlNode["Process Groups"];
+
+  parseProcessorNode(processorsNode, group.get());
+  parseRemoteProcessGroup(remoteProcessingGroupsNode, group.get());
+  parseFunnels(funnelsNode, group.get());
+  // parse connections last to give feedback if the source and/or destination
+  // is not in the same process group
+  parseConnection(connectionsNode, group.get());
+
+  if (childProcessGroupNodeSeq && childProcessGroupNodeSeq.isSequence()) {
+    for (const auto childProcessGroupNode : childProcessGroupNodeSeq) {

Review Comment:
   should this be `const auto&`?  (also in a few other places in this file)



##########
libminifi/src/core/flow/StructuredConfiguration.cpp:
##########
@@ -0,0 +1,896 @@
+/**
+ *
+ * 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 <memory>
+#include <vector>
+#include <set>
+#include <cinttypes>
+
+#include "core/flow/StructuredConfiguration.h"
+#include "core/flow/CheckRequiredField.h"
+#include "core/flow/StructuredConnectionParser.h"
+#include "core/state/Value.h"
+#include "Defaults.h"
+#include "utils/TimeUtil.h"
+#include "utils/RegexUtils.h"
+
+namespace org::apache::nifi::minifi::core::flow {
+
+std::shared_ptr<utils::IdGenerator> StructuredConfiguration::id_generator_ = utils::IdGenerator::getIdGenerator();
+
+StructuredConfiguration::StructuredConfiguration(ConfigurationContext ctx, std::shared_ptr<logging::Logger> logger)
+    : FlowConfiguration(std::move(ctx)),
+      logger_(std::move(logger)) {}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::parseRootProcessGroup(const Node& root_flow_node) {
+  auto flow_controller_node = root_flow_node[CONFIG_FLOW_CONTROLLER_KEY];
+  auto root_group = parseProcessGroup(flow_controller_node, root_flow_node, true);
+  this->name_ = root_group->getName();
+  return root_group;
+}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::createProcessGroup(const Node& node, bool is_root) {
+  int version = 0;
+
+  checkRequiredField(node, "name", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+  auto flowName = node["name"].getString().value();
+
+  utils::Identifier uuid;
+  // assignment throws on invalid uuid
+  uuid = getOrGenerateId(node);
+
+  if (node["version"]) {
+    version = node["version"].getInt().value();
+  }
+
+  logger_->log_debug("parseRootProcessGroup: id => [%s], name => [%s]", uuid.to_string(), flowName);
+  std::unique_ptr<core::ProcessGroup> group;
+  if (is_root) {
+    group = FlowConfiguration::createRootProcessGroup(flowName, uuid, version);
+  } else {
+    group = FlowConfiguration::createSimpleProcessGroup(flowName, uuid, version);
+  }
+
+  if (node["onschedule retry interval"]) {
+    auto onScheduleRetryPeriod = node["onschedule retry interval"].getString().value();
+    logger_->log_debug("parseRootProcessGroup: onschedule retry period => [%s]", onScheduleRetryPeriod);
+
+    auto on_schedule_retry_period_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(onScheduleRetryPeriod);
+    if (on_schedule_retry_period_value.has_value() && group) {
+      logger_->log_debug("parseRootProcessGroup: onschedule retry => [%" PRId64 "] ms", on_schedule_retry_period_value->count());
+      group->setOnScheduleRetryPeriod(on_schedule_retry_period_value->count());
+    }
+  }
+
+  return group;
+}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::parseProcessGroup(const Node& headerNode, const Node& yamlNode, bool is_root) {
+  auto group = createProcessGroup(headerNode, is_root);
+  Node processorsNode = yamlNode[CONFIG_PROCESSORS_KEY];
+  Node connectionsNode = yamlNode[StructuredConnectionParser::CONFIG_CONNECTIONS_KEY];
+  Node funnelsNode = yamlNode[CONFIG_FUNNELS_KEY];
+  Node remoteProcessingGroupsNode = [&] {
+    // assignment is not supported on invalid Yaml nodes
+    Node candidate = yamlNode[CONFIG_REMOTE_PROCESS_GROUP_KEY];
+    if (candidate) {
+      return candidate;
+    }
+    return yamlNode[CONFIG_REMOTE_PROCESS_GROUP_KEY_V3];
+  }();
+  Node childProcessGroupNodeSeq = yamlNode["Process Groups"];
+
+  parseProcessorNode(processorsNode, group.get());
+  parseRemoteProcessGroup(remoteProcessingGroupsNode, group.get());
+  parseFunnels(funnelsNode, group.get());
+  // parse connections last to give feedback if the source and/or destination
+  // is not in the same process group
+  parseConnection(connectionsNode, group.get());
+
+  if (childProcessGroupNodeSeq && childProcessGroupNodeSeq.isSequence()) {
+    for (const auto childProcessGroupNode : childProcessGroupNodeSeq) {
+      group->addProcessGroup(parseProcessGroup(childProcessGroupNode, childProcessGroupNode));
+    }
+  }
+  return group;
+}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::getRootFrom(const Node& rootYamlNode) {
+  uuids_.clear();
+  Node controllerServiceNode = rootYamlNode[CONFIG_CONTROLLER_SERVICES_KEY];
+  Node provenanceReportNode = rootYamlNode[CONFIG_PROVENANCE_REPORT_KEY];
+
+  parseControllerServices(controllerServiceNode);
+  // Create the root process group
+  std::unique_ptr<core::ProcessGroup> root = parseRootProcessGroup(rootYamlNode);
+  parseProvenanceReporting(provenanceReportNode, root.get());
+
+  // set the controller services into the root group.
+  for (const auto& controller_service : controller_services_->getAllControllerServices()) {
+    root->addControllerService(controller_service->getName(), controller_service);
+    root->addControllerService(controller_service->getUUIDStr(), controller_service);
+  }
+
+  return root;
+}
+
+void StructuredConfiguration::parseProcessorNode(const Node& processors_node, core::ProcessGroup* parentGroup) {
+  int64_t runDurationNanos = -1;
+  utils::Identifier uuid;
+  std::unique_ptr<core::Processor> processor;
+
+  if (!parentGroup) {
+    logger_->log_error("parseProcessNodeYaml: no parent group exists");
+    return;
+  }
+
+  if (!processors_node) {
+    throw std::invalid_argument("Cannot instantiate a MiNiFi instance without a defined Processors configuration node.");
+  }
+  if (!processors_node.isSequence()) {
+    throw std::invalid_argument(
+        "Cannot instantiate a MiNiFi instance without a defined Processors configuration node.");
+  }
+  // Evaluate sequence of processors
+  for (const auto procNode : processors_node) {
+    core::ProcessorConfig procCfg;
+
+    checkRequiredField(procNode, "name", CONFIG_PROCESSORS_KEY);
+    procCfg.name = procNode["name"].getString().value();
+    procCfg.id = getOrGenerateId(procNode);
+
+    uuid = procCfg.id;
+    logger_->log_debug("parseProcessorNode: name => [%s] id => [%s]", procCfg.name, procCfg.id);
+    checkRequiredField(procNode, "class", CONFIG_PROCESSORS_KEY);
+    procCfg.javaClass = procNode["class"].getString().value();
+    logger_->log_debug("parseProcessorNode: class => [%s]", procCfg.javaClass);
+
+    // Determine the processor name only from the Java class
+    auto lastOfIdx = procCfg.javaClass.find_last_of('.');
+    if (lastOfIdx != std::string::npos) {
+      lastOfIdx++;  // if a value is found, increment to move beyond the .
+      std::string processorName = procCfg.javaClass.substr(lastOfIdx);
+      processor = this->createProcessor(processorName, procCfg.javaClass, uuid);
+    } else {
+      // Allow unqualified class names for core processors
+      processor = this->createProcessor(procCfg.javaClass, uuid);
+    }
+
+    if (!processor) {
+      logger_->log_error("Could not create a processor %s with id %s", procCfg.name, procCfg.id);
+      throw std::invalid_argument("Could not create processor " + procCfg.name);
+    }
+
+    processor->setName(procCfg.name);
+
+    processor->setFlowIdentifier(flow_version_->getFlowIdentifier());
+
+    procCfg.schedulingStrategy = getOptionalField(procNode, "scheduling strategy", DEFAULT_SCHEDULING_STRATEGY, CONFIG_PROCESSORS_KEY);
+    logger_->log_debug("parseProcessorNode: scheduling strategy => [%s]", procCfg.schedulingStrategy);
+
+    procCfg.schedulingPeriod = getOptionalField(procNode, "scheduling period", DEFAULT_SCHEDULING_PERIOD_STR, CONFIG_PROCESSORS_KEY);
+
+    logger_->log_debug("parseProcessorNode: scheduling period => [%s]", procCfg.schedulingPeriod);
+
+    if (auto tasksNode = procNode["max concurrent tasks"]) {
+      if (auto int_val = tasksNode.getUInt64()) {
+        procCfg.maxConcurrentTasks = std::to_string(int_val.value());
+      } else {
+        procCfg.maxConcurrentTasks = tasksNode.getString().value();
+      }
+      logger_->log_debug("parseProcessorNode: max concurrent tasks => [%s]", procCfg.maxConcurrentTasks);
+    }
+
+    if (procNode["penalization period"]) {
+      procCfg.penalizationPeriod = procNode["penalization period"].getString().value();
+      logger_->log_debug("parseProcessorNode: penalization period => [%s]", procCfg.penalizationPeriod);
+    }
+
+    if (procNode["yield period"]) {
+      procCfg.yieldPeriod = procNode["yield period"].getString().value();
+      logger_->log_debug("parseProcessorNode: yield period => [%s]", procCfg.yieldPeriod);
+    }
+
+    if (auto runNode = procNode["run duration nanos"]) {
+      if (auto int_val = runNode.getUInt64()) {
+        procCfg.runDurationNanos = std::to_string(int_val.value());
+      } else {
+        procCfg.runDurationNanos = runNode.getString().value();
+      }
+      logger_->log_debug("parseProcessorNode: run duration nanos => [%s]", procCfg.runDurationNanos);
+    }
+
+    // handle auto-terminated relationships
+    if (procNode["auto-terminated relationships list"]) {
+      Node autoTerminatedSequence = procNode["auto-terminated relationships list"];
+      std::vector<std::string> rawAutoTerminatedRelationshipValues;
+      if (autoTerminatedSequence.isSequence() && autoTerminatedSequence.size() > 0) {
+        for (const auto autoTerminatedRel : autoTerminatedSequence) {
+          rawAutoTerminatedRelationshipValues.push_back(autoTerminatedRel.getString().value());
+        }
+      }
+      procCfg.autoTerminatedRelationships = rawAutoTerminatedRelationshipValues;
+    }
+
+    // handle processor properties
+    if (procNode["Properties"]) {
+      Node propertiesNode = procNode["Properties"];
+      parsePropertiesNode(propertiesNode, *processor, procCfg.name, CONFIG_PROCESSORS_KEY);
+    }
+
+    // Take care of scheduling
+
+    if (procCfg.schedulingStrategy == "TIMER_DRIVEN" || procCfg.schedulingStrategy == "EVENT_DRIVEN") {
+      if (auto scheduling_period = utils::timeutils::StringToDuration<std::chrono::nanoseconds>(procCfg.schedulingPeriod)) {
+        logger_->log_debug("convert: parseProcessorNode: schedulingPeriod => [%" PRId64 "] ns", scheduling_period->count());
+        processor->setSchedulingPeriodNano(*scheduling_period);
+      }
+    } else {
+      processor->setCronPeriod(procCfg.schedulingPeriod);
+    }
+
+    if (auto penalization_period = utils::timeutils::StringToDuration<std::chrono::milliseconds>(procCfg.penalizationPeriod)) {
+      logger_->log_debug("convert: parseProcessorNode: penalizationPeriod => [%" PRId64 "] ms", penalization_period->count());
+      processor->setPenalizationPeriod(penalization_period.value());
+    }
+
+    if (auto yield_period = utils::timeutils::StringToDuration<std::chrono::milliseconds>(procCfg.yieldPeriod)) {
+      logger_->log_debug("convert: parseProcessorNode: yieldPeriod => [%" PRId64 "] ms", yield_period->count());
+      processor->setYieldPeriodMsec(yield_period.value());
+    }
+
+    // Default to running
+    processor->setScheduledState(core::RUNNING);
+
+    if (procCfg.schedulingStrategy == "TIMER_DRIVEN") {
+      processor->setSchedulingStrategy(core::TIMER_DRIVEN);
+      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
+    } else if (procCfg.schedulingStrategy == "EVENT_DRIVEN") {
+      processor->setSchedulingStrategy(core::EVENT_DRIVEN);
+      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
+    } else {
+      processor->setSchedulingStrategy(core::CRON_DRIVEN);
+      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
+    }
+
+    int32_t maxConcurrentTasks;
+    if (core::Property::StringToInt(procCfg.maxConcurrentTasks, maxConcurrentTasks)) {
+      logger_->log_debug("parseProcessorNode: maxConcurrentTasks => [%d]", maxConcurrentTasks);
+      processor->setMaxConcurrentTasks((uint8_t) maxConcurrentTasks);
+    }
+
+    if (core::Property::StringToInt(procCfg.runDurationNanos, runDurationNanos)) {
+      logger_->log_debug("parseProcessorNode: runDurationNanos => [%d]", runDurationNanos);
+      processor->setRunDurationNano(std::chrono::nanoseconds(runDurationNanos));
+    }
+
+    std::vector<core::Relationship> autoTerminatedRelationships;
+    for (auto &&relString : procCfg.autoTerminatedRelationships) {
+      core::Relationship relationship(relString, "");
+      logger_->log_debug("parseProcessorNode: autoTerminatedRelationship  => [%s]", relString);
+      autoTerminatedRelationships.push_back(relationship);
+    }
+
+    processor->setAutoTerminatedRelationships(autoTerminatedRelationships);
+
+    parentGroup->addProcessor(std::move(processor));
+  }
+}
+
+void StructuredConfiguration::parseRemoteProcessGroup(const Node& rpg_node_seq, core::ProcessGroup* parentGroup) {
+  utils::Identifier uuid;
+  std::string id;
+
+  if (!parentGroup) {
+    logger_->log_error("parseRemoteProcessGroupYaml: no parent group exists");
+    return;
+  }
+
+  if (!rpg_node_seq || !rpg_node_seq.isSequence()) {
+    return;
+  }
+  for (const auto currRpgNode : rpg_node_seq) {
+    checkRequiredField(currRpgNode, "name", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+    auto name = currRpgNode["name"].getString().value();
+    id = getOrGenerateId(currRpgNode);
+
+    logger_->log_debug("parseRemoteProcessGroupYaml: name => [%s], id => [%s]", name, id);
+
+    auto url = getOptionalField(currRpgNode, "url", "", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+
+    logger_->log_debug("parseRemoteProcessGroupYaml: url => [%s]", url);
+
+    uuid = id;
+    auto group = createRemoteProcessGroup(name, uuid);
+    group->setParent(parentGroup);
+
+    if (currRpgNode["yield period"]) {
+      auto yieldPeriod = currRpgNode["yield period"].getString().value();
+      logger_->log_debug("parseRemoteProcessGroupYaml: yield period => [%s]", yieldPeriod);
+
+      auto yield_period_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(yieldPeriod);
+      if (yield_period_value.has_value() && group) {
+        logger_->log_debug("parseRemoteProcessGroupYaml: yieldPeriod => [%" PRId64 "] ms", yield_period_value->count());
+        group->setYieldPeriodMsec(*yield_period_value);
+      }
+    }
+
+    if (currRpgNode["timeout"]) {
+      auto timeout = currRpgNode["timeout"].getString().value();
+      logger_->log_debug("parseRemoteProcessGroupYaml: timeout => [%s]", timeout);
+
+      auto timeout_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(timeout);
+      if (timeout_value.has_value() && group) {
+        logger_->log_debug("parseRemoteProcessGroupYaml: timeoutValue => [%" PRId64 "] ms", timeout_value->count());
+        group->setTimeout(timeout_value->count());
+      }
+    }
+
+    if (currRpgNode["local network interface"]) {
+      auto interface = currRpgNode["local network interface"].getString().value();
+      logger_->log_debug("parseRemoteProcessGroupYaml: local network interface => [%s]", interface);
+      group->setInterface(interface);
+    }
+
+    if (currRpgNode["transport protocol"]) {
+      auto transport_protocol = currRpgNode["transport protocol"].getString().value();
+      logger_->log_debug("parseRemoteProcessGroupYaml: transport protocol => [%s]", transport_protocol);
+      if (transport_protocol == "HTTP") {
+        group->setTransportProtocol(transport_protocol);
+        if (currRpgNode["proxy host"]) {
+          auto http_proxy_host = currRpgNode["proxy host"].getString().value();
+          logger_->log_debug("parseRemoteProcessGroupYaml: proxy host => [%s]", http_proxy_host);
+          group->setHttpProxyHost(http_proxy_host);
+          if (currRpgNode["proxy user"]) {
+            auto http_proxy_username = currRpgNode["proxy user"].getString().value();
+            logger_->log_debug("parseRemoteProcessGroupYaml: proxy user => [%s]", http_proxy_username);
+            group->setHttpProxyUserName(http_proxy_username);
+          }
+          if (currRpgNode["proxy password"]) {
+            auto http_proxy_password = currRpgNode["proxy password"].getString().value();
+            logger_->log_debug("parseRemoteProcessGroupYaml: proxy password => [%s]", http_proxy_password);
+            group->setHttpProxyPassWord(http_proxy_password);
+          }
+          if (currRpgNode["proxy port"]) {
+            auto http_proxy_port = currRpgNode["proxy port"].getString().value();
+            int32_t port;
+            if (core::Property::StringToInt(http_proxy_port, port)) {
+              logger_->log_debug("parseRemoteProcessGroupYaml: proxy port => [%d]", port);
+              group->setHttpProxyPort(port);
+            }
+          }
+        }
+      } else if (transport_protocol == "RAW") {
+        group->setTransportProtocol(transport_protocol);
+      } else {
+        std::stringstream stream;
+        stream << "Invalid transport protocol " << transport_protocol;
+        throw minifi::Exception(ExceptionType::SITE2SITE_EXCEPTION, stream.str().c_str());
+      }
+    }
+
+    group->setTransmitting(true);
+    group->setURL(url);
+
+    checkRequiredField(currRpgNode, "Input Ports", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+    auto inputPorts = currRpgNode["Input Ports"];
+    if (inputPorts && inputPorts.isSequence()) {
+      for (const auto currPort : inputPorts) {
+        parsePort(currPort, group.get(), sitetosite::SEND);
+      }  // for node
+    }
+    auto outputPorts = currRpgNode["Output Ports"];
+    if (outputPorts && outputPorts.isSequence()) {
+      for (const auto currPort : outputPorts) {
+        logger_->log_debug("Got a current port, iterating...");
+
+        parsePort(currPort, group.get(), sitetosite::RECEIVE);
+      }  // for node
+    }
+    parentGroup->addProcessGroup(std::move(group));
+  }
+}
+
+void StructuredConfiguration::parseProvenanceReporting(const Node& node, core::ProcessGroup* parent_group) {
+  utils::Identifier port_uuid;
+
+  if (!parent_group) {
+    logger_->log_error("parseProvenanceReportingYaml: no parent group exists");
+    return;
+  }
+
+  if (!node || node.isNull()) {
+    logger_->log_debug("no provenance reporting task specified");
+    return;
+  }
+
+  auto reportTask = createProvenanceReportTask();
+
+  checkRequiredField(node, "scheduling strategy", CONFIG_PROVENANCE_REPORT_KEY);
+  auto schedulingStrategyStr = node["scheduling strategy"].getString().value();
+  checkRequiredField(node, "scheduling period", CONFIG_PROVENANCE_REPORT_KEY);
+  auto schedulingPeriodStr = node["scheduling period"].getString().value();
+
+  if (auto scheduling_period = utils::timeutils::StringToDuration<std::chrono::nanoseconds>(schedulingPeriodStr)) {
+    logger_->log_debug("ProvenanceReportingTask schedulingPeriod %" PRId64 " ns", scheduling_period->count());
+    reportTask->setSchedulingPeriodNano(*scheduling_period);
+  }
+
+  if (schedulingStrategyStr == "TIMER_DRIVEN") {
+    reportTask->setSchedulingStrategy(core::TIMER_DRIVEN);
+    logger_->log_debug("ProvenanceReportingTask scheduling strategy %s", schedulingStrategyStr);
+  } else {
+    throw std::invalid_argument("Invalid scheduling strategy " + schedulingStrategyStr);
+  }
+
+  int64_t lvalue;
+  if (node["host"] && node["port"]) {
+    auto hostStr = node["host"].getString().value();
+
+    std::string portStr;
+    if (auto int_val = node["port"].getInt()) {
+      portStr = std::to_string(int_val.value());
+    } else {
+      portStr = node["port"].getString().value();
+    }

Review Comment:
   We do this sort of thing a lot, and what used to be 1 line is now 6 lines.
   
   I think it would be worth adding a `getScalarAsString()` function (or something with a better name) which mimics the old `as<std::string>()` function.  It would make the interface less elegant, but more usable.
   
   Any other solution which makes this 1 line instead of 6 would be fine, too.



##########
libminifi/src/core/flow/StructuredConnectionParser.cpp:
##########
@@ -46,18 +46,18 @@ void YamlConnectionParser::addFunnelRelationshipToConnection(minifi::Connection&
   }
 }
 
-void YamlConnectionParser::configureConnectionSourceRelationshipsFromYaml(minifi::Connection& connection) const {
+void StructuredConnectionParser::configureConnectionSourceRelationships(minifi::Connection& connection) const {
   // Configure connection source
-  if (connectionNode_.as<YAML::Node>()["source relationship name"] && !connectionNode_["source relationship name"].as<std::string>().empty()) {
-    addNewRelationshipToConnection(connectionNode_["source relationship name"].as<std::string>(), connection);
-  } else if (connectionNode_.as<YAML::Node>()["source relationship names"]) {
+  if (connectionNode_["source relationship name"] && !connectionNode_["source relationship name"].getString().value().empty()) {
+    addNewRelationshipToConnection(connectionNode_["source relationship name"].getString().value(), connection);
+  } else if (connectionNode_["source relationship names"]) {
     auto relList = connectionNode_["source relationship names"];

Review Comment:
   ```suggestion
       const auto& relList = connectionNode_["source relationship names"];
   ```
   ?



##########
libminifi/src/core/json/JsonConfiguration.cpp:
##########
@@ -0,0 +1,89 @@
+/**
+ *
+ * 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 <memory>
+#include <vector>
+#include <set>
+#include <cinttypes>
+#include <variant>
+
+#include "core/json/JsonConfiguration.h"
+#include "core/json/JsonNode.h"
+#include "core/state/Value.h"
+#include "Defaults.h"
+#include "utils/TimeUtil.h"
+
+#include "rapidjson/rapidjson.h"
+#include "rapidjson/document.h"
+
+namespace org::apache::nifi::minifi::core {
+
+namespace {
+
+}  // namespace
+
+
+JsonConfiguration::JsonConfiguration(ConfigurationContext ctx)
+    : StructuredConfiguration(([&] {
+                                if (!ctx.path) {
+                                  ctx.path = DEFAULT_NIFI_CONFIG_JSON;
+                                }
+                                return std::move(ctx);
+                              })(),
+                              logging::LoggerFactory<JsonConfiguration>::getLogger()) {}
+
+std::unique_ptr<core::ProcessGroup> JsonConfiguration::getRoot() {
+  if (!config_path_) {
+    logger_->log_error("Cannot instantiate flow, no config file is set.");
+    throw Exception(ExceptionType::FLOW_EXCEPTION, "No config file specified");
+  }
+  const auto configuration = filesystem_->read(config_path_.value());
+  if (!configuration) {
+    // non-existence of flow config file is not a dealbreaker, the caller might fetch it from network
+    return nullptr;
+  }
+  try {
+    rapidjson::Document doc;
+    rapidjson::ParseResult res = doc.Parse(configuration->c_str(), configuration->length());
+    if (!res) {
+      throw std::runtime_error("Could not parse json file");
+    }
+    flow::Node root{std::make_shared<JsonNode>(&doc)};
+    return getRootFrom(root);
+  } catch(...) {
+    logger_->log_error("Invalid json configuration file");
+    throw;
+  }
+}
+
+std::unique_ptr<core::ProcessGroup> JsonConfiguration::getRootFromPayload(const std::string &yamlConfigPayload) {

Review Comment:
   stray `yaml` (sub)string here



##########
libminifi/src/core/yaml/YamlConfiguration.cpp:
##########
@@ -22,900 +22,66 @@
 #include <cinttypes>
 
 #include "core/yaml/YamlConfiguration.h"
-#include "core/yaml/CheckRequiredField.h"
-#include "core/yaml/YamlConnectionParser.h"
 #include "core/state/Value.h"
 #include "Defaults.h"
 #include "utils/TimeUtil.h"
-
-#ifdef YAML_CONFIGURATION_USE_REGEX
+#include "yaml-cpp/yaml.h"
+#include "core/yaml/YamlNode.h"
 #include "utils/RegexUtils.h"
-#endif  // YAML_CONFIGURATION_USE_REGEX
 
 namespace org::apache::nifi::minifi::core {
 
-std::shared_ptr<utils::IdGenerator> YamlConfiguration::id_generator_ = utils::IdGenerator::getIdGenerator();
-
-YamlConfiguration::YamlConfiguration(const std::shared_ptr<core::Repository>& repo, const std::shared_ptr<core::Repository>& flow_file_repo,
-                                     const std::shared_ptr<core::ContentRepository>& content_repo, const std::shared_ptr<io::StreamFactory>& stream_factory,
-                                     const std::shared_ptr<Configure>& configuration, const std::optional<std::string>& path,
-                                     const std::shared_ptr<utils::file::FileSystem>& filesystem)
-    : FlowConfiguration(repo, flow_file_repo, content_repo, stream_factory, configuration,
-                        path.value_or(DEFAULT_NIFI_CONFIG_YML), filesystem),
-      stream_factory_(stream_factory),
-      logger_(logging::LoggerFactory<YamlConfiguration>::getLogger()) {}
-
-std::unique_ptr<core::ProcessGroup> YamlConfiguration::parseRootProcessGroupYaml(const YAML::Node& rootFlowNode) {
-  auto flowControllerNode = rootFlowNode[CONFIG_YAML_FLOW_CONTROLLER_KEY];
-  auto rootGroup = parseProcessGroupYaml(flowControllerNode, rootFlowNode, true);
-  this->name_ = rootGroup->getName();
-  return rootGroup;
-}
-
-std::unique_ptr<core::ProcessGroup> YamlConfiguration::createProcessGroup(const YAML::Node& yamlNode, bool is_root) {
-  int version = 0;
-
-  yaml::checkRequiredField(yamlNode, "name", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
-  auto flowName = yamlNode["name"].as<std::string>();
-
-  utils::Identifier uuid;
-  // assignment throws on invalid uuid
-  uuid = getOrGenerateId(yamlNode);
-
-  if (yamlNode["version"]) {
-    version = yamlNode["version"].as<int>();
-  }
-
-  logger_->log_debug("parseRootProcessGroup: id => [%s], name => [%s]", uuid.to_string(), flowName);
-  std::unique_ptr<core::ProcessGroup> group;
-  if (is_root) {
-    group = FlowConfiguration::createRootProcessGroup(flowName, uuid, version);
-  } else {
-    group = FlowConfiguration::createSimpleProcessGroup(flowName, uuid, version);
-  }
-
-  if (yamlNode["onschedule retry interval"]) {
-    auto onScheduleRetryPeriod = yamlNode["onschedule retry interval"].as<std::string>();
-    logger_->log_debug("parseRootProcessGroup: onschedule retry period => [%s]", onScheduleRetryPeriod);
-
-    auto on_schedule_retry_period_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(onScheduleRetryPeriod);
-    if (on_schedule_retry_period_value.has_value() && group) {
-      logger_->log_debug("parseRootProcessGroup: onschedule retry => [%" PRId64 "] ms", on_schedule_retry_period_value->count());
-      group->setOnScheduleRetryPeriod(on_schedule_retry_period_value->count());
-    }
-  }
-
-  return group;
-}
-
-std::unique_ptr<core::ProcessGroup> YamlConfiguration::parseProcessGroupYaml(const YAML::Node& headerNode, const YAML::Node& yamlNode, bool is_root) {
-  auto group = createProcessGroup(headerNode, is_root);
-  YAML::Node processorsNode = yamlNode[CONFIG_YAML_PROCESSORS_KEY];
-  YAML::Node connectionsNode = yamlNode[yaml::YamlConnectionParser::CONFIG_YAML_CONNECTIONS_KEY];
-  YAML::Node funnelsNode = yamlNode[CONFIG_YAML_FUNNELS_KEY];
-  YAML::Node remoteProcessingGroupsNode = [&] {
-    // assignment is not supported on invalid Yaml nodes
-    YAML::Node candidate = yamlNode[CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY];
-    if (candidate) {
-      return candidate;
-    }
-    return yamlNode[CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY_V3];
-  }();
-  YAML::Node childProcessGroupNodeSeq = yamlNode["Process Groups"];
-
-  parseProcessorNodeYaml(processorsNode, group.get());
-  parseRemoteProcessGroupYaml(remoteProcessingGroupsNode, group.get());
-  parseFunnelsYaml(funnelsNode, group.get());
-  // parse connections last to give feedback if the source and/or destination
-  // is not in the same process group
-  parseConnectionYaml(connectionsNode, group.get());
-
-  if (childProcessGroupNodeSeq && childProcessGroupNodeSeq.IsSequence()) {
-    for (YAML::const_iterator it = childProcessGroupNodeSeq.begin(); it != childProcessGroupNodeSeq.end(); ++it) {
-      auto childProcessGroupNode = it->as<YAML::Node>();
-      group->addProcessGroup(parseProcessGroupYaml(childProcessGroupNode, childProcessGroupNode));
-    }
-  }
-  return group;
-}
-
-std::unique_ptr<core::ProcessGroup> YamlConfiguration::getYamlRoot(const YAML::Node& rootYamlNode) {
-  uuids_.clear();
-  YAML::Node controllerServiceNode = rootYamlNode[CONFIG_YAML_CONTROLLER_SERVICES_KEY];
-  YAML::Node provenanceReportNode = rootYamlNode[CONFIG_YAML_PROVENANCE_REPORT_KEY];
-
-  parseControllerServices(controllerServiceNode);
-  // Create the root process group
-  std::unique_ptr<core::ProcessGroup> root = parseRootProcessGroupYaml(rootYamlNode);
-  parseProvenanceReportingYaml(provenanceReportNode, root.get());
-
-  // set the controller services into the root group.
-  for (const auto& controller_service : controller_services_->getAllControllerServices()) {
-    root->addControllerService(controller_service->getName(), controller_service);
-    root->addControllerService(controller_service->getUUIDStr(), controller_service);
-  }
-
-  return root;
-}
-
-void YamlConfiguration::parseProcessorNodeYaml(const YAML::Node& processorsNode, core::ProcessGroup* parentGroup) {
-  int64_t runDurationNanos = -1;
-  utils::Identifier uuid;
-  std::unique_ptr<core::Processor> processor;
-
-  if (!parentGroup) {
-    logger_->log_error("parseProcessNodeYaml: no parent group exists");
-    return;
-  }
-
-  if (!processorsNode) {
-    throw std::invalid_argument("Cannot instantiate a MiNiFi instance without a defined Processors configuration node.");
-  }
-  if (!processorsNode.IsSequence()) {
-    throw std::invalid_argument(
-        "Cannot instantiate a MiNiFi instance without a defined Processors configuration node.");
-  }
-  // Evaluate sequence of processors
-  for (YAML::const_iterator iter = processorsNode.begin(); iter != processorsNode.end(); ++iter) {
-    core::ProcessorConfig procCfg;
-    const auto procNode = iter->as<YAML::Node>();
-
-    yaml::checkRequiredField(procNode, "name", CONFIG_YAML_PROCESSORS_KEY);
-    procCfg.name = procNode["name"].as<std::string>();
-    procCfg.id = getOrGenerateId(procNode);
-
-    uuid = procCfg.id;
-    logger_->log_debug("parseProcessorNode: name => [%s] id => [%s]", procCfg.name, procCfg.id);
-    yaml::checkRequiredField(procNode, "class", CONFIG_YAML_PROCESSORS_KEY);
-    procCfg.javaClass = procNode["class"].as<std::string>();
-    logger_->log_debug("parseProcessorNode: class => [%s]", procCfg.javaClass);
-
-    // Determine the processor name only from the Java class
-    auto lastOfIdx = procCfg.javaClass.find_last_of('.');
-    if (lastOfIdx != std::string::npos) {
-      lastOfIdx++;  // if a value is found, increment to move beyond the .
-      std::string processorName = procCfg.javaClass.substr(lastOfIdx);
-      processor = this->createProcessor(processorName, procCfg.javaClass, uuid);
-    } else {
-      // Allow unqualified class names for core processors
-      processor = this->createProcessor(procCfg.javaClass, uuid);
-    }
-
-    if (!processor) {
-      logger_->log_error("Could not create a processor %s with id %s", procCfg.name, procCfg.id);
-      throw std::invalid_argument("Could not create processor " + procCfg.name);
-    }
-
-    processor->setName(procCfg.name);
-
-    processor->setFlowIdentifier(flow_version_->getFlowIdentifier());
-
-    auto strategyNode = getOptionalField(procNode, "scheduling strategy", YAML::Node(DEFAULT_SCHEDULING_STRATEGY),
-    CONFIG_YAML_PROCESSORS_KEY);
-    procCfg.schedulingStrategy = strategyNode.as<std::string>();
-    logger_->log_debug("parseProcessorNode: scheduling strategy => [%s]", procCfg.schedulingStrategy);
-
-    auto periodNode = getOptionalField(procNode, "scheduling period", YAML::Node(DEFAULT_SCHEDULING_PERIOD_STR),
-    CONFIG_YAML_PROCESSORS_KEY);
-
-    procCfg.schedulingPeriod = periodNode.as<std::string>();
-    logger_->log_debug("parseProcessorNode: scheduling period => [%s]", procCfg.schedulingPeriod);
-
-    if (procNode["max concurrent tasks"]) {
-      procCfg.maxConcurrentTasks = procNode["max concurrent tasks"].as<std::string>();
-      logger_->log_debug("parseProcessorNode: max concurrent tasks => [%s]", procCfg.maxConcurrentTasks);
-    }
-
-    if (procNode["penalization period"]) {
-      procCfg.penalizationPeriod = procNode["penalization period"].as<std::string>();
-      logger_->log_debug("parseProcessorNode: penalization period => [%s]", procCfg.penalizationPeriod);
-    }
-
-    if (procNode["yield period"]) {
-      procCfg.yieldPeriod = procNode["yield period"].as<std::string>();
-      logger_->log_debug("parseProcessorNode: yield period => [%s]", procCfg.yieldPeriod);
-    }
-
-    if (procNode["run duration nanos"]) {
-      procCfg.runDurationNanos = procNode["run duration nanos"].as<std::string>();
-      logger_->log_debug("parseProcessorNode: run duration nanos => [%s]", procCfg.runDurationNanos);
-    }
-
-    // handle auto-terminated relationships
-    if (procNode["auto-terminated relationships list"]) {
-      YAML::Node autoTerminatedSequence = procNode["auto-terminated relationships list"];
-      std::vector<std::string> rawAutoTerminatedRelationshipValues;
-      if (autoTerminatedSequence.IsSequence() && !autoTerminatedSequence.IsNull() && autoTerminatedSequence.size() > 0) {
-        for (YAML::const_iterator relIter = autoTerminatedSequence.begin(); relIter != autoTerminatedSequence.end(); ++relIter) {
-          auto autoTerminatedRel = relIter->as<std::string>();
-          rawAutoTerminatedRelationshipValues.push_back(autoTerminatedRel);
-        }
-      }
-      procCfg.autoTerminatedRelationships = rawAutoTerminatedRelationshipValues;
-    }
-
-    // handle processor properties
-    if (procNode["Properties"]) {
-      YAML::Node propertiesNode = procNode["Properties"];
-      parsePropertiesNodeYaml(propertiesNode, *processor, procCfg.name, CONFIG_YAML_PROCESSORS_KEY);
-    }
-
-    // Take care of scheduling
-
-    if (procCfg.schedulingStrategy == "TIMER_DRIVEN" || procCfg.schedulingStrategy == "EVENT_DRIVEN") {
-      if (auto scheduling_period = utils::timeutils::StringToDuration<std::chrono::nanoseconds>(procCfg.schedulingPeriod)) {
-        logger_->log_debug("convert: parseProcessorNode: schedulingPeriod => [%" PRId64 "] ns", scheduling_period->count());
-        processor->setSchedulingPeriodNano(*scheduling_period);
-      }
-    } else {
-      processor->setCronPeriod(procCfg.schedulingPeriod);
-    }
-
-    if (auto penalization_period = utils::timeutils::StringToDuration<std::chrono::milliseconds>(procCfg.penalizationPeriod)) {
-      logger_->log_debug("convert: parseProcessorNode: penalizationPeriod => [%" PRId64 "] ms", penalization_period->count());
-      processor->setPenalizationPeriod(penalization_period.value());
-    }
-
-    if (auto yield_period = utils::timeutils::StringToDuration<std::chrono::milliseconds>(procCfg.yieldPeriod)) {
-      logger_->log_debug("convert: parseProcessorNode: yieldPeriod => [%" PRId64 "] ms", yield_period->count());
-      processor->setYieldPeriodMsec(yield_period.value());
-    }
-
-    // Default to running
-    processor->setScheduledState(core::RUNNING);
-
-    if (procCfg.schedulingStrategy == "TIMER_DRIVEN") {
-      processor->setSchedulingStrategy(core::TIMER_DRIVEN);
-      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
-    } else if (procCfg.schedulingStrategy == "EVENT_DRIVEN") {
-      processor->setSchedulingStrategy(core::EVENT_DRIVEN);
-      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
-    } else {
-      processor->setSchedulingStrategy(core::CRON_DRIVEN);
-      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
-    }
-
-    int32_t maxConcurrentTasks;
-    if (core::Property::StringToInt(procCfg.maxConcurrentTasks, maxConcurrentTasks)) {
-      logger_->log_debug("parseProcessorNode: maxConcurrentTasks => [%d]", maxConcurrentTasks);
-      processor->setMaxConcurrentTasks((uint8_t) maxConcurrentTasks);
-    }
-
-    if (core::Property::StringToInt(procCfg.runDurationNanos, runDurationNanos)) {
-      logger_->log_debug("parseProcessorNode: runDurationNanos => [%d]", runDurationNanos);
-      processor->setRunDurationNano(std::chrono::nanoseconds(runDurationNanos));
-    }
-
-    std::vector<core::Relationship> autoTerminatedRelationships;
-    for (auto &&relString : procCfg.autoTerminatedRelationships) {
-      core::Relationship relationship(relString, "");
-      logger_->log_debug("parseProcessorNode: autoTerminatedRelationship  => [%s]", relString);
-      autoTerminatedRelationships.push_back(relationship);
-    }
-
-    processor->setAutoTerminatedRelationships(autoTerminatedRelationships);
-
-    parentGroup->addProcessor(std::move(processor));
-  }
-}
-
-void YamlConfiguration::parseRemoteProcessGroupYaml(const YAML::Node& rpgNode, core::ProcessGroup* parentGroup) {
-  utils::Identifier uuid;
-  std::string id;
-
-  if (!parentGroup) {
-    logger_->log_error("parseRemoteProcessGroupYaml: no parent group exists");
-    return;
-  }
-
-  if (!rpgNode || !rpgNode.IsSequence()) {
-    return;
-  }
-  for (YAML::const_iterator iter = rpgNode.begin(); iter != rpgNode.end(); ++iter) {
-    auto currRpgNode = iter->as<YAML::Node>();
-
-    yaml::checkRequiredField(currRpgNode, "name", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
-    auto name = currRpgNode["name"].as<std::string>();
-    id = getOrGenerateId(currRpgNode);
-
-    logger_->log_debug("parseRemoteProcessGroupYaml: name => [%s], id => [%s]", name, id);
-
-    auto urlNode = getOptionalField(currRpgNode, "url", YAML::Node(""),
-    CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
-
-    auto url = urlNode.as<std::string>();
-    logger_->log_debug("parseRemoteProcessGroupYaml: url => [%s]", url);
-
-    uuid = id;
-    auto group = this->createRemoteProcessGroup(name, uuid);
-    group->setParent(parentGroup);
-
-    if (currRpgNode["yield period"]) {
-      auto yieldPeriod = currRpgNode["yield period"].as<std::string>();
-      logger_->log_debug("parseRemoteProcessGroupYaml: yield period => [%s]", yieldPeriod);
-
-      auto yield_period_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(yieldPeriod);
-      if (yield_period_value.has_value() && group) {
-        logger_->log_debug("parseRemoteProcessGroupYaml: yieldPeriod => [%" PRId64 "] ms", yield_period_value->count());
-        group->setYieldPeriodMsec(*yield_period_value);
-      }
-    }
-
-    if (currRpgNode["timeout"]) {
-      auto timeout = currRpgNode["timeout"].as<std::string>();
-      logger_->log_debug("parseRemoteProcessGroupYaml: timeout => [%s]", timeout);
-
-      auto timeout_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(timeout);
-      if (timeout_value.has_value() && group) {
-        logger_->log_debug("parseRemoteProcessGroupYaml: timeoutValue => [%" PRId64 "] ms", timeout_value->count());
-        group->setTimeout(timeout_value->count());
-      }
-    }
-
-    if (currRpgNode["local network interface"]) {
-      auto interface = currRpgNode["local network interface"].as<std::string>();
-      logger_->log_debug("parseRemoteProcessGroupYaml: local network interface => [%s]", interface);
-      group->setInterface(interface);
-    }
-
-    if (currRpgNode["transport protocol"]) {
-      auto transport_protocol = currRpgNode["transport protocol"].as<std::string>();
-      logger_->log_debug("parseRemoteProcessGroupYaml: transport protocol => [%s]", transport_protocol);
-      if (transport_protocol == "HTTP") {
-        group->setTransportProtocol(transport_protocol);
-        if (currRpgNode["proxy host"]) {
-          auto http_proxy_host = currRpgNode["proxy host"].as<std::string>();
-          logger_->log_debug("parseRemoteProcessGroupYaml: proxy host => [%s]", http_proxy_host);
-          group->setHttpProxyHost(http_proxy_host);
-          if (currRpgNode["proxy user"]) {
-            auto http_proxy_username = currRpgNode["proxy user"].as<std::string>();
-            logger_->log_debug("parseRemoteProcessGroupYaml: proxy user => [%s]", http_proxy_username);
-            group->setHttpProxyUserName(http_proxy_username);
-          }
-          if (currRpgNode["proxy password"]) {
-            auto http_proxy_password = currRpgNode["proxy password"].as<std::string>();
-            logger_->log_debug("parseRemoteProcessGroupYaml: proxy password => [%s]", http_proxy_password);
-            group->setHttpProxyPassWord(http_proxy_password);
+YamlConfiguration::YamlConfiguration(ConfigurationContext ctx)
+    : StructuredConfiguration(([&] {
+          if (!ctx.path) {
+            ctx.path = DEFAULT_NIFI_CONFIG_YML;
           }
-          if (currRpgNode["proxy port"]) {
-            auto http_proxy_port = currRpgNode["proxy port"].as<std::string>();
-            int32_t port;
-            if (core::Property::StringToInt(http_proxy_port, port)) {
-              logger_->log_debug("parseRemoteProcessGroupYaml: proxy port => [%d]", port);
-              group->setHttpProxyPort(port);
-            }
-          }
-        }
-      } else if (transport_protocol == "RAW") {
-        group->setTransportProtocol(transport_protocol);
-      } else {
-        std::stringstream stream;
-        stream << "Invalid transport protocol " << transport_protocol;
-        throw minifi::Exception(ExceptionType::SITE2SITE_EXCEPTION, stream.str().c_str());
-      }
-    }
-
-    group->setTransmitting(true);
-    group->setURL(url);
-
-    yaml::checkRequiredField(currRpgNode, "Input Ports", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
-    auto inputPorts = currRpgNode["Input Ports"].as<YAML::Node>();
-    if (inputPorts && inputPorts.IsSequence()) {
-      for (YAML::const_iterator portIter = inputPorts.begin(); portIter != inputPorts.end(); ++portIter) {
-        auto currPort = portIter->as<YAML::Node>();
-
-        this->parsePortYaml(currPort, group.get(), sitetosite::SEND);
-      }  // for node
-    }
-    auto outputPorts = currRpgNode["Output Ports"].as<YAML::Node>();
-    if (outputPorts && outputPorts.IsSequence()) {
-      for (YAML::const_iterator portIter = outputPorts.begin(); portIter != outputPorts.end(); ++portIter) {
-        logger_->log_debug("Got a current port, iterating...");
-
-        auto currPort = portIter->as<YAML::Node>();
-
-        this->parsePortYaml(currPort, group.get(), sitetosite::RECEIVE);
-      }  // for node
-    }
-    parentGroup->addProcessGroup(std::move(group));
-  }
-}
-
-void YamlConfiguration::parseProvenanceReportingYaml(const YAML::Node& reportNode, core::ProcessGroup* parentGroup) {
-  utils::Identifier port_uuid;
+          return std::move(ctx);
+        })(),
+        logging::LoggerFactory<YamlConfiguration>::getLogger()) {}
 
-  if (!parentGroup) {
-    logger_->log_error("parseProvenanceReportingYaml: no parent group exists");
-    return;
+std::unique_ptr<core::ProcessGroup> YamlConfiguration::getRoot() {
+  if (!config_path_) {
+    logger_->log_error("Cannot instantiate flow, no config file is set.");
+    throw Exception(ExceptionType::FLOW_EXCEPTION, "No config file specified");
   }
-
-  if (!reportNode || !reportNode.IsDefined() || reportNode.IsNull()) {
-    logger_->log_debug("no provenance reporting task specified");
-    return;
-  }
-
-  auto reportTask = createProvenanceReportTask();
-
-  const auto node = reportNode.as<YAML::Node>();
-
-  yaml::checkRequiredField(node, "scheduling strategy", CONFIG_YAML_PROVENANCE_REPORT_KEY);
-  auto schedulingStrategyStr = node["scheduling strategy"].as<std::string>();
-  yaml::checkRequiredField(node, "scheduling period", CONFIG_YAML_PROVENANCE_REPORT_KEY);
-  auto schedulingPeriodStr = node["scheduling period"].as<std::string>();
-
-  if (auto scheduling_period = utils::timeutils::StringToDuration<std::chrono::nanoseconds>(schedulingPeriodStr)) {
-    logger_->log_debug("ProvenanceReportingTask schedulingPeriod %" PRId64 " ns", scheduling_period->count());
-    reportTask->setSchedulingPeriodNano(*scheduling_period);
+  const auto configuration = filesystem_->read(config_path_.value());
+  if (!configuration) {
+  // non-existence of flow config file is not a dealbreaker, the caller might fetch it from network
+  return nullptr;
   }
-
-  if (schedulingStrategyStr == "TIMER_DRIVEN") {
-    reportTask->setSchedulingStrategy(core::TIMER_DRIVEN);
-    logger_->log_debug("ProvenanceReportingTask scheduling strategy %s", schedulingStrategyStr);
-  } else {
-    throw std::invalid_argument("Invalid scheduling strategy " + schedulingStrategyStr);
-  }
-
-  int64_t lvalue;
-  if (node["host"] && node["port"]) {
-    auto hostStr = node["host"].as<std::string>();
-
-    auto portStr = node["port"].as<std::string>();
-    if (core::Property::StringToInt(portStr, lvalue) && !hostStr.empty()) {
-      logger_->log_debug("ProvenanceReportingTask port %" PRId64, lvalue);
-      std::string url = hostStr + ":" + portStr;
-      reportTask->setURL(url);
-    }
-  }
-
-  if (node["url"]) {
-    auto urlStr = node["url"].as<std::string>();
-    if (!urlStr.empty()) {
-      reportTask->setURL(urlStr);
-      logger_->log_debug("ProvenanceReportingTask URL %s", urlStr);
-    }
-  }
-  yaml::checkRequiredField(node, "port uuid", CONFIG_YAML_PROVENANCE_REPORT_KEY);
-  auto portUUIDStr = node["port uuid"].as<std::string>();
-  yaml::checkRequiredField(node, "batch size", CONFIG_YAML_PROVENANCE_REPORT_KEY);
-  auto batchSizeStr = node["batch size"].as<std::string>();
-
-  logger_->log_debug("ProvenanceReportingTask port uuid %s", portUUIDStr);
-  port_uuid = portUUIDStr;
-  reportTask->setPortUUID(port_uuid);
-
-  if (core::Property::StringToInt(batchSizeStr, lvalue)) {
-    reportTask->setBatchSize(gsl::narrow<int>(lvalue));
-  }
-
-  reportTask->initialize();
-
-  // add processor to parent
-  reportTask->setScheduledState(core::RUNNING);
-  parentGroup->addProcessor(std::move(reportTask));
-}
-
-void YamlConfiguration::parseControllerServices(const YAML::Node& controllerServicesNode) {
-  if (!controllerServicesNode || !controllerServicesNode.IsSequence()) {
-    return;
-  }
-  for (const auto& iter : controllerServicesNode) {
-    const auto controllerServiceNode = iter.as<YAML::Node>();
-    try {
-      yaml::checkRequiredField(controllerServiceNode, "name", CONFIG_YAML_CONTROLLER_SERVICES_KEY);
-
-      auto type = yaml::getRequiredField(controllerServiceNode, std::vector<std::string>{"class", "type"}, CONFIG_YAML_CONTROLLER_SERVICES_KEY);
-      logger_->log_debug("Using type %s for controller service node", type);
-
-      std::string fullType = type;
-      auto lastOfIdx = type.find_last_of('.');
-      if (lastOfIdx != std::string::npos) {
-        lastOfIdx++;  // if a value is found, increment to move beyond the .
-        type = type.substr(lastOfIdx);
-      }
-
-      auto name = controllerServiceNode["name"].as<std::string>();
-      auto id = getRequiredIdField(controllerServiceNode, CONFIG_YAML_CONTROLLER_SERVICES_KEY);
-
-      utils::Identifier uuid;
-      uuid = id;
-      std::shared_ptr<core::controller::ControllerServiceNode> controller_service_node = createControllerService(type, fullType, name, uuid);
-      if (nullptr != controller_service_node) {
-        logger_->log_debug("Created Controller Service with UUID %s and name %s", id, name);
-        controller_service_node->initialize();
-        YAML::Node propertiesNode = controllerServiceNode["Properties"];
-        // we should propagate properties to the node and to the implementation
-        parsePropertiesNodeYaml(propertiesNode, *controller_service_node, name, CONFIG_YAML_CONTROLLER_SERVICES_KEY);
-        if (auto controllerServiceImpl = controller_service_node->getControllerServiceImplementation(); controllerServiceImpl) {
-          parsePropertiesNodeYaml(propertiesNode, *controllerServiceImpl, name, CONFIG_YAML_CONTROLLER_SERVICES_KEY);
-        }
-      } else {
-        logger_->log_debug("Could not locate %s", type);
-      }
-      controller_services_->put(id, controller_service_node);
-      controller_services_->put(name, controller_service_node);
-    } catch (YAML::InvalidNode &) {
-      throw Exception(ExceptionType::GENERAL_EXCEPTION, "Name, id, and class must be specified for controller services");
-    }
-  }
-}
-
-void YamlConfiguration::parseConnectionYaml(const YAML::Node& connectionsNode, core::ProcessGroup* parent) {
-  if (!parent) {
-    logger_->log_error("parseProcessNode: no parent group was provided");
-    return;
-  }
-  if (!connectionsNode || !connectionsNode.IsSequence()) {
-    return;
-  }
-
-  for (YAML::const_iterator iter = connectionsNode.begin(); iter != connectionsNode.end(); ++iter) {
-    const auto connectionNode = iter->as<YAML::Node>();
-
-    // Configure basic connection
-    const std::string id = getOrGenerateId(connectionNode);
-
-    // Default name to be same as ID
-    // If name is specified in configuration, use the value
-    const auto name = connectionNode["name"].as<std::string>(id);
-
-    const auto uuid = utils::Identifier::parse(id) | utils::orElse([this] {
-      logger_->log_debug("Incorrect connection UUID format.");
-      throw Exception(ExceptionType::GENERAL_EXCEPTION, "Incorrect connection UUID format.");
-    });
-
-    auto connection = createConnection(name, uuid.value());
-    logger_->log_debug("Created connection with UUID %s and name %s", id, name);
-    const yaml::YamlConnectionParser connectionParser(connectionNode, name, gsl::not_null<core::ProcessGroup*>{ parent }, logger_);
-    connectionParser.configureConnectionSourceRelationshipsFromYaml(*connection);
-    connection->setMaxQueueSize(connectionParser.getWorkQueueSizeFromYaml());
-    connection->setMaxQueueDataSize(connectionParser.getWorkQueueDataSizeFromYaml());
-    connection->setSwapThreshold(connectionParser.getSwapThresholdFromYaml());
-    connection->setSourceUUID(connectionParser.getSourceUUIDFromYaml());
-    connection->setDestinationUUID(connectionParser.getDestinationUUIDFromYaml());
-    connection->setFlowExpirationDuration(connectionParser.getFlowFileExpirationFromYaml());
-    connection->setDropEmptyFlowFiles(connectionParser.getDropEmptyFromYaml());
-
-    parent->addConnection(std::move(connection));
-  }
-}
-
-void YamlConfiguration::parsePortYaml(const YAML::Node& portNode, core::ProcessGroup* parent, sitetosite::TransferDirection direction) {
-  utils::Identifier uuid;
-
-  if (!parent) {
-    logger_->log_error("parseProcessNode: no parent group existed");
-    return;
-  }
-
-  const auto inputPortsObj = portNode.as<YAML::Node>();
-
-  // Check for required fields
-  yaml::checkRequiredField(inputPortsObj, "name", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
-  auto nameStr = inputPortsObj["name"].as<std::string>();
-  auto portId = getRequiredIdField(inputPortsObj, CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY,
-    "The field 'id' is required for "
-    "the port named '" + nameStr + "' in the YAML Config. If this port "
-    "is an input port for a NiFi Remote Process Group, the port "
-    "id should match the corresponding id specified in the NiFi configuration. "
-    "This is a UUID of the format XXXXXXXX-XXXX-XXXX-XXXX-XXXXXXXXXXXX.");
-  uuid = portId;
-
-  auto port = std::make_unique<minifi::RemoteProcessorGroupPort>(
-          stream_factory_, nameStr, parent->getURL(), this->configuration_, uuid);
-  port->setDirection(direction);
-  port->setTimeout(parent->getTimeout());
-  port->setTransmitting(true);
-  port->setYieldPeriodMsec(parent->getYieldPeriodMsec());
-  port->initialize();
-  if (!parent->getInterface().empty())
-    port->setInterface(parent->getInterface());
-  if (parent->getTransportProtocol() == "HTTP") {
-    port->enableHTTP();
-    if (!parent->getHttpProxyHost().empty())
-      port->setHTTPProxy(parent->getHTTPProxy());
-  }
-  // else defaults to RAW
-
-  // handle port properties
-  const auto nodeVal = portNode.as<YAML::Node>();
-  YAML::Node propertiesNode = nodeVal["Properties"];
-  parsePropertiesNodeYaml(propertiesNode, *port, nameStr, CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
-
-  // add processor to parent
-  auto& processor = *port;
-  parent->addProcessor(std::move(port));
-  processor.setScheduledState(core::RUNNING);
-
-  if (inputPortsObj["max concurrent tasks"]) {
-    auto rawMaxConcurrentTasks = inputPortsObj["max concurrent tasks"].as<std::string>();
-    int32_t maxConcurrentTasks;
-    if (core::Property::StringToInt(rawMaxConcurrentTasks, maxConcurrentTasks)) {
-      processor.setMaxConcurrentTasks(maxConcurrentTasks);
-    }
-    logger_->log_debug("parseProcessorNode: maxConcurrentTasks => [%d]", maxConcurrentTasks);
-    processor.setMaxConcurrentTasks(maxConcurrentTasks);
-  }
-}
-
-void YamlConfiguration::parsePropertyValueSequence(const std::string& propertyName, const YAML::Node& propertyValueNode, core::ConfigurableComponent& component) {
-  for (const auto& iter : propertyValueNode) {
-    if (iter.IsDefined()) {
-      const auto nodeVal = iter.as<YAML::Node>();
-      YAML::Node propertiesNode = nodeVal["value"];
-      // must insert the sequence in differently.
-      const auto rawValueString = propertiesNode.as<std::string>();
-      logger_->log_debug("Found %s=%s", propertyName, rawValueString);
-      if (!component.updateProperty(propertyName, rawValueString)) {
-        auto proc = dynamic_cast<core::Connectable*>(&component);
-        if (proc) {
-          logger_->log_warn("Received property %s with value %s but is not one of the properties for %s. Attempting to add as dynamic property.", propertyName, rawValueString, proc->getName());
-          if (!component.setDynamicProperty(propertyName, rawValueString)) {
-            logger_->log_warn("Unable to set the dynamic property %s with value %s", propertyName, rawValueString);
-          } else {
-            logger_->log_warn("Dynamic property %s with value %s set", propertyName, rawValueString);
-          }
-        }
-      }
-    }
-  }
-}
-
-PropertyValue YamlConfiguration::getValidatedProcessorPropertyForDefaultTypeInfo(const core::Property& propertyFromProcessor, const YAML::Node& propertyValueNode) {
-  PropertyValue defaultValue;
-  defaultValue = propertyFromProcessor.getDefaultValue();
-  const std::type_index defaultType = defaultValue.getTypeInfo();
   try {
-    PropertyValue coercedValue = defaultValue;
-    if (defaultType == typeid(int64_t)) {
-      coercedValue = propertyValueNode.as<int64_t>();
-    } else if (defaultType == typeid(uint64_t)) {
-      uint64_t integer_value;
-      if (YAML::convert<uint64_t>::decode(propertyValueNode, integer_value)) {
-        coercedValue = integer_value;
-      } else {
-        coercedValue = propertyValueNode.as<std::string>();
-      }
-    } else if (defaultType == typeid(int)) {
-      coercedValue = propertyValueNode.as<int>();
-    } else if (defaultType == typeid(bool)) {
-      coercedValue = propertyValueNode.as<bool>();
-    } else {
-      coercedValue = propertyValueNode.as<std::string>();
-    }
-    return coercedValue;
-  } catch (const std::exception& e) {
-    logger_->log_error("Fetching property failed with an exception of %s", e.what());
-    logger_->log_error("Invalid conversion for field %s. Value %s", propertyFromProcessor.getName(), propertyValueNode.as<std::string>());
-  } catch (...) {
-    logger_->log_error("Invalid conversion for field %s. Value %s", propertyFromProcessor.getName(), propertyValueNode.as<std::string>());
-  }
-  return defaultValue;
-}
-
-void YamlConfiguration::parseSingleProperty(const std::string& propertyName, const YAML::Node& propertyValueNode, core::ConfigurableComponent& processor) {
-  core::Property myProp(propertyName, "", "");
-  processor.getProperty(propertyName, myProp);
-  const PropertyValue coercedValue = getValidatedProcessorPropertyForDefaultTypeInfo(myProp, propertyValueNode);
-  bool property_set = false;
-  try {
-    property_set = processor.setProperty(myProp, coercedValue);
-  } catch(const utils::internal::InvalidValueException&) {
-    auto component = dynamic_cast<core::CoreComponent*>(&processor);
-    if (component == nullptr) {
-      logger_->log_error("processor was not a CoreComponent for property '%s'", propertyName);
-    } else {
-      logger_->log_error("Invalid value was set for property '%s' creating component '%s'", propertyName, component->getName());
-    }
+    YAML::Node rootYamlNode = YAML::Load(configuration.value());
+    flow::Node root{std::make_shared<YamlNode>(rootYamlNode)};
+    return getRootFrom(root);
+  } catch(...) {
+    logger_->log_error("Invalid yaml configuration file");
     throw;
   }
-  const auto rawValueString = propertyValueNode.as<std::string>();
-  if (!property_set) {
-    auto proc = dynamic_cast<core::Connectable*>(&processor);
-    if (proc) {
-      logger_->log_warn("Received property %s with value %s but is not one of the properties for %s. Attempting to add as dynamic property.", propertyName, rawValueString, proc->getName());
-      if (!processor.setDynamicProperty(propertyName, rawValueString)) {
-        logger_->log_warn("Unable to set the dynamic property %s with value %s", propertyName, rawValueString);
-      } else {
-        logger_->log_warn("Dynamic property %s with value %s set", propertyName, rawValueString);
-      }
-    }
-  } else {
-    logger_->log_debug("Property %s with value %s set", propertyName, rawValueString);
-  }
-}
-
-void YamlConfiguration::parsePropertyNodeElement(const std::string& propertyName, const YAML::Node& propertyValueNode, core::ConfigurableComponent& processor) {
-  logger_->log_trace("Encountered %s", propertyName);
-  if (propertyValueNode.IsNull() || !propertyValueNode.IsDefined()) {
-    return;
-  }
-  if (propertyValueNode.IsSequence()) {
-    parsePropertyValueSequence(propertyName, propertyValueNode, processor);
-  } else {
-    parseSingleProperty(propertyName, propertyValueNode, processor);
-  }
-}
-
-void YamlConfiguration::parsePropertiesNodeYaml(const YAML::Node& propertiesNode, core::ConfigurableComponent& component, const std::string& component_name,
-    const std::string& yaml_section) {
-  // Treat generically as a YAML node so we can perform inspection on entries to ensure they are populated
-  logger_->log_trace("Entered %s", component_name);
-  for (const auto& propertyElem : propertiesNode) {
-    const auto propertyName = propertyElem.first.as<std::string>();
-    const YAML::Node propertyValueNode = propertyElem.second;
-    parsePropertyNodeElement(propertyName, propertyValueNode, component);
-  }
-
-  validateComponentProperties(component, component_name, yaml_section);
 }
 
-void YamlConfiguration::parseFunnelsYaml(const YAML::Node& node, core::ProcessGroup* parent) {
-  if (!parent) {
-    logger_->log_error("parseFunnelsYaml: no parent group was provided");
-    return;
-  }
-  if (!node || !node.IsSequence()) {
-    return;
-  }
-
-  for (const auto& element : node) {
-    const auto funnel_node = element.as<YAML::Node>();
-
-    std::string id = getOrGenerateId(funnel_node);
-
-    // Default name to be same as ID
-    const auto name = funnel_node["name"].as<std::string>(id);
-
-    const auto uuid = utils::Identifier::parse(id) | utils::orElse([this] {
-      logger_->log_debug("Incorrect funnel UUID format.");
-      throw Exception(ExceptionType::GENERAL_EXCEPTION, "Incorrect funnel UUID format.");
-    });
-
-    auto funnel = std::make_unique<core::Funnel>(name, uuid.value());
-    logger_->log_debug("Created funnel with UUID %s and name %s", id, name);
-    funnel->setScheduledState(core::RUNNING);
-    funnel->setSchedulingStrategy(core::EVENT_DRIVEN);
-    parent->addProcessor(std::move(funnel));
-  }
-}
-
-void YamlConfiguration::validateComponentProperties(ConfigurableComponent& component, const std::string &component_name, const std::string &yaml_section) const {
-  const auto &component_properties = component.getProperties();
-
-  // Validate required properties
-  for (const auto &prop_pair : component_properties) {
-    if (prop_pair.second.getRequired()) {
-      if (prop_pair.second.getValue().to_string().empty()) {
-        std::string reason = utils::StringUtils::join_pack("required property '", prop_pair.second.getName(), "' is not set");
-        raiseComponentError(component_name, yaml_section, reason);
-      } else if (!prop_pair.second.getValue().validate(prop_pair.first).valid()) {
-        std::string reason = utils::StringUtils::join_pack("the value '", prop_pair.first, "' is not valid for property '", prop_pair.second.getName(), "'");
-        raiseComponentError(component_name, yaml_section, reason);
-      }
-    }
-  }
-
-  // Validate dependent properties
-  for (const auto &prop_pair : component_properties) {
-    const auto &dep_props = prop_pair.second.getDependentProperties();
-
-    if (prop_pair.second.getValue().to_string().empty()) {
-      continue;
-    }
-
-    for (const auto &dep_prop_key : dep_props) {
-      if (component_properties.at(dep_prop_key).getValue().to_string().empty()) {
-        std::string reason = utils::StringUtils::join_pack("property '", prop_pair.second.getName(),
-            "' depends on property '", dep_prop_key, "' which is not set");
-        raiseComponentError(component_name, yaml_section, reason);
-      }
-    }
-  }
-
-#ifdef YAML_CONFIGURATION_USE_REGEX
-  // Validate mutually-exclusive properties
-  for (const auto &prop_pair : component_properties) {
-    const auto &excl_props = prop_pair.second.getExclusiveOfProperties();
-
-    if (prop_pair.second.getValue().empty()) {
-      continue;
-    }
-
-    for (const auto &excl_pair : excl_props) {
-      utils::Regex excl_expr(excl_pair.second);
-      if (utils::regexMatch(component_properties.at(excl_pair.first).getValue().to_string(), excl_expr)) {
-        std::string reason = utils::StringUtils::join_pack("property '", prop_pair.second.getName(),
-            "' must not be set when the value of property '", excl_pair.first, "' matches '", excl_pair.second, "'");
-        raiseComponentError(component_name, yaml_section, reason);
-      }
-    }
-  }
-
-  // Validate regex properties
-  for (const auto &prop_pair : component_properties) {
-    const auto &prop_regex_str = prop_pair.second.getValidRegex();
-
-    if (!prop_regex_str.empty()) {
-      utils::Regex prop_regex(prop_regex_str);
-      if (!utils::regexMatch(prop_pair.second.getValue().to_string(), prop_regex)) {
-        std::string reason = utils::StringUtils::join_pack("property '", prop_pair.second.getName(), "' does not match validation pattern '", prop_regex_str, "'");
-        raiseComponentError(component_name, yaml_section, reason);
-      }
-    }
-  }
-#else
-  logging::LOG_INFO(logger_) << "Validation of mutally-exclusive properties is disabled in this build.";
-  logging::LOG_INFO(logger_) << "Regex validation of properties is not available in this build.";
-#endif  // YAML_CONFIGURATION_USE_REGEX
-}
-
-void YamlConfiguration::raiseComponentError(const std::string &component_name, const std::string &yaml_section, const std::string &reason) const {
-  std::string err_msg = "Unable to parse configuration file for component named '";
-  err_msg.append(component_name);
-  err_msg.append("' because " + reason);
-  if (!yaml_section.empty()) {
-    err_msg.append(" [in '" + yaml_section + "' section of configuration file]");
-  }
-
-  logging::LOG_ERROR(logger_) << err_msg;
-
-  throw std::invalid_argument(err_msg);
-}
-
-std::string YamlConfiguration::getOrGenerateId(const YAML::Node& yamlNode, const std::string& idField) {
-  std::string id;
-  auto node = yamlNode.as<YAML::Node>();
-
-  if (node[idField]) {
-    if (YAML::NodeType::Scalar == node[idField].Type()) {
-      id = node[idField].as<std::string>();
-      addNewId(id);
-      return id;
-    }
-    throw std::invalid_argument("getOrGenerateId: idField is expected to reference YAML::Node of YAML::NodeType::Scalar.");
-  }
-
-  id = id_generator_->generate().to_string();
-  logger_->log_debug("Generating random ID: id => [%s]", id);
-  return id;
-}
-
-std::string YamlConfiguration::getRequiredIdField(const YAML::Node& yaml_node, std::string_view yaml_section, std::string_view error_message) {
-  yaml::checkRequiredField(yaml_node, "id", yaml_section, error_message);
-  auto id = yaml_node["id"].as<std::string>();
-  addNewId(id);
-  return id;
-}
-
-YAML::Node YamlConfiguration::getOptionalField(const YAML::Node& yamlNode, const std::string& fieldName, const YAML::Node& defaultValue, const std::string& yamlSection,
-                                               const std::string& providedInfoMessage) {
-  std::string infoMessage = providedInfoMessage;
-  auto result = yamlNode.as<YAML::Node>()[fieldName];
-  if (!result) {
-    if (infoMessage.empty()) {
-      // Build a helpful info message for the user to inform them that a default is being used
-      infoMessage =
-          yamlNode.as<YAML::Node>()["name"] ?
-              "Using default value for optional field '" + fieldName + "' in component named '" + yamlNode.as<YAML::Node>()["name"].as<std::string>() + "'" :
-              "Using default value for optional field '" + fieldName + "' ";
-      if (!yamlSection.empty()) {
-        infoMessage += " [in '" + yamlSection + "' section of configuration file]: ";
-      }
-
-      infoMessage += defaultValue.as<std::string>();
-    }
-    logging::LOG_INFO(logger_) << infoMessage;
-    result = defaultValue;
+std::unique_ptr<core::ProcessGroup> YamlConfiguration::getYamlRoot(std::istream &yamlConfigStream) {
+  try {
+    YAML::Node rootYamlNode = YAML::Load(yamlConfigStream);
+    flow::Node root{std::make_shared<YamlNode>(rootYamlNode)};
+    return getRootFrom(root);
+  } catch (const YAML::ParserException &pe) {
+    logger_->log_error(pe.what());
+    std::rethrow_exception(std::current_exception());

Review Comment:
   Is this different from `throw;`?  If not, I think that would be better.



##########
libminifi/include/core/json/JsonNode.h:
##########
@@ -0,0 +1,248 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string>
+#include <utility>
+#include <memory>
+
+#include "core/flow/Node.h"
+#include "rapidjson/document.h"
+#include "utils/gsl.h"
+#include "utils/ValueCaster.h"
+
+namespace org::apache::nifi::minifi::core {
+
+class JsonNode : public flow::Node::NodeImpl {
+ public:
+  explicit JsonNode(const rapidjson::Value* node): node_(node) {}
+
+  explicit operator bool() const override {
+    return node_ != nullptr;
+  }
+  bool isSequence() const override {
+    return node_ ? node_->IsArray() : false;
+  }
+  bool isMap() const override {
+    return node_ ? node_->IsObject() : false;
+  }
+  bool isNull() const override {
+    return node_ ? node_->IsNull() : false;
+  }
+
+  nonstd::expected<std::string, std::exception_ptr> getString() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get string of invalid json value");
+      }
+      if (!node_->IsString()) {
+        throw std::runtime_error("Cannot get string of non-string json value");
+      }
+      return std::string{node_->GetString(), node_->GetStringLength()};
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());
+    }
+  }
+
+  nonstd::expected<int, std::exception_ptr> getInt() const override {
+    return getNumber<int>("int");
+  }
+  nonstd::expected<unsigned int, std::exception_ptr> getUInt() const override {
+    return getNumber<unsigned int>("unsigned int");
+  }
+  nonstd::expected<int64_t, std::exception_ptr> getInt64() const override {
+    return getNumber<int64_t>("int64_t");
+  }
+  nonstd::expected<uint64_t, std::exception_ptr> getUInt64() const override {
+    return getNumber<uint64_t>("uint64_t");
+  }
+
+  nonstd::expected<bool, std::exception_ptr> getBool() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get bool of invalid json value");
+      }
+      if (!node_->IsBool()) {
+        throw std::runtime_error("Cannot get bool of non-bool json value");
+      }
+      return node_->GetBool();
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());
+    }
+  }
+
+  std::string getDebugString() const override {
+    if (!node_) return "<invalid>";
+    if (node_->IsObject()) return "<Map>";
+    if (node_->IsArray()) return "<Array>";
+    if (node_->IsNull()) return "null";
+    if (node_->IsInt()) return std::to_string(node_->GetInt());
+    if (node_->IsUint()) return std::to_string(node_->GetUint());
+    if (node_->IsInt64()) return std::to_string(node_->GetInt64());
+    if (node_->IsUint64()) return std::to_string(node_->GetUint64());
+    if (node_->IsTrue()) return "true";
+    if (node_->IsFalse()) return "false";
+    if (node_->IsDouble()) return std::to_string(node_->GetDouble());
+    if (node_->IsString()) return '"' + std::string(node_->GetString(), node_->GetStringLength()) + '"';
+    return "<unknown>";
+  }
+
+  size_t size() const override {
+    if (!node_) {
+      throw std::runtime_error("Cannot get size of invalid json value");
+    }
+    if (!node_->IsArray()) {
+      throw std::runtime_error("Cannot get size of non-array json value");
+    }
+    return node_->Size();
+  }
+  flow::Node::Iterator begin() const override;
+  flow::Node::Iterator end() const override;
+
+  flow::Node operator[](std::string_view key) const override {
+    if (!node_) {
+      throw std::runtime_error("Cannot get member of invalid json value");
+    }
+    if (!node_->IsObject()) {
+      return flow::Node{std::make_shared<JsonNode>(nullptr)};
+    }
+    auto it = node_->FindMember(rapidjson::Value(rapidjson::StringRef(key.data(), key.length())));
+    if (it == node_->MemberEnd()) {
+      return flow::Node{std::make_shared<JsonNode>(nullptr)};
+    }
+    return flow::Node{std::make_shared<JsonNode>(&it->value)};
+  }
+
+  std::optional<flow::Node::Cursor> getCursor() const override {
+    return std::nullopt;
+  }
+
+ private:
+  template<typename T, typename U>
+  std::optional<T> narrow(const U& value) {
+    T res = static_cast<T>(value);
+    if (static_cast<U>(res) != value) {
+      return std::nullopt;
+    }
+    return res;
+  }

Review Comment:
   it looks like this has been replaced by `cast_if_in_range` and can be deleted



##########
libminifi/src/core/flow/StructuredConfiguration.cpp:
##########
@@ -0,0 +1,896 @@
+/**
+ *
+ * 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 <memory>
+#include <vector>
+#include <set>
+#include <cinttypes>
+
+#include "core/flow/StructuredConfiguration.h"
+#include "core/flow/CheckRequiredField.h"
+#include "core/flow/StructuredConnectionParser.h"
+#include "core/state/Value.h"
+#include "Defaults.h"
+#include "utils/TimeUtil.h"
+#include "utils/RegexUtils.h"
+
+namespace org::apache::nifi::minifi::core::flow {
+
+std::shared_ptr<utils::IdGenerator> StructuredConfiguration::id_generator_ = utils::IdGenerator::getIdGenerator();
+
+StructuredConfiguration::StructuredConfiguration(ConfigurationContext ctx, std::shared_ptr<logging::Logger> logger)
+    : FlowConfiguration(std::move(ctx)),
+      logger_(std::move(logger)) {}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::parseRootProcessGroup(const Node& root_flow_node) {
+  auto flow_controller_node = root_flow_node[CONFIG_FLOW_CONTROLLER_KEY];
+  auto root_group = parseProcessGroup(flow_controller_node, root_flow_node, true);
+  this->name_ = root_group->getName();
+  return root_group;
+}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::createProcessGroup(const Node& node, bool is_root) {
+  int version = 0;
+
+  checkRequiredField(node, "name", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+  auto flowName = node["name"].getString().value();
+
+  utils::Identifier uuid;
+  // assignment throws on invalid uuid
+  uuid = getOrGenerateId(node);
+
+  if (node["version"]) {
+    version = node["version"].getInt().value();
+  }
+
+  logger_->log_debug("parseRootProcessGroup: id => [%s], name => [%s]", uuid.to_string(), flowName);
+  std::unique_ptr<core::ProcessGroup> group;
+  if (is_root) {
+    group = FlowConfiguration::createRootProcessGroup(flowName, uuid, version);
+  } else {
+    group = FlowConfiguration::createSimpleProcessGroup(flowName, uuid, version);
+  }
+
+  if (node["onschedule retry interval"]) {
+    auto onScheduleRetryPeriod = node["onschedule retry interval"].getString().value();
+    logger_->log_debug("parseRootProcessGroup: onschedule retry period => [%s]", onScheduleRetryPeriod);
+
+    auto on_schedule_retry_period_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(onScheduleRetryPeriod);
+    if (on_schedule_retry_period_value.has_value() && group) {
+      logger_->log_debug("parseRootProcessGroup: onschedule retry => [%" PRId64 "] ms", on_schedule_retry_period_value->count());
+      group->setOnScheduleRetryPeriod(on_schedule_retry_period_value->count());
+    }
+  }
+
+  return group;
+}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::parseProcessGroup(const Node& headerNode, const Node& yamlNode, bool is_root) {
+  auto group = createProcessGroup(headerNode, is_root);
+  Node processorsNode = yamlNode[CONFIG_PROCESSORS_KEY];
+  Node connectionsNode = yamlNode[StructuredConnectionParser::CONFIG_CONNECTIONS_KEY];
+  Node funnelsNode = yamlNode[CONFIG_FUNNELS_KEY];
+  Node remoteProcessingGroupsNode = [&] {
+    // assignment is not supported on invalid Yaml nodes
+    Node candidate = yamlNode[CONFIG_REMOTE_PROCESS_GROUP_KEY];
+    if (candidate) {
+      return candidate;
+    }
+    return yamlNode[CONFIG_REMOTE_PROCESS_GROUP_KEY_V3];
+  }();
+  Node childProcessGroupNodeSeq = yamlNode["Process Groups"];
+
+  parseProcessorNode(processorsNode, group.get());
+  parseRemoteProcessGroup(remoteProcessingGroupsNode, group.get());
+  parseFunnels(funnelsNode, group.get());
+  // parse connections last to give feedback if the source and/or destination
+  // is not in the same process group
+  parseConnection(connectionsNode, group.get());
+
+  if (childProcessGroupNodeSeq && childProcessGroupNodeSeq.isSequence()) {
+    for (const auto childProcessGroupNode : childProcessGroupNodeSeq) {
+      group->addProcessGroup(parseProcessGroup(childProcessGroupNode, childProcessGroupNode));
+    }
+  }
+  return group;
+}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::getRootFrom(const Node& rootYamlNode) {
+  uuids_.clear();
+  Node controllerServiceNode = rootYamlNode[CONFIG_CONTROLLER_SERVICES_KEY];
+  Node provenanceReportNode = rootYamlNode[CONFIG_PROVENANCE_REPORT_KEY];
+
+  parseControllerServices(controllerServiceNode);
+  // Create the root process group
+  std::unique_ptr<core::ProcessGroup> root = parseRootProcessGroup(rootYamlNode);
+  parseProvenanceReporting(provenanceReportNode, root.get());
+
+  // set the controller services into the root group.
+  for (const auto& controller_service : controller_services_->getAllControllerServices()) {
+    root->addControllerService(controller_service->getName(), controller_service);
+    root->addControllerService(controller_service->getUUIDStr(), controller_service);
+  }
+
+  return root;
+}
+
+void StructuredConfiguration::parseProcessorNode(const Node& processors_node, core::ProcessGroup* parentGroup) {
+  int64_t runDurationNanos = -1;
+  utils::Identifier uuid;
+  std::unique_ptr<core::Processor> processor;
+
+  if (!parentGroup) {
+    logger_->log_error("parseProcessNodeYaml: no parent group exists");
+    return;
+  }
+
+  if (!processors_node) {
+    throw std::invalid_argument("Cannot instantiate a MiNiFi instance without a defined Processors configuration node.");
+  }
+  if (!processors_node.isSequence()) {
+    throw std::invalid_argument(
+        "Cannot instantiate a MiNiFi instance without a defined Processors configuration node.");
+  }
+  // Evaluate sequence of processors
+  for (const auto procNode : processors_node) {
+    core::ProcessorConfig procCfg;
+
+    checkRequiredField(procNode, "name", CONFIG_PROCESSORS_KEY);
+    procCfg.name = procNode["name"].getString().value();
+    procCfg.id = getOrGenerateId(procNode);
+
+    uuid = procCfg.id;
+    logger_->log_debug("parseProcessorNode: name => [%s] id => [%s]", procCfg.name, procCfg.id);
+    checkRequiredField(procNode, "class", CONFIG_PROCESSORS_KEY);
+    procCfg.javaClass = procNode["class"].getString().value();
+    logger_->log_debug("parseProcessorNode: class => [%s]", procCfg.javaClass);
+
+    // Determine the processor name only from the Java class
+    auto lastOfIdx = procCfg.javaClass.find_last_of('.');
+    if (lastOfIdx != std::string::npos) {
+      lastOfIdx++;  // if a value is found, increment to move beyond the .
+      std::string processorName = procCfg.javaClass.substr(lastOfIdx);
+      processor = this->createProcessor(processorName, procCfg.javaClass, uuid);
+    } else {
+      // Allow unqualified class names for core processors
+      processor = this->createProcessor(procCfg.javaClass, uuid);
+    }
+
+    if (!processor) {
+      logger_->log_error("Could not create a processor %s with id %s", procCfg.name, procCfg.id);
+      throw std::invalid_argument("Could not create processor " + procCfg.name);
+    }
+
+    processor->setName(procCfg.name);
+
+    processor->setFlowIdentifier(flow_version_->getFlowIdentifier());
+
+    procCfg.schedulingStrategy = getOptionalField(procNode, "scheduling strategy", DEFAULT_SCHEDULING_STRATEGY, CONFIG_PROCESSORS_KEY);
+    logger_->log_debug("parseProcessorNode: scheduling strategy => [%s]", procCfg.schedulingStrategy);
+
+    procCfg.schedulingPeriod = getOptionalField(procNode, "scheduling period", DEFAULT_SCHEDULING_PERIOD_STR, CONFIG_PROCESSORS_KEY);
+
+    logger_->log_debug("parseProcessorNode: scheduling period => [%s]", procCfg.schedulingPeriod);
+
+    if (auto tasksNode = procNode["max concurrent tasks"]) {
+      if (auto int_val = tasksNode.getUInt64()) {
+        procCfg.maxConcurrentTasks = std::to_string(int_val.value());
+      } else {
+        procCfg.maxConcurrentTasks = tasksNode.getString().value();
+      }
+      logger_->log_debug("parseProcessorNode: max concurrent tasks => [%s]", procCfg.maxConcurrentTasks);
+    }
+
+    if (procNode["penalization period"]) {
+      procCfg.penalizationPeriod = procNode["penalization period"].getString().value();
+      logger_->log_debug("parseProcessorNode: penalization period => [%s]", procCfg.penalizationPeriod);
+    }
+
+    if (procNode["yield period"]) {
+      procCfg.yieldPeriod = procNode["yield period"].getString().value();
+      logger_->log_debug("parseProcessorNode: yield period => [%s]", procCfg.yieldPeriod);
+    }
+
+    if (auto runNode = procNode["run duration nanos"]) {
+      if (auto int_val = runNode.getUInt64()) {
+        procCfg.runDurationNanos = std::to_string(int_val.value());
+      } else {
+        procCfg.runDurationNanos = runNode.getString().value();
+      }
+      logger_->log_debug("parseProcessorNode: run duration nanos => [%s]", procCfg.runDurationNanos);
+    }
+
+    // handle auto-terminated relationships
+    if (procNode["auto-terminated relationships list"]) {
+      Node autoTerminatedSequence = procNode["auto-terminated relationships list"];
+      std::vector<std::string> rawAutoTerminatedRelationshipValues;
+      if (autoTerminatedSequence.isSequence() && autoTerminatedSequence.size() > 0) {
+        for (const auto autoTerminatedRel : autoTerminatedSequence) {
+          rawAutoTerminatedRelationshipValues.push_back(autoTerminatedRel.getString().value());
+        }
+      }
+      procCfg.autoTerminatedRelationships = rawAutoTerminatedRelationshipValues;
+    }
+
+    // handle processor properties
+    if (procNode["Properties"]) {
+      Node propertiesNode = procNode["Properties"];
+      parsePropertiesNode(propertiesNode, *processor, procCfg.name, CONFIG_PROCESSORS_KEY);
+    }
+
+    // Take care of scheduling
+
+    if (procCfg.schedulingStrategy == "TIMER_DRIVEN" || procCfg.schedulingStrategy == "EVENT_DRIVEN") {
+      if (auto scheduling_period = utils::timeutils::StringToDuration<std::chrono::nanoseconds>(procCfg.schedulingPeriod)) {
+        logger_->log_debug("convert: parseProcessorNode: schedulingPeriod => [%" PRId64 "] ns", scheduling_period->count());
+        processor->setSchedulingPeriodNano(*scheduling_period);
+      }
+    } else {
+      processor->setCronPeriod(procCfg.schedulingPeriod);
+    }
+
+    if (auto penalization_period = utils::timeutils::StringToDuration<std::chrono::milliseconds>(procCfg.penalizationPeriod)) {
+      logger_->log_debug("convert: parseProcessorNode: penalizationPeriod => [%" PRId64 "] ms", penalization_period->count());
+      processor->setPenalizationPeriod(penalization_period.value());
+    }
+
+    if (auto yield_period = utils::timeutils::StringToDuration<std::chrono::milliseconds>(procCfg.yieldPeriod)) {
+      logger_->log_debug("convert: parseProcessorNode: yieldPeriod => [%" PRId64 "] ms", yield_period->count());
+      processor->setYieldPeriodMsec(yield_period.value());
+    }
+
+    // Default to running
+    processor->setScheduledState(core::RUNNING);
+
+    if (procCfg.schedulingStrategy == "TIMER_DRIVEN") {
+      processor->setSchedulingStrategy(core::TIMER_DRIVEN);
+      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
+    } else if (procCfg.schedulingStrategy == "EVENT_DRIVEN") {
+      processor->setSchedulingStrategy(core::EVENT_DRIVEN);
+      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
+    } else {
+      processor->setSchedulingStrategy(core::CRON_DRIVEN);
+      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
+    }
+
+    int32_t maxConcurrentTasks;
+    if (core::Property::StringToInt(procCfg.maxConcurrentTasks, maxConcurrentTasks)) {
+      logger_->log_debug("parseProcessorNode: maxConcurrentTasks => [%d]", maxConcurrentTasks);
+      processor->setMaxConcurrentTasks((uint8_t) maxConcurrentTasks);
+    }
+
+    if (core::Property::StringToInt(procCfg.runDurationNanos, runDurationNanos)) {
+      logger_->log_debug("parseProcessorNode: runDurationNanos => [%d]", runDurationNanos);
+      processor->setRunDurationNano(std::chrono::nanoseconds(runDurationNanos));
+    }
+
+    std::vector<core::Relationship> autoTerminatedRelationships;
+    for (auto &&relString : procCfg.autoTerminatedRelationships) {
+      core::Relationship relationship(relString, "");
+      logger_->log_debug("parseProcessorNode: autoTerminatedRelationship  => [%s]", relString);
+      autoTerminatedRelationships.push_back(relationship);
+    }
+
+    processor->setAutoTerminatedRelationships(autoTerminatedRelationships);
+
+    parentGroup->addProcessor(std::move(processor));
+  }
+}
+
+void StructuredConfiguration::parseRemoteProcessGroup(const Node& rpg_node_seq, core::ProcessGroup* parentGroup) {
+  utils::Identifier uuid;
+  std::string id;
+
+  if (!parentGroup) {
+    logger_->log_error("parseRemoteProcessGroupYaml: no parent group exists");
+    return;
+  }
+
+  if (!rpg_node_seq || !rpg_node_seq.isSequence()) {
+    return;
+  }
+  for (const auto currRpgNode : rpg_node_seq) {
+    checkRequiredField(currRpgNode, "name", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+    auto name = currRpgNode["name"].getString().value();
+    id = getOrGenerateId(currRpgNode);
+
+    logger_->log_debug("parseRemoteProcessGroupYaml: name => [%s], id => [%s]", name, id);
+
+    auto url = getOptionalField(currRpgNode, "url", "", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+
+    logger_->log_debug("parseRemoteProcessGroupYaml: url => [%s]", url);
+
+    uuid = id;
+    auto group = createRemoteProcessGroup(name, uuid);
+    group->setParent(parentGroup);
+
+    if (currRpgNode["yield period"]) {
+      auto yieldPeriod = currRpgNode["yield period"].getString().value();
+      logger_->log_debug("parseRemoteProcessGroupYaml: yield period => [%s]", yieldPeriod);
+
+      auto yield_period_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(yieldPeriod);
+      if (yield_period_value.has_value() && group) {
+        logger_->log_debug("parseRemoteProcessGroupYaml: yieldPeriod => [%" PRId64 "] ms", yield_period_value->count());
+        group->setYieldPeriodMsec(*yield_period_value);
+      }
+    }
+
+    if (currRpgNode["timeout"]) {
+      auto timeout = currRpgNode["timeout"].getString().value();
+      logger_->log_debug("parseRemoteProcessGroupYaml: timeout => [%s]", timeout);
+
+      auto timeout_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(timeout);
+      if (timeout_value.has_value() && group) {
+        logger_->log_debug("parseRemoteProcessGroupYaml: timeoutValue => [%" PRId64 "] ms", timeout_value->count());
+        group->setTimeout(timeout_value->count());
+      }
+    }
+
+    if (currRpgNode["local network interface"]) {
+      auto interface = currRpgNode["local network interface"].getString().value();
+      logger_->log_debug("parseRemoteProcessGroupYaml: local network interface => [%s]", interface);
+      group->setInterface(interface);
+    }
+
+    if (currRpgNode["transport protocol"]) {
+      auto transport_protocol = currRpgNode["transport protocol"].getString().value();
+      logger_->log_debug("parseRemoteProcessGroupYaml: transport protocol => [%s]", transport_protocol);
+      if (transport_protocol == "HTTP") {
+        group->setTransportProtocol(transport_protocol);
+        if (currRpgNode["proxy host"]) {
+          auto http_proxy_host = currRpgNode["proxy host"].getString().value();
+          logger_->log_debug("parseRemoteProcessGroupYaml: proxy host => [%s]", http_proxy_host);
+          group->setHttpProxyHost(http_proxy_host);
+          if (currRpgNode["proxy user"]) {
+            auto http_proxy_username = currRpgNode["proxy user"].getString().value();
+            logger_->log_debug("parseRemoteProcessGroupYaml: proxy user => [%s]", http_proxy_username);
+            group->setHttpProxyUserName(http_proxy_username);
+          }
+          if (currRpgNode["proxy password"]) {
+            auto http_proxy_password = currRpgNode["proxy password"].getString().value();
+            logger_->log_debug("parseRemoteProcessGroupYaml: proxy password => [%s]", http_proxy_password);
+            group->setHttpProxyPassWord(http_proxy_password);
+          }
+          if (currRpgNode["proxy port"]) {
+            auto http_proxy_port = currRpgNode["proxy port"].getString().value();
+            int32_t port;
+            if (core::Property::StringToInt(http_proxy_port, port)) {
+              logger_->log_debug("parseRemoteProcessGroupYaml: proxy port => [%d]", port);
+              group->setHttpProxyPort(port);
+            }
+          }
+        }
+      } else if (transport_protocol == "RAW") {
+        group->setTransportProtocol(transport_protocol);
+      } else {
+        std::stringstream stream;
+        stream << "Invalid transport protocol " << transport_protocol;
+        throw minifi::Exception(ExceptionType::SITE2SITE_EXCEPTION, stream.str().c_str());
+      }
+    }
+
+    group->setTransmitting(true);
+    group->setURL(url);
+
+    checkRequiredField(currRpgNode, "Input Ports", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+    auto inputPorts = currRpgNode["Input Ports"];
+    if (inputPorts && inputPorts.isSequence()) {
+      for (const auto currPort : inputPorts) {
+        parsePort(currPort, group.get(), sitetosite::SEND);
+      }  // for node
+    }
+    auto outputPorts = currRpgNode["Output Ports"];
+    if (outputPorts && outputPorts.isSequence()) {
+      for (const auto currPort : outputPorts) {
+        logger_->log_debug("Got a current port, iterating...");
+
+        parsePort(currPort, group.get(), sitetosite::RECEIVE);
+      }  // for node
+    }
+    parentGroup->addProcessGroup(std::move(group));
+  }
+}
+
+void StructuredConfiguration::parseProvenanceReporting(const Node& node, core::ProcessGroup* parent_group) {
+  utils::Identifier port_uuid;
+
+  if (!parent_group) {
+    logger_->log_error("parseProvenanceReportingYaml: no parent group exists");
+    return;
+  }
+
+  if (!node || node.isNull()) {
+    logger_->log_debug("no provenance reporting task specified");
+    return;
+  }
+
+  auto reportTask = createProvenanceReportTask();
+
+  checkRequiredField(node, "scheduling strategy", CONFIG_PROVENANCE_REPORT_KEY);
+  auto schedulingStrategyStr = node["scheduling strategy"].getString().value();
+  checkRequiredField(node, "scheduling period", CONFIG_PROVENANCE_REPORT_KEY);
+  auto schedulingPeriodStr = node["scheduling period"].getString().value();
+
+  if (auto scheduling_period = utils::timeutils::StringToDuration<std::chrono::nanoseconds>(schedulingPeriodStr)) {
+    logger_->log_debug("ProvenanceReportingTask schedulingPeriod %" PRId64 " ns", scheduling_period->count());
+    reportTask->setSchedulingPeriodNano(*scheduling_period);
+  }
+
+  if (schedulingStrategyStr == "TIMER_DRIVEN") {
+    reportTask->setSchedulingStrategy(core::TIMER_DRIVEN);
+    logger_->log_debug("ProvenanceReportingTask scheduling strategy %s", schedulingStrategyStr);
+  } else {
+    throw std::invalid_argument("Invalid scheduling strategy " + schedulingStrategyStr);
+  }
+
+  int64_t lvalue;
+  if (node["host"] && node["port"]) {
+    auto hostStr = node["host"].getString().value();
+
+    std::string portStr;
+    if (auto int_val = node["port"].getInt()) {
+      portStr = std::to_string(int_val.value());
+    } else {
+      portStr = node["port"].getString().value();
+    }
+    if (core::Property::StringToInt(portStr, lvalue) && !hostStr.empty()) {
+      logger_->log_debug("ProvenanceReportingTask port %" PRId64, lvalue);
+      std::string url = hostStr + ":" + portStr;
+      reportTask->setURL(url);
+    }
+  }
+
+  if (node["url"]) {
+    auto urlStr = node["url"].getString().value();
+    if (!urlStr.empty()) {
+      reportTask->setURL(urlStr);
+      logger_->log_debug("ProvenanceReportingTask URL %s", urlStr);
+    }
+  }
+  checkRequiredField(node, "port uuid", CONFIG_PROVENANCE_REPORT_KEY);
+  auto portUUIDStr = node["port uuid"].getString().value();
+  checkRequiredField(node, "batch size", CONFIG_PROVENANCE_REPORT_KEY);
+  auto batchSizeStr = node["batch size"].getString().value();
+
+  logger_->log_debug("ProvenanceReportingTask port uuid %s", portUUIDStr);
+  port_uuid = portUUIDStr;
+  reportTask->setPortUUID(port_uuid);
+
+  if (core::Property::StringToInt(batchSizeStr, lvalue)) {
+    reportTask->setBatchSize(gsl::narrow<int>(lvalue));
+  }
+
+  reportTask->initialize();
+
+  // add processor to parent
+  reportTask->setScheduledState(core::RUNNING);
+  parent_group->addProcessor(std::move(reportTask));
+}
+
+void StructuredConfiguration::parseControllerServices(const Node& controllerServicesNode) {
+  if (!controllerServicesNode || !controllerServicesNode.isSequence()) {
+    return;
+  }
+  for (const auto& controllerServiceNode : controllerServicesNode) {
+    checkRequiredField(controllerServiceNode, "name", CONFIG_CONTROLLER_SERVICES_KEY);
+
+    auto type = getRequiredField(controllerServiceNode, std::vector<std::string>{"class", "type"}, CONFIG_CONTROLLER_SERVICES_KEY);
+    logger_->log_debug("Using type %s for controller service node", type);
+
+    std::string fullType = type;
+    auto lastOfIdx = type.find_last_of('.');
+    if (lastOfIdx != std::string::npos) {
+      lastOfIdx++;  // if a value is found, increment to move beyond the .
+      type = type.substr(lastOfIdx);
+    }
+
+    auto name = controllerServiceNode["name"].getString().value();
+    auto id = getRequiredIdField(controllerServiceNode, CONFIG_CONTROLLER_SERVICES_KEY);
+
+    utils::Identifier uuid;
+    uuid = id;
+    std::shared_ptr<core::controller::ControllerServiceNode> controller_service_node = createControllerService(type, fullType, name, uuid);
+    if (nullptr != controller_service_node) {
+      logger_->log_debug("Created Controller Service with UUID %s and name %s", id, name);
+      controller_service_node->initialize();
+      if (Node propertiesNode = controllerServiceNode["Properties"]) {
+        // we should propagate properties to the node and to the implementation
+        parsePropertiesNode(propertiesNode, *controller_service_node, name, CONFIG_CONTROLLER_SERVICES_KEY);
+        if (auto controllerServiceImpl = controller_service_node->getControllerServiceImplementation(); controllerServiceImpl) {
+          parsePropertiesNode(propertiesNode, *controllerServiceImpl, name, CONFIG_CONTROLLER_SERVICES_KEY);
+        }
+      }
+    } else {
+      logger_->log_debug("Could not locate %s", type);
+    }
+    controller_services_->put(id, controller_service_node);
+    controller_services_->put(name, controller_service_node);
+  }
+}
+
+void StructuredConfiguration::parseConnection(const Node& connection_node_seq, core::ProcessGroup* parent) {
+  if (!parent) {
+    logger_->log_error("parseProcessNode: no parent group was provided");
+    return;
+  }
+  if (!connection_node_seq || !connection_node_seq.isSequence()) {
+    return;
+  }
+
+  for (const auto& connection_node : connection_node_seq) {
+    if (!connection_node || !connection_node.isMap()) {
+      logger_->log_error("Invalid connection node, ignoring");
+      continue;
+    }

Review Comment:
   Is it OK to ignore the invalid connection node?  I would expect parsing to fail in this case.



##########
libminifi/src/core/yaml/YamlConfiguration.cpp:
##########
@@ -22,900 +22,66 @@
 #include <cinttypes>
 
 #include "core/yaml/YamlConfiguration.h"
-#include "core/yaml/CheckRequiredField.h"
-#include "core/yaml/YamlConnectionParser.h"
 #include "core/state/Value.h"
 #include "Defaults.h"
 #include "utils/TimeUtil.h"
-
-#ifdef YAML_CONFIGURATION_USE_REGEX
+#include "yaml-cpp/yaml.h"
+#include "core/yaml/YamlNode.h"
 #include "utils/RegexUtils.h"
-#endif  // YAML_CONFIGURATION_USE_REGEX
 
 namespace org::apache::nifi::minifi::core {
 
-std::shared_ptr<utils::IdGenerator> YamlConfiguration::id_generator_ = utils::IdGenerator::getIdGenerator();
-
-YamlConfiguration::YamlConfiguration(const std::shared_ptr<core::Repository>& repo, const std::shared_ptr<core::Repository>& flow_file_repo,
-                                     const std::shared_ptr<core::ContentRepository>& content_repo, const std::shared_ptr<io::StreamFactory>& stream_factory,
-                                     const std::shared_ptr<Configure>& configuration, const std::optional<std::string>& path,
-                                     const std::shared_ptr<utils::file::FileSystem>& filesystem)
-    : FlowConfiguration(repo, flow_file_repo, content_repo, stream_factory, configuration,
-                        path.value_or(DEFAULT_NIFI_CONFIG_YML), filesystem),
-      stream_factory_(stream_factory),
-      logger_(logging::LoggerFactory<YamlConfiguration>::getLogger()) {}
-
-std::unique_ptr<core::ProcessGroup> YamlConfiguration::parseRootProcessGroupYaml(const YAML::Node& rootFlowNode) {
-  auto flowControllerNode = rootFlowNode[CONFIG_YAML_FLOW_CONTROLLER_KEY];
-  auto rootGroup = parseProcessGroupYaml(flowControllerNode, rootFlowNode, true);
-  this->name_ = rootGroup->getName();
-  return rootGroup;
-}
-
-std::unique_ptr<core::ProcessGroup> YamlConfiguration::createProcessGroup(const YAML::Node& yamlNode, bool is_root) {
-  int version = 0;
-
-  yaml::checkRequiredField(yamlNode, "name", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
-  auto flowName = yamlNode["name"].as<std::string>();
-
-  utils::Identifier uuid;
-  // assignment throws on invalid uuid
-  uuid = getOrGenerateId(yamlNode);
-
-  if (yamlNode["version"]) {
-    version = yamlNode["version"].as<int>();
-  }
-
-  logger_->log_debug("parseRootProcessGroup: id => [%s], name => [%s]", uuid.to_string(), flowName);
-  std::unique_ptr<core::ProcessGroup> group;
-  if (is_root) {
-    group = FlowConfiguration::createRootProcessGroup(flowName, uuid, version);
-  } else {
-    group = FlowConfiguration::createSimpleProcessGroup(flowName, uuid, version);
-  }
-
-  if (yamlNode["onschedule retry interval"]) {
-    auto onScheduleRetryPeriod = yamlNode["onschedule retry interval"].as<std::string>();
-    logger_->log_debug("parseRootProcessGroup: onschedule retry period => [%s]", onScheduleRetryPeriod);
-
-    auto on_schedule_retry_period_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(onScheduleRetryPeriod);
-    if (on_schedule_retry_period_value.has_value() && group) {
-      logger_->log_debug("parseRootProcessGroup: onschedule retry => [%" PRId64 "] ms", on_schedule_retry_period_value->count());
-      group->setOnScheduleRetryPeriod(on_schedule_retry_period_value->count());
-    }
-  }
-
-  return group;
-}
-
-std::unique_ptr<core::ProcessGroup> YamlConfiguration::parseProcessGroupYaml(const YAML::Node& headerNode, const YAML::Node& yamlNode, bool is_root) {
-  auto group = createProcessGroup(headerNode, is_root);
-  YAML::Node processorsNode = yamlNode[CONFIG_YAML_PROCESSORS_KEY];
-  YAML::Node connectionsNode = yamlNode[yaml::YamlConnectionParser::CONFIG_YAML_CONNECTIONS_KEY];
-  YAML::Node funnelsNode = yamlNode[CONFIG_YAML_FUNNELS_KEY];
-  YAML::Node remoteProcessingGroupsNode = [&] {
-    // assignment is not supported on invalid Yaml nodes
-    YAML::Node candidate = yamlNode[CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY];
-    if (candidate) {
-      return candidate;
-    }
-    return yamlNode[CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY_V3];
-  }();
-  YAML::Node childProcessGroupNodeSeq = yamlNode["Process Groups"];
-
-  parseProcessorNodeYaml(processorsNode, group.get());
-  parseRemoteProcessGroupYaml(remoteProcessingGroupsNode, group.get());
-  parseFunnelsYaml(funnelsNode, group.get());
-  // parse connections last to give feedback if the source and/or destination
-  // is not in the same process group
-  parseConnectionYaml(connectionsNode, group.get());
-
-  if (childProcessGroupNodeSeq && childProcessGroupNodeSeq.IsSequence()) {
-    for (YAML::const_iterator it = childProcessGroupNodeSeq.begin(); it != childProcessGroupNodeSeq.end(); ++it) {
-      auto childProcessGroupNode = it->as<YAML::Node>();
-      group->addProcessGroup(parseProcessGroupYaml(childProcessGroupNode, childProcessGroupNode));
-    }
-  }
-  return group;
-}
-
-std::unique_ptr<core::ProcessGroup> YamlConfiguration::getYamlRoot(const YAML::Node& rootYamlNode) {
-  uuids_.clear();
-  YAML::Node controllerServiceNode = rootYamlNode[CONFIG_YAML_CONTROLLER_SERVICES_KEY];
-  YAML::Node provenanceReportNode = rootYamlNode[CONFIG_YAML_PROVENANCE_REPORT_KEY];
-
-  parseControllerServices(controllerServiceNode);
-  // Create the root process group
-  std::unique_ptr<core::ProcessGroup> root = parseRootProcessGroupYaml(rootYamlNode);
-  parseProvenanceReportingYaml(provenanceReportNode, root.get());
-
-  // set the controller services into the root group.
-  for (const auto& controller_service : controller_services_->getAllControllerServices()) {
-    root->addControllerService(controller_service->getName(), controller_service);
-    root->addControllerService(controller_service->getUUIDStr(), controller_service);
-  }
-
-  return root;
-}
-
-void YamlConfiguration::parseProcessorNodeYaml(const YAML::Node& processorsNode, core::ProcessGroup* parentGroup) {
-  int64_t runDurationNanos = -1;
-  utils::Identifier uuid;
-  std::unique_ptr<core::Processor> processor;
-
-  if (!parentGroup) {
-    logger_->log_error("parseProcessNodeYaml: no parent group exists");
-    return;
-  }
-
-  if (!processorsNode) {
-    throw std::invalid_argument("Cannot instantiate a MiNiFi instance without a defined Processors configuration node.");
-  }
-  if (!processorsNode.IsSequence()) {
-    throw std::invalid_argument(
-        "Cannot instantiate a MiNiFi instance without a defined Processors configuration node.");
-  }
-  // Evaluate sequence of processors
-  for (YAML::const_iterator iter = processorsNode.begin(); iter != processorsNode.end(); ++iter) {
-    core::ProcessorConfig procCfg;
-    const auto procNode = iter->as<YAML::Node>();
-
-    yaml::checkRequiredField(procNode, "name", CONFIG_YAML_PROCESSORS_KEY);
-    procCfg.name = procNode["name"].as<std::string>();
-    procCfg.id = getOrGenerateId(procNode);
-
-    uuid = procCfg.id;
-    logger_->log_debug("parseProcessorNode: name => [%s] id => [%s]", procCfg.name, procCfg.id);
-    yaml::checkRequiredField(procNode, "class", CONFIG_YAML_PROCESSORS_KEY);
-    procCfg.javaClass = procNode["class"].as<std::string>();
-    logger_->log_debug("parseProcessorNode: class => [%s]", procCfg.javaClass);
-
-    // Determine the processor name only from the Java class
-    auto lastOfIdx = procCfg.javaClass.find_last_of('.');
-    if (lastOfIdx != std::string::npos) {
-      lastOfIdx++;  // if a value is found, increment to move beyond the .
-      std::string processorName = procCfg.javaClass.substr(lastOfIdx);
-      processor = this->createProcessor(processorName, procCfg.javaClass, uuid);
-    } else {
-      // Allow unqualified class names for core processors
-      processor = this->createProcessor(procCfg.javaClass, uuid);
-    }
-
-    if (!processor) {
-      logger_->log_error("Could not create a processor %s with id %s", procCfg.name, procCfg.id);
-      throw std::invalid_argument("Could not create processor " + procCfg.name);
-    }
-
-    processor->setName(procCfg.name);
-
-    processor->setFlowIdentifier(flow_version_->getFlowIdentifier());
-
-    auto strategyNode = getOptionalField(procNode, "scheduling strategy", YAML::Node(DEFAULT_SCHEDULING_STRATEGY),
-    CONFIG_YAML_PROCESSORS_KEY);
-    procCfg.schedulingStrategy = strategyNode.as<std::string>();
-    logger_->log_debug("parseProcessorNode: scheduling strategy => [%s]", procCfg.schedulingStrategy);
-
-    auto periodNode = getOptionalField(procNode, "scheduling period", YAML::Node(DEFAULT_SCHEDULING_PERIOD_STR),
-    CONFIG_YAML_PROCESSORS_KEY);
-
-    procCfg.schedulingPeriod = periodNode.as<std::string>();
-    logger_->log_debug("parseProcessorNode: scheduling period => [%s]", procCfg.schedulingPeriod);
-
-    if (procNode["max concurrent tasks"]) {
-      procCfg.maxConcurrentTasks = procNode["max concurrent tasks"].as<std::string>();
-      logger_->log_debug("parseProcessorNode: max concurrent tasks => [%s]", procCfg.maxConcurrentTasks);
-    }
-
-    if (procNode["penalization period"]) {
-      procCfg.penalizationPeriod = procNode["penalization period"].as<std::string>();
-      logger_->log_debug("parseProcessorNode: penalization period => [%s]", procCfg.penalizationPeriod);
-    }
-
-    if (procNode["yield period"]) {
-      procCfg.yieldPeriod = procNode["yield period"].as<std::string>();
-      logger_->log_debug("parseProcessorNode: yield period => [%s]", procCfg.yieldPeriod);
-    }
-
-    if (procNode["run duration nanos"]) {
-      procCfg.runDurationNanos = procNode["run duration nanos"].as<std::string>();
-      logger_->log_debug("parseProcessorNode: run duration nanos => [%s]", procCfg.runDurationNanos);
-    }
-
-    // handle auto-terminated relationships
-    if (procNode["auto-terminated relationships list"]) {
-      YAML::Node autoTerminatedSequence = procNode["auto-terminated relationships list"];
-      std::vector<std::string> rawAutoTerminatedRelationshipValues;
-      if (autoTerminatedSequence.IsSequence() && !autoTerminatedSequence.IsNull() && autoTerminatedSequence.size() > 0) {
-        for (YAML::const_iterator relIter = autoTerminatedSequence.begin(); relIter != autoTerminatedSequence.end(); ++relIter) {
-          auto autoTerminatedRel = relIter->as<std::string>();
-          rawAutoTerminatedRelationshipValues.push_back(autoTerminatedRel);
-        }
-      }
-      procCfg.autoTerminatedRelationships = rawAutoTerminatedRelationshipValues;
-    }
-
-    // handle processor properties
-    if (procNode["Properties"]) {
-      YAML::Node propertiesNode = procNode["Properties"];
-      parsePropertiesNodeYaml(propertiesNode, *processor, procCfg.name, CONFIG_YAML_PROCESSORS_KEY);
-    }
-
-    // Take care of scheduling
-
-    if (procCfg.schedulingStrategy == "TIMER_DRIVEN" || procCfg.schedulingStrategy == "EVENT_DRIVEN") {
-      if (auto scheduling_period = utils::timeutils::StringToDuration<std::chrono::nanoseconds>(procCfg.schedulingPeriod)) {
-        logger_->log_debug("convert: parseProcessorNode: schedulingPeriod => [%" PRId64 "] ns", scheduling_period->count());
-        processor->setSchedulingPeriodNano(*scheduling_period);
-      }
-    } else {
-      processor->setCronPeriod(procCfg.schedulingPeriod);
-    }
-
-    if (auto penalization_period = utils::timeutils::StringToDuration<std::chrono::milliseconds>(procCfg.penalizationPeriod)) {
-      logger_->log_debug("convert: parseProcessorNode: penalizationPeriod => [%" PRId64 "] ms", penalization_period->count());
-      processor->setPenalizationPeriod(penalization_period.value());
-    }
-
-    if (auto yield_period = utils::timeutils::StringToDuration<std::chrono::milliseconds>(procCfg.yieldPeriod)) {
-      logger_->log_debug("convert: parseProcessorNode: yieldPeriod => [%" PRId64 "] ms", yield_period->count());
-      processor->setYieldPeriodMsec(yield_period.value());
-    }
-
-    // Default to running
-    processor->setScheduledState(core::RUNNING);
-
-    if (procCfg.schedulingStrategy == "TIMER_DRIVEN") {
-      processor->setSchedulingStrategy(core::TIMER_DRIVEN);
-      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
-    } else if (procCfg.schedulingStrategy == "EVENT_DRIVEN") {
-      processor->setSchedulingStrategy(core::EVENT_DRIVEN);
-      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
-    } else {
-      processor->setSchedulingStrategy(core::CRON_DRIVEN);
-      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
-    }
-
-    int32_t maxConcurrentTasks;
-    if (core::Property::StringToInt(procCfg.maxConcurrentTasks, maxConcurrentTasks)) {
-      logger_->log_debug("parseProcessorNode: maxConcurrentTasks => [%d]", maxConcurrentTasks);
-      processor->setMaxConcurrentTasks((uint8_t) maxConcurrentTasks);
-    }
-
-    if (core::Property::StringToInt(procCfg.runDurationNanos, runDurationNanos)) {
-      logger_->log_debug("parseProcessorNode: runDurationNanos => [%d]", runDurationNanos);
-      processor->setRunDurationNano(std::chrono::nanoseconds(runDurationNanos));
-    }
-
-    std::vector<core::Relationship> autoTerminatedRelationships;
-    for (auto &&relString : procCfg.autoTerminatedRelationships) {
-      core::Relationship relationship(relString, "");
-      logger_->log_debug("parseProcessorNode: autoTerminatedRelationship  => [%s]", relString);
-      autoTerminatedRelationships.push_back(relationship);
-    }
-
-    processor->setAutoTerminatedRelationships(autoTerminatedRelationships);
-
-    parentGroup->addProcessor(std::move(processor));
-  }
-}
-
-void YamlConfiguration::parseRemoteProcessGroupYaml(const YAML::Node& rpgNode, core::ProcessGroup* parentGroup) {
-  utils::Identifier uuid;
-  std::string id;
-
-  if (!parentGroup) {
-    logger_->log_error("parseRemoteProcessGroupYaml: no parent group exists");
-    return;
-  }
-
-  if (!rpgNode || !rpgNode.IsSequence()) {
-    return;
-  }
-  for (YAML::const_iterator iter = rpgNode.begin(); iter != rpgNode.end(); ++iter) {
-    auto currRpgNode = iter->as<YAML::Node>();
-
-    yaml::checkRequiredField(currRpgNode, "name", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
-    auto name = currRpgNode["name"].as<std::string>();
-    id = getOrGenerateId(currRpgNode);
-
-    logger_->log_debug("parseRemoteProcessGroupYaml: name => [%s], id => [%s]", name, id);
-
-    auto urlNode = getOptionalField(currRpgNode, "url", YAML::Node(""),
-    CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
-
-    auto url = urlNode.as<std::string>();
-    logger_->log_debug("parseRemoteProcessGroupYaml: url => [%s]", url);
-
-    uuid = id;
-    auto group = this->createRemoteProcessGroup(name, uuid);
-    group->setParent(parentGroup);
-
-    if (currRpgNode["yield period"]) {
-      auto yieldPeriod = currRpgNode["yield period"].as<std::string>();
-      logger_->log_debug("parseRemoteProcessGroupYaml: yield period => [%s]", yieldPeriod);
-
-      auto yield_period_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(yieldPeriod);
-      if (yield_period_value.has_value() && group) {
-        logger_->log_debug("parseRemoteProcessGroupYaml: yieldPeriod => [%" PRId64 "] ms", yield_period_value->count());
-        group->setYieldPeriodMsec(*yield_period_value);
-      }
-    }
-
-    if (currRpgNode["timeout"]) {
-      auto timeout = currRpgNode["timeout"].as<std::string>();
-      logger_->log_debug("parseRemoteProcessGroupYaml: timeout => [%s]", timeout);
-
-      auto timeout_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(timeout);
-      if (timeout_value.has_value() && group) {
-        logger_->log_debug("parseRemoteProcessGroupYaml: timeoutValue => [%" PRId64 "] ms", timeout_value->count());
-        group->setTimeout(timeout_value->count());
-      }
-    }
-
-    if (currRpgNode["local network interface"]) {
-      auto interface = currRpgNode["local network interface"].as<std::string>();
-      logger_->log_debug("parseRemoteProcessGroupYaml: local network interface => [%s]", interface);
-      group->setInterface(interface);
-    }
-
-    if (currRpgNode["transport protocol"]) {
-      auto transport_protocol = currRpgNode["transport protocol"].as<std::string>();
-      logger_->log_debug("parseRemoteProcessGroupYaml: transport protocol => [%s]", transport_protocol);
-      if (transport_protocol == "HTTP") {
-        group->setTransportProtocol(transport_protocol);
-        if (currRpgNode["proxy host"]) {
-          auto http_proxy_host = currRpgNode["proxy host"].as<std::string>();
-          logger_->log_debug("parseRemoteProcessGroupYaml: proxy host => [%s]", http_proxy_host);
-          group->setHttpProxyHost(http_proxy_host);
-          if (currRpgNode["proxy user"]) {
-            auto http_proxy_username = currRpgNode["proxy user"].as<std::string>();
-            logger_->log_debug("parseRemoteProcessGroupYaml: proxy user => [%s]", http_proxy_username);
-            group->setHttpProxyUserName(http_proxy_username);
-          }
-          if (currRpgNode["proxy password"]) {
-            auto http_proxy_password = currRpgNode["proxy password"].as<std::string>();
-            logger_->log_debug("parseRemoteProcessGroupYaml: proxy password => [%s]", http_proxy_password);
-            group->setHttpProxyPassWord(http_proxy_password);
+YamlConfiguration::YamlConfiguration(ConfigurationContext ctx)
+    : StructuredConfiguration(([&] {
+          if (!ctx.path) {
+            ctx.path = DEFAULT_NIFI_CONFIG_YML;
           }
-          if (currRpgNode["proxy port"]) {
-            auto http_proxy_port = currRpgNode["proxy port"].as<std::string>();
-            int32_t port;
-            if (core::Property::StringToInt(http_proxy_port, port)) {
-              logger_->log_debug("parseRemoteProcessGroupYaml: proxy port => [%d]", port);
-              group->setHttpProxyPort(port);
-            }
-          }
-        }
-      } else if (transport_protocol == "RAW") {
-        group->setTransportProtocol(transport_protocol);
-      } else {
-        std::stringstream stream;
-        stream << "Invalid transport protocol " << transport_protocol;
-        throw minifi::Exception(ExceptionType::SITE2SITE_EXCEPTION, stream.str().c_str());
-      }
-    }
-
-    group->setTransmitting(true);
-    group->setURL(url);
-
-    yaml::checkRequiredField(currRpgNode, "Input Ports", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
-    auto inputPorts = currRpgNode["Input Ports"].as<YAML::Node>();
-    if (inputPorts && inputPorts.IsSequence()) {
-      for (YAML::const_iterator portIter = inputPorts.begin(); portIter != inputPorts.end(); ++portIter) {
-        auto currPort = portIter->as<YAML::Node>();
-
-        this->parsePortYaml(currPort, group.get(), sitetosite::SEND);
-      }  // for node
-    }
-    auto outputPorts = currRpgNode["Output Ports"].as<YAML::Node>();
-    if (outputPorts && outputPorts.IsSequence()) {
-      for (YAML::const_iterator portIter = outputPorts.begin(); portIter != outputPorts.end(); ++portIter) {
-        logger_->log_debug("Got a current port, iterating...");
-
-        auto currPort = portIter->as<YAML::Node>();
-
-        this->parsePortYaml(currPort, group.get(), sitetosite::RECEIVE);
-      }  // for node
-    }
-    parentGroup->addProcessGroup(std::move(group));
-  }
-}
-
-void YamlConfiguration::parseProvenanceReportingYaml(const YAML::Node& reportNode, core::ProcessGroup* parentGroup) {
-  utils::Identifier port_uuid;
+          return std::move(ctx);
+        })(),
+        logging::LoggerFactory<YamlConfiguration>::getLogger()) {}
 
-  if (!parentGroup) {
-    logger_->log_error("parseProvenanceReportingYaml: no parent group exists");
-    return;
+std::unique_ptr<core::ProcessGroup> YamlConfiguration::getRoot() {
+  if (!config_path_) {
+    logger_->log_error("Cannot instantiate flow, no config file is set.");
+    throw Exception(ExceptionType::FLOW_EXCEPTION, "No config file specified");
   }
-
-  if (!reportNode || !reportNode.IsDefined() || reportNode.IsNull()) {
-    logger_->log_debug("no provenance reporting task specified");
-    return;
-  }
-
-  auto reportTask = createProvenanceReportTask();
-
-  const auto node = reportNode.as<YAML::Node>();
-
-  yaml::checkRequiredField(node, "scheduling strategy", CONFIG_YAML_PROVENANCE_REPORT_KEY);
-  auto schedulingStrategyStr = node["scheduling strategy"].as<std::string>();
-  yaml::checkRequiredField(node, "scheduling period", CONFIG_YAML_PROVENANCE_REPORT_KEY);
-  auto schedulingPeriodStr = node["scheduling period"].as<std::string>();
-
-  if (auto scheduling_period = utils::timeutils::StringToDuration<std::chrono::nanoseconds>(schedulingPeriodStr)) {
-    logger_->log_debug("ProvenanceReportingTask schedulingPeriod %" PRId64 " ns", scheduling_period->count());
-    reportTask->setSchedulingPeriodNano(*scheduling_period);
+  const auto configuration = filesystem_->read(config_path_.value());
+  if (!configuration) {
+  // non-existence of flow config file is not a dealbreaker, the caller might fetch it from network
+  return nullptr;
   }
-
-  if (schedulingStrategyStr == "TIMER_DRIVEN") {
-    reportTask->setSchedulingStrategy(core::TIMER_DRIVEN);
-    logger_->log_debug("ProvenanceReportingTask scheduling strategy %s", schedulingStrategyStr);
-  } else {
-    throw std::invalid_argument("Invalid scheduling strategy " + schedulingStrategyStr);
-  }
-
-  int64_t lvalue;
-  if (node["host"] && node["port"]) {
-    auto hostStr = node["host"].as<std::string>();
-
-    auto portStr = node["port"].as<std::string>();
-    if (core::Property::StringToInt(portStr, lvalue) && !hostStr.empty()) {
-      logger_->log_debug("ProvenanceReportingTask port %" PRId64, lvalue);
-      std::string url = hostStr + ":" + portStr;
-      reportTask->setURL(url);
-    }
-  }
-
-  if (node["url"]) {
-    auto urlStr = node["url"].as<std::string>();
-    if (!urlStr.empty()) {
-      reportTask->setURL(urlStr);
-      logger_->log_debug("ProvenanceReportingTask URL %s", urlStr);
-    }
-  }
-  yaml::checkRequiredField(node, "port uuid", CONFIG_YAML_PROVENANCE_REPORT_KEY);
-  auto portUUIDStr = node["port uuid"].as<std::string>();
-  yaml::checkRequiredField(node, "batch size", CONFIG_YAML_PROVENANCE_REPORT_KEY);
-  auto batchSizeStr = node["batch size"].as<std::string>();
-
-  logger_->log_debug("ProvenanceReportingTask port uuid %s", portUUIDStr);
-  port_uuid = portUUIDStr;
-  reportTask->setPortUUID(port_uuid);
-
-  if (core::Property::StringToInt(batchSizeStr, lvalue)) {
-    reportTask->setBatchSize(gsl::narrow<int>(lvalue));
-  }
-
-  reportTask->initialize();
-
-  // add processor to parent
-  reportTask->setScheduledState(core::RUNNING);
-  parentGroup->addProcessor(std::move(reportTask));
-}
-
-void YamlConfiguration::parseControllerServices(const YAML::Node& controllerServicesNode) {
-  if (!controllerServicesNode || !controllerServicesNode.IsSequence()) {
-    return;
-  }
-  for (const auto& iter : controllerServicesNode) {
-    const auto controllerServiceNode = iter.as<YAML::Node>();
-    try {
-      yaml::checkRequiredField(controllerServiceNode, "name", CONFIG_YAML_CONTROLLER_SERVICES_KEY);
-
-      auto type = yaml::getRequiredField(controllerServiceNode, std::vector<std::string>{"class", "type"}, CONFIG_YAML_CONTROLLER_SERVICES_KEY);
-      logger_->log_debug("Using type %s for controller service node", type);
-
-      std::string fullType = type;
-      auto lastOfIdx = type.find_last_of('.');
-      if (lastOfIdx != std::string::npos) {
-        lastOfIdx++;  // if a value is found, increment to move beyond the .
-        type = type.substr(lastOfIdx);
-      }
-
-      auto name = controllerServiceNode["name"].as<std::string>();
-      auto id = getRequiredIdField(controllerServiceNode, CONFIG_YAML_CONTROLLER_SERVICES_KEY);
-
-      utils::Identifier uuid;
-      uuid = id;
-      std::shared_ptr<core::controller::ControllerServiceNode> controller_service_node = createControllerService(type, fullType, name, uuid);
-      if (nullptr != controller_service_node) {
-        logger_->log_debug("Created Controller Service with UUID %s and name %s", id, name);
-        controller_service_node->initialize();
-        YAML::Node propertiesNode = controllerServiceNode["Properties"];
-        // we should propagate properties to the node and to the implementation
-        parsePropertiesNodeYaml(propertiesNode, *controller_service_node, name, CONFIG_YAML_CONTROLLER_SERVICES_KEY);
-        if (auto controllerServiceImpl = controller_service_node->getControllerServiceImplementation(); controllerServiceImpl) {
-          parsePropertiesNodeYaml(propertiesNode, *controllerServiceImpl, name, CONFIG_YAML_CONTROLLER_SERVICES_KEY);
-        }
-      } else {
-        logger_->log_debug("Could not locate %s", type);
-      }
-      controller_services_->put(id, controller_service_node);
-      controller_services_->put(name, controller_service_node);
-    } catch (YAML::InvalidNode &) {
-      throw Exception(ExceptionType::GENERAL_EXCEPTION, "Name, id, and class must be specified for controller services");
-    }
-  }
-}
-
-void YamlConfiguration::parseConnectionYaml(const YAML::Node& connectionsNode, core::ProcessGroup* parent) {
-  if (!parent) {
-    logger_->log_error("parseProcessNode: no parent group was provided");
-    return;
-  }
-  if (!connectionsNode || !connectionsNode.IsSequence()) {
-    return;
-  }
-
-  for (YAML::const_iterator iter = connectionsNode.begin(); iter != connectionsNode.end(); ++iter) {
-    const auto connectionNode = iter->as<YAML::Node>();
-
-    // Configure basic connection
-    const std::string id = getOrGenerateId(connectionNode);
-
-    // Default name to be same as ID
-    // If name is specified in configuration, use the value
-    const auto name = connectionNode["name"].as<std::string>(id);
-
-    const auto uuid = utils::Identifier::parse(id) | utils::orElse([this] {
-      logger_->log_debug("Incorrect connection UUID format.");
-      throw Exception(ExceptionType::GENERAL_EXCEPTION, "Incorrect connection UUID format.");
-    });
-
-    auto connection = createConnection(name, uuid.value());
-    logger_->log_debug("Created connection with UUID %s and name %s", id, name);
-    const yaml::YamlConnectionParser connectionParser(connectionNode, name, gsl::not_null<core::ProcessGroup*>{ parent }, logger_);
-    connectionParser.configureConnectionSourceRelationshipsFromYaml(*connection);
-    connection->setMaxQueueSize(connectionParser.getWorkQueueSizeFromYaml());
-    connection->setMaxQueueDataSize(connectionParser.getWorkQueueDataSizeFromYaml());
-    connection->setSwapThreshold(connectionParser.getSwapThresholdFromYaml());
-    connection->setSourceUUID(connectionParser.getSourceUUIDFromYaml());
-    connection->setDestinationUUID(connectionParser.getDestinationUUIDFromYaml());
-    connection->setFlowExpirationDuration(connectionParser.getFlowFileExpirationFromYaml());
-    connection->setDropEmptyFlowFiles(connectionParser.getDropEmptyFromYaml());
-
-    parent->addConnection(std::move(connection));
-  }
-}
-
-void YamlConfiguration::parsePortYaml(const YAML::Node& portNode, core::ProcessGroup* parent, sitetosite::TransferDirection direction) {
-  utils::Identifier uuid;
-
-  if (!parent) {
-    logger_->log_error("parseProcessNode: no parent group existed");
-    return;
-  }
-
-  const auto inputPortsObj = portNode.as<YAML::Node>();
-
-  // Check for required fields
-  yaml::checkRequiredField(inputPortsObj, "name", CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
-  auto nameStr = inputPortsObj["name"].as<std::string>();
-  auto portId = getRequiredIdField(inputPortsObj, CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY,
-    "The field 'id' is required for "
-    "the port named '" + nameStr + "' in the YAML Config. If this port "
-    "is an input port for a NiFi Remote Process Group, the port "
-    "id should match the corresponding id specified in the NiFi configuration. "
-    "This is a UUID of the format XXXXXXXX-XXXX-XXXX-XXXX-XXXXXXXXXXXX.");
-  uuid = portId;
-
-  auto port = std::make_unique<minifi::RemoteProcessorGroupPort>(
-          stream_factory_, nameStr, parent->getURL(), this->configuration_, uuid);
-  port->setDirection(direction);
-  port->setTimeout(parent->getTimeout());
-  port->setTransmitting(true);
-  port->setYieldPeriodMsec(parent->getYieldPeriodMsec());
-  port->initialize();
-  if (!parent->getInterface().empty())
-    port->setInterface(parent->getInterface());
-  if (parent->getTransportProtocol() == "HTTP") {
-    port->enableHTTP();
-    if (!parent->getHttpProxyHost().empty())
-      port->setHTTPProxy(parent->getHTTPProxy());
-  }
-  // else defaults to RAW
-
-  // handle port properties
-  const auto nodeVal = portNode.as<YAML::Node>();
-  YAML::Node propertiesNode = nodeVal["Properties"];
-  parsePropertiesNodeYaml(propertiesNode, *port, nameStr, CONFIG_YAML_REMOTE_PROCESS_GROUP_KEY);
-
-  // add processor to parent
-  auto& processor = *port;
-  parent->addProcessor(std::move(port));
-  processor.setScheduledState(core::RUNNING);
-
-  if (inputPortsObj["max concurrent tasks"]) {
-    auto rawMaxConcurrentTasks = inputPortsObj["max concurrent tasks"].as<std::string>();
-    int32_t maxConcurrentTasks;
-    if (core::Property::StringToInt(rawMaxConcurrentTasks, maxConcurrentTasks)) {
-      processor.setMaxConcurrentTasks(maxConcurrentTasks);
-    }
-    logger_->log_debug("parseProcessorNode: maxConcurrentTasks => [%d]", maxConcurrentTasks);
-    processor.setMaxConcurrentTasks(maxConcurrentTasks);
-  }
-}
-
-void YamlConfiguration::parsePropertyValueSequence(const std::string& propertyName, const YAML::Node& propertyValueNode, core::ConfigurableComponent& component) {
-  for (const auto& iter : propertyValueNode) {
-    if (iter.IsDefined()) {
-      const auto nodeVal = iter.as<YAML::Node>();
-      YAML::Node propertiesNode = nodeVal["value"];
-      // must insert the sequence in differently.
-      const auto rawValueString = propertiesNode.as<std::string>();
-      logger_->log_debug("Found %s=%s", propertyName, rawValueString);
-      if (!component.updateProperty(propertyName, rawValueString)) {
-        auto proc = dynamic_cast<core::Connectable*>(&component);
-        if (proc) {
-          logger_->log_warn("Received property %s with value %s but is not one of the properties for %s. Attempting to add as dynamic property.", propertyName, rawValueString, proc->getName());
-          if (!component.setDynamicProperty(propertyName, rawValueString)) {
-            logger_->log_warn("Unable to set the dynamic property %s with value %s", propertyName, rawValueString);
-          } else {
-            logger_->log_warn("Dynamic property %s with value %s set", propertyName, rawValueString);
-          }
-        }
-      }
-    }
-  }
-}
-
-PropertyValue YamlConfiguration::getValidatedProcessorPropertyForDefaultTypeInfo(const core::Property& propertyFromProcessor, const YAML::Node& propertyValueNode) {
-  PropertyValue defaultValue;
-  defaultValue = propertyFromProcessor.getDefaultValue();
-  const std::type_index defaultType = defaultValue.getTypeInfo();
   try {
-    PropertyValue coercedValue = defaultValue;
-    if (defaultType == typeid(int64_t)) {
-      coercedValue = propertyValueNode.as<int64_t>();
-    } else if (defaultType == typeid(uint64_t)) {
-      uint64_t integer_value;
-      if (YAML::convert<uint64_t>::decode(propertyValueNode, integer_value)) {
-        coercedValue = integer_value;
-      } else {
-        coercedValue = propertyValueNode.as<std::string>();
-      }
-    } else if (defaultType == typeid(int)) {
-      coercedValue = propertyValueNode.as<int>();
-    } else if (defaultType == typeid(bool)) {
-      coercedValue = propertyValueNode.as<bool>();
-    } else {
-      coercedValue = propertyValueNode.as<std::string>();
-    }
-    return coercedValue;
-  } catch (const std::exception& e) {
-    logger_->log_error("Fetching property failed with an exception of %s", e.what());
-    logger_->log_error("Invalid conversion for field %s. Value %s", propertyFromProcessor.getName(), propertyValueNode.as<std::string>());
-  } catch (...) {
-    logger_->log_error("Invalid conversion for field %s. Value %s", propertyFromProcessor.getName(), propertyValueNode.as<std::string>());
-  }
-  return defaultValue;
-}
-
-void YamlConfiguration::parseSingleProperty(const std::string& propertyName, const YAML::Node& propertyValueNode, core::ConfigurableComponent& processor) {
-  core::Property myProp(propertyName, "", "");
-  processor.getProperty(propertyName, myProp);
-  const PropertyValue coercedValue = getValidatedProcessorPropertyForDefaultTypeInfo(myProp, propertyValueNode);
-  bool property_set = false;
-  try {
-    property_set = processor.setProperty(myProp, coercedValue);
-  } catch(const utils::internal::InvalidValueException&) {
-    auto component = dynamic_cast<core::CoreComponent*>(&processor);
-    if (component == nullptr) {
-      logger_->log_error("processor was not a CoreComponent for property '%s'", propertyName);
-    } else {
-      logger_->log_error("Invalid value was set for property '%s' creating component '%s'", propertyName, component->getName());
-    }
+    YAML::Node rootYamlNode = YAML::Load(configuration.value());
+    flow::Node root{std::make_shared<YamlNode>(rootYamlNode)};
+    return getRootFrom(root);
+  } catch(...) {
+    logger_->log_error("Invalid yaml configuration file");
     throw;
   }
-  const auto rawValueString = propertyValueNode.as<std::string>();
-  if (!property_set) {
-    auto proc = dynamic_cast<core::Connectable*>(&processor);
-    if (proc) {
-      logger_->log_warn("Received property %s with value %s but is not one of the properties for %s. Attempting to add as dynamic property.", propertyName, rawValueString, proc->getName());
-      if (!processor.setDynamicProperty(propertyName, rawValueString)) {
-        logger_->log_warn("Unable to set the dynamic property %s with value %s", propertyName, rawValueString);
-      } else {
-        logger_->log_warn("Dynamic property %s with value %s set", propertyName, rawValueString);
-      }
-    }
-  } else {
-    logger_->log_debug("Property %s with value %s set", propertyName, rawValueString);
-  }
-}
-
-void YamlConfiguration::parsePropertyNodeElement(const std::string& propertyName, const YAML::Node& propertyValueNode, core::ConfigurableComponent& processor) {
-  logger_->log_trace("Encountered %s", propertyName);
-  if (propertyValueNode.IsNull() || !propertyValueNode.IsDefined()) {
-    return;
-  }
-  if (propertyValueNode.IsSequence()) {
-    parsePropertyValueSequence(propertyName, propertyValueNode, processor);
-  } else {
-    parseSingleProperty(propertyName, propertyValueNode, processor);
-  }
-}
-
-void YamlConfiguration::parsePropertiesNodeYaml(const YAML::Node& propertiesNode, core::ConfigurableComponent& component, const std::string& component_name,
-    const std::string& yaml_section) {
-  // Treat generically as a YAML node so we can perform inspection on entries to ensure they are populated
-  logger_->log_trace("Entered %s", component_name);
-  for (const auto& propertyElem : propertiesNode) {
-    const auto propertyName = propertyElem.first.as<std::string>();
-    const YAML::Node propertyValueNode = propertyElem.second;
-    parsePropertyNodeElement(propertyName, propertyValueNode, component);
-  }
-
-  validateComponentProperties(component, component_name, yaml_section);
 }
 
-void YamlConfiguration::parseFunnelsYaml(const YAML::Node& node, core::ProcessGroup* parent) {
-  if (!parent) {
-    logger_->log_error("parseFunnelsYaml: no parent group was provided");
-    return;
-  }
-  if (!node || !node.IsSequence()) {
-    return;
-  }
-
-  for (const auto& element : node) {
-    const auto funnel_node = element.as<YAML::Node>();
-
-    std::string id = getOrGenerateId(funnel_node);
-
-    // Default name to be same as ID
-    const auto name = funnel_node["name"].as<std::string>(id);
-
-    const auto uuid = utils::Identifier::parse(id) | utils::orElse([this] {
-      logger_->log_debug("Incorrect funnel UUID format.");
-      throw Exception(ExceptionType::GENERAL_EXCEPTION, "Incorrect funnel UUID format.");
-    });
-
-    auto funnel = std::make_unique<core::Funnel>(name, uuid.value());
-    logger_->log_debug("Created funnel with UUID %s and name %s", id, name);
-    funnel->setScheduledState(core::RUNNING);
-    funnel->setSchedulingStrategy(core::EVENT_DRIVEN);
-    parent->addProcessor(std::move(funnel));
-  }
-}
-
-void YamlConfiguration::validateComponentProperties(ConfigurableComponent& component, const std::string &component_name, const std::string &yaml_section) const {
-  const auto &component_properties = component.getProperties();
-
-  // Validate required properties
-  for (const auto &prop_pair : component_properties) {
-    if (prop_pair.second.getRequired()) {
-      if (prop_pair.second.getValue().to_string().empty()) {
-        std::string reason = utils::StringUtils::join_pack("required property '", prop_pair.second.getName(), "' is not set");
-        raiseComponentError(component_name, yaml_section, reason);
-      } else if (!prop_pair.second.getValue().validate(prop_pair.first).valid()) {
-        std::string reason = utils::StringUtils::join_pack("the value '", prop_pair.first, "' is not valid for property '", prop_pair.second.getName(), "'");
-        raiseComponentError(component_name, yaml_section, reason);
-      }
-    }
-  }
-
-  // Validate dependent properties
-  for (const auto &prop_pair : component_properties) {
-    const auto &dep_props = prop_pair.second.getDependentProperties();
-
-    if (prop_pair.second.getValue().to_string().empty()) {
-      continue;
-    }
-
-    for (const auto &dep_prop_key : dep_props) {
-      if (component_properties.at(dep_prop_key).getValue().to_string().empty()) {
-        std::string reason = utils::StringUtils::join_pack("property '", prop_pair.second.getName(),
-            "' depends on property '", dep_prop_key, "' which is not set");
-        raiseComponentError(component_name, yaml_section, reason);
-      }
-    }
-  }
-
-#ifdef YAML_CONFIGURATION_USE_REGEX
-  // Validate mutually-exclusive properties
-  for (const auto &prop_pair : component_properties) {
-    const auto &excl_props = prop_pair.second.getExclusiveOfProperties();
-
-    if (prop_pair.second.getValue().empty()) {
-      continue;
-    }
-
-    for (const auto &excl_pair : excl_props) {
-      utils::Regex excl_expr(excl_pair.second);
-      if (utils::regexMatch(component_properties.at(excl_pair.first).getValue().to_string(), excl_expr)) {
-        std::string reason = utils::StringUtils::join_pack("property '", prop_pair.second.getName(),
-            "' must not be set when the value of property '", excl_pair.first, "' matches '", excl_pair.second, "'");
-        raiseComponentError(component_name, yaml_section, reason);
-      }
-    }
-  }
-
-  // Validate regex properties
-  for (const auto &prop_pair : component_properties) {
-    const auto &prop_regex_str = prop_pair.second.getValidRegex();
-
-    if (!prop_regex_str.empty()) {
-      utils::Regex prop_regex(prop_regex_str);
-      if (!utils::regexMatch(prop_pair.second.getValue().to_string(), prop_regex)) {
-        std::string reason = utils::StringUtils::join_pack("property '", prop_pair.second.getName(), "' does not match validation pattern '", prop_regex_str, "'");
-        raiseComponentError(component_name, yaml_section, reason);
-      }
-    }
-  }
-#else
-  logging::LOG_INFO(logger_) << "Validation of mutally-exclusive properties is disabled in this build.";
-  logging::LOG_INFO(logger_) << "Regex validation of properties is not available in this build.";
-#endif  // YAML_CONFIGURATION_USE_REGEX
-}
-
-void YamlConfiguration::raiseComponentError(const std::string &component_name, const std::string &yaml_section, const std::string &reason) const {
-  std::string err_msg = "Unable to parse configuration file for component named '";
-  err_msg.append(component_name);
-  err_msg.append("' because " + reason);
-  if (!yaml_section.empty()) {
-    err_msg.append(" [in '" + yaml_section + "' section of configuration file]");
-  }
-
-  logging::LOG_ERROR(logger_) << err_msg;
-
-  throw std::invalid_argument(err_msg);
-}
-
-std::string YamlConfiguration::getOrGenerateId(const YAML::Node& yamlNode, const std::string& idField) {
-  std::string id;
-  auto node = yamlNode.as<YAML::Node>();
-
-  if (node[idField]) {
-    if (YAML::NodeType::Scalar == node[idField].Type()) {
-      id = node[idField].as<std::string>();
-      addNewId(id);
-      return id;
-    }
-    throw std::invalid_argument("getOrGenerateId: idField is expected to reference YAML::Node of YAML::NodeType::Scalar.");
-  }
-
-  id = id_generator_->generate().to_string();
-  logger_->log_debug("Generating random ID: id => [%s]", id);
-  return id;
-}
-
-std::string YamlConfiguration::getRequiredIdField(const YAML::Node& yaml_node, std::string_view yaml_section, std::string_view error_message) {
-  yaml::checkRequiredField(yaml_node, "id", yaml_section, error_message);
-  auto id = yaml_node["id"].as<std::string>();
-  addNewId(id);
-  return id;
-}
-
-YAML::Node YamlConfiguration::getOptionalField(const YAML::Node& yamlNode, const std::string& fieldName, const YAML::Node& defaultValue, const std::string& yamlSection,
-                                               const std::string& providedInfoMessage) {
-  std::string infoMessage = providedInfoMessage;
-  auto result = yamlNode.as<YAML::Node>()[fieldName];
-  if (!result) {
-    if (infoMessage.empty()) {
-      // Build a helpful info message for the user to inform them that a default is being used
-      infoMessage =
-          yamlNode.as<YAML::Node>()["name"] ?
-              "Using default value for optional field '" + fieldName + "' in component named '" + yamlNode.as<YAML::Node>()["name"].as<std::string>() + "'" :
-              "Using default value for optional field '" + fieldName + "' ";
-      if (!yamlSection.empty()) {
-        infoMessage += " [in '" + yamlSection + "' section of configuration file]: ";
-      }
-
-      infoMessage += defaultValue.as<std::string>();
-    }
-    logging::LOG_INFO(logger_) << infoMessage;
-    result = defaultValue;
+std::unique_ptr<core::ProcessGroup> YamlConfiguration::getYamlRoot(std::istream &yamlConfigStream) {
+  try {
+    YAML::Node rootYamlNode = YAML::Load(yamlConfigStream);
+    flow::Node root{std::make_shared<YamlNode>(rootYamlNode)};
+    return getRootFrom(root);
+  } catch (const YAML::ParserException &pe) {
+    logger_->log_error(pe.what());
+    std::rethrow_exception(std::current_exception());
   }
-
-  return result;
+  return nullptr;

Review Comment:
   This is unreachable code (also at line 84).  Does one of our supported compilers insist on it?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r989811291


##########
libminifi/src/core/flow/StructuredConfiguration.cpp:
##########
@@ -0,0 +1,896 @@
+/**
+ *
+ * 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 <memory>
+#include <vector>
+#include <set>
+#include <cinttypes>
+
+#include "core/flow/StructuredConfiguration.h"
+#include "core/flow/CheckRequiredField.h"
+#include "core/flow/StructuredConnectionParser.h"
+#include "core/state/Value.h"
+#include "Defaults.h"
+#include "utils/TimeUtil.h"
+#include "utils/RegexUtils.h"
+
+namespace org::apache::nifi::minifi::core::flow {
+
+std::shared_ptr<utils::IdGenerator> StructuredConfiguration::id_generator_ = utils::IdGenerator::getIdGenerator();
+
+StructuredConfiguration::StructuredConfiguration(ConfigurationContext ctx, std::shared_ptr<logging::Logger> logger)
+    : FlowConfiguration(std::move(ctx)),
+      logger_(std::move(logger)) {}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::parseRootProcessGroup(const Node& root_flow_node) {
+  auto flow_controller_node = root_flow_node[CONFIG_FLOW_CONTROLLER_KEY];
+  auto root_group = parseProcessGroup(flow_controller_node, root_flow_node, true);
+  this->name_ = root_group->getName();
+  return root_group;
+}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::createProcessGroup(const Node& node, bool is_root) {
+  int version = 0;
+
+  checkRequiredField(node, "name", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+  auto flowName = node["name"].getString().value();
+
+  utils::Identifier uuid;
+  // assignment throws on invalid uuid
+  uuid = getOrGenerateId(node);
+
+  if (node["version"]) {
+    version = node["version"].getInt().value();
+  }
+
+  logger_->log_debug("parseRootProcessGroup: id => [%s], name => [%s]", uuid.to_string(), flowName);
+  std::unique_ptr<core::ProcessGroup> group;
+  if (is_root) {
+    group = FlowConfiguration::createRootProcessGroup(flowName, uuid, version);
+  } else {
+    group = FlowConfiguration::createSimpleProcessGroup(flowName, uuid, version);
+  }
+
+  if (node["onschedule retry interval"]) {
+    auto onScheduleRetryPeriod = node["onschedule retry interval"].getString().value();
+    logger_->log_debug("parseRootProcessGroup: onschedule retry period => [%s]", onScheduleRetryPeriod);
+
+    auto on_schedule_retry_period_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(onScheduleRetryPeriod);
+    if (on_schedule_retry_period_value.has_value() && group) {
+      logger_->log_debug("parseRootProcessGroup: onschedule retry => [%" PRId64 "] ms", on_schedule_retry_period_value->count());
+      group->setOnScheduleRetryPeriod(on_schedule_retry_period_value->count());
+    }
+  }
+
+  return group;
+}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::parseProcessGroup(const Node& headerNode, const Node& yamlNode, bool is_root) {
+  auto group = createProcessGroup(headerNode, is_root);
+  Node processorsNode = yamlNode[CONFIG_PROCESSORS_KEY];
+  Node connectionsNode = yamlNode[StructuredConnectionParser::CONFIG_CONNECTIONS_KEY];
+  Node funnelsNode = yamlNode[CONFIG_FUNNELS_KEY];
+  Node remoteProcessingGroupsNode = [&] {
+    // assignment is not supported on invalid Yaml nodes
+    Node candidate = yamlNode[CONFIG_REMOTE_PROCESS_GROUP_KEY];
+    if (candidate) {
+      return candidate;
+    }
+    return yamlNode[CONFIG_REMOTE_PROCESS_GROUP_KEY_V3];
+  }();
+  Node childProcessGroupNodeSeq = yamlNode["Process Groups"];
+
+  parseProcessorNode(processorsNode, group.get());
+  parseRemoteProcessGroup(remoteProcessingGroupsNode, group.get());
+  parseFunnels(funnelsNode, group.get());
+  // parse connections last to give feedback if the source and/or destination
+  // is not in the same process group
+  parseConnection(connectionsNode, group.get());
+
+  if (childProcessGroupNodeSeq && childProcessGroupNodeSeq.isSequence()) {
+    for (const auto childProcessGroupNode : childProcessGroupNodeSeq) {
+      group->addProcessGroup(parseProcessGroup(childProcessGroupNode, childProcessGroupNode));
+    }
+  }
+  return group;
+}
+
+std::unique_ptr<core::ProcessGroup> StructuredConfiguration::getRootFrom(const Node& rootYamlNode) {
+  uuids_.clear();
+  Node controllerServiceNode = rootYamlNode[CONFIG_CONTROLLER_SERVICES_KEY];
+  Node provenanceReportNode = rootYamlNode[CONFIG_PROVENANCE_REPORT_KEY];
+
+  parseControllerServices(controllerServiceNode);
+  // Create the root process group
+  std::unique_ptr<core::ProcessGroup> root = parseRootProcessGroup(rootYamlNode);
+  parseProvenanceReporting(provenanceReportNode, root.get());
+
+  // set the controller services into the root group.
+  for (const auto& controller_service : controller_services_->getAllControllerServices()) {
+    root->addControllerService(controller_service->getName(), controller_service);
+    root->addControllerService(controller_service->getUUIDStr(), controller_service);
+  }
+
+  return root;
+}
+
+void StructuredConfiguration::parseProcessorNode(const Node& processors_node, core::ProcessGroup* parentGroup) {
+  int64_t runDurationNanos = -1;
+  utils::Identifier uuid;
+  std::unique_ptr<core::Processor> processor;
+
+  if (!parentGroup) {
+    logger_->log_error("parseProcessNodeYaml: no parent group exists");
+    return;
+  }
+
+  if (!processors_node) {
+    throw std::invalid_argument("Cannot instantiate a MiNiFi instance without a defined Processors configuration node.");
+  }
+  if (!processors_node.isSequence()) {
+    throw std::invalid_argument(
+        "Cannot instantiate a MiNiFi instance without a defined Processors configuration node.");
+  }
+  // Evaluate sequence of processors
+  for (const auto procNode : processors_node) {
+    core::ProcessorConfig procCfg;
+
+    checkRequiredField(procNode, "name", CONFIG_PROCESSORS_KEY);
+    procCfg.name = procNode["name"].getString().value();
+    procCfg.id = getOrGenerateId(procNode);
+
+    uuid = procCfg.id;
+    logger_->log_debug("parseProcessorNode: name => [%s] id => [%s]", procCfg.name, procCfg.id);
+    checkRequiredField(procNode, "class", CONFIG_PROCESSORS_KEY);
+    procCfg.javaClass = procNode["class"].getString().value();
+    logger_->log_debug("parseProcessorNode: class => [%s]", procCfg.javaClass);
+
+    // Determine the processor name only from the Java class
+    auto lastOfIdx = procCfg.javaClass.find_last_of('.');
+    if (lastOfIdx != std::string::npos) {
+      lastOfIdx++;  // if a value is found, increment to move beyond the .
+      std::string processorName = procCfg.javaClass.substr(lastOfIdx);
+      processor = this->createProcessor(processorName, procCfg.javaClass, uuid);
+    } else {
+      // Allow unqualified class names for core processors
+      processor = this->createProcessor(procCfg.javaClass, uuid);
+    }
+
+    if (!processor) {
+      logger_->log_error("Could not create a processor %s with id %s", procCfg.name, procCfg.id);
+      throw std::invalid_argument("Could not create processor " + procCfg.name);
+    }
+
+    processor->setName(procCfg.name);
+
+    processor->setFlowIdentifier(flow_version_->getFlowIdentifier());
+
+    procCfg.schedulingStrategy = getOptionalField(procNode, "scheduling strategy", DEFAULT_SCHEDULING_STRATEGY, CONFIG_PROCESSORS_KEY);
+    logger_->log_debug("parseProcessorNode: scheduling strategy => [%s]", procCfg.schedulingStrategy);
+
+    procCfg.schedulingPeriod = getOptionalField(procNode, "scheduling period", DEFAULT_SCHEDULING_PERIOD_STR, CONFIG_PROCESSORS_KEY);
+
+    logger_->log_debug("parseProcessorNode: scheduling period => [%s]", procCfg.schedulingPeriod);
+
+    if (auto tasksNode = procNode["max concurrent tasks"]) {
+      if (auto int_val = tasksNode.getUInt64()) {
+        procCfg.maxConcurrentTasks = std::to_string(int_val.value());
+      } else {
+        procCfg.maxConcurrentTasks = tasksNode.getString().value();
+      }
+      logger_->log_debug("parseProcessorNode: max concurrent tasks => [%s]", procCfg.maxConcurrentTasks);
+    }
+
+    if (procNode["penalization period"]) {
+      procCfg.penalizationPeriod = procNode["penalization period"].getString().value();
+      logger_->log_debug("parseProcessorNode: penalization period => [%s]", procCfg.penalizationPeriod);
+    }
+
+    if (procNode["yield period"]) {
+      procCfg.yieldPeriod = procNode["yield period"].getString().value();
+      logger_->log_debug("parseProcessorNode: yield period => [%s]", procCfg.yieldPeriod);
+    }
+
+    if (auto runNode = procNode["run duration nanos"]) {
+      if (auto int_val = runNode.getUInt64()) {
+        procCfg.runDurationNanos = std::to_string(int_val.value());
+      } else {
+        procCfg.runDurationNanos = runNode.getString().value();
+      }
+      logger_->log_debug("parseProcessorNode: run duration nanos => [%s]", procCfg.runDurationNanos);
+    }
+
+    // handle auto-terminated relationships
+    if (procNode["auto-terminated relationships list"]) {
+      Node autoTerminatedSequence = procNode["auto-terminated relationships list"];
+      std::vector<std::string> rawAutoTerminatedRelationshipValues;
+      if (autoTerminatedSequence.isSequence() && autoTerminatedSequence.size() > 0) {
+        for (const auto autoTerminatedRel : autoTerminatedSequence) {
+          rawAutoTerminatedRelationshipValues.push_back(autoTerminatedRel.getString().value());
+        }
+      }
+      procCfg.autoTerminatedRelationships = rawAutoTerminatedRelationshipValues;
+    }
+
+    // handle processor properties
+    if (procNode["Properties"]) {
+      Node propertiesNode = procNode["Properties"];
+      parsePropertiesNode(propertiesNode, *processor, procCfg.name, CONFIG_PROCESSORS_KEY);
+    }
+
+    // Take care of scheduling
+
+    if (procCfg.schedulingStrategy == "TIMER_DRIVEN" || procCfg.schedulingStrategy == "EVENT_DRIVEN") {
+      if (auto scheduling_period = utils::timeutils::StringToDuration<std::chrono::nanoseconds>(procCfg.schedulingPeriod)) {
+        logger_->log_debug("convert: parseProcessorNode: schedulingPeriod => [%" PRId64 "] ns", scheduling_period->count());
+        processor->setSchedulingPeriodNano(*scheduling_period);
+      }
+    } else {
+      processor->setCronPeriod(procCfg.schedulingPeriod);
+    }
+
+    if (auto penalization_period = utils::timeutils::StringToDuration<std::chrono::milliseconds>(procCfg.penalizationPeriod)) {
+      logger_->log_debug("convert: parseProcessorNode: penalizationPeriod => [%" PRId64 "] ms", penalization_period->count());
+      processor->setPenalizationPeriod(penalization_period.value());
+    }
+
+    if (auto yield_period = utils::timeutils::StringToDuration<std::chrono::milliseconds>(procCfg.yieldPeriod)) {
+      logger_->log_debug("convert: parseProcessorNode: yieldPeriod => [%" PRId64 "] ms", yield_period->count());
+      processor->setYieldPeriodMsec(yield_period.value());
+    }
+
+    // Default to running
+    processor->setScheduledState(core::RUNNING);
+
+    if (procCfg.schedulingStrategy == "TIMER_DRIVEN") {
+      processor->setSchedulingStrategy(core::TIMER_DRIVEN);
+      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
+    } else if (procCfg.schedulingStrategy == "EVENT_DRIVEN") {
+      processor->setSchedulingStrategy(core::EVENT_DRIVEN);
+      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
+    } else {
+      processor->setSchedulingStrategy(core::CRON_DRIVEN);
+      logger_->log_debug("setting scheduling strategy as %s", procCfg.schedulingStrategy);
+    }
+
+    int32_t maxConcurrentTasks;
+    if (core::Property::StringToInt(procCfg.maxConcurrentTasks, maxConcurrentTasks)) {
+      logger_->log_debug("parseProcessorNode: maxConcurrentTasks => [%d]", maxConcurrentTasks);
+      processor->setMaxConcurrentTasks((uint8_t) maxConcurrentTasks);
+    }
+
+    if (core::Property::StringToInt(procCfg.runDurationNanos, runDurationNanos)) {
+      logger_->log_debug("parseProcessorNode: runDurationNanos => [%d]", runDurationNanos);
+      processor->setRunDurationNano(std::chrono::nanoseconds(runDurationNanos));
+    }
+
+    std::vector<core::Relationship> autoTerminatedRelationships;
+    for (auto &&relString : procCfg.autoTerminatedRelationships) {
+      core::Relationship relationship(relString, "");
+      logger_->log_debug("parseProcessorNode: autoTerminatedRelationship  => [%s]", relString);
+      autoTerminatedRelationships.push_back(relationship);
+    }
+
+    processor->setAutoTerminatedRelationships(autoTerminatedRelationships);
+
+    parentGroup->addProcessor(std::move(processor));
+  }
+}
+
+void StructuredConfiguration::parseRemoteProcessGroup(const Node& rpg_node_seq, core::ProcessGroup* parentGroup) {
+  utils::Identifier uuid;
+  std::string id;
+
+  if (!parentGroup) {
+    logger_->log_error("parseRemoteProcessGroupYaml: no parent group exists");
+    return;
+  }
+
+  if (!rpg_node_seq || !rpg_node_seq.isSequence()) {
+    return;
+  }
+  for (const auto currRpgNode : rpg_node_seq) {
+    checkRequiredField(currRpgNode, "name", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+    auto name = currRpgNode["name"].getString().value();
+    id = getOrGenerateId(currRpgNode);
+
+    logger_->log_debug("parseRemoteProcessGroupYaml: name => [%s], id => [%s]", name, id);
+
+    auto url = getOptionalField(currRpgNode, "url", "", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+
+    logger_->log_debug("parseRemoteProcessGroupYaml: url => [%s]", url);
+
+    uuid = id;
+    auto group = createRemoteProcessGroup(name, uuid);
+    group->setParent(parentGroup);
+
+    if (currRpgNode["yield period"]) {
+      auto yieldPeriod = currRpgNode["yield period"].getString().value();
+      logger_->log_debug("parseRemoteProcessGroupYaml: yield period => [%s]", yieldPeriod);
+
+      auto yield_period_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(yieldPeriod);
+      if (yield_period_value.has_value() && group) {
+        logger_->log_debug("parseRemoteProcessGroupYaml: yieldPeriod => [%" PRId64 "] ms", yield_period_value->count());
+        group->setYieldPeriodMsec(*yield_period_value);
+      }
+    }
+
+    if (currRpgNode["timeout"]) {
+      auto timeout = currRpgNode["timeout"].getString().value();
+      logger_->log_debug("parseRemoteProcessGroupYaml: timeout => [%s]", timeout);
+
+      auto timeout_value = utils::timeutils::StringToDuration<std::chrono::milliseconds>(timeout);
+      if (timeout_value.has_value() && group) {
+        logger_->log_debug("parseRemoteProcessGroupYaml: timeoutValue => [%" PRId64 "] ms", timeout_value->count());
+        group->setTimeout(timeout_value->count());
+      }
+    }
+
+    if (currRpgNode["local network interface"]) {
+      auto interface = currRpgNode["local network interface"].getString().value();
+      logger_->log_debug("parseRemoteProcessGroupYaml: local network interface => [%s]", interface);
+      group->setInterface(interface);
+    }
+
+    if (currRpgNode["transport protocol"]) {
+      auto transport_protocol = currRpgNode["transport protocol"].getString().value();
+      logger_->log_debug("parseRemoteProcessGroupYaml: transport protocol => [%s]", transport_protocol);
+      if (transport_protocol == "HTTP") {
+        group->setTransportProtocol(transport_protocol);
+        if (currRpgNode["proxy host"]) {
+          auto http_proxy_host = currRpgNode["proxy host"].getString().value();
+          logger_->log_debug("parseRemoteProcessGroupYaml: proxy host => [%s]", http_proxy_host);
+          group->setHttpProxyHost(http_proxy_host);
+          if (currRpgNode["proxy user"]) {
+            auto http_proxy_username = currRpgNode["proxy user"].getString().value();
+            logger_->log_debug("parseRemoteProcessGroupYaml: proxy user => [%s]", http_proxy_username);
+            group->setHttpProxyUserName(http_proxy_username);
+          }
+          if (currRpgNode["proxy password"]) {
+            auto http_proxy_password = currRpgNode["proxy password"].getString().value();
+            logger_->log_debug("parseRemoteProcessGroupYaml: proxy password => [%s]", http_proxy_password);
+            group->setHttpProxyPassWord(http_proxy_password);
+          }
+          if (currRpgNode["proxy port"]) {
+            auto http_proxy_port = currRpgNode["proxy port"].getString().value();
+            int32_t port;
+            if (core::Property::StringToInt(http_proxy_port, port)) {
+              logger_->log_debug("parseRemoteProcessGroupYaml: proxy port => [%d]", port);
+              group->setHttpProxyPort(port);
+            }
+          }
+        }
+      } else if (transport_protocol == "RAW") {
+        group->setTransportProtocol(transport_protocol);
+      } else {
+        std::stringstream stream;
+        stream << "Invalid transport protocol " << transport_protocol;
+        throw minifi::Exception(ExceptionType::SITE2SITE_EXCEPTION, stream.str().c_str());
+      }
+    }
+
+    group->setTransmitting(true);
+    group->setURL(url);
+
+    checkRequiredField(currRpgNode, "Input Ports", CONFIG_REMOTE_PROCESS_GROUP_KEY);
+    auto inputPorts = currRpgNode["Input Ports"];
+    if (inputPorts && inputPorts.isSequence()) {
+      for (const auto currPort : inputPorts) {
+        parsePort(currPort, group.get(), sitetosite::SEND);
+      }  // for node
+    }
+    auto outputPorts = currRpgNode["Output Ports"];
+    if (outputPorts && outputPorts.isSequence()) {
+      for (const auto currPort : outputPorts) {
+        logger_->log_debug("Got a current port, iterating...");
+
+        parsePort(currPort, group.get(), sitetosite::RECEIVE);
+      }  // for node
+    }
+    parentGroup->addProcessGroup(std::move(group));
+  }
+}
+
+void StructuredConfiguration::parseProvenanceReporting(const Node& node, core::ProcessGroup* parent_group) {
+  utils::Identifier port_uuid;
+
+  if (!parent_group) {
+    logger_->log_error("parseProvenanceReportingYaml: no parent group exists");
+    return;
+  }
+
+  if (!node || node.isNull()) {
+    logger_->log_debug("no provenance reporting task specified");
+    return;
+  }
+
+  auto reportTask = createProvenanceReportTask();
+
+  checkRequiredField(node, "scheduling strategy", CONFIG_PROVENANCE_REPORT_KEY);
+  auto schedulingStrategyStr = node["scheduling strategy"].getString().value();
+  checkRequiredField(node, "scheduling period", CONFIG_PROVENANCE_REPORT_KEY);
+  auto schedulingPeriodStr = node["scheduling period"].getString().value();
+
+  if (auto scheduling_period = utils::timeutils::StringToDuration<std::chrono::nanoseconds>(schedulingPeriodStr)) {
+    logger_->log_debug("ProvenanceReportingTask schedulingPeriod %" PRId64 " ns", scheduling_period->count());
+    reportTask->setSchedulingPeriodNano(*scheduling_period);
+  }
+
+  if (schedulingStrategyStr == "TIMER_DRIVEN") {
+    reportTask->setSchedulingStrategy(core::TIMER_DRIVEN);
+    logger_->log_debug("ProvenanceReportingTask scheduling strategy %s", schedulingStrategyStr);
+  } else {
+    throw std::invalid_argument("Invalid scheduling strategy " + schedulingStrategyStr);
+  }
+
+  int64_t lvalue;
+  if (node["host"] && node["port"]) {
+    auto hostStr = node["host"].getString().value();
+
+    std::string portStr;
+    if (auto int_val = node["port"].getInt()) {
+      portStr = std::to_string(int_val.value());
+    } else {
+      portStr = node["port"].getString().value();
+    }

Review Comment:
   good idea, added `getIntegerAsString`



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r989810617


##########
libminifi/include/core/json/JsonNode.h:
##########
@@ -0,0 +1,248 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <string>
+#include <utility>
+#include <memory>
+
+#include "core/flow/Node.h"
+#include "rapidjson/document.h"
+#include "utils/gsl.h"
+#include "utils/ValueCaster.h"
+
+namespace org::apache::nifi::minifi::core {
+
+class JsonNode : public flow::Node::NodeImpl {
+ public:
+  explicit JsonNode(const rapidjson::Value* node): node_(node) {}
+
+  explicit operator bool() const override {
+    return node_ != nullptr;
+  }
+  bool isSequence() const override {
+    return node_ ? node_->IsArray() : false;
+  }
+  bool isMap() const override {
+    return node_ ? node_->IsObject() : false;
+  }
+  bool isNull() const override {
+    return node_ ? node_->IsNull() : false;
+  }
+
+  nonstd::expected<std::string, std::exception_ptr> getString() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get string of invalid json value");
+      }
+      if (!node_->IsString()) {
+        throw std::runtime_error("Cannot get string of non-string json value");
+      }
+      return std::string{node_->GetString(), node_->GetStringLength()};
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());
+    }
+  }
+
+  nonstd::expected<int, std::exception_ptr> getInt() const override {
+    return getNumber<int>("int");
+  }
+  nonstd::expected<unsigned int, std::exception_ptr> getUInt() const override {
+    return getNumber<unsigned int>("unsigned int");
+  }
+  nonstd::expected<int64_t, std::exception_ptr> getInt64() const override {
+    return getNumber<int64_t>("int64_t");
+  }
+  nonstd::expected<uint64_t, std::exception_ptr> getUInt64() const override {
+    return getNumber<uint64_t>("uint64_t");
+  }
+
+  nonstd::expected<bool, std::exception_ptr> getBool() const override {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get bool of invalid json value");
+      }
+      if (!node_->IsBool()) {
+        throw std::runtime_error("Cannot get bool of non-bool json value");
+      }
+      return node_->GetBool();
+    } catch (...) {
+      return nonstd::make_unexpected(std::current_exception());
+    }
+  }
+
+  std::string getDebugString() const override {
+    if (!node_) return "<invalid>";
+    if (node_->IsObject()) return "<Map>";
+    if (node_->IsArray()) return "<Array>";
+    if (node_->IsNull()) return "null";
+    if (node_->IsInt()) return std::to_string(node_->GetInt());
+    if (node_->IsUint()) return std::to_string(node_->GetUint());
+    if (node_->IsInt64()) return std::to_string(node_->GetInt64());
+    if (node_->IsUint64()) return std::to_string(node_->GetUint64());
+    if (node_->IsTrue()) return "true";
+    if (node_->IsFalse()) return "false";
+    if (node_->IsDouble()) return std::to_string(node_->GetDouble());
+    if (node_->IsString()) return '"' + std::string(node_->GetString(), node_->GetStringLength()) + '"';
+    return "<unknown>";
+  }
+
+  size_t size() const override {
+    if (!node_) {
+      throw std::runtime_error("Cannot get size of invalid json value");
+    }
+    if (!node_->IsArray()) {
+      throw std::runtime_error("Cannot get size of non-array json value");
+    }
+    return node_->Size();
+  }
+  flow::Node::Iterator begin() const override;
+  flow::Node::Iterator end() const override;
+
+  flow::Node operator[](std::string_view key) const override {
+    if (!node_) {
+      throw std::runtime_error("Cannot get member of invalid json value");
+    }
+    if (!node_->IsObject()) {
+      return flow::Node{std::make_shared<JsonNode>(nullptr)};
+    }
+    auto it = node_->FindMember(rapidjson::Value(rapidjson::StringRef(key.data(), key.length())));
+    if (it == node_->MemberEnd()) {
+      return flow::Node{std::make_shared<JsonNode>(nullptr)};
+    }
+    return flow::Node{std::make_shared<JsonNode>(&it->value)};
+  }
+
+  std::optional<flow::Node::Cursor> getCursor() const override {
+    return std::nullopt;
+  }
+
+ private:
+  template<typename T, typename U>
+  std::optional<T> narrow(const U& value) {
+    T res = static_cast<T>(value);
+    if (static_cast<U>(res) != value) {
+      return std::nullopt;
+    }
+    return res;
+  }
+
+  template<typename T>
+  nonstd::expected<T, std::exception_ptr> getNumber(const char* type_name) const {
+    try {
+      if (!node_) {
+        throw std::runtime_error("Cannot get " + std::string(type_name) + " of invalid json value");
+      }
+      T result;
+      if (node_->IsInt() && utils::internal::cast_if_in_range(node_->GetInt(), result)) return result;
+      if (node_->IsUint() && utils::internal::cast_if_in_range(node_->GetUint(), result)) return result;
+      if (node_->IsInt64() && utils::internal::cast_if_in_range(node_->GetInt64(), result)) return result;
+      if (node_->IsUint64() && utils::internal::cast_if_in_range(node_->GetUint64(), result)) return result;
+      throw std::runtime_error("Cannot get " + std::string(type_name) + " of non-numeric json value");

Review Comment:
   changed



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r989813976


##########
libminifi/src/core/json/JsonConfiguration.cpp:
##########
@@ -0,0 +1,89 @@
+/**
+ *
+ * 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 <memory>
+#include <vector>
+#include <set>
+#include <cinttypes>
+#include <variant>
+
+#include "core/json/JsonConfiguration.h"
+#include "core/json/JsonNode.h"
+#include "core/state/Value.h"
+#include "Defaults.h"
+#include "utils/TimeUtil.h"
+
+#include "rapidjson/rapidjson.h"
+#include "rapidjson/document.h"
+
+namespace org::apache::nifi::minifi::core {
+
+namespace {
+
+}  // namespace
+
+
+JsonConfiguration::JsonConfiguration(ConfigurationContext ctx)
+    : StructuredConfiguration(([&] {
+                                if (!ctx.path) {
+                                  ctx.path = DEFAULT_NIFI_CONFIG_JSON;
+                                }
+                                return std::move(ctx);
+                              })(),
+                              logging::LoggerFactory<JsonConfiguration>::getLogger()) {}
+
+std::unique_ptr<core::ProcessGroup> JsonConfiguration::getRoot() {
+  if (!config_path_) {
+    logger_->log_error("Cannot instantiate flow, no config file is set.");
+    throw Exception(ExceptionType::FLOW_EXCEPTION, "No config file specified");
+  }
+  const auto configuration = filesystem_->read(config_path_.value());
+  if (!configuration) {
+    // non-existence of flow config file is not a dealbreaker, the caller might fetch it from network
+    return nullptr;
+  }
+  try {
+    rapidjson::Document doc;
+    rapidjson::ParseResult res = doc.Parse(configuration->c_str(), configuration->length());
+    if (!res) {
+      throw std::runtime_error("Could not parse json file");
+    }
+    flow::Node root{std::make_shared<JsonNode>(&doc)};
+    return getRootFrom(root);
+  } catch(...) {
+    logger_->log_error("Invalid json configuration file");
+    throw;
+  }
+}
+
+std::unique_ptr<core::ProcessGroup> JsonConfiguration::getRootFromPayload(const std::string &yamlConfigPayload) {

Review Comment:
   removed



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1391: MINIFICPP-1846 - Json configuration support part 1

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1391:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1391#discussion_r989818493


##########
libminifi/src/core/json/JsonConfiguration.cpp:
##########
@@ -0,0 +1,89 @@
+/**
+ *
+ * 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 <memory>
+#include <vector>
+#include <set>
+#include <cinttypes>
+#include <variant>
+
+#include "core/json/JsonConfiguration.h"
+#include "core/json/JsonNode.h"
+#include "core/state/Value.h"
+#include "Defaults.h"
+#include "utils/TimeUtil.h"
+
+#include "rapidjson/rapidjson.h"
+#include "rapidjson/document.h"
+
+namespace org::apache::nifi::minifi::core {
+
+namespace {
+
+}  // namespace
+
+
+JsonConfiguration::JsonConfiguration(ConfigurationContext ctx)
+    : StructuredConfiguration(([&] {
+                                if (!ctx.path) {
+                                  ctx.path = DEFAULT_NIFI_CONFIG_JSON;
+                                }
+                                return std::move(ctx);
+                              })(),
+                              logging::LoggerFactory<JsonConfiguration>::getLogger()) {}
+
+std::unique_ptr<core::ProcessGroup> JsonConfiguration::getRoot() {
+  if (!config_path_) {
+    logger_->log_error("Cannot instantiate flow, no config file is set.");
+    throw Exception(ExceptionType::FLOW_EXCEPTION, "No config file specified");
+  }
+  const auto configuration = filesystem_->read(config_path_.value());
+  if (!configuration) {
+    // non-existence of flow config file is not a dealbreaker, the caller might fetch it from network
+    return nullptr;
+  }
+  try {
+    rapidjson::Document doc;
+    rapidjson::ParseResult res = doc.Parse(configuration->c_str(), configuration->length());
+    if (!res) {
+      throw std::runtime_error("Could not parse json file");
+    }
+    flow::Node root{std::make_shared<JsonNode>(&doc)};
+    return getRootFrom(root);
+  } catch(...) {
+    logger_->log_error("Invalid json configuration file");
+    throw;
+  }

Review Comment:
   aside from the different error messages, they seem to be the same, but I didn't want to change them due to the different logs



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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