KAFKA-9313: Set use_all_dns_ips as the new default for client.dns.lookup (KIP-602)#8644
Conversation
| } else { | ||
| // ClientDnsLookup.USE_ALL_DNS_IPS == clientDnsLookup || ClientDnsLookup.DEFAULT == clientDnsLookup | ||
| return filterPreferredAddresses(addresses); | ||
| } |
There was a problem hiding this comment.
@badaiaqrandista We should replace this if/else with a switch statement so that we are forced to handle every case. That makes it easier to avoid issues if we add new elements to the enum. Also, can we please add tests?
There was a problem hiding this comment.
Btw, it seems that we are changing the behavior of RESOLVE_CANONICAL_BOOTSTRAP_SERVERS_ONLY as well, right? Before, we would only use the first DNS IP for that option in the non bootstrap path. It would be good to make that clear in the KIP.
There was a problem hiding this comment.
Yes, you are correct. If client.dns.lookup=RESOLVE_CANONICAL_BOOTSTRAP_SERVERS_ONLY , then it will resolves the hostname to IP address using the default behaviour.
There was a problem hiding this comment.
I've changed this into switch/case and updated the KIP.
There was a problem hiding this comment.
Can we remove the default since we handled all the cases?
There was a problem hiding this comment.
ok. will remove the default.
| USE_ALL_DNS_IPS("use_all_dns_ips"), | ||
| RESOLVE_CANONICAL_BOOTSTRAP_SERVERS_ONLY("resolve_canonical_bootstrap_servers_only"); | ||
| RESOLVE_CANONICAL_BOOTSTRAP_SERVERS_ONLY("resolve_canonical_bootstrap_servers_only"), | ||
| USE_FIRST_DNS_IP("use_first_dns_ip"); |
There was a problem hiding this comment.
We said this would be deprecated for removal in 3.0. Let's mention that in the documentation and add a warning if this is selected.
There was a problem hiding this comment.
Thinking about it, maybe we should not add this at all. We could deprecate default and switch the config default to use_all_dns_ips. Thoughts @badaiaqrandista and @rajinisivaram?
There was a problem hiding this comment.
I think that is a better idea, instead of introducing new value that would be removed soon.
ClientDnsLookup.DEFAULT is used in few places in core (server):
https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/KafkaServer.scala#L520
https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/ReplicaFetcherBlockingSend.scala#L92
https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/controller/ControllerChannelManager.scala#L156
https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala#L82
And a couple of tools:
https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala#L482
https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/admin/BrokerApiVersionsCommand.scala#L299
It is also used literally in a lot of test cases under clients.
I did not want to touch too many code in the first go. Should I change them all in this KIP or leave them until we remove ClientDnsLookup.DEFAULT in 3.0 ?
There was a problem hiding this comment.
@badaiaqrandista I think you should change it in the non test code in this PR. For the test code, we can do it in a separate PR.
There was a problem hiding this comment.
Ok. I have changed core code to use ClientDnsLookup.USE_ALL_DNS_IPS and leaving all clients tests except ClientUtilsTest.
….USE_ALL_DNS_IPS in all clients tests
….USE_ALL_DNS_IPS in all clients tests
…nsLookup.USE_ALL_DNS_IPS in all clients tests" This reverts commit 9f07b9d.
…hub.com/badaiaqrandista/kafka into KIP-602_make-use_all_dns_ips-as-default
…nsLookup.USE_ALL_DNS_IPS in all clients tests" This reverts commit 9f07b9d.
|
We should also add a note to upgrade.html. |
ijuma
left a comment
There was a problem hiding this comment.
Thanks for the updates. I think we're close. A few more comments below and please also add a note to upgrade.html since we are adjusting the default behavior slightly here.
| return Collections.singletonList(addresses[0]); | ||
| switch (clientDnsLookup) { | ||
| case DEFAULT: | ||
| log.warn("Configuration 'client.dns.lookup' value 'default' is deprecated and will be removed in the future version."); |
There was a problem hiding this comment.
This will warn on every resolve. We should probably add the warning when we get the config value from ConsumerConfig, ProducerConfig and AdminClientConfig.
There was a problem hiding this comment.
ok. will move the warn there.
There was a problem hiding this comment.
I'm adding the check and the warning to the constructor of KafkaProducer, KafkaConsumer and KafkaAdminClient because the log context does not exist in the Config classes.
There was a problem hiding this comment.
Nevermind. I've added the warning to Config classes.
| } else { | ||
| // ClientDnsLookup.USE_ALL_DNS_IPS == clientDnsLookup || ClientDnsLookup.DEFAULT == clientDnsLookup | ||
| return filterPreferredAddresses(addresses); | ||
| } |
There was a problem hiding this comment.
Can we remove the default since we handled all the cases?
| + " If set to <code>use_all_dns_ips</code>, attempt to connect to all IP addresses returned by the lookup and use the first one that connects successfully." | ||
| + " If set to <code>default</code>, connect to the first IP address returned by the lookup, even if the lookup returns multiple IP addresses." | ||
| + " If set to <code>resolve_canonical_bootstrap_servers_only</code>, each entry will be resolved and expanded into a list of canonical names." | ||
| + " Note that <code>default</code> is deprecated and will be removed in future release."; |
There was a problem hiding this comment.
Can we clarify that resolve_canonical_bootstrap_servers_only applies to the boostrap urls only. For advertised servers, both use_all_dns_ips and resolve_canonical_bootstrap_servers behave the same.
There was a problem hiding this comment.
Sorry, I do not understand why you are referring to "advertised servers". Instead, I have changed the explanation to clarify that this only applies "if bootstrap hostname is an alias to multiple canonical names".
There was a problem hiding this comment.
Each of these configs have an impact on bootstrap and advertized servers. So, we should be clear on what they do for each case.
There was a problem hiding this comment.
Config doc updated.
| @Test | ||
| public void testResolveDnsLookup() throws UnknownHostException { | ||
| assertEquals(1, ClientUtils.resolve("localhost", ClientDnsLookup.DEFAULT).size()); | ||
| assertEquals(1, ClientUtils.resolve("kafka.apache.org", ClientDnsLookup.DEFAULT).size()); |
There was a problem hiding this comment.
because kafka.apache.org resolves to 2 IP addresses. and I want to be sure that DEFAULT only returns 1 of them.
localhost resolves to 1 IP address.
| return Collections.singletonList(addresses[0]); | ||
| case USE_ALL_DNS_IPS: | ||
| case RESOLVE_CANONICAL_BOOTSTRAP_SERVERS_ONLY: | ||
| return filterPreferredAddresses(addresses); |
There was a problem hiding this comment.
Can we add a test that passes RESOLVE_CANONICAL_BOOTSTRAP_SERVERS_ONLY showing that we now get multiple addresses instead of only the first one. That's one of the changes in this PR.
|
Can you please rebase against trunk. |
|
@ijuma Merged against trunk, fixed conflict in ConsumerConfig, and updated upgrade.html. |
…use_all_dns_ips-as-default * apache-github/trunk: KAFKA-9788; Use distinct names for transaction and group load time sensors (apache#8784) KAFKA-9514; The protocol generator generated useless condition when a field is made nullable and flexible version is used (apache#8793) MINOR: Update to Gradle 6.5 and tweak build jvm config (apache#8751) MINOR: Upgrade spotbugs and spotbugsPlugin (apache#8790) KAFKA-10089 The stale ssl engine factory is not closed after reconfigure (apache#8792) KAFKA-10080; Fix race condition on txn completion which can cause duplicate appends (apache#8782) KAFKA-10084: Fix EosTestDriver end offset (apache#8785) KAFKA-10083: fix failed testReassignmentWithRandomSubscriptionsAndChanges tests (apache#8786)
use_all_dns_ips as the new default for client.dns.lookup (KIP-602)
ijuma
left a comment
There was a problem hiding this comment.
@badaiaqrandista I pushed some changes. This PR looks good to me now. Can you review, please?
|
@ijuma LGTM. Much clearer. Thanks. |
|
retest this please |
| if (clientDnsLookupValue.equals(ClientDnsLookup.DEFAULT.toString())) | ||
| log.warn("Configuration '{}' with value '{}' is deprecated and will be removed in " + | ||
| "future version. Please use '{}' or another non-deprecated value.", | ||
| CLIENT_DNS_LOOKUP_CONFIG, ClientDnsLookup.DEFAULT.toString(), |
There was a problem hiding this comment.
We don't need to call toString()
…lookup` (KIP-602) (#8644) This applies to the producer, consumer, admin client, connect worker and inter broker communication. `ClientDnsLookup.DEFAULT` has been deprecated and a warning will be logged if it's explicitly set in a client config. Reviewers: Mickael Maison <[email protected]>, Ismael Juma <[email protected]>
|
Thanks for the contribution! Merged to trunk and 2.6 branches. |
* 'trunk' of github.com:apache/kafka: (46 commits) MINOR: improve code encapsulation between StreamThread and TaskManager (apache#8819) Fixing KAFKA-10094 (apache#8797) KAFKA-9851: Revoking Connect tasks due to connectivity issues should also clear the running assignment (apache#8804) KAFKA-9840; Skip End Offset validation when the leader epoch is not reliable (apache#8486) HOT_FIX: Update javadoc since imports added (apache#8817) KAFKA-8011: Fix flaky RegexSourceIntegrationTest (apache#8799) KAFKA-9570: Define SSL configs in all worker config classes, not just distributed (apache#8135) KAFKA-10111: Make SinkTaskContext.errantRecordReporter() a default method (apache#8814) KAFKA-10110: Corrected potential NPE when null label value added to KafkaMetricsContext (apache#8811) MINOR: Change the order that Connect calls `config()` and `validate()` to avoid validating if the required ConfigDef is null (apache#8810) MINOR: fix backwards incompatibility in JmxReporter introduced by KIP-606 MINOR: Fix javadoc warnings (apache#8809) KAFKA-9441: Improve Kafka Streams task management (apache#8776) fix the broken links of streams javadoc (apache#8789) KAFKA-10040; Make computing the PreferredReplicaImbalanceCount metric more efficient (apache#8724) KAFKA-10066: TestOutputTopic should pass record headers into deserializers (apache#8759) MINOR: Add explanation for disabling forwarding from value transformers (apache#8771) KAFKA-10033: Throw UnknownTopicOrPartitionException if altering configs of non-existing topic KAFKA-9434: automated protocol for alterReplicaLogDirs (apache#8311) KAFKA-9313: Set `use_all_dns_ips` as the new default for `client.dns.lookup` (KIP-602) (apache#8644) ...
…lookup` (KIP-602) (apache#8644) This applies to the producer, consumer, admin client, connect worker and inter broker communication. `ClientDnsLookup.DEFAULT` has been deprecated and a warning will be logged if it's explicitly set in a client config. Reviewers: Mickael Maison <[email protected]>, Ismael Juma <[email protected]>
* KAFKA-6863 Kafka clients should try to use multiple DNS resolved IP (apache#4987) Implementation of KIP-302: Based on the new client configuration `client.dns.lookup`, a NetworkClient can use InetAddress.getAllByName to find all IPs and iterate over them when they fail to connect. Only uses either IPv4 or IPv6 addresses similar to the default mode. Co-authored-by: Edoardo Comar <[email protected]> Co-authored-by: Mickael Maison <[email protected]> Reviewers: Rajini Sivaram <[email protected]> * [LI-HOTFIX] Ignore the failed test ClusterConnectionStatesTest#testMultipleIPsWithUseAll (#116) TICKET = N/A LI_DESCRIPTION = The test fails since the domain kafka.apache.org used to return 3 IPs and is now only returning two IPs. Furthermore, the upstream fix identified below cannot be cleanly cherry picked. EXIT_CRITERIA = when the commit 131d475 is picked from upstream: KAFKA-12193: Re-resolve IPs after a client disconnects apache#9902 * Ignoring the failed tests (#188) [LI-HOTFIX] Ignoring the failed tests (#188) TICKET = N/A LI_DESCRIPTION = Several tests are failing since the domain kafka.apache.org that used to resolve to more than 1 IPv4 addresses are not only resolving to 1 IPv4 address. The upstream code has overhauled the ClusterConnectionStatesTest. We are simply ignoring these tests for now, and will get the new logic from upstream after a major version rebase. EXIT_CRITERIA = This hotfix can be removed in the next major version rebase * Fix for KAFKA-7974: Avoid zombie AdminClient when node host isn't resolvable (apache#6305) * Fix for KAFKA-7974: Avoid calling disconnect() when not connecting * Resolve host only when currentAddress() is called Moves away from automatically resolving the host when the connection entry is constructed, which can leave ClusterConnectionStates in a confused state. Instead, resolution is done on demand, ensuring that the entry in the connection list is present even if the resolution failed. * Add Javadoc to ClusterConnectionStates.connecting() * KAFKA-9313: Set `use_all_dns_ips` as the new default for `client.dns.lookup` (KIP-602) (apache#8644) This applies to the producer, consumer, admin client, connect worker and inter broker communication. `ClientDnsLookup.DEFAULT` has been deprecated and a warning will be logged if it's explicitly set in a client config. Reviewers: Mickael Maison <[email protected]>, Ismael Juma <[email protected]> * Update NetworkClient usage in SSLNetworkClient * [LI-HOTFIX] Bypass cluster metadata auto refresh code path by default (#329) The PR #228 attempted to resolve provided boostrap servers when the metadata is exceeding a staleness threshold. The config is coverred both on producer and consumer, and default behavior without configured value is setting timeout to Long.MAX_VALUE. However, cruise-control is affected by the behavior as it implements a similar mechanism on its own and directly uses of NetworkClient. The code would fail if empty bootstrap server is passed to NetworkClient, which is the case for internal use of CC. To resolve this, this patch aims to make default value as -1, and omit the code path referencing bootstrap server when we see -1. EXIT_CRITERIA = When #228 is ejected Co-authored-by: Xiongqi Wu <[email protected]> Co-authored-by: Edoardo Comar <[email protected]> Co-authored-by: Lucas Wang <[email protected]> Co-authored-by: Nicholas Parker <[email protected]> Co-authored-by: Badai Aqrandista <[email protected]> Co-authored-by: Joseph (Ting-Chou) Lin <[email protected]>
This applies to the producer, consumer, admin client, connect worker
and inter broker communication.
ClientDnsLookup.DEFAULThas been deprecated and a warningwill be logged if it's explicitly set in a client config.
Committer Checklist (excluded from commit message)