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/22 08:44:12 UTC

[GitHub] [nifi-minifi-cpp] lordgamez opened a new pull request, #1420: MINIFICPP-1925 Ensure compatibility with the MiNiFi C2 server

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

   Ensure that config update and heartbeats from MiNiFi C2 server are handled properly in MiNiFi agent
   
   - Read flowid from configuration update parameter
   - Make C2 operation strings case-insensitive
   - Handle `null` in requested operations
   - Remove `Accept` header temporarily until MiNiFi C2 handling is fixed
   - Add test coverage for MiNiFi C2 compatibility
   
   https://issues.apache.org/jira/browse/MINIFICPP-1925
   
   ---------
   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] szaszm closed pull request #1420: MINIFICPP-1925 Ensure compatibility with the MiNiFi C2 server

Posted by GitBox <gi...@apache.org>.
szaszm closed pull request #1420: MINIFICPP-1925 Ensure compatibility with the MiNiFi C2 server
URL: https://github.com/apache/nifi-minifi-cpp/pull/1420


-- 
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] ihor-sokoliuk-exa commented on pull request #1420: MINIFICPP-1925 Ensure compatibility with the MiNiFi C2 server

Posted by GitBox <gi...@apache.org>.
ihor-sokoliuk-exa commented on PR #1420:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1420#issuecomment-1282622132

   @lordgamez you rock!
   
   Everything works like a charm!
   
   Thank you for your contribution and the instructions particularly.


-- 
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 #1420: MINIFICPP-1925 Ensure compatibility with the MiNiFi C2 server

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


##########
docker/test/integration/minifi/core/ImageStore.py:
##########
@@ -105,6 +107,14 @@ def __build_minifi_cpp_image(self):
                 RUN echo nifi.metrics.publisher.class=PrometheusMetricsPublisher >> {minifi_root}/conf/minifi.properties
                 RUN echo nifi.metrics.publisher.PrometheusMetricsPublisher.port=9936 >> {minifi_root}/conf/minifi.properties
                 RUN echo nifi.metrics.publisher.metrics=RepositoryMetrics,QueueMetrics,GetFileMetrics,GetTCPMetrics,FlowInformation,DeviceInfoNode >> {minifi_root}/conf/minifi.properties
+                RUN echo nifi.c2.enable=true  >> {minifi_root}/conf/minifi.properties

Review Comment:
   Would it be possible to only enable this for C2 tests? Just to avoid spamming the logs with C2 HTTP errors, and hopefully save a bit of resources.



##########
docker/test/integration/resources/minifi-c2-server-ssl/config.yml:
##########
@@ -0,0 +1,43 @@
+MiNiFi Config Version: 3
+Flow Controller:
+  name: MiNiFi Flow
+Processors:
+- name: Get files from /tmp/input
+  id: 2f2a3b47-f5ba-49f6-82b5-bc1c86b96e27
+  class: org.apache.nifi.minifi.processors.GetFile
+  scheduling strategy: TIMER_DRIVEN
+  scheduling period: 1000 ms
+  Properties:
+    Input Directory: /tmp/input
+- name: Put files to /tmp/output
+  id: e143601d-de4f-44ba-a6ec-d1f97d77ec94
+  class: org.apache.nifi.minifi.processors.PutFile
+  scheduling strategy: EVENT_DRIVEN
+  auto-terminated relationships list:
+  - failure
+  - success
+  Properties:
+    Conflict Resolution Strategy: fail
+    Create Missing Directories: 'true'
+    Directory: /tmp/output
+Connections:
+- name: GetFile/success/PutFile
+  id: 098a56ba-f4bf-4323-a3f3-6f8a5e3586bf
+  source id: 2f2a3b47-f5ba-49f6-82b5-bc1c86b96e27
+  source relationship names:
+  - success
+  destination id: e143601d-de4f-44ba-a6ec-d1f97d77ec94
+Controller Services:
+  - name: SSLContextService
+    id: 2438e3c8-015a-1000-79ca-83af40ec1994
+    class: SSLContextService

Review Comment:
   This is not referenced anywhere in the file. Is it somehow picked up automatically by the C2 client?



