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/11/13 20:34:40 UTC

[GitHub] [nifi] ChrisSamo632 opened a new pull request, #6658: NIFI-10797 add customisable Elasticsearch REST Client config and Elasticsearch Cluster Sniffer

ChrisSamo632 opened a new pull request, #6658:
URL: https://github.com/apache/nifi/pull/6658

   # Summary
   
   [NIFI-10797](https://issues.apache.org/jira/browse/NIFI-10797) add customisable Elasticsearch REST Client config and Elasticsearch Cluster Sniffer
   
   # Tracking
   
   Please complete the following tracking steps prior to pull request creation.
   
   ### Issue Tracking
   
   - [x] [Apache NiFi Jira](https://issues.apache.org/jira/browse/NIFI) issue created
   
   ### Pull Request Tracking
   
   - [x] Pull Request title starts with Apache NiFi Jira issue number, such as `NIFI-00000`
   - [x] Pull Request commit message starts with Apache NiFi Jira issue number, as such `NIFI-00000`
   
   ### Pull Request Formatting
   
   - [x] Pull Request based on current revision of the `main` branch
   - [x] Pull Request refers to a feature branch with one commit containing changes
   
   # Verification
   
   Please indicate the verification steps performed prior to pull request creation.
   
   ### Build
   
   - [x] Build completed using `mvn clean install -P contrib-check`
     - [ ] JDK 8
     - [ ] JDK 11
     - [x] JDK 17
   
   ### Licensing
   
   - ~[ ] New dependencies are compatible with the [Apache License 2.0](https://apache.org/licenses/LICENSE-2.0) according to the [License Policy](https://www.apache.org/legal/resolved.html)~
   - ~[ ] New dependencies are documented in applicable `LICENSE` and `NOTICE` files~
   
   ### Documentation
   
   - ~[ ] Documentation formatting appears as expected in rendered files~
   


-- 
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] gresockj commented on a diff in pull request #6658: NIFI-10797 add customisable Elasticsearch REST Client config and Elasticsearch Cluster Sniffer

Posted by "gresockj (via GitHub)" <gi...@apache.org>.
gresockj commented on code in PR #6658:
URL: https://github.com/apache/nifi/pull/6658#discussion_r1103780177


##########
nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/main/resources/docs/org.apache.nifi.elasticsearch.ElasticSearchClientServiceImpl/additionalDetails.html:
##########
@@ -0,0 +1,54 @@
+<!DOCTYPE html>
+<html lang="en">
+<!--
+  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.
+-->
+<head>
+    <meta charset="utf-8" />
+    <title>ElasticSearchClientServiceImpl</title>
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css" />
+</head>
+
+<body>
+
+<h2>Sniffing</h2>
+<p>
+    The Elasticsearch Sniffer can be used to locate Elasticsearch Nodes within a Cluster to which you are connecting.
+    This can be beneficial if your cluster dynamically changes over time, e.g. new Nodes are added to maintain performance during heavy load.
+</p>
+<p>
+    Sniffing can also be used to update the list of Hosts within the Cluster if a connection Failure is encountered during operation.
+    In order to "Sniff on Failure", you <b>must</b> also enable "Sniff Cluster Nodes".
+</p>
+<p>
+    Not all situations make sense to use Sniffing, for example if:
+    <ul>
+        <li>Elasticsearch is situated behind a load balancer, which dynamically routes connections from NiFi</li>
+        <li>Elasticsearch is on a different network to NiFi</li>
+    </ul>
+</p>
+<p>
+    There may also be need to set some of the <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/modules-network.html">
+    Elasticsearch Networking Advanced Settings</a>, such as <code>network.publish_host</code> to ensure that
+    the HTTP Hosts found by the Sniffer are accessible by NiFi. For example, Elasticsearch may use a network internal
+    <code>publish_address</code> that is inaccessible to NiFi, but instead should use an address/IP that NiFi understands.
+    It may also be necessary to add this same address to Elasticsearch's <code>network.bind_address</code> list.

Review Comment:
   `network.bind_host`



##########
nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service-api/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchClientService.java:
##########
@@ -161,6 +157,118 @@ public interface ElasticSearchClientService extends ControllerService, Verifiabl
             .required(true)
             .build();
 
+    PropertyDescriptor COMPRESSION = new PropertyDescriptor.Builder()
+            .name("el-cs-enable-compression")
+            .displayName("Enable Compression")
+            .description("Whether the REST client should compress requests using gzip content encoding and add the " +
+                    "\"Accept-Encoding: gzip\" header to receive compressed responses")
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true)
+            .build();
+
+    PropertyDescriptor SEND_META_HEADER = new PropertyDescriptor.Builder()
+            .name("el-cs-send-meta-header")
+            .displayName("Send Meta Header")
+            .description("Whether to send a \"X-Elastic-Client-Meta\" header that describes the runtime environment. " +
+                    "It contains information that is similar to what could be found in User-Agent. " +
+                    "Using a separate header allows applications to use User-Agent for their own needs, " +
+                    "e.g. to identify application version or other environment information")
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    PropertyDescriptor STRICT_DEPRECATION = new PropertyDescriptor.Builder()
+            .name("el-cs-strict-deprecation")
+            .displayName("Strict Deprecation")
+            .description("Whether the REST client should return any response containing at least one warning header as a failure")
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true)
+            .build();
+
+    AllowableValue NODE_SELECTOR_ANY = new AllowableValue("ANY", "Any",
+            "Select any Elasticsearch node to handle requests");
+    AllowableValue NODE_SELECTOR_SKIP_DEDICATED_MASTERS = new AllowableValue("SKIP_DEDICATED_MASTERS", "Skip Dedicated Masters",
+            "Skip dedicated Elasticsearch master nodes for handling request");
+
+    PropertyDescriptor NODE_SELECTOR = new PropertyDescriptor.Builder()
+            .name("el-cs-node-selector")
+            .displayName("Node Selector")
+            .description("Selects Elasticsearch nodes that can receive requests. Used to keep requests away from dedicated Elasticsearch master nodes")
+            .allowableValues(NODE_SELECTOR_ANY, NODE_SELECTOR_SKIP_DEDICATED_MASTERS)
+            .defaultValue(NODE_SELECTOR_ANY.getValue())
+            .required(true)
+            .build();
+
+    PropertyDescriptor PATH_PREFIX = new PropertyDescriptor.Builder()
+            .name("el-cs-path-prefix")
+            .displayName("Path Prefix")
+            .description("Sets the path's prefix for every request used by the http client. " +
+                    "For example, if this is set to \"/my/path\", then any client request will become \"/my/path/\" + endpoint. " +
+                    "In essence, every request's endpoint is prefixed by this pathPrefix. " +
+                    "The path prefix is useful for when Elasticsearch is behind a proxy that provides a base path or a proxy that requires all paths to start with '/'; " +
+                    "it is not intended for other purposes and it should not be supplied in other scenarios")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    PropertyDescriptor SNIFF_CLUSTER_NODES = new PropertyDescriptor.Builder()
+            .name("el-cs-sniff-cluster-nodes")
+            .displayName("Sniff Cluster Nodes")
+            .description("Periodically sniff for nodes within the Elasticsearch cluster via the Elasticsearch Node Info API. " +
+                    "If Elasticsearch security features are enabled (default to \"true\" for 8.x+), the Elasticsearch user must " +
+                    "have the \"monitor\" or \"manage\" cluster privilege to use this API." +
+                    "Note that all " + HTTP_HOSTS.getDisplayName() + " (and those that may be discovered within the cluster " +
+                    "using the Sniffer) must use the same protocol, e.g. http or https, and be contactable using the same client settings. " +
+                    "Finally the Elasticsearch \"network.publish_address\" must match one of the \"network.bind_address(es)\" " +

Review Comment:
   I'm assuming you mean [network.publish_host](https://www.elastic.co/guide/en/elasticsearch/reference/current/modules-network.html#advanced-network-settings) and `network.bind_host` here.



-- 
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] ChrisSamo632 commented on pull request #6658: NIFI-10797 add customisable Elasticsearch REST Client config and Elasticsearch Cluster Sniffer

