You know what, it's likely this is all because I'm running a bad fork of
Kafka 0.7.2 for Scala 2.10 (on the producers/consumers) since that's the
version we've standardized on.

Behavior in 2.9.2 with the official Kafka 0.7.2 release seems much more
normal -- I'm working on downgrading all our clients and I'll report back.

Sorry for the spam.


On Thu, Jun 13, 2013 at 12:31 PM, Brett Hoerner <br...@bretthoerner.com>wrote:

> As an update, this continues to affect us.
>
> First I'd like to note ways in which my issues seems different than
> KAFKA-278,
>
> * I did not add a new broker or a new topic, this topic has been in use on
> two existing brokers for months
> * The topic definitely exists on both brokers. The topic/data directory
> exists on both, and as noted above both brokers even show it in ZK
>
> That said, I went ahead and did the "work around" from the ticket, which
> in my case basically means "restart the brokers" (because the topic/data
> directory already exists).
>
> After bouncing both brokers I *did* get data to both brokers for a while.
> I'm not yet sure if this only lasts until I have to restart my *producers*
> (as I've had to update them a bit lately), but that is my current guess.
>
> When I start a producer now (both brokers up, data looks exactly like ZK
> in original post), I get output like this:
>
> 2013-06-13_15:46:11.64496 Broker Topic Path => /brokers/topics
> 2013-06-13_15:46:11.99990 15:46:11.999 [run-main] INFO
> kafka.producer.ProducerPool - Creating async producer for broker id = 1 at
> 10.10.150.16:9092
> 2013-06-13_15:46:12.00109 15:46:12.001 [run-main] INFO
> kafka.producer.ProducerPool - Creating async producer for broker id = 0 at
> 10.10.71.113:9092
> 2013-06-13_15:46:16.77956 15:46:16.779 [ProducerSendThread-1375847990]
> INFO  kafka.producer.SyncProducer - Connected to 10.10.150.16:9092 for
> producing
>
> The last line repeats as the SyncProducer does its periodic reconnect
> thing, but note that it is *always* broker 1 at 10.10.150.16:9092, even
> though it seems like broker 0 is "seen."
>
> Thanks for your help!
>
>
>
> On Tue, Jun 4, 2013 at 6:24 PM, Neha Narkhede <neha.narkh...@gmail.com>wrote:
>
>> You are probably hitting https://issues.apache.org/jira/browse/KAFKA-278.
>> Can you please try the workaround mentioned in the JIRA description?
>>
>> Thanks,
>> Neha
>>
>>
>> On Tue, Jun 4, 2013 at 4:21 PM, Brett Hoerner <br...@bretthoerner.com
>> >wrote:
>>
>> > (version 0.7.2)
>> >
>> > For some reason, my producers are only picking up the partition on 1 of
>> my
>> > 2 brokers. I've been digging through the code, and I don't see any
>> issues
>> > given the state of my ZK nodes. The producer never seems to locate a
>> > partition on Broker0, even though ZK clearly states that it has 1 (just
>> > like Broker1 does).
>> >
>> > The result of this is that the producer works, but only sends data to
>> one
>> > Broker.
>> >
>> > Has anyone seen something like this before? I'm stumped. Thanks.
>> >
>> > # zk information follows:
>> > [zk: localhost:2181(CONNECTED) 5] ls /kafka/brokers/ids
>> > [1, 0]
>> >
>> > [zk: localhost:2181(CONNECTED) 6] get /kafka/brokers/ids/0
>> > 10.10.71.113-1365733477001:10.10.71.113:9092
>> >
>> > [zk: localhost:2181(CONNECTED) 7] get /kafka/brokers/ids/1
>> > 10.10.150.16-1369236663861:10.10.150.16:9092
>> >
>> > [zk: localhost:2181(CONNECTED) 10] ls /kafka/brokers/topics/test
>> > [1, 0]
>> >
>> > # this is the most confusing one
>> > [zk: localhost:2181(CONNECTED) 11] get /kafka/brokers/topics/test/0
>> > 1
>> >
>> > [zk: localhost:2181(CONNECTED) 12] get /kafka/brokers/topics/test/1
>> > 1
>> >
>> > # kafka producer ZK information DEBUG log, as you can see it finds 0
>> > partitions on Broker0:
>> > Broker Topic Path => /brokers/topics
>> > DEBUG [2013-06-04 23:14:30,689] kafka.producer.ZKBrokerPartitionInfo:
>> > Broker ids and # of partitions on each for topic: test =
>> ArrayBuffer((0,0),
>> > (1,1))
>> > DEBUG [2013-06-04 23:14:30,690] kafka.producer.ZKBrokerPartitionInfo:
>> > Sorted list of broker ids and partition ids on each for topic: test =
>> > TreeSet(1-0)
>> > DEBUG [2013-06-04 23:14:30,819]
>> > kafka.producer.ZKBrokerPartitionInfo$BrokerTopicsListener:
>> > [BrokerTopicsListener] Creating broker topics listener to watch the
>> > following paths -
>> > /broker/topics, /broker/topics/topic, /broker/ids
>> > DEBUG [2013-06-04 23:14:30,823]
>> > kafka.producer.ZKBrokerPartitionInfo$BrokerTopicsListener:
>> > [BrokerTopicsListener] Initialized this broker topics listener with
>> initial
>> > mapping of broker id to partition id per topic with Map(test ->
>> > TreeSet(1-0))
>> > DEBUG [2013-06-04 23:14:30,904] kafka.producer.ZKBrokerPartitionInfo:
>> > Registering listener on path: /brokers/topics/test
>> >
>>
>
>

Reply via email to