-- 
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 #1420: MINIFICPP-1925 Ensure compatibility with the MiNiFi C2 server

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


##########
docker/test/integration/resources/minifi-c2-server-ssl/config.yml:
##########
@@ -0,0 +1,43 @@
+MiNiFi Config Version: 3
+Flow Controller:
+  name: MiNiFi Flow
+Processors:
+- name: Get files from /tmp/input
+  id: 2f2a3b47-f5ba-49f6-82b5-bc1c86b96e27
+  class: org.apache.nifi.minifi.processors.GetFile
+  scheduling strategy: TIMER_DRIVEN
+  scheduling period: 1000 ms
+  Properties:
+    Input Directory: /tmp/input
+- name: Put files to /tmp/output
+  id: e143601d-de4f-44ba-a6ec-d1f97d77ec94
+  class: org.apache.nifi.minifi.processors.PutFile
+  scheduling strategy: EVENT_DRIVEN
+  auto-terminated relationships list:
+  - failure
+  - success
+  Properties:
+    Conflict Resolution Strategy: fail
+    Create Missing Directories: 'true'
+    Directory: /tmp/output
+Connections:
+- name: GetFile/success/PutFile
+  id: 098a56ba-f4bf-4323-a3f3-6f8a5e3586bf
+  source id: 2f2a3b47-f5ba-49f6-82b5-bc1c86b96e27
+  source relationship names:
+  - success
+  destination id: e143601d-de4f-44ba-a6ec-d1f97d77ec94
+Controller Services:
+  - name: SSLContextService
+    id: 2438e3c8-015a-1000-79ca-83af40ec1994
+    class: SSLContextService

Review Comment:
   It is retrieved through the `nifi.c2.rest.ssl.context.service` property where its name is set.



-- 
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 #1420: MINIFICPP-1925 Ensure compatibility with the MiNiFi C2 server

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


##########
libminifi/src/core/FlowConfiguration.cpp:
##########
@@ -97,24 +97,24 @@ std::unique_ptr<core::reporting::SiteToSiteProvenanceReportingTask> FlowConfigur
   return processor;
 }
 