Posted by "ChrisSamo632 (via GitHub)" <gi...@apache.org>.
ChrisSamo632 commented on PR #6658:
URL: https://github.com/apache/nifi/pull/6658#issuecomment-1426492811

   The `integration-tests` (with update `verifySniffer` logic) just revealed that the Elasticsearch instance's `network.publish_address` **must** be one of the `network.bind_address`(es) for the Sniffer to work - this wasn't the case by default for the TestContainer running Elasticsearch, so it could be an issue like that - again, I've added a note to the `Sniffer` property and the `integration-tests` now set `network.bind_address` to [site](https://www.elastic.co/guide/en/elasticsearch/reference/current/modules-network.html#network-interface-values), which allows them to pass on both Elasticsearch 7.x and 8.x (no TLS but with Auth enabled)
   
   Going back to the earlier Docker question - is it that you're running Elasticsearch as a Docker container but NiFi "natively" on your machine, connecting to Elasticsearch using `localhost` and then the Elasticsearch instance is reporting back the Docker-internal address of `172.17.0.3`? What do you see under the `http` section of the response from a call to your Elasticsearch's `GET /_nodes/http` API?
   
   With an Elasticsearch Docker container running, I get:
   ```json
   "http": {
     "bound_address": [
       "0.0.0.0:9200"
     ],
     "publish_address": "172.20.0.2:9200",
     "max_content_length_in_bytes": 104857600
   }
   ```
   which isn't contactable from my browser/terminal - so I suspect this is indeed what's going on for you, i.e. your Elasticsearch instance is reporting back its "internal" IP as its `publish_address` and you'd need to re-configure the instance's `network.publish_address` (or run NiFi within the Docker network **or** use the `host` network for your Elasticsearch container) so that the correct combination of `publish_address` and host network routing can be used


