You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2016/09/02 07:18:20 UTC
[jira] [Commented] (KAFKA-4116) Specifying 0.0.0.0 in "listeners"
doesn't work
[ https://issues.apache.org/jira/browse/KAFKA-4116?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15457768#comment-15457768 ]
ASF GitHub Bot commented on KAFKA-4116:
---------------------------------------
GitHub user kawamuray opened a pull request:
https://github.com/apache/kafka/pull/1816
KAFKA-4116: Handle 0.0.0.0 as a special case when using advertised.listeners
Issue: https://issues.apache.org/jira/browse/KAFKA-4116
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/kawamuray/kafka KAFKA-4116-listeners
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/kafka/pull/1816.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #1816
----
commit 0eec0393b41e4e75d942b3431e328d5acc18ca7f
Author: Yuto Kawamura <ka...@gmail.com>
Date: 2016-09-02T07:11:04Z
KAFKA-4116: Handle 0.0.0.0 as a special case when using advertised.listeners
----
> Specifying 0.0.0.0 in "listeners" doesn't work
> ----------------------------------------------
>
> Key: KAFKA-4116
> URL: https://issues.apache.org/jira/browse/KAFKA-4116
> Project: Kafka
> Issue Type: Bug
> Components: core
> Affects Versions: 0.9.0.1, 0.10.0.1
> Reporter: Yuto Kawamura
> Assignee: Yuto Kawamura
> Fix For: 0.10.0.2
>
>
> The document of {{listeners}} says:
> "Specify hostname as 0.0.0.0 to bind to all interfaces."
> However when I give config such as below, a started kafka broker can't join the cluster due to invalid address advertised on zk.
> {code}
> listeners=PLAINTEXT://0.0.0.0:9092
> # advertised.listeners=
> {code}
> This is because of:
> - {{advertised.listeners}} which is used as an address to publish on zk defaults to {{listeners}}
> - KafkaHealthcheck#register isn't considering the host "0.0.0.0" as a special case : https://github.com/apache/kafka/blob/8f3462552fa4d6a6d70a837c2ef7439bba512657/core/src/main/scala/kafka/server/KafkaHealthcheck.scala#L60-L61
> h3. Proof
> Test environment:
> - kafka-broker version 0.10.1.0-SNAPSHOT(build from trunk)
> - Brokers HOST-A, HOST-B, HOST-C
> - Controller: HOST-A
> - topic-A has 3 replicas, 3 partitions
> Update HOST-B's server.properties with updating listeners to below and restart the broker.
> {code}
> listeners=PLAINTEXT://0.0.0.0:9092
> {code}
> Then HOST-B registeres it's broker info to ZK path {{/brokers/ids/2}}, but "0.0.0.0" is used as it's host:
> {code}
> [zk: ZKHOST1:2181,ZKHOST2:2181,ZKHOST3:2181/kafka-test(CONNECTED) 8] get /brokers/ids/2
> {"jmx_port":12345,"timestamp":"1472796372181","endpoints":["PLAINTEXT://0.0.0.0:9092"],"host":"0.0.0.0","version":3,"port":9092}
> {code}
> Controller tries to send an request to the above address but of course it will never reach to the HOST-B.
> controller.log:
> {code}
> [2016-09-02 15:06:12,206] INFO [Controller-1-to-broker-2-send-thread], Controller 1 connected to 0.0.0.0:9092 (id: 2 rack: null) for sending state change requests (kafka.controller.RequestSendThread)
> {code}
> I'm guessing maybe controller sending a request to itself(kafka broker working on the same instance), as calling connect("0.0.0.0") results to connect to localhost, which sounds scary but havn't digged into.
> So the ISR won't recovered even through a broker starts up.
> {code}
> ./kafka-topics.sh ... --describe --topic topic-A
> Topic:topic-A PartitionCount:3 ReplicationFactor:3 Configs:retention.ms=86400000,min.insync.replicas=2
> Topic: topic-A Partition: 0 Leader: 3 Replicas: 3,2,1 Isr: 1,3
> Topic: topic-A Partition: 1 Leader: 1 Replicas: 1,3,2 Isr: 1,3
> Topic: topic-A Partition: 2 Leader: 1 Replicas: 2,1,3 Isr: 1,3
> {code}
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)