Skip to content

KAFKA-9313: Set use_all_dns_ips as the new default for client.dns.lookup (KIP-602)#8644

Merged
ijuma merged 22 commits intoapache:trunkfrom
badaiaqrandista:KIP-602_make-use_all_dns_ips-as-default
Jun 4, 2020
Merged

KAFKA-9313: Set use_all_dns_ips as the new default for client.dns.lookup (KIP-602)#8644
ijuma merged 22 commits intoapache:trunkfrom
badaiaqrandista:KIP-602_make-use_all_dns_ips-as-default

Conversation

@badaiaqrandista
Copy link
Copy Markdown
Contributor

@badaiaqrandista badaiaqrandista commented May 10, 2020

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.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

Comment thread clients/src/main/java/org/apache/kafka/clients/ClientDnsLookup.java Outdated
} else {
// ClientDnsLookup.USE_ALL_DNS_IPS == clientDnsLookup || ClientDnsLookup.DEFAULT == clientDnsLookup
return filterPreferredAddresses(addresses);
}
Copy link
Copy Markdown
Member

@ijuma ijuma May 28, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I've changed this into switch/case and updated the KIP.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we remove the default since we handled all the cases?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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");
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Copy Markdown
Member

@ijuma ijuma May 28, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok. I have changed core code to use ClientDnsLookup.USE_ALL_DNS_IPS and leaving all clients tests except ClientUtilsTest.

@ijuma
Copy link
Copy Markdown
Member

ijuma commented May 30, 2020

We should also add a note to upgrade.html.

Copy link
Copy Markdown
Member

@ijuma ijuma left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.");
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This will warn on every resolve. We should probably add the warning when we get the config value from ConsumerConfig, ProducerConfig and AdminClientConfig.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok. will move the warn there.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nevermind. I've added the warning to Config classes.

} else {
// ClientDnsLookup.USE_ALL_DNS_IPS == clientDnsLookup || ClientDnsLookup.DEFAULT == clientDnsLookup
return filterPreferredAddresses(addresses);
}
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.";
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Each of these configs have an impact on bootstrap and advertized servers. So, we should be clear on what they do for each case.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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());
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do we need this change?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you add a comment about this?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added.

return Collections.singletonList(addresses[0]);
case USE_ALL_DNS_IPS:
case RESOLVE_CANONICAL_BOOTSTRAP_SERVERS_ONLY:
return filterPreferredAddresses(addresses);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added.

@ijuma
Copy link
Copy Markdown
Member

ijuma commented Jun 2, 2020

Can you please rebase against trunk.

@badaiaqrandista
Copy link
Copy Markdown
Contributor Author

@ijuma Merged against trunk, fixed conflict in ConsumerConfig, and updated upgrade.html.

@ijuma ijuma changed the title KAFKA-9313: [WIP] Make use_all_dns_ips as the default for client.dns.lookup KAFKA-9313: Make use_all_dns_ips the default for client.dns.lookup (KIP-XX) Jun 4, 2020
@ijuma ijuma changed the title KAFKA-9313: Make use_all_dns_ips the default for client.dns.lookup (KIP-XX) KAFKA-9313: Make use_all_dns_ips the default for client.dns.lookup (KIP-602) Jun 4, 2020
ijuma added 2 commits June 3, 2020 19:01
…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)
@ijuma ijuma changed the title KAFKA-9313: Make use_all_dns_ips the default for client.dns.lookup (KIP-602) KAFKA-9313: Set use_all_dns_ips as the new default for client.dns.lookup (KIP-602) Jun 4, 2020
Copy link
Copy Markdown
Member

@ijuma ijuma left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@badaiaqrandista I pushed some changes. This PR looks good to me now. Can you review, please?

@badaiaqrandista
Copy link
Copy Markdown
Contributor Author

@ijuma LGTM. Much clearer. Thanks.

@ijuma
Copy link
Copy Markdown
Member

ijuma commented Jun 4, 2020

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(),
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We don't need to call toString()

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done.

@badaiaqrandista badaiaqrandista requested a review from mimaison June 4, 2020 11:24
Copy link
Copy Markdown
Member

@mimaison mimaison left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the quick update

@ijuma ijuma merged commit 50c3012 into apache:trunk Jun 4, 2020
ijuma pushed a commit that referenced this pull request Jun 4, 2020
…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]>
@ijuma
Copy link
Copy Markdown
Member

ijuma commented Jun 4, 2020

Thanks for the contribution! Merged to trunk and 2.6 branches.

Kvicii pushed a commit to Kvicii/kafka that referenced this pull request Jun 6, 2020
* '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)
  ...
ZIDAZ pushed a commit to linkedin/kafka that referenced this pull request Jun 14, 2022
…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]>
ZIDAZ pushed a commit to linkedin/kafka that referenced this pull request Jun 14, 2022
* 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]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants