Skip to content

KAFKA-6863 Kafka clients should try to use multiple DNS resolved IP#4987

Merged
rajinisivaram merged 2 commits intoapache:trunkfrom
edoardocomar:KAFKA-6863
Oct 11, 2018
Merged

KAFKA-6863 Kafka clients should try to use multiple DNS resolved IP#4987
rajinisivaram merged 2 commits intoapache:trunkfrom
edoardocomar:KAFKA-6863

Conversation

@edoardocomar
Copy link
Copy Markdown
Contributor

@edoardocomar edoardocomar commented May 9, 2018

  • Implementation of KIP-302
  • Based on the new client configuration client.dns.lookup a
    Selector can use InetAddress.getAllByName to find all IPs and iterates
    over them when they fail to connect
  • Only use IPv4 or IPv6 addresses
  • Unit test

Co-authored-by: Edoardo Comar [email protected]
Co-authored-by: Mickael Maison [email protected]

On attempting to connect to a resolved IP that does not respond, there appears to be a 75 second timeout - this is reflected in the unit test.
If the resolved IP refuses the connection the selector would move the next IP quickly

Note that we also fixed SelectorTst.testConnectionRefused()

Committer Checklist (excluded from commit message)

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

@edoardocomar
Copy link
Copy Markdown
Contributor Author

If we could control the timeout on the unblocking NIO connection the test could be made to pass quickly. Please feedback welcome

@ijuma
Copy link
Copy Markdown
Member

ijuma commented May 9, 2018

Thanks for the PR. This is not what I thought you were asking for. I thought you wanted the bootstrap brokers to resolve all the IPs, but the behaviour implemented is more than that. Can you elaborate on why we would want this? It seems like it could be a bit dangerous.

@edoardocomar
Copy link
Copy Markdown
Contributor Author

Hi @ijuma why do you think this could be dangerous ?
The client is simply using the information returned by the DNS.
If you don't trust your DNS to resolve your hostnames we enter a different scenario.

Why we think this JIRA/PR is useful :
Normally you'd expect a symbolic name to resolve to one IP address, but that is not necessarily the case.
As explained in the JIRA, our use case is for multiple load balancers fronting the Kafka cluster. The Kafka advertised listeners advertise hostnames that the DNS server maps to the IP of all LBs not just one.
If one LBs isn't available, but the client is able to use another IP for the same hostname (it connects to the 2nd LB for example) the service stays available.

As we have not introduced any new client configturation, we thought this PR can be reviewed without a KIP. If that's not the case please let us know as we'd like this functionality to make it in v2.0.0

@edoardocomar
Copy link
Copy Markdown
Contributor Author

@ijuma @rajinisivaram any comments ? As said this is an internal improvment that we'd like to get in for 2.0.0
It has no impact on users whose clusters hostnames are resolved to a single IP

@rajinisivaram
Copy link
Copy Markdown
Contributor

@edoardocomar Thanks for the PR. KAFKA-6195 addresses a similar but slightly different requirement. It also started as an internal improvement, but it would have broken some environments. The KIP that was added later (KIP-235) was useful because in the end a config was added to enable/disable auto-lookups.

Like with KIP-235, I am not sure we want this behaviour as default. Both this PR and the KIP are targeted at multiple DNS entries and use all resolved entries. One uses (canonicalHostName, IP) and the other uses (hostName, IP). In a secure environment, host name matters. We need to clarify how these two approaches co-exist and when to use each approach.

The JIRA suggests that the use case is to resolve load balancers, which sort of points towards bootstrap list. You have kafka.example.com resolving to {(kafka01.example.com, ip1), (kafka02.example.com, ip2), (kafka03.example.com, ip3)} and want to use kafka.example.com in the bootstrap list. This scenario is addressed by KIP-235. Instead of using (kafka.example.com, ip1) as the single bootstrap server, KIP-235 changes the bootstrap list to {(kafka01.example.com, ip1), (kafka02.example.com, ip2), (kafka03.example.com, ip3)} .