-- 
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] gresockj commented on pull request #6658: NIFI-10797 add customisable Elasticsearch REST Client config and Elasticsearch Cluster Sniffer

Posted by "gresockj (via GitHub)" <gi...@apache.org>.
gresockj commented on PR #6658:
URL: https://github.com/apache/nifi/pull/6658#issuecomment-1427007337

   
   > Going back to the earlier Docker question - is it that you're running Elasticsearch as a Docker container but NiFi "natively" on your machine, connecting to Elasticsearch using `localhost` and then the Elasticsearch instance is reporting back the Docker-internal address of `172.17.0.3`? What do you see under the `http` section of the response from a call to your Elasticsearch's `GET /_nodes/http` API?
   
   Yes, that is my setup, and I get similar output to what you posted.
   
   > which isn't contactable from my browser/terminal - so I suspect this is indeed what's going on for you, i.e. your Elasticsearch instance is reporting back its "internal" IP as its `publish_address` and you'd need to re-configure the instance's `network.publish_address` (or run NiFi within the Docker network **or** use the `host` network for your Elasticsearch container) so that the correct combination of `publish_address` and host network routing can be used
   
   Good catch -- when I added `-e network.publish_host=192.168.1.X` to my docker run command, and used the same IP in the NiFi ES client service, the sniffer worked correctly.


-- 
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] ChrisSamo632 commented on pull request #6658: NIFI-10797 add customisable Elasticsearch REST Client config and Elasticsearch Cluster Sniffer

Posted by "ChrisSamo632 (via GitHub)" <gi...@apache.org>.
ChrisSamo632 commented on PR #6658:
URL: https://github.com/apache/nifi/pull/6658#issuecomment-1426874715

   @gresockj the [Elasticsearch documentation/blogs](https://www.elastic.co/blog/elasticsearch-sniffing-best-practices-what-when-why-how) indicate that this is indeed what's going on (I've added an "additional details" section for the Controller Service and linked to these docs).
   
   I also added a connectivity check to the "Verify" steps when the Sniffer is enabled - the `integration-tests` were updated to cater for this too (see the updated `AbstractElasticsearchITBase` for the settings I changed to make the `TestContainer` contactable after Sniffing as well as by the standard RestClient setup)


-- 
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] ChrisSamo632 commented on pull request #6658: NIFI-10797 add customisable Elasticsearch REST Client config and Elasticsearch Cluster Sniffer

