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/09/23 16:54:40 UTC

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

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