Return-Path: X-Original-To: apmail-incubator-kafka-dev-archive@minotaur.apache.org Delivered-To: apmail-incubator-kafka-dev-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 596387A80 for ; Sun, 11 Dec 2011 18:27:47 +0000 (UTC) Received: (qmail 56588 invoked by uid 500); 11 Dec 2011 18:27:47 -0000 Delivered-To: apmail-incubator-kafka-dev-archive@incubator.apache.org Received: (qmail 56533 invoked by uid 500); 11 Dec 2011 18:27:47 -0000 Mailing-List: contact kafka-dev-help@incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: kafka-dev@incubator.apache.org Delivered-To: mailing list kafka-dev@incubator.apache.org Received: (qmail 56524 invoked by uid 99); 11 Dec 2011 18:27:47 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Sun, 11 Dec 2011 18:27:47 +0000 X-ASF-Spam-Status: No, hits=1.5 required=5.0 tests=HTML_MESSAGE,RCVD_IN_DNSWL_LOW,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (nike.apache.org: domain of neha.narkhede@gmail.com designates 209.85.220.175 as permitted sender) Received: from [209.85.220.175] (HELO mail-vx0-f175.google.com) (209.85.220.175) by apache.org (qpsmtpd/0.29) with ESMTP; Sun, 11 Dec 2011 18:27:34 +0000 Received: by vcbfo13 with SMTP id fo13so3373614vcb.6 for ; Sun, 11 Dec 2011 10:27:13 -0800 (PST) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=gamma; h=mime-version:in-reply-to:references:date:message-id:subject:from:to :content-type; bh=irhlmoWJ9PyM14aXqYAfHyHa/INqA+i0EfZltjeENXE=; b=ukQfDXAJorSkhkdtm/HkIcokXBJ+ARqsaASv+RbHaTgS6apYR9e5+3YdG0MViZOa6I DJQbgNZar542AkuOtZy5xtWes+sBe2YzAqB5XfY8feiTrkb4apyU3HtVNnrC2hSGD3J4 KLMhkQHhYOxTCGdQK48h0pgrzx8bCEfzTy4zY= MIME-Version: 1.0 Received: by 10.52.66.205 with SMTP id h13mr8603385vdt.1.1323628033524; Sun, 11 Dec 2011 10:27:13 -0800 (PST) Received: by 10.220.17.147 with HTTP; Sun, 11 Dec 2011 10:27:13 -0800 (PST) In-Reply-To: References: Date: Sun, 11 Dec 2011 10:27:13 -0800 Message-ID: Subject: Re: ZooKeeper-based Producer behavior before there are topics From: Neha Narkhede To: kafka-dev@incubator.apache.org Content-Type: multipart/alternative; boundary=20cf307d04dcc0c46d04b3d52a4f X-Virus-Checked: Checked by ClamAV on apache.org --20cf307d04dcc0c46d04b3d52a4f Content-Type: text/plain; charset=ISO-8859-1 Inder, Zookeeper is used on the producer side for load balancing. It is used for automatic detection of new and failed brokers. Thanks, Neha On Sun, Dec 11, 2011 at 2:44 AM, Inder Pall wrote: > why do u want to integrate zk at producer side. can you please elaborate > your usecase. > > is it for load balancing. > On Dec 7, 2011 12:35 AM, "Jun Rao" wrote: > > > > David, > > > > Could you also comment whether you want a programming api or a monitoring > > api? > > > > Thanks, > > > > Jun > > > > On Tue, Dec 6, 2011 at 10:03 AM, David Ormsbee > wrote: > > > > > Jun, Neha: > > > > > > Thank you for pointing me the right way on the Producer issue. :-) > > > > > > As for the metadata requests, I've filed a ticket here: > > > > > > https://issues.apache.org/jira/browse/KAFKA-223 > > > > > > Take care. > > > > > > Dave > > > > > > On Tue, Dec 6, 2011 at 12:05 PM, Neha Narkhede < > neha.narkhede@gmail.com > > > >wrote: > > > > > > > Hi Dave, > > > > > > > > >> Or should I just make the initial assumption > > > > that all brokers have one partition, send messages to those to create > the > > > > topic on the brokers, and then readjust when the the brokers publish > > > their > > > > partitions-per-topic information to ZK after the initial writes? > > > > > > > > That's exactly what we do. You can take a look at > > > > bootstrapWithExistingBrokers() in > kafka.producer.ZKBrokerPartitionInfo. > > > > > > > > >> As an aside, is there any way to query metadata about the broker > > > > directly? > > > > So that a simple consumer might be able to ask a broker for what > topics > > > > they have, how many partitions, what version it's running, etc? > > > > > > > > Today, we don't have such APIs. But I was thinking we can have > something > > > > similar to the 4-letter commands in zookeeper. > > > > For example, > > > > > > > > echo topics | nc broker-host broker-port (Returns all topics > with > > > > number of partitions on each broker) > > > > echo srvr | nc broker-host broker-port (Returns basic broker > > > stats > > > > like version, number of topics etc) > > > > > > > > If the above functionality is useful, lets file a JIRA for it. > > > > > > > > Thanks > > > > Neha > > > > > > > > On Tue, Dec 6, 2011 at 6:34 AM, David Ormsbee > > > wrote: > > > > > > > > > Hi folks, > > > > > > > > > > I'm implementing ZooKeeper support in our Python client, and I was > > > > > wondering what the expected behavior of the Producer is when the > > > brokers > > > > > are completely empty. At this point, the brokers have registered > > > > themselves > > > > > in /brokers/ids/[0..N], but there are no topics registered under > > > > > /brokers/topics. If someone wants to create a Producer at this > point > > > and > > > > > start sending messages to a new topic, is that an error because no > > > > brokers > > > > > are registered for that topic? Or should I just make the initial > > > > assumption > > > > > that all brokers have one partition, send messages to those to > create > > > the > > > > > topic on the brokers, and then readjust when the the brokers > publish > > > > their > > > > > partitions-per-topic information to ZK after the initial writes? > > > > > > > > > > I'm sure I'm missing something obvious here, since we'd run into a > > > > similar > > > > > problem for new, empty brokers joining the pool. > > > > > > > > > > As an aside, is there any way to query metadata about the broker > > > > directly? > > > > > So that a simple consumer might be able to ask a broker for what > topics > > > > > they have, how many partitions, what version it's running, etc? Or > is > > > > that > > > > > only ever published through ZooKeeper? > > > > > > > > > > Thank you. > > > > > > > > > > Dave > > > > > > > > > > > > > --20cf307d04dcc0c46d04b3d52a4f--