Posted by "ChrisSamo632 (via GitHub)" <gi...@apache.org>.
ChrisSamo632 commented on PR #6658:
URL: https://github.com/apache/nifi/pull/6658#issuecomment-1426428989

   @gresockj thanks for the feedback
   
   The "Sniff on Failure" error looks to be due to a bug in the new code - I'd not added the `Sniffer` into the `SniffOnFailureListener`, so I've corrected that.
   
   Are you connecting to Elasticsearch over `http` or `https` (8.x comes with TLS out of the box, but can be turned off)?
   
   I've tried to cater for both protocols in this code, but quite possible there's a problem there. The Sniffer only returns the hostname and port number (no protocol) so we have to tell it whether to use `http` or `https` when generating the host URL - for this, I'm using the "first" of the specified `HTTP_HOSTS` and checking whether it starts with `https://` (otherwise assuming it's `http://`) - I'll add a note to the `Sniffer` property to indicate that all `HTTP_HOSTS` must use the same protocol (which I suspect NiFi and probably Elasticsearch would struggle with anyway if they didn't).
   
   Looking at the IP in your error message, I'm guessing you're running Elasticsearch as a Docker container, is NiFi running in the same Docker network (although if NiFi can connect to the same Elasticsearch instance when the Sniffer is **not** configured, then I suspect it's not a networking issue)?
   
   The Sniffer uses the `Nodes Info` API in Elasticsearch, the only [pre-requisite](https://www.elastic.co/guide/en/elasticsearch/reference/current/cluster-nodes-info.html#cluster-nodes-info-api-prereqs) for use is that if Elasticsearch Security is enabled (default to `true` in 8.x) then the user connecting must have the `monitor` or `manage` [cluster privilege](https://www.elastic.co/guide/en/elasticsearch/reference/current/security-privileges.html#privileges-list-cluster) to successfully list the cluster nodes - I'll note this in the `Sniffer` property too. But I'd expect that to throw a `403` (or similar) error response rather than causing a connection timeout.


-- 
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] asfgit closed pull request #6658: NIFI-10797 add customisable Elasticsearch REST Client config and Elasticsearch Cluster Sniffer

Posted by "asfgit (via GitHub)" <gi...@apache.org>.
asfgit closed pull request #6658: NIFI-10797 add customisable Elasticsearch REST Client config and Elasticsearch Cluster Sniffer
URL: https://github.com/apache/nifi/pull/6658


-- 
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] gresockj commented on pull request #6658: NIFI-10797 add customisable Elasticsearch REST Client config and Elasticsearch Cluster Sniffer

Posted by "gresockj (via GitHub)" <gi...@apache.org>.
gresockj commented on PR #6658:
URL: https://github.com/apache/nifi/pull/6658#issuecomment-1419563321

   Neat ideas for additional ES client configuration.  Reviewing now


-- 
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] ChrisSamo632 commented on a diff in pull request #6658: NIFI-10797 add customisable Elasticsearch REST Client config and Elasticsearch Cluster Sniffer

Posted by "ChrisSamo632 (via GitHub)" <gi...@apache.org>.
ChrisSamo632 commented on code in PR #6658:
URL: https://github.com/apache/nifi/pull/6658#discussion_r1103566353


##########
nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchClientServiceImpl.java:
##########
@@ -220,16 +285,36 @@ public List<ConfigVerificationResult> verify(final ConfigurationContext context,
                         .explanation("Elasticsearch Rest Client not configured");
                 warningsResult.outcome(ConfigVerificationResult.Outcome.SKIPPED)
                         .explanation("Elasticsearch Rest Client not configured");
+                snifferResult.outcome(ConfigVerificationResult.Outcome.SKIPPED)
+                        .explanation("Elasticsearch Rest Client not configured");
             }
 
             results.add(clientSetup);
             results.add(connectionResult.build());
             results.add(warningsResult.build());
+            results.add(snifferResult.build());
         }
 
         return results;
     }
 
