Hi Jonathan thanks for your comments > This kind of sounds related to the environment you're running to me. > What is the rationale behind using the advertised listeners to do your load > balancing advertisement rather than a top level alias that has everything ? > > It sounds like in your case there is a mismatch between bootstrap.servers and > advertised.listeners, and you want advertised.listeners to take precedence > and have the client iterate over what is returned by the broker. > So the extra parameter doesn't only have to do with DNS but it's also > appending from the broker, maybe the parameter name should reflect this ?
There is no mismatch for us. Our KIP-302 and associated PR allow the network client - both at bootstrap and when using the cluster's metadata response - to connect to the brokers using alternative IPs in case the first one fails. This KIP changes the client behaviour only if the DNS is configured to resolve a single name with multiple valid IP addresses. KIP-235 use case seems targeted to enrich the bootstrap list with more hostnames as long as they all resolve. And I understand it to be Kerberos-motivated. I can't see a similar list expansion being applied to the hostnames returned in a metadata response. KIP-235 doesn't seem to resolve our use case, and KIP-302 doesn't seem to help with yours. I keep seeing as addressing different concerns, the only commonality is ... they have DNS in their titles :-) Edoardo > Jonathan Skrzypek > > > -----Original Message----- > From: Skrzypek, Jonathan [Tech] > Sent: 14 May 2018 14:46 > To: dev@kafka.apache.org > Subject: RE: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved > IP addresses > > Hi, > > I see you noted the similarities with KIP-235. > But KIP-235 might also solve what this KIP is trying to achieve. > > When parsing bootstrap.servers, KIP-235 has the client add all underlying > hostnames and IPs. > And this happens before hitting the NetworkClient. > > So to me the client will try every single endpoint behind any > bootstrap.servers record. > > See > https://github.com/apache/kafka/pull/4485/commits/24757eb7b06bcf8c7d7649c85232c52b5d54f0e4#diff-89ef153462e64c250a21bd324ae1a851 > which calls getAllByName like you suggested > > Jonathan Skrzypek > > > -----Original Message----- > From: Edoardo Comar [mailto:edoco...@gmail.com] > Sent: 14 May 2018 14:17 > To: dev@kafka.apache.org > Subject: [DISCUSS] KIP-302 - Enable Kafka clients to use all DNS resolved IP > addresses > > Hi all, > > We just opened a KIP to add support for the client to use all IPs returned > by DNS for the brokers > > The details are here - > > https://urldefense.proofpoint.com/v2/url?u=https-3A__cwiki.apache.org_confluence_display_KAFKA_KIP-2D302-2B-2D-2BEnable-2BKafka-2Bclients-2Bto-2Buse-2Ball-2BDNS-2Bresolved-2BIP-2Baddresses&d=DwIBaQ&c=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4&r=nNmJlu1rR_QFAPdxGlafmDu9_r6eaCbPOM0NM1EHo-E&m=EJafFl1clRyolgtcu2uCc4_cIOJnlxb1r1n-D2Dti4k&s=C-UZ6KUG7JFiPD_CnHczDOVqH9-XC5f_OFkw4BTNrI4&e= > > The JIRA and provisional PR (where the discussion lead to the creation of > this KIP) are : > > https://urldefense.proofpoint.com/v2/url?u=https-3A__issues.apache.org_jira_browse_KAFKA-2D6863&d=DwIBaQ&c=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4&r=nNmJlu1rR_QFAPdxGlafmDu9_r6eaCbPOM0NM1EHo-E&m=EJafFl1clRyolgtcu2uCc4_cIOJnlxb1r1n-D2Dti4k&s=3Puqs5iYoPsw6hARQr6gvokdFE-H5USMiNVGOUtNkJI&e= > > https://urldefense.proofpoint.com/v2/url?u=https-3A__github.com_apache_kafka_pull_4987&d=DwIBaQ&c=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4&r=nNmJlu1rR_QFAPdxGlafmDu9_r6eaCbPOM0NM1EHo-E&m=EJafFl1clRyolgtcu2uCc4_cIOJnlxb1r1n-D2Dti4k&s=Hqn5dOgQy4-MHTIJLE49O8bNomry3SoGq9OVoHU-CRA&e= > > Looking forward to the community's feedback. > It would be amazing to have it voted by May 22nd :-) :-) > > Edoardo & Mickael -- "When the people fear their government, there is tyranny; when the government fears the people, there is liberty." [Thomas Jefferson]