You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/04/12 22:43:18 UTC

[GitHub] [kafka] kpatelatwork opened a new pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

kpatelatwork opened a new pull request #10530:
URL: https://github.com/apache/kafka/pull/10530


   …ertised uri is invalid and other nodes can't reach it. Node names have rules about what characters they can have and maximum length like in RFC-1123. The node-node communication over REST API won't happen if this node's advertised URL to the cluster has an invalid host name, and the error message in logs isn't very helpful. 
   
   This PR adds a new behavior by using the java IDN class to expose the detailed error message and fails the server bootstrap. 
   
   @C0urante , @rhauch and @kkonstantine  please review
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


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

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



[GitHub] [kafka] C0urante commented on a change in pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
C0urante commented on a change in pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#discussion_r612518884



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
##########
@@ -369,9 +370,31 @@ else if (serverConnector != null && serverConnector.getHost() != null && serverC
         else if (serverConnector != null && serverConnector.getPort() > 0)
             builder.port(serverConnector.getPort());
 
-        log.info("Advertised URI: {}", builder.build());
+        URI uri = builder.build();
+        validateUriHost(uri);
+        log.info("Advertised URI: {}", uri);
 
-        return builder.build();
+        return uri;
+    }
+
+    /**
+     * Parses the uri and throws a more definitive error
+     * when the internal node to node communication can't happen due to an invalid host name.
+     */
+    private void validateUriHost(URI uri) {
+        if (uri.getHost() == null) {
+            String host = Utils.getHost(uri.getAuthority());
+            String errorMsg = "Invalid host=" + host + ", in url=" + uri.toString();

Review comment:
       If `host` is null (due to a parsing failure in `Utils::getHost`), won't this lead to a similarly-confusing error message for the user? I'm not sure `Invalid host=null, in url ...` would be easy to decipher, and although it's unclear if we could ever trigger that code path at the moment, it may be worth considering in case later refactoring of this and other classes makes it possible.
   
   Perhaps something like "Could not parse host from advertised URL <url>" would help shed more light 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.

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



[GitHub] [kafka] kpatelatwork edited a comment on pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
kpatelatwork edited a comment on pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#issuecomment-822663074


   @tombentley could you please review this 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.

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



[GitHub] [kafka] C0urante commented on a change in pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
C0urante commented on a change in pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#discussion_r612513524



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
##########
@@ -369,9 +370,31 @@ else if (serverConnector != null && serverConnector.getHost() != null && serverC
         else if (serverConnector != null && serverConnector.getPort() > 0)
             builder.port(serverConnector.getPort());
 
-        log.info("Advertised URI: {}", builder.build());
+        URI uri = builder.build();
+        validateUriHost(uri);
+        log.info("Advertised URI: {}", uri);
 
-        return builder.build();
+        return uri;
+    }
+
+    /**
+     * Parses the uri and throws a more definitive error
+     * when the internal node to node communication can't happen due to an invalid host name.
+     */
+    private void validateUriHost(URI uri) {
+        if (uri.getHost() == null) {
+            String host = Utils.getHost(uri.getAuthority());

Review comment:
       I was wondering the same thing. Some local testing revealed that `Uri::getHost` returns `null` if the hostname is invalid, which also explains the null check at the line above.




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

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



[GitHub] [kafka] C0urante commented on pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
C0urante commented on pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#issuecomment-824847306


   I've just realized--this may not be a safe change to make. Although we should fail fast on startup if a worker tries to join an existing cluster with a bad hostname, it's still possible right now to run a single worker in that scenario. Although running a single worker in distributed mode may sound like an anti-pattern, there are still benefits to such a setup, such as persistent storage of configurations in a Kafka topic. It's also possible that there are Dockerized quickstarts and demos that use a single Connect worker and may have a bad hostname; these would suddenly break on upgrade if we merge this change.
   
   At the very least, I think we should be more explicit about _why_ we're failing workers on startup with our error message, try to give users a better picture of how workers with a bad hostname can be dangerous (since right now they don't fail on startup but instead begin to fail silently when forwarding user requests or task configs to the leader), and even call out Dockerized setups with instructions on how to fix the worker config in that case by changing the advertised URL to use a valid hostname and, if running a multi-node cluster, making sure that the worker is reachable from other workers with that advertised URL.
   
   But I think at this point we're doing a fair bit of work to try to circumvent this issue instead of addressing it head-on. Using a different HTTP client has already been discussed as an option but punted on in favor of a smaller, simpler change; given that this change seems less simple now, I wonder if it's worth reconsidering.


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

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



[GitHub] [kafka] kpatelatwork commented on a change in pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
kpatelatwork commented on a change in pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#discussion_r617556919



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
##########
@@ -369,9 +370,34 @@ else if (serverConnector != null && serverConnector.getHost() != null && serverC
         else if (serverConnector != null && serverConnector.getPort() > 0)
             builder.port(serverConnector.getPort());
 
-        log.info("Advertised URI: {}", builder.build());
+        URI uri = builder.build();
+        validateUriHost(uri);
+        log.info("Advertised URI: {}", uri);
 
-        return builder.build();
+        return uri;
+    }
+
+    /**
+     * Parses the uri and throws a more definitive error
+     * when the internal node to node communication can't happen due to an invalid host name.
+     */
+    static void validateUriHost(URI uri) {
+        //java URI parsing will fail silently returning null in the host if the host name contains invalid characters like _
+        //this bubbles up later when the Herder tries to communicate on the advertised url and the current HttpClient fails with an ambiguous message
+        //we need to revisit this when we upgrade to a better HttpClient that can communicate with such host names or throws a better error message

Review comment:
       @tombentley  excellent points about not parsing the hostname from URI and I also felt Utils.getHost was broken as it didn't worked on a case with path in the URL and that's why I passed authority to it.
   
   Let me work on a change using the approach you suggested and get back to you. I was doing uri.getHost null check to avoid the blast radius in case we didn't anticipated some pattern so I think we should still use the null check but as you said use the hostname that is available to us instead of parsing it from authority and uri.




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

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



[GitHub] [kafka] kpatelatwork commented on pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
kpatelatwork commented on pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#issuecomment-819667966


   @kkonstantine and @rhauch Could one of you please review and merge this.


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

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



[GitHub] [kafka] tombentley commented on a change in pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
tombentley commented on a change in pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#discussion_r616468886



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
##########
@@ -369,9 +370,31 @@ else if (serverConnector != null && serverConnector.getHost() != null && serverC
         else if (serverConnector != null && serverConnector.getPort() > 0)
             builder.port(serverConnector.getPort());
 
-        log.info("Advertised URI: {}", builder.build());
+        URI uri = builder.build();
+        validateUriHost(uri);
+        log.info("Advertised URI: {}", uri);
 
-        return builder.build();
+        return uri;
+    }
+
+    /**
+     * Parses the uri and throws a more definitive error
+     * when the internal node to node communication can't happen due to an invalid host name.
+     */
+    static void validateUriHost(URI uri) {
+        if (uri.getHost() == null) {
+            String host = Utils.getHost(uri.getAuthority());
+            String errorMsg = "Could not parse host from advertised URL=" + uri.toString();

Review comment:
       We don't tend to use `=` in exception messages like this, so perhaps it's better to format the String something link `"Could not parse host '%s' from advertised URL"`

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
##########
@@ -369,9 +370,31 @@ else if (serverConnector != null && serverConnector.getHost() != null && serverC
         else if (serverConnector != null && serverConnector.getPort() > 0)
             builder.port(serverConnector.getPort());
 
-        log.info("Advertised URI: {}", builder.build());
+        URI uri = builder.build();
+        validateUriHost(uri);
+        log.info("Advertised URI: {}", uri);
 
-        return builder.build();
+        return uri;
+    }
+
+    /**
+     * Parses the uri and throws a more definitive error
+     * when the internal node to node communication can't happen due to an invalid host name.
+     */
+    static void validateUriHost(URI uri) {
+        if (uri.getHost() == null) {
+            String host = Utils.getHost(uri.getAuthority());
+            String errorMsg = "Could not parse host from advertised URL=" + uri.toString();
+            if (host != null) {
+                try {
+                    IDN.toASCII(host, IDN.USE_STD3_ASCII_RULES);
+                } catch (IllegalArgumentException e) {
+                    errorMsg += ", as it doesn't conform to RFC 1123 specification, reason=" + e.getMessage();

Review comment:
       Same comment about `=`, here I'd write `", as it doesn't conform to RFC 1123 specification: " + e.getMessage();`
   

##########
File path: connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java
##########
@@ -185,6 +189,25 @@ public void testAdvertisedUri() {
         Assert.assertEquals("http://plaintext-localhost:4761/", server.advertisedUrl().toString());
     }
 
+    @Test
+    public void testValidateUriHost() {
+        validateUriHost(URI.create("http://localhost:8080"));
+        validateUriHost(URI.create("http://172.217.2.110:80"));
+        validateUriHost(URI.create("http://[2607:f8b0:4006:818::2004]:80"));

Review comment:
       Could we add some valid IDN hostnames here too?

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
##########
@@ -369,9 +370,31 @@ else if (serverConnector != null && serverConnector.getHost() != null && serverC
         else if (serverConnector != null && serverConnector.getPort() > 0)
             builder.port(serverConnector.getPort());
 
-        log.info("Advertised URI: {}", builder.build());
+        URI uri = builder.build();
+        validateUriHost(uri);
+        log.info("Advertised URI: {}", uri);
 
-        return builder.build();
+        return uri;
+    }
+
+    /**
+     * Parses the uri and throws a more definitive error
+     * when the internal node to node communication can't happen due to an invalid host name.
+     */
+    private void validateUriHost(URI uri) {
+        if (uri.getHost() == null) {
+            String host = Utils.getHost(uri.getAuthority());

Review comment:
       I do wonder whether it would be simpler to just validate the given `REST_ADVERTISED_HOST_NAME_CONFIG` around line 362. That way we don't need to mess around with URIs and are just validating what's been supplied.
   
   If we do need to keep this stuff with URIs, I think this is worth a comment to explain that `uri.getHost()` will only be null if the host was undefined or the URI couldn't be parsed as a URI containing a host. Also, it's possible that `getAuthority` can return null, so best to check 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.

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



[GitHub] [kafka] kpatelatwork removed a comment on pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
kpatelatwork removed a comment on pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#issuecomment-819667966


   @kkonstantine and @rhauch Could one of you please review and merge this.


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

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



[GitHub] [kafka] tombentley commented on a change in pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
tombentley commented on a change in pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#discussion_r618372208



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
##########
@@ -357,21 +358,53 @@ public URI advertisedUrl() {
         ServerConnector serverConnector = findConnector(advertisedSecurityProtocol);
         builder.scheme(advertisedSecurityProtocol);
 
-        String advertisedHostname = config.getString(WorkerConfig.REST_ADVERTISED_HOST_NAME_CONFIG);
-        if (advertisedHostname != null && !advertisedHostname.isEmpty())
-            builder.host(advertisedHostname);
-        else if (serverConnector != null && serverConnector.getHost() != null && serverConnector.getHost().length() > 0)
-            builder.host(serverConnector.getHost());
+        String hostNameOverride = hostNameOverride(serverConnector);
+        if (hostNameOverride != null) {
+            builder.host(hostNameOverride);
+        }
 
         Integer advertisedPort = config.getInt(WorkerConfig.REST_ADVERTISED_PORT_CONFIG);
         if (advertisedPort != null)
             builder.port(advertisedPort);
         else if (serverConnector != null && serverConnector.getPort() > 0)
             builder.port(serverConnector.getPort());
 
-        log.info("Advertised URI: {}", builder.build());
+        URI uri = builder.build();
+        maybeThrowInvalidHostNameException(uri, hostNameOverride);
+        log.info("Advertised URI: {}", uri);
 
-        return builder.build();
+        return uri;
+    }
+
+    private String hostNameOverride(ServerConnector serverConnector) {
+        String advertisedHostname = config.getString(WorkerConfig.REST_ADVERTISED_HOST_NAME_CONFIG);
+        if (advertisedHostname != null && !advertisedHostname.isEmpty())
+            return advertisedHostname;
+        else if (serverConnector != null && serverConnector.getHost() != null && serverConnector.getHost().length() > 0)
+            return serverConnector.getHost();
+        return null;
+    }
+
+    /**
+     * Parses the uri and throws a more definitive error
+     * when the internal node to node communication can't happen due to an invalid host name.
+     */
+    static void maybeThrowInvalidHostNameException(URI uri, String hostNameOverride) {
+        //java URI parsing will fail silently returning null in the host if the host name contains invalid characters like _
+        //this bubbles up later when the Herder tries to communicate on the advertised url and the current HttpClient fails with an ambiguous message
+        if (uri.getHost() == null) {
+            String errorMsg = "Could not parse host from advertised URL: '"  + uri.toString() + "'";
+            if (hostNameOverride != null) {
+                //validate hostname using IDN class to see if it can bubble up the real cause and we can show the user a more detailed exception
+                try {
+                    IDN.toASCII(hostNameOverride, IDN.USE_STD3_ASCII_RULES);
+                } catch (IllegalArgumentException e) {
+                    errorMsg += ", as it doesn't conform to RFC 1123 specification: " + e.getMessage();

Review comment:
       If we're going to mention the RFC let's mention section 2.1 specifically. We're referring the user to a nearly 100 page document and it's only a tiny part of it which is relevant here.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
##########
@@ -357,21 +358,53 @@ public URI advertisedUrl() {
         ServerConnector serverConnector = findConnector(advertisedSecurityProtocol);
         builder.scheme(advertisedSecurityProtocol);
 
-        String advertisedHostname = config.getString(WorkerConfig.REST_ADVERTISED_HOST_NAME_CONFIG);
-        if (advertisedHostname != null && !advertisedHostname.isEmpty())
-            builder.host(advertisedHostname);
-        else if (serverConnector != null && serverConnector.getHost() != null && serverConnector.getHost().length() > 0)
-            builder.host(serverConnector.getHost());
+        String hostNameOverride = hostNameOverride(serverConnector);
+        if (hostNameOverride != null) {
+            builder.host(hostNameOverride);
+        }
 
         Integer advertisedPort = config.getInt(WorkerConfig.REST_ADVERTISED_PORT_CONFIG);
         if (advertisedPort != null)
             builder.port(advertisedPort);
         else if (serverConnector != null && serverConnector.getPort() > 0)
             builder.port(serverConnector.getPort());
 
-        log.info("Advertised URI: {}", builder.build());
+        URI uri = builder.build();
+        maybeThrowInvalidHostNameException(uri, hostNameOverride);
+        log.info("Advertised URI: {}", uri);
 
-        return builder.build();
+        return uri;
+    }
+
+    private String hostNameOverride(ServerConnector serverConnector) {
+        String advertisedHostname = config.getString(WorkerConfig.REST_ADVERTISED_HOST_NAME_CONFIG);
+        if (advertisedHostname != null && !advertisedHostname.isEmpty())
+            return advertisedHostname;
+        else if (serverConnector != null && serverConnector.getHost() != null && serverConnector.getHost().length() > 0)
+            return serverConnector.getHost();
+        return null;
+    }
+
+    /**
+     * Parses the uri and throws a more definitive error
+     * when the internal node to node communication can't happen due to an invalid host name.
+     */
+    static void maybeThrowInvalidHostNameException(URI uri, String hostNameOverride) {
+        //java URI parsing will fail silently returning null in the host if the host name contains invalid characters like _
+        //this bubbles up later when the Herder tries to communicate on the advertised url and the current HttpClient fails with an ambiguous message

Review comment:
       It's not that URI parsing _fails_, it actually succeeds, but it doesn't result in a server-based authority (it's a registry-based authority instead, precisely because what would be the hostname part cannot be a hostname because of the illegal syntax) and so there really is no host. Can we amend the comment to something like this:
   
   ```suggestion
           // When the hostname contains illegal characters (e.g. _) the URI authority will be registry-based, lacking a host
           // so detect it here to prevent a confusing error later when the Herder tries to communicate on the advertised url
   ```




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

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



[GitHub] [kafka] kpatelatwork commented on a change in pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
kpatelatwork commented on a change in pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#discussion_r617556919



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
##########
@@ -369,9 +370,34 @@ else if (serverConnector != null && serverConnector.getHost() != null && serverC
         else if (serverConnector != null && serverConnector.getPort() > 0)
             builder.port(serverConnector.getPort());
 
-        log.info("Advertised URI: {}", builder.build());
+        URI uri = builder.build();
+        validateUriHost(uri);
+        log.info("Advertised URI: {}", uri);
 
-        return builder.build();
+        return uri;
+    }
+
+    /**
+     * Parses the uri and throws a more definitive error
+     * when the internal node to node communication can't happen due to an invalid host name.
+     */
+    static void validateUriHost(URI uri) {
+        //java URI parsing will fail silently returning null in the host if the host name contains invalid characters like _
+        //this bubbles up later when the Herder tries to communicate on the advertised url and the current HttpClient fails with an ambiguous message
+        //we need to revisit this when we upgrade to a better HttpClient that can communicate with such host names or throws a better error message

Review comment:
       @tombentley  excellent points about not parsing the hostname from URI and I also felt Utils.getHost was broken as it didn't worked on a case with path in the URL and that's why I passed authority to it.
   
   Let me work on a change using the approach you suggested and get back to you. I was doing uri.getHost null check to avoid the blast radius in case we didn't anticipated some pattern so I think we should still use the null check but as you said use the hostname that is available to us instead of parsing it from authority and host.




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

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



[GitHub] [kafka] kpatelatwork commented on pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
kpatelatwork commented on pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#issuecomment-824873571


   Great catch @C0urante . Let me investigate what it takes to upgrade to an Apache HttpClient and how much of an effort it is.  I Agree a seemingly simple change doesn't seem simple anymore so why not put the effort in the right future direction to use a client that supports IDNs.


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

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



[GitHub] [kafka] kpatelatwork edited a comment on pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
kpatelatwork edited a comment on pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#issuecomment-823359381


   @tombentley Thanks for the comments, I resolved all of them :). When you get time could you please review again to see if it looks good?


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

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



[GitHub] [kafka] kpatelatwork commented on pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
kpatelatwork commented on pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#issuecomment-824141840


   @tombentley I took another shot of implementing the PR using the above suggestions.  
   
   Could you please check if it still needs improvement?
   
   I didn't do explicit IPV4 and IPV6 checks because the `uri.getHost()` won't be null if it's a valid address.


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

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



[GitHub] [kafka] tombentley commented on a change in pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
tombentley commented on a change in pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#discussion_r612219113



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
##########
@@ -369,9 +370,31 @@ else if (serverConnector != null && serverConnector.getHost() != null && serverC
         else if (serverConnector != null && serverConnector.getPort() > 0)
             builder.port(serverConnector.getPort());
 
-        log.info("Advertised URI: {}", builder.build());
+        URI uri = builder.build();
+        validateUriHost(uri);
+        log.info("Advertised URI: {}", uri);
 
-        return builder.build();
+        return uri;
+    }
+
+    /**
+     * Parses the uri and throws a more definitive error
+     * when the internal node to node communication can't happen due to an invalid host name.
+     */
+    private void validateUriHost(URI uri) {
+        if (uri.getHost() == null) {
+            String host = Utils.getHost(uri.getAuthority());

Review comment:
       Why use `Utils.getHost()` rather than `uri.getHost()`?




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

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



[GitHub] [kafka] kpatelatwork commented on pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
kpatelatwork commented on pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#issuecomment-822663074


   @tombentley could you please review and merge this 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.

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



[GitHub] [kafka] kpatelatwork commented on a change in pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
kpatelatwork commented on a change in pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#discussion_r612798221



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
##########
@@ -369,9 +370,31 @@ else if (serverConnector != null && serverConnector.getHost() != null && serverC
         else if (serverConnector != null && serverConnector.getPort() > 0)
             builder.port(serverConnector.getPort());
 
-        log.info("Advertised URI: {}", builder.build());
+        URI uri = builder.build();
+        validateUriHost(uri);
+        log.info("Advertised URI: {}", uri);
 
-        return builder.build();
+        return uri;
+    }
+
+    /**
+     * Parses the uri and throws a more definitive error
+     * when the internal node to node communication can't happen due to an invalid host name.
+     */
+    private void validateUriHost(URI uri) {
+        if (uri.getHost() == null) {
+            String host = Utils.getHost(uri.getAuthority());
+            String errorMsg = "Invalid host=" + host + ", in url=" + uri.toString();

Review comment:
       Fixed the error message as per your suggestions.




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

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



[GitHub] [kafka] kpatelatwork commented on pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
kpatelatwork commented on pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#issuecomment-823359381


   @tombentley Thanks for the comments, I resolved all of them :). Could you please review again and merge?


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

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



[GitHub] [kafka] kpatelatwork commented on a change in pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
kpatelatwork commented on a change in pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#discussion_r613415919



##########
File path: connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java
##########
@@ -185,6 +188,36 @@ public void testAdvertisedUri() {
         Assert.assertEquals("http://plaintext-localhost:4761/", server.advertisedUrl().toString());
     }
 
+    @Test
+    public void testValidateUriHost() {
+        Map<String, String> configMap = new HashMap<>(baseWorkerProps());
+        configMap.put(WorkerConfig.LISTENERS_CONFIG, "http://localhost:8080,https://localhost:8443");
+        DistributedConfig config = new DistributedConfig(configMap);
+
+        server = new RestServer(config);

Review comment:
       @C0urante I made it static and I Agree it made the testing clean.




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

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



[GitHub] [kafka] kpatelatwork commented on a change in pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
kpatelatwork commented on a change in pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#discussion_r612545585



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
##########
@@ -369,9 +370,31 @@ else if (serverConnector != null && serverConnector.getHost() != null && serverC
         else if (serverConnector != null && serverConnector.getPort() > 0)
             builder.port(serverConnector.getPort());
 
-        log.info("Advertised URI: {}", builder.build());
+        URI uri = builder.build();
+        validateUriHost(uri);
+        log.info("Advertised URI: {}", uri);
 
-        return builder.build();
+        return uri;
+    }
+
+    /**
+     * Parses the uri and throws a more definitive error
+     * when the internal node to node communication can't happen due to an invalid host name.
+     */
+    private void validateUriHost(URI uri) {
+        if (uri.getHost() == null) {
+            String host = Utils.getHost(uri.getAuthority());

Review comment:
       You are right, if the domain name is invalid then Uri parsing gives up and this is why authority will have the "host:port" pattern but the host and port will be null and thats why I am using  Utils.getHost uses regex to parse this out of authority(not reinventing the wheel as the method was already there). 
   
    The other alternative I had considered was "new URL(uri.toString()).getHost()" but in the future if java fixes the bug then it may also start returning null like URI.




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

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



[GitHub] [kafka] kpatelatwork commented on a change in pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
kpatelatwork commented on a change in pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#discussion_r616783892



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
##########
@@ -369,9 +370,31 @@ else if (serverConnector != null && serverConnector.getHost() != null && serverC
         else if (serverConnector != null && serverConnector.getPort() > 0)
             builder.port(serverConnector.getPort());
 
-        log.info("Advertised URI: {}", builder.build());
+        URI uri = builder.build();
+        validateUriHost(uri);
+        log.info("Advertised URI: {}", uri);
 
-        return builder.build();
+        return uri;
+    }
+
+    /**
+     * Parses the uri and throws a more definitive error
+     * when the internal node to node communication can't happen due to an invalid host name.
+     */
+    static void validateUriHost(URI uri) {
+        if (uri.getHost() == null) {
+            String host = Utils.getHost(uri.getAuthority());
+            String errorMsg = "Could not parse host from advertised URL=" + uri.toString();

Review comment:
       fixed

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
##########
@@ -369,9 +370,31 @@ else if (serverConnector != null && serverConnector.getHost() != null && serverC
         else if (serverConnector != null && serverConnector.getPort() > 0)
             builder.port(serverConnector.getPort());
 
-        log.info("Advertised URI: {}", builder.build());
+        URI uri = builder.build();
+        validateUriHost(uri);
+        log.info("Advertised URI: {}", uri);
 
-        return builder.build();
+        return uri;
+    }
+
+    /**
+     * Parses the uri and throws a more definitive error
+     * when the internal node to node communication can't happen due to an invalid host name.
+     */
+    static void validateUriHost(URI uri) {
+        if (uri.getHost() == null) {
+            String host = Utils.getHost(uri.getAuthority());
+            String errorMsg = "Could not parse host from advertised URL=" + uri.toString();
+            if (host != null) {
+                try {
+                    IDN.toASCII(host, IDN.USE_STD3_ASCII_RULES);
+                } catch (IllegalArgumentException e) {
+                    errorMsg += ", as it doesn't conform to RFC 1123 specification, reason=" + e.getMessage();

Review comment:
       fixed

##########
File path: connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java
##########
@@ -185,6 +189,25 @@ public void testAdvertisedUri() {
         Assert.assertEquals("http://plaintext-localhost:4761/", server.advertisedUrl().toString());
     }
 
+    @Test
+    public void testValidateUriHost() {
+        validateUriHost(URI.create("http://localhost:8080"));
+        validateUriHost(URI.create("http://172.217.2.110:80"));
+        validateUriHost(URI.create("http://[2607:f8b0:4006:818::2004]:80"));

Review comment:
       done




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

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



[GitHub] [kafka] kpatelatwork edited a comment on pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
kpatelatwork edited a comment on pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#issuecomment-825190538


   @tombentley and @C0urante  Thanks for the above comment.
   
    I peeked at the code for upgrading to Apache HttpClient and this seems like a big work especially around the SSL factory and other parameters. This has the potential to cause even bigger disruption. I am a new committer and the benefits of doing this upgrade don't seem worth it to me as this bug seems like a very corner case.  All we want to do is report a better error message to the user that he needs to fix the hostname.
   
   At this point, my suggestion is to pick between the below two choices:
   
   1. close the PR, given it's a corner case, and revisit in the future.
   2. Go back to the original PR where the plan was to validate the URL when Herder tries to forward the request to another node here https://github.com/kpatelatwork/kafka/blob/7c8fb346a976903cc67e66c4ccfe6cc9858b5048/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestClient.java#L107 .   This would require parsing the URL to create a URI and see if the host is null then parse the host out of the URL.  I tried 
   `System.out.println(new java.net.URL("http://kafka_connect-0.dev-2:8080").getHost());` 
   and it seems to do return the proper hostname that later we can validate using IDN class.  
   
   What do you think if we go #2 approach?
   


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

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



[GitHub] [kafka] C0urante commented on a change in pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
C0urante commented on a change in pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#discussion_r613339599



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
##########
@@ -369,9 +370,31 @@ else if (serverConnector != null && serverConnector.getHost() != null && serverC
         else if (serverConnector != null && serverConnector.getPort() > 0)
             builder.port(serverConnector.getPort());
 
-        log.info("Advertised URI: {}", builder.build());
+        URI uri = builder.build();
+        validateUriHost(uri);
+        log.info("Advertised URI: {}", uri);
 
-        return builder.build();
+        return uri;
+    }
+
+    /**
+     * Parses the uri and throws a more definitive error
+     * when the internal node to node communication can't happen due to an invalid host name.
+     */
+    private void validateUriHost(URI uri) {
+        if (uri.getHost() == null) {
+            String host = Utils.getHost(uri.getAuthority());
+            String errorMsg = "Invalid host=" + host + ", in url=" + uri.toString();

Review comment:
       Yeah, changing to a new REST client is probably fine as a follow-up ticket 👍 

##########
File path: connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java
##########
@@ -185,6 +188,36 @@ public void testAdvertisedUri() {
         Assert.assertEquals("http://plaintext-localhost:4761/", server.advertisedUrl().toString());
     }
 
+    @Test
+    public void testValidateUriHost() {
+        Map<String, String> configMap = new HashMap<>(baseWorkerProps());
+        configMap.put(WorkerConfig.LISTENERS_CONFIG, "http://localhost:8080,https://localhost:8443");
+        DistributedConfig config = new DistributedConfig(configMap);
+
+        server = new RestServer(config);
+        server.validateUriHost(URI.create("http://localhost:8080"));
+        server.validateUriHost(URI.create("http://172.217.2.110:80"));
+        server.validateUriHost(URI.create("http://[2607:f8b0:4006:818::2004]:80"));
+    }
+
+    @Test
+    public void testValidateUriInvalidHost() {
+        Map<String, String> configMap = new HashMap<>(baseWorkerProps());
+        configMap.put(WorkerConfig.LISTENERS_CONFIG, "http://localhost:8080,https://localhost:8443");
+        DistributedConfig config = new DistributedConfig(configMap);
+
+        server = new RestServer(config);
+
+        ConnectException exception = assertThrows(ConnectException.class, () -> server.validateUriHost(URI.create("http://kafka_connect-0.dev-2:8080")));
+        assertTrue(exception.getMessage().contains("RFC 1123"));
+        //invalid uri with / in the end

Review comment:
       Nice case!

##########
File path: connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java
##########
@@ -185,6 +188,36 @@ public void testAdvertisedUri() {
         Assert.assertEquals("http://plaintext-localhost:4761/", server.advertisedUrl().toString());
     }
 
+    @Test
+    public void testValidateUriHost() {
+        Map<String, String> configMap = new HashMap<>(baseWorkerProps());
+        configMap.put(WorkerConfig.LISTENERS_CONFIG, "http://localhost:8080,https://localhost:8443");
+        DistributedConfig config = new DistributedConfig(configMap);
+
+        server = new RestServer(config);

Review comment:
       Nit: seems a little strange to have to instantiate a `RestServer` with one set of REST listeners and no advertised listeners, but then use it to validate a different set of advertised listeners. Would it make sense to change `RestServer::validateUriHost` to a `static` method to make the testing here cleaner?




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

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



[GitHub] [kafka] kpatelatwork commented on a change in pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
kpatelatwork commented on a change in pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#discussion_r612800672



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
##########
@@ -369,9 +370,31 @@ else if (serverConnector != null && serverConnector.getHost() != null && serverC
         else if (serverConnector != null && serverConnector.getPort() > 0)
             builder.port(serverConnector.getPort());
 
-        log.info("Advertised URI: {}", builder.build());
+        URI uri = builder.build();
+        validateUriHost(uri);
+        log.info("Advertised URI: {}", uri);
 
-        return builder.build();
+        return uri;
+    }
+
+    /**
+     * Parses the uri and throws a more definitive error
+     * when the internal node to node communication can't happen due to an invalid host name.
+     */
+    private void validateUriHost(URI uri) {
+        if (uri.getHost() == null) {
+            String host = Utils.getHost(uri.getAuthority());
+            String errorMsg = "Invalid host=" + host + ", in url=" + uri.toString();

Review comment:
       The new check in the code happens only when uri.getHost is null so its not worse than what we have in trunk but I do agree Apache HttpClient may fix this. I had thought about using Apache HttpClient but isn't this a big library change so do we want to do that in a separate ticket as I wasn't sure about out library rules as I didn't found Apache HttpClient being used anywhere in our code except in 1 test case.

##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
##########
@@ -369,9 +370,31 @@ else if (serverConnector != null && serverConnector.getHost() != null && serverC
         else if (serverConnector != null && serverConnector.getPort() > 0)
             builder.port(serverConnector.getPort());
 
-        log.info("Advertised URI: {}", builder.build());
+        URI uri = builder.build();
+        validateUriHost(uri);
+        log.info("Advertised URI: {}", uri);
 
-        return builder.build();
+        return uri;
+    }
+
+    /**
+     * Parses the uri and throws a more definitive error
+     * when the internal node to node communication can't happen due to an invalid host name.
+     */
+    private void validateUriHost(URI uri) {
+        if (uri.getHost() == null) {
+            String host = Utils.getHost(uri.getAuthority());
+            String errorMsg = "Invalid host=" + host + ", in url=" + uri.toString();

Review comment:
       @C0urante  The new check in the code happens only when uri.getHost is null so its not worse than what we have in trunk but I do agree Apache HttpClient may fix this. I had thought about using Apache HttpClient but isn't this a big library change so do we want to do that in a separate ticket as I wasn't sure about out library rules as I didn't found Apache HttpClient being used anywhere in our code except in 1 test 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.

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



[GitHub] [kafka] kpatelatwork commented on pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
kpatelatwork commented on pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#issuecomment-825190538


   @tombentley and @C0urante  Thanks for the above comment.
   
    I peeked at the code for upgrading to Apache HttpClient and this seems like a big work especially around the SSL factory and other parameters. This has the potential to cause even bigger disruption. I am a new committer and the benefits of doing this upgrade don't seem worth it to me as this bug seems like a very corner case.  All we want to do is report a better error message to the user that he needs to fix the hostname.
   
   At this point, my suggestion is to pick between the below two choices:
   
   1. close the PR, given it's a corner case, and revisit in the future.
   2. Go back to the original PR where the plan was to validate the URL when Herder tries to forward the request to another node here https://github.com/kpatelatwork/kafka/blob/7c8fb346a976903cc67e66c4ccfe6cc9858b5048/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestClient.java#L107 .   This would require parsing the URL to create a URI and see if the host is null then parse the host out of the URL.  I tried 
   `System.out.println(new java.net.URL("http://kafka_connect-0.dev-2:8080").getHost());` 
   and it seems to do return the proper hostname.  
   
   What do you think if we go #2 approach?
   


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

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



[GitHub] [kafka] tombentley commented on pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
tombentley commented on pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#issuecomment-824864967


   Great spot @C0urante! I completely agree we shouldn't break working single-node distributed installs and that trying to fix this validation approach is going to end up being a lot of work.


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

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



[GitHub] [kafka] tombentley commented on a change in pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
tombentley commented on a change in pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#discussion_r618109470



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
##########
@@ -357,21 +358,53 @@ public URI advertisedUrl() {
         ServerConnector serverConnector = findConnector(advertisedSecurityProtocol);
         builder.scheme(advertisedSecurityProtocol);
 
-        String advertisedHostname = config.getString(WorkerConfig.REST_ADVERTISED_HOST_NAME_CONFIG);
-        if (advertisedHostname != null && !advertisedHostname.isEmpty())
-            builder.host(advertisedHostname);
-        else if (serverConnector != null && serverConnector.getHost() != null && serverConnector.getHost().length() > 0)
-            builder.host(serverConnector.getHost());
+        String hostNameOverride = hostNameOverride(serverConnector);
+        if (hostNameOverride != null) {
+            builder.host(hostNameOverride);
+        }
 
         Integer advertisedPort = config.getInt(WorkerConfig.REST_ADVERTISED_PORT_CONFIG);
         if (advertisedPort != null)
             builder.port(advertisedPort);
         else if (serverConnector != null && serverConnector.getPort() > 0)
             builder.port(serverConnector.getPort());
 
-        log.info("Advertised URI: {}", builder.build());
+        URI uri = builder.build();
+        maybeThrowInvalidHostNameException(uri, hostNameOverride);
+        log.info("Advertised URI: {}", uri);
 
-        return builder.build();
+        return uri;
+    }
+
+    private String hostNameOverride(ServerConnector serverConnector) {
+        String advertisedHostname = config.getString(WorkerConfig.REST_ADVERTISED_HOST_NAME_CONFIG);
+        if (advertisedHostname != null && !advertisedHostname.isEmpty())
+            return advertisedHostname;
+        else if (serverConnector != null && serverConnector.getHost() != null && serverConnector.getHost().length() > 0)
+            return serverConnector.getHost();
+        return null;
+    }
+
+    /**
+     * Parses the uri and throws a more definitive error
+     * when the internal node to node communication can't happen due to an invalid host name.
+     */
+    static void maybeThrowInvalidHostNameException(URI uri, String hostNameOverride) {
+        //java URI parsing will fail silently returning null in the host if the host name contains invalid characters like _
+        //this bubbles up later when the Herder tries to communicate on the advertised url and the current HttpClient fails with an ambiguous message
+        if (uri.getHost() == null) {
+            String errorMsg = "Could not parse host from advertised URL: '"  + uri.toString() + "'";
+            if (hostNameOverride != null) {
+                //validate hostname using IDN class to see if it can bubble up the real cause and we can show the user a more detailed exception

Review comment:
       Can we mention that hostNameOverride won't contain any non-ascii characters at this point; we're just calling toASCII for it's RFC-1123 validation.




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

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



[GitHub] [kafka] tombentley commented on a change in pull request #10530: KAFKA-10231 fail bootstrap of Rest server if the host name in the adv…

Posted by GitBox <gi...@apache.org>.
tombentley commented on a change in pull request #10530:
URL: https://github.com/apache/kafka/pull/10530#discussion_r616833243



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java
##########
@@ -369,9 +370,34 @@ else if (serverConnector != null && serverConnector.getHost() != null && serverC
         else if (serverConnector != null && serverConnector.getPort() > 0)
             builder.port(serverConnector.getPort());
 
-        log.info("Advertised URI: {}", builder.build());
+        URI uri = builder.build();
+        validateUriHost(uri);
+        log.info("Advertised URI: {}", uri);
 
-        return builder.build();
+        return uri;
+    }
+
+    /**
+     * Parses the uri and throws a more definitive error
+     * when the internal node to node communication can't happen due to an invalid host name.
+     */
+    static void validateUriHost(URI uri) {
+        //java URI parsing will fail silently returning null in the host if the host name contains invalid characters like _
+        //this bubbles up later when the Herder tries to communicate on the advertised url and the current HttpClient fails with an ambiguous message
+        //we need to revisit this when we upgrade to a better HttpClient that can communicate with such host names or throws a better error message

Review comment:
       The place for plans for future changes is JIRA. Statements about changes which may or may not happen in the future means the comment could become stale (in this case when the client does get changed), or at least means someone has to know to amend the comment when the client does change. So best drop the last line imo.




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

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