+    private void verifySniffer(final ConfigurationContext context, final RestClient verifyClient, final ConfigVerificationResult.Builder snifferResult) {
+        try (final Sniffer verifySniffer = setupSniffer(context, verifyClient)) {
+            if (verifySniffer != null && elasticsearchNodesSniffer != null) {
+                final List<Node> nodes = elasticsearchNodesSniffer.sniff();

Review Comment:
   Now that we get a definite set of Nodes through the Sniffer during verification, we could take the opportunity to confirm connectivity - as noted in the PR conversations, it can be that the Elasticsearch instance `publish_address` is an "internal" area that can't actually be contacted by NiFi
   
   While we can't change how the Sniffer works here, we can provide helpful guidance for NiFi users using the verification checks, e.g. "Found 1 node, 0 nodes were contactable, check the Elasticsearch Network Settings". This wouldn't prevent connectivity issues when using the Sniffer for real (it would still get the publish_address from Elasticsearch), but should help NiFi users understand what needs changing in Elasticsearch (or whether the Sniffer is the right option in their use case)
   
   Maybe also link to the Elasticsearch docs from this Controller's additionalDetails?



-- 
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] ChrisSamo632 commented on a diff in pull request #6658: NIFI-10797 add customisable Elasticsearch REST Client config and Elasticsearch Cluster Sniffer

Posted by "ChrisSamo632 (via GitHub)" <gi...@apache.org>.
ChrisSamo632 commented on code in PR #6658:
URL: https://github.com/apache/nifi/pull/6658#discussion_r1104180861


##########
nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service-api/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchClientService.java:
##########
@@ -161,6 +157,118 @@ public interface ElasticSearchClientService extends ControllerService, Verifiabl
             .required(true)
             .build();
 
+    PropertyDescriptor COMPRESSION = new PropertyDescriptor.Builder()
+            .name("el-cs-enable-compression")
+            .displayName("Enable Compression")
+            .description("Whether the REST client should compress requests using gzip content encoding and add the " +
+                    "\"Accept-Encoding: gzip\" header to receive compressed responses")
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true)
+            .build();
+
+    PropertyDescriptor SEND_META_HEADER = new PropertyDescriptor.Builder()
+            .name("el-cs-send-meta-header")
+            .displayName("Send Meta Header")
+            .description("Whether to send a \"X-Elastic-Client-Meta\" header that describes the runtime environment. " +
+                    "It contains information that is similar to what could be found in User-Agent. " +
+                    "Using a separate header allows applications to use User-Agent for their own needs, " +
+                    "e.g. to identify application version or other environment information")
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
+    PropertyDescriptor STRICT_DEPRECATION = new PropertyDescriptor.Builder()
+            .name("el-cs-strict-deprecation")
+            .displayName("Strict Deprecation")
+            .description("Whether the REST client should return any response containing at least one warning header as a failure")
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true)
+            .build();
+
+    AllowableValue NODE_SELECTOR_ANY = new AllowableValue("ANY", "Any",
+            "Select any Elasticsearch node to handle requests");
+    AllowableValue NODE_SELECTOR_SKIP_DEDICATED_MASTERS = new AllowableValue("SKIP_DEDICATED_MASTERS", "Skip Dedicated Masters",
+            "Skip dedicated Elasticsearch master nodes for handling request");
+
+    PropertyDescriptor NODE_SELECTOR = new PropertyDescriptor.Builder()
+            .name("el-cs-node-selector")
+            .displayName("Node Selector")
+            .description("Selects Elasticsearch nodes that can receive requests. Used to keep requests away from dedicated Elasticsearch master nodes")
+            .allowableValues(NODE_SELECTOR_ANY, NODE_SELECTOR_SKIP_DEDICATED_MASTERS)
+            .defaultValue(NODE_SELECTOR_ANY.getValue())
+            .required(true)
+            .build();
+
+    PropertyDescriptor PATH_PREFIX = new PropertyDescriptor.Builder()
+            .name("el-cs-path-prefix")
+            .displayName("Path Prefix")
+            .description("Sets the path's prefix for every request used by the http client. " +
+                    "For example, if this is set to \"/my/path\", then any client request will become \"/my/path/\" + endpoint. " +
+                    "In essence, every request's endpoint is prefixed by this pathPrefix. " +
+                    "The path prefix is useful for when Elasticsearch is behind a proxy that provides a base path or a proxy that requires all paths to start with '/'; " +
+                    "it is not intended for other purposes and it should not be supplied in other scenarios")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .build();
+
+    PropertyDescriptor SNIFF_CLUSTER_NODES = new PropertyDescriptor.Builder()
+            .name("el-cs-sniff-cluster-nodes")
+            .displayName("Sniff Cluster Nodes")
+            .description("Periodically sniff for nodes within the Elasticsearch cluster via the Elasticsearch Node Info API. " +
+                    "If Elasticsearch security features are enabled (default to \"true\" for 8.x+), the Elasticsearch user must " +
+                    "have the \"monitor\" or \"manage\" cluster privilege to use this API." +
+                    "Note that all " + HTTP_HOSTS.getDisplayName() + " (and those that may be discovered within the cluster " +
+                    "using the Sniffer) must use the same protocol, e.g. http or https, and be contactable using the same client settings. " +
+                    "Finally the Elasticsearch \"network.publish_address\" must match one of the \"network.bind_address(es)\" " +

Review Comment:
   Darn, for some reason I keep typing `_address`, good catch though, think I\ve updated it everywhere for both `bind_` and `publish_` settings



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