But this PR suggests that you have more than one proxy in front of each broker for load balancing and want to resolve not just bootstrap list, but also hosts returned as advertised host names. So you have kafka01.example.com resolving to {(kafka01.example.com, ip1a), (kafka01.example.com, ip1b)}. Is that really the problem you are trying to solve?

In any case, I think a KIP that explains the motivation, the reason why KIP-235 is not sufficient and how the two approaches will co-exist would help. Or you could piggy-back on KIP-235 since if we have to add a config, it may be possible to adapt the config proposed in KIP-235.

@edoardocomar
Copy link
Copy Markdown
Contributor Author

But this PR suggests that you have more than one proxy in front of each broker for load balancing and want to resolve not just bootstrap list, but also hosts returned as advertised host names. So you have kafka01.example.com resolving to {(kafka01.example.com, ip1a), (kafka01.example.com, ip1b)}. Is that really the problem you are trying to solve?

yes exactly

@edoardocomar
Copy link
Copy Markdown
Contributor Author

edoardocomar commented May 11, 2018

I am not sure we want this behaviour as default.

well for most cases where the the DNS will resolve a name to a single IP, nothing changes

@rajinisivaram
Copy link
Copy Markdown
Contributor

@edoardocomar In some environments, something does change - hence you added the PR :-) Do all those environments expect connections to be attempted to all resolved IPs? If not, we need a config. The default value could be what most of those environments expect as long as we don't break anyone who didn't expect this to happen (it could be to enable this change by default, just needs discussion)

As an example, I have both IPv4 and IPv6 addresses for localhost in /etc/hosts. If I run InetAddress.getAllByName("localhost"), I get two entries, one with an IPv4 address and another with an IPv6 address. I don't really want two connections to be attempted since one is not a fallback for the other.

Another reason why a KIP would be useful for discussion is that the PR currently uses two proxies in active/standby mode. So it is not balancing load, it is using a standby proxy for HA if connection to the active proxy fails. Do we want to randomize like we do with bootstrap brokers? Again these scenarios are easier to discuss in the context of a KIP.

@edoardocomar
Copy link
Copy Markdown
Contributor Author

edoardocomar commented Sep 26, 2018

Hi @rajinisivaram @ijuma we've overhauled the PR by implementing KIP-302 i.e. the conditional behavior based on the new config option defined by KIP-235

Note that we also fixed SelectorTest.testConnectionRefused()

@mimaison
Copy link
Copy Markdown
Member

@lindong28 @harshach @omkreddy @cmccabe Can one of you help with the review? Thanks

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

@edoardocomar Since this PR is using the same config as #4485, it will need to get merged on top of that right?

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 copied all common files from #4485 (at the same time addressing the points we raised there) so they can be merged in any order.

Copy link
Copy Markdown
Contributor

@rajinisivaram rajinisivaram left a comment

Choose a reason for hiding this comment

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

@mimaison I think this PR is very big because it has changed the Selectable interface. Is there a reason why iteration through resolved addresses can't be handled in the client layer rather than the network layer?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Why can we iterate through addresses in NetworkClient instead of Selector?

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 the client calls connect which then hands off to NIO the asynchronous connection
and it's in handling the keys that we can see if one IP was "bad" and all this without the client being involved at all.

It may be not the smallest change possible because of the footprint in the tests, but to us this was the change that seemed to make sense.

We are biased by the fact that the existing Selectable.connect signature wasn't great also for KIP-81

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Network layer code is already rather complex and I would avoid this type of client logic into this layer unless there was no other way. For disconnections, we already propagate channel state to clients and it would be quite easy to add another state to indicate that it may want to retry. A few advantages of doing it this way:

  1. Keep network layer changes to minimum
  2. We want to make this change only for clients, so the keeping the change in the client layer makes sense (making it easier to read the code when debugging broker issues)
  3. Reduces the footprint of this change
  4. Keeps the handling of different variations of this config in the same layer (since the other values of this config are handled in the client layer)

