Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id CEA90200B78 for ; Fri, 2 Sep 2016 08:54:22 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id CD5AB160AAB; Fri, 2 Sep 2016 06:54:22 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 21A65160AAC for ; Fri, 2 Sep 2016 08:54:21 +0200 (CEST) Received: (qmail 41875 invoked by uid 500); 2 Sep 2016 06:54:21 -0000 Mailing-List: contact dev-help@kafka.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@kafka.apache.org Delivered-To: mailing list dev@kafka.apache.org Received: (qmail 41613 invoked by uid 99); 2 Sep 2016 06:54:20 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 02 Sep 2016 06:54:20 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id 97A5C2C014E for ; Fri, 2 Sep 2016 06:54:20 +0000 (UTC) Date: Fri, 2 Sep 2016 06:54:20 +0000 (UTC) From: "Yuto Kawamura (JIRA)" To: dev@kafka.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Created] (KAFKA-4116) Specifying 0.0.0.0 in "listeners" doesn't work MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Fri, 02 Sep 2016 06:54:23 -0000 Yuto Kawamura created KAFKA-4116: ------------------------------------ Summary: 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.10.0.1, 0.9.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)