KAFKA-6195: Resolve DNS aliases in bootstrap.server#4485
KAFKA-6195: Resolve DNS aliases in bootstrap.server#4485rajinisivaram merged 36 commits intoapache:trunkfrom
Conversation
Reverse DNS lookup performed regardless of the SecurityProtocol used if bootstrap.reverse.dns.lookup is set to true.
| import org.apache.kafka.clients.KafkaClient; | ||
| import org.apache.kafka.clients.Metadata; | ||
| import org.apache.kafka.clients.NetworkClient; | ||
| import org.apache.kafka.clients.*; |
There was a problem hiding this comment.
I don't think we want star imports.
| this.metadata = metadata; | ||
| List<InetSocketAddress> addresses = ClientUtils.parseAndValidateAddresses( | ||
| config.getList(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG)); | ||
| List<InetSocketAddress> addresses = ClientUtils.parseAndValidateAddresses(config.getList(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG),config.getString(CommonClientConfigs.CLIENT_DNS_LOOKUP)); |
There was a problem hiding this comment.
This line is too long. We need the linebreaks back here
| checkWithoutLookup("127.0.0.1:8000"); | ||
| checkWithoutLookup("mydomain.com:8080"); | ||
| checkWithoutLookup("[::1]:8000"); | ||
| checkWithoutLookup("[2001:db8:85a3:8d3:1319:8a2e:370:7348]:1234", "mydomain.com:10000"); |
There was a problem hiding this comment.
Some day, mydomain.com will be changed and all of this will break. Additionally, this prevents people from running the unit tests without network connectivity. Our unit tests shouldn't have this dependency on an external network resource.
We should be able to figure out what the IP address is for localhost and go from there.
If we want to validate a non-localhost address as well, we can use the standard "example.com" (See https://en.wikipedia.org/wiki/Example.com )
There was a problem hiding this comment.
Good point. I was merely following what was already in place.
Would we want to address this now or could the feature be merged as is and I can take a stab at having something better for this test next week?
KAFKA-6195: Add new parameter to toggle reverse dns lookup KAFKA-6195: Remove specific handling for SSL. Reverse DNS lookup performed regardless of the SecurityProtocol used if bootstrap.reverse.dns.lookup is set to true. KAFKA-6195 introduce enum to drive dns lookup behaviour Remove star import adding apache license header
| + "servers (you may want more than one, though, in case a server is down)."; | ||
|
|
||
| public static final String CLIENT_DNS_LOOKUP = "client.dns.lookup"; | ||
| public static final String CLIENT_DNS_LOOKUP_DOC = "Drives whether the client performs DNS lookups on entries of bootstrap.servers"; |
There was a problem hiding this comment.
I would phrase this as "To enable client DNS lookups on bootstrap.servers"
|
@lepolac overall looks good to me. Left a minor nit and there are few un-addressed comments from previous reviewers. Can you address these and rebase your commits into single one. |
|
@harshach I'm a bit struggling to rebase the commits, started from a fresh directory, cloned my fork, checked out my KAFKA-6195 branch. Then added official repo as a remote, fetched it. A git merge-base KAFKA-6195 kafka/trunk gives me hash b5da5f8, but doing a rebase -i on that hash doesn't show me anything to pick. Any ideas ? |
|
@lepolac sorry didn't see your comment. The approach I usually go for is to update the trunk branch in my fork and merge with my JIRA branch. |
|
@harshach Ok, managed to push, hopefully this looks ok ? |
|
retest this please |
|
@lepolac couple of compilation errors are coming. You can run |
|
@lepolac @edoardocomar @mimaison I have reviewed this as well as PR #4987. If we can the config naming consistent today, address review comments and get clean builds today, then we can merge both to 2.1.0 and trunk. |
…ry-catch, adding config in WorkerConfig
|
@lepolac I have merged the other PR. Can you rebase this one please? |
|
Ok, looks like ClientDnsLookup from the other KIP was put in another module and uses other value lookup methods, will merge this. |
|
@lepolac I think |
|
Probably faster for us to change on this PR I think as other one is already merged |
|
@lepolac @adammilnesmith There are PR build failures, can you take a look? |
rajinisivaram
left a comment
There was a problem hiding this comment.
@lepolac Thanks for the updates, looks good. Left some minor comments. I think I can merge this once they are addressed.
| String resolvedCanonicalName = inetAddress.getCanonicalHostName(); | ||
| InetSocketAddress address = new InetSocketAddress(resolvedCanonicalName, port); | ||
| if (address.isUnresolved()) { | ||
| log.warn("Couldn't resolve server {} from {} as DNS resolution of the canonical hostname failed for {}", url, CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, host); |
There was a problem hiding this comment.
Can we include resolvedCanonicalName in the warning?
| @@ -162,7 +168,9 @@ public class AdminClientConfig extends AbstractConfig { | |||
| .define(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG, | |||
There was a problem hiding this comment.
Change from CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG to CLIENT_DNS_LOOKUP_CONFIG?
| ClientDnsLookup.USE_ALL_DNS_IPS.toString(), | ||
| ClientDnsLookup.RESOLVE_CANONICAL_BOOTSTRAP_SERVERS_ONLY.toString()), | ||
| Importance.MEDIUM, | ||
| CommonClientConfigs.CLIENT_DNS_LOOKUP_DOC) |
There was a problem hiding this comment.
Change from CommonClientConfigs.CLIENT_DNS_LOOKUP_DOC to CLIENT_DNS_LOOKUP_DOC?
| new ConfigDef.NonNullValidator(), | ||
| Importance.HIGH, | ||
| CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) | ||
| .define(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG, |
There was a problem hiding this comment.
Change CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG to CLIENT_DNS_LOOKUP_CONFIG?
| import scala.collection.mutable.HashMap | ||
| import scala.collection.{Set, mutable} | ||
| import org.apache.kafka.common.config.ClientDnsLookup | ||
| import org.apache.kafka.clients.ClientDnsLookup |
There was a problem hiding this comment.
There is a wildcard import in this class for org.apache.kafka.clients, so this import is not required.
| import collection.JavaConverters._ | ||
| import scala.collection.{concurrent, immutable} | ||
| import org.apache.kafka.common.config.ClientDnsLookup | ||
| import org.apache.kafka.clients.ClientDnsLookup |
There was a problem hiding this comment.
There is a wildcard import in this class for org.apache.kafka.clients, so this import is not required.
|
|
||
| import scala.collection.JavaConverters._ | ||
| import org.apache.kafka.common.config.ClientDnsLookup | ||
| import org.apache.kafka.clients.ClientDnsLookup |
There was a problem hiding this comment.
There is a wildcard import in this class for org.apache.kafka.clients, so this import is not required.
|
|
||
| import scala.collection.JavaConverters._ | ||
| import org.apache.kafka.common.config.ClientDnsLookup | ||
| import org.apache.kafka.clients.ClientDnsLookup |
There was a problem hiding this comment.
There is a wildcard import in this class for org.apache.kafka.clients, so this import is not required.
| import org.apache.kafka.common.Cluster; | ||
| import org.apache.kafka.common.Node; | ||
| import org.apache.kafka.common.config.ClientDnsLookup; | ||
| import org.apache.kafka.clients.ClientDnsLookup; |
There was a problem hiding this comment.
Can we just move this above alongside client imports?
|
Looks like one of the build fails with unrelated tests ? |
rajinisivaram
left a comment
There was a problem hiding this comment.
@lepolac Thanks for the updates, LGTM. I will make some minor updates to reorganize imports where package changed and merge the PR.
Adds `client.dns.lookup=resolve_canonical_bootstrap_servers_only` option to perform full dns resolution of bootstrap addresses Reviewers: Colin Patrick McCabe <[email protected]>, Sriharsha Chintalapani <[email protected]>, Edoardo Comar <[email protected]>, Mickael Maison <[email protected]>, Manikumar Reddy <[email protected]>, Rajini Sivaram <[email protected]>
|
Merged to trunk and 2.1. |
|
Thanks !! |
…#4485) Adds `client.dns.lookup=resolve_canonical_bootstrap_servers_only` option to perform full dns resolution of bootstrap addresses Reviewers: Colin Patrick McCabe <[email protected]>, Sriharsha Chintalapani <[email protected]>, Edoardo Comar <[email protected]>, Mickael Maison <[email protected]>, Manikumar Reddy <[email protected]>, Rajini Sivaram <[email protected]>
Change described in KIP-235
https://cwiki.apache.org/confluence/display/KAFKA/KIP-235%3A+Add+DNS+alias+support+for+secured+connection
I license the work to the Apache Kafka project under the project's open source license.