I haven't followed KIP-81 PR discussions, so if there are other reasons to make this change in Selector, then perhaps it may make sense. But just looking at this one, it feels like it should be the responsibility of NetworkClient to iterate through the addresses.

@hachikuji Since you reviewed the KIP-81 PR, what do you think?

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.

Many thanks @rajinisivaram for taking a look under heavy load!
We see the point of not polluting the Network layer and having retries done by the client only.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

The KIP says use.all.dns.ips and the other PR uses dot separated name.

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 but for consistency @mimaison commented on KIP-235
#4485 (comment)

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.

of course we'd update the KIP if this change was acceptable. it breaks no one yet ...

@edoardocomar
Copy link
Copy Markdown
Contributor Author

Hi @rajinisivaram we had another go, this time using the ClientDnsLookup configuration in NetworkClient and would welcome your feedback !

Please note that some of the tests added to ClientUtilsTest rely on the fact that kafka.apche.org resolves to TWO different IPs.
Convenient for us, but maybe not too reliable moving forward. We could fix that by accessing the inner class in NodeConnectionState directly in the unit test.

Copy link
Copy Markdown
Contributor

@rajinisivaram rajinisivaram left a comment

Choose a reason for hiding this comment

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

@edoardocomar @mimaison Thanks for the updates, this looks good. Left a few minor comments (just nits). Also need to rebase to resolve conflicts. I think the only issue remaining to merge this is the naming of the config value (which is currently different in PR #4485). If that is agreed today, then we can merge this to 2.1.0 and trunk.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

nit: Prefered => Preferred

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

as before preferred

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Preferred

edoardocomar and others added 2 commits October 11, 2018 14:20
* 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 use IPv4 or IPv6 addresses
* Unit test

Co-authored-by: Edoardo Comar <[email protected]>
Co-authored-by: Mickael Maison <[email protected]>
fixed spelling as per review

Co-authored-by: Edoardo Comar <[email protected]>
Co-authored-by: Mickael Maison <[email protected]>
@mimaison
Copy link
Copy Markdown
Member

@rajinisivaram Thanks for the review. We've rebased on trunk and fixed the typo!

Copy link
Copy Markdown
Contributor

@rajinisivaram rajinisivaram left a comment

Choose a reason for hiding this comment

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

@edoardocomar @mimaison Thanks for the updates, LGTM. As @lindong28 has agreed for this to go into 2.1.0, I will merge to 2.1.0 and trunk after the builds complete.

@edoardocomar
Copy link
Copy Markdown
Contributor Author

Thanks - will update the KIP

@rajinisivaram
Copy link
Copy Markdown
Contributor

Test failures look unrelated, merging to trunk and 2.1.

@rajinisivaram rajinisivaram merged commit f393b2f into apache:trunk Oct 11, 2018
rajinisivaram pushed a commit that referenced this pull request Oct 11, 2018
…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]>
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.common.config;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

@edoardocomar @mimaison org.apache.kafka.common.config is a public package and this is not a public API as-per the KIPs. I dont think we include similar enums in the config package. Since this is used only for client-side with NetworkClient, we should move ClientDnsLookup into org.apache.kafka.clients as it is in the other PR. Can you submit a follow-on PR to fix this?

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.

Sure, we'll do that now

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.

I see they just made that change in #4485.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

@mimaison thanks, it can be updated in that PR if they find it easier.

@edoardocomar edoardocomar deleted the KAFKA-6863 branch October 12, 2018 22:25
pengxiaolong pushed a commit to pengxiaolong/kafka that referenced this pull request Jun 14, 2019
…pache#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]>
ZIDAZ pushed a commit to linkedin/kafka that referenced this pull request Jun 14, 2022
…pache#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]>
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.

4 participants