-std::unique_ptr<core::ProcessGroup> FlowConfiguration::updateFromPayload(const std::string& url, const std::string& yamlConfigPayload) {
+std::unique_ptr<core::ProcessGroup> FlowConfiguration::updateFromPayload(const std::string& url, const std::string& yamlConfigPayload, const std::optional<std::string>& flow_id) {
   auto old_services = controller_services_;
   auto old_provider = service_provider_;
   controller_services_ = std::make_shared<core::controller::ControllerServiceMap>();
   service_provider_ = std::make_shared<core::controller::StandardControllerServiceProvider>(controller_services_, nullptr, configuration_);
   auto payload = getRootFromPayload(yamlConfigPayload);
   if (!url.empty() && payload != nullptr) {
-    std::string flow_id;
+    std::string payload_flow_id;
     std::string bucket_id;
     auto path_split = utils::StringUtils::split(url, "/");
     for (auto it = path_split.cbegin(); it != path_split.cend(); ++it) {
       if (*it == "flows" && std::next(it) != path_split.cend()) {
-        flow_id = *++it;
+        payload_flow_id = *++it;
       } else if (*it == "buckets" && std::next(it) != path_split.cend()) {
         bucket_id = *++it;
       }
     }
-    flow_version_->setFlowVersion(url, bucket_id, flow_id);
+    flow_version_->setFlowVersion(url, bucket_id, flow_id ? *flow_id : payload_flow_id);

Review Comment:
   Is it normal behavior if the flow ID in the payload is different from the original flow ID?  If not, we could log that something unexpected has happened.



##########
docker/test/integration/minifi/core/MinifiC2ServerSslContainer.py:
##########
@@ -0,0 +1,36 @@
+# 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.
+
+
+from .Container import Container
+
+
+class MinifiC2ServerSslContainer(Container):
+    def __init__(self, name, vols, network, image_store, command=None):
+        super().__init__(name, 'minifi-c2-server-ssl', vols, network, image_store, command)

Review Comment:
   Since the name of the engine is the only difference between `MinifiC2ServerContainer` and `MinifiC2ServerSslContainer`, could the name of the engine be a parameter of the constructor, so we can have one class instead of two?



-- 
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 #1420: MINIFICPP-1925 Ensure compatibility with the MiNiFi C2 server

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


##########
docker/test/integration/minifi/core/ImageStore.py:
##########
@@ -105,6 +107,14 @@ def __build_minifi_cpp_image(self):
                 RUN echo nifi.metrics.publisher.class=PrometheusMetricsPublisher >> {minifi_root}/conf/minifi.properties
                 RUN echo nifi.metrics.publisher.PrometheusMetricsPublisher.port=9936 >> {minifi_root}/conf/minifi.properties
                 RUN echo nifi.metrics.publisher.metrics=RepositoryMetrics,QueueMetrics,GetFileMetrics,GetTCPMetrics,FlowInformation,DeviceInfoNode >> {minifi_root}/conf/minifi.properties
+                RUN echo nifi.c2.enable=true  >> {minifi_root}/conf/minifi.properties

Review Comment:
   Added ticket https://issues.apache.org/jira/browse/MINIFICPP-1983



-- 
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 pull request #1420: MINIFICPP-1925 Ensure compatibility with the MiNiFi C2 server

Posted by GitBox <gi...@apache.org>.
lordgamez commented on PR #1420:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1420#issuecomment-1272862179

   > Hey @lordgamez Great PR!
   > 
   > I've tried your changes, and they do not work with NiFi C2 over HTTPS. I am getting the next error:
   > 
   > `[2022-10-05 23:07:51.015] [org::apache::nifi::minifi::extensions::curl::HTTPClient] [error] curl_easy_perform() failed SSL peer certificate or SSH remote key was not OK on https://nifi-c2:8880/c2/config/heartbeat, error code 60`
   > 
   > Do you have experience in setting up the secured connection with NiFi C2?
   
   Hi @ihor-sokoliuk-exa ,
   
   I don't have any experience setting up NiFi C2 over SSL, but I'll investigate this use case and try to update the PR with this option.


-- 
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 #1420: MINIFICPP-1925 Ensure compatibility with the MiNiFi C2 server

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


##########
libminifi/src/core/FlowConfiguration.cpp:
##########
@@ -97,24 +97,24 @@ std::unique_ptr<core::reporting::SiteToSiteProvenanceReportingTask> FlowConfigur
   return processor;
 }
 
-std::unique_ptr<core::ProcessGroup> FlowConfiguration::updateFromPayload(const std::string& url, const std::string& yamlConfigPayload) {
+std::unique_ptr<core::ProcessGroup> FlowConfiguration::updateFromPayload(const std::string& url, const std::string& yamlConfigPayload, const std::optional<std::string>& flow_id) {
   auto old_services = controller_services_;
   auto old_provider = service_provider_;
   controller_services_ = std::make_shared<core::controller::ControllerServiceMap>();
   service_provider_ = std::make_shared<core::controller::StandardControllerServiceProvider>(controller_services_, nullptr, configuration_);
   auto payload = getRootFromPayload(yamlConfigPayload);
   if (!url.empty() && payload != nullptr) {
-    std::string flow_id;
+    std::string payload_flow_id;
     std::string bucket_id;
     auto path_split = utils::StringUtils::split(url, "/");
     for (auto it = path_split.cbegin(); it != path_split.cend(); ++it) {
       if (*it == "flows" && std::next(it) != path_split.cend()) {
-        flow_id = *++it;
+        payload_flow_id = *++it;
       } else if (*it == "buckets" && std::next(it) != path_split.cend()) {
         bucket_id = *++it;
       }
     }
-    flow_version_->setFlowVersion(url, bucket_id, flow_id);
+    flow_version_->setFlowVersion(url, bucket_id, flow_id ? *flow_id : payload_flow_id);

Review Comment:
   This is normal in this case when a flow update happens, in that case the C2 server can generate a new flow id for the new flow which is then sent in the payload_flow_id.



-- 
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 pull request #1420: MINIFICPP-1925 Ensure compatibility with the MiNiFi C2 server

Posted by GitBox <gi...@apache.org>.
lordgamez commented on PR #1420:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1420#issuecomment-1280741097

   Hi @ihor-sokoliuk-exa,
   
   I pushed a new commit 88a3bc2ed9cac02c8d4f0e8caff7ca7b47e01877 that updates the PR to support SSL connection in C2 messages. To enable the SSL connection you need to configure the following:
   
   On C2 server side, in the c2.properties file the `minifi.c2.server.secure=true` should be set and the corresponding keystore and truststore should be configured as well. The `authorizations.yaml` file should also be changed to allow the agent's connections through SSL. For this you should define the agent class's DN in the `authorities.yaml` and set the `allow` action for this class in the `authorizations.yaml` for each endpoint. See the added test files for reference.
   
   On the MiNiFi side you should add an `SSLContextService` controller in the `config.yml` file where you can set the client certificate paths for the SSL connection. After that the `nifi.c2.rest.ssl.context.service` property should be set to the name of the `SSLContextService` in the `minifi.properties` file for the MiNiFi agent to use that SSL service for the C2 connection.


-- 
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] ihor-sokoliuk-exa commented on pull request #1420: MINIFICPP-1925 Ensure compatibility with the MiNiFi C2 server

Posted by GitBox <gi...@apache.org>.
ihor-sokoliuk-exa commented on PR #1420:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1420#issuecomment-1269091030

   Hey @lordgamez 
   Great PR!
   
   I've tried your changes, and they do not work with NiFi C2 over HTTPS.
   I am getting the next error:
   
   `[2022-10-05 23:07:51.015] [org::apache::nifi::minifi::extensions::curl::HTTPClient] [error] curl_easy_perform() failed SSL peer certificate or SSH remote key was not OK on https://nifi-c2:8880/c2/config/heartbeat, error code 60`
   
   Do you have experience in setting up the secured connection with NiFi C2?
   


-- 
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 pull request #1420: MINIFICPP-1925 Ensure compatibility with the MiNiFi C2 server

Posted by GitBox <gi...@apache.org>.
lordgamez commented on PR #1420:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1420#issuecomment-1283794118

   > @lordgamez you rock!
   > 
   > Everything works like a charm!
   > 
   > Thank you for your contribution and the instructions particularly.
   
   You're welcome, I'm glad it worked :)


-- 
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 #1420: MINIFICPP-1925 Ensure compatibility with the MiNiFi C2 server

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


##########
docker/test/integration/minifi/core/MinifiC2ServerSslContainer.py:
##########
@@ -0,0 +1,36 @@
+# 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.
+
+
+from .Container import Container
+
+
+class MinifiC2ServerSslContainer(Container):
+    def __init__(self, name, vols, network, image_store, command=None):
+        super().__init__(name, 'minifi-c2-server-ssl', vols, network, image_store, command)

Review Comment:
   Good point, added an `ssl` contructor argument to figure out which version should be instantiated in fd13cbd5ad0819f5a3be0e01cbdb4cac5db3fadd (passing the engine as parameter is more risky as any engine could be passed not just a C2 server engine)



-- 
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 #1420: MINIFICPP-1925 Ensure compatibility with the MiNiFi C2 server

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


##########
docker/test/integration/minifi/core/ImageStore.py:
##########
@@ -105,6 +107,14 @@ def __build_minifi_cpp_image(self):
                 RUN echo nifi.metrics.publisher.class=PrometheusMetricsPublisher >> {minifi_root}/conf/minifi.properties
                 RUN echo nifi.metrics.publisher.PrometheusMetricsPublisher.port=9936 >> {minifi_root}/conf/minifi.properties
                 RUN echo nifi.metrics.publisher.metrics=RepositoryMetrics,QueueMetrics,GetFileMetrics,GetTCPMetrics,FlowInformation,DeviceInfoNode >> {minifi_root}/conf/minifi.properties
+                RUN echo nifi.c2.enable=true  >> {minifi_root}/conf/minifi.properties

Review Comment:
   Well it would be possible, but that would require creating a separate minifi image containing a different minifi.properties file and change the tests where we want to use that image instead of the default minifi image. Now we already have a provenance repo and https version besides this which is already too many. A better solution would be to refactor the tests to have a single minifi image and use the configuration file as a bind mount. Then the specific config file could be set in the test description, but that change should be done in a separate PR.



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