Re: [VOTE RESULT] Apache Kafka 0.8.1.1 Release Candidate 1

2014-04-29 Thread Michael G. Noll

Many thanks for this release, everyone!

--Michael



On 2014-04-26 14:57, Joe Stein wrote:
Including my +1 the vote passes with 4 x +1 binding and 2 x +1 
non-binding

(0 x -1 and 0x 0 votes).

I will push the maven artifacts to maven central and upload the 
binaries

for the dist mirrors.

Once all that shows up I will update the download page and send the
announcement (probably by monday everything should be mirrored).

Thanks to everyone that got into making this release happen.

/***
 Joe Stein
 Founder, Principal Consultant
 Big Data Open Source Security LLC
 http://www.stealth.ly
 Twitter: @allthingshadoop 
/


[jira] [Commented] (KAFKA-1029) Zookeeper leader election stuck in ephemeral node retry loop

2014-04-29 Thread Michael Noll (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1029?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984265#comment-13984265
 ] 

Michael Noll commented on KAFKA-1029:
-

Interestingly I ran into the very same issue while doing basic validation of 
Kafka 0.8.1.1.

Here is an example log message, which was repeated indefinitely:

{code}
[2014-04-29 09:48:27,207] INFO conflict in /controller data: 
{"version":1,"brokerid":0,"timestamp":"1398764901156"} stored data: 
{"version":1,"brokerid":0,"timestamp":"1398764894941"} (kafka.utils.ZkUtils$)
[2014-04-29 09:48:27,218] INFO I wrote this conflicted ephemeral node 
[{"version":1,"brokerid":0,"timestamp":"1398764901156"}] at /controller a while 
back in a different session, hence I will backoff for this node to be deleted 
by Zookeeper and retry (kafka.utils.ZkUtils$)
{code}

*How to reproduce (unsolved)*

Unfortunately I cannot consistently reproduce the issue and, to be honest, I am 
still not sure what actually triggers the bug.  As such I can only summarize 
what I did before and around the time when this bug was triggered, which I 
could observe through errors in log files and through errors being displayed 
after running certain commands.  So yes, it's a bit like shooting in the dark.

Here's an overview of my test setup:

- I deployed Kafka 0.8.1.1 to one machine {{kafka1}} and ZooKeeper 3.4.5 to a 
second machine {{zookeeper1}}.
- I followed the Kafka 0.8.1 quick start guide to create a topic "test" with 1 
partition and a replication factor of 1.
- I sent test messages to the topic "test" via the console producer.
- I read test messages from the topic "test" via the console consumer.
- Apart from producing and consuming a handful of test messages I also ran some 
supposedly read-only admin commands such as "describing" the topic, and running 
the consumer offset checker tool.
- At "some" point, Kafka was caught in an indefinite loop complaining about 
"conflicted ephemeral node".


The following paragraphs list in more detail what I did before the error popped 
up.

Producer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# This command returned no topics at this point = worked as expected
$ bin/kafka-topics.sh --list --zookeeper zookeeper1

# I created a topic, worked as expected
$ bin/kafka-topics.sh --create --zookeeper zookeeper1 --replication-factor 1 
--partitions 1 --topic test

# I requested details of topic "test", worked as expected
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic test

# I started a console producer and manually send a handful of test messages, 
worked as expected (see consumer below)
$ bin/kafka-console-producer.sh --broker-list localhost:9092 --topic test
{code}

Consumer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# I started a console consumer, worked as expected (i.e. could read messages 
sent by producer, see above)
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning
{code}

Up to that point, everything worked.  But then the Kafka broker went the way of 
the dodo.  As I said I can't pinpoint the cause, and re-running the same 
commands on a fresh Kafka/ZooKeeper deployment (fresh VMs etc.) didn't 
consistently trigger the issue like I hoped.

Here's what I did after the commands above, and at some point I eventually did 
observe the original error described in this JIRA ticket.  Again, at the moment 
I cannot tell what actually triggered the bug.

Producer:

{code}
# Test-driving the consumer offset checker
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1
# At this point consumer "foo" was not expected to exist.
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1 --broker-info --group foo

#
# I then re-started the console producer (see below), now configured to use the 
group id "foo".
#
{code}

Consumer:

{code}
# I re-started the console consumer, now configured to use the group id "foo".
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning --group foo
{code}

At this point "describing" the topic gave the following info, indicating that 
there was a problem (e.g. no leader for partition):

{code}
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic testTopic:test  
PartitionCount:1ReplicationFactor:1 Configs:
Topic: test Partition: 0Leader: -1  Replicas: 0 Isr:
{code}

Log files such as {{state-change.log}} showed these error messages:

{code}
[2014-04-29 07:44:47,816] TRACE Broker 0 cached leader info 
(LeaderAndIsrInfo:(Leader:0,ISR:0,LeaderEpoch:0,ControllerEp
och:1),ReplicationFactor:1),AllReplicas:0) for partition [test,0] in response 
to UpdateMetadata request sent by controll
er 0 epoch 1 with correlation id 7 (state.change.logger)
[2014-04-29 07:44:47,818] TRACE Controller 0 epoch 1 received response 
correlationId 7 for a request sent to brok

[jira] [Comment Edited] (KAFKA-1029) Zookeeper leader election stuck in ephemeral node retry loop

2014-04-29 Thread Michael Noll (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1029?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984265#comment-13984265
 ] 

Michael Noll edited comment on KAFKA-1029 at 4/29/14 12:59 PM:
---

Interestingly I ran into the very same issue while doing basic validation of 
Kafka 0.8.1.1.  Note that unlike Jason Rosenberg's case I was only using 
code/tools/scripts that are shipped with Kafka (e.g. console producer and 
console consumer).

Here is an example log message, which was repeated indefinitely:

{code}
[2014-04-29 09:48:27,207] INFO conflict in /controller data: 
{"version":1,"brokerid":0,"timestamp":"1398764901156"} stored data: 
{"version":1,"brokerid":0,"timestamp":"1398764894941"} (kafka.utils.ZkUtils$)
[2014-04-29 09:48:27,218] INFO I wrote this conflicted ephemeral node 
[{"version":1,"brokerid":0,"timestamp":"1398764901156"}] at /controller a while 
back in a different session, hence I will backoff for this node to be deleted 
by Zookeeper and retry (kafka.utils.ZkUtils$)
{code}

*How to reproduce (unsolved)*

Unfortunately I cannot consistently reproduce the issue and, to be honest, I am 
still not sure what actually triggers the bug.  As such I can only summarize 
what I did before and around the time when this bug was triggered, which I 
could observe through errors in log files and through errors being displayed 
after running certain commands.  So yes, it's a bit like shooting in the dark.

Here's an overview of my test setup:

- I deployed Kafka 0.8.1.1 to one machine {{kafka1}} and ZooKeeper 3.4.5 to a 
second machine {{zookeeper1}}.
- I followed the Kafka 0.8.1 quick start guide to create a topic "test" with 1 
partition and a replication factor of 1.
- I sent test messages to the topic "test" via the console producer.
- I read test messages from the topic "test" via the console consumer.
- Apart from producing and consuming a handful of test messages I also ran some 
supposedly read-only admin commands such as "describing" the topic, and running 
the consumer offset checker tool.
- At "some" point, Kafka was caught in an indefinite loop complaining about 
"conflicted ephemeral node".


The following paragraphs list in more detail what I did before the error popped 
up.

Producer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# This command returned no topics at this point = worked as expected
$ bin/kafka-topics.sh --list --zookeeper zookeeper1

# I created a topic, worked as expected
$ bin/kafka-topics.sh --create --zookeeper zookeeper1 --replication-factor 1 
--partitions 1 --topic test

# I requested details of topic "test", worked as expected
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic test

# I started a console producer and manually send a handful of test messages, 
worked as expected (see consumer below)
$ bin/kafka-console-producer.sh --broker-list localhost:9092 --topic test
{code}

Consumer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# I started a console consumer, worked as expected (i.e. could read messages 
sent by producer, see above)
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning
{code}

Up to that point, everything worked.  But then the Kafka broker went the way of 
the dodo.  As I said I can't pinpoint the cause, and re-running the same 
commands on a fresh Kafka/ZooKeeper deployment (fresh VMs etc.) didn't 
consistently trigger the issue like I hoped.

Here's what I did after the commands above, and at some point I eventually did 
observe the original error described in this JIRA ticket.  Again, at the moment 
I cannot tell what actually triggered the bug.

Producer:

{code}
# Test-driving the consumer offset checker
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1
# At this point consumer "foo" was not expected to exist.
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1 --broker-info --group foo

#
# I then re-started the console producer (see below), now configured to use the 
group id "foo".
#
{code}

Consumer:

{code}
# I re-started the console consumer, now configured to use the group id "foo".
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning --group foo
{code}

At this point "describing" the topic gave the following info, indicating that 
there was a problem (e.g. no leader for partition):

{code}
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic testTopic:test  
PartitionCount:1ReplicationFactor:1 Configs:
Topic: test Partition: 0Leader: -1  Replicas: 0 Isr:
{code}

Log files such as {{state-change.log}} showed these error messages:

{code}
[2014-04-29 07:44:47,816] TRACE Broker 0 cached leader info 
(LeaderAndIsrInfo:(Leader:0,ISR:0,LeaderEpoch:0,ControllerEp
och:1),ReplicationFactor:1),AllReplicas:0) for partition [test,0] in response 
to UpdateM

[jira] [Comment Edited] (KAFKA-1029) Zookeeper leader election stuck in ephemeral node retry loop

2014-04-29 Thread Michael Noll (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1029?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984265#comment-13984265
 ] 

Michael Noll edited comment on KAFKA-1029 at 4/29/14 1:02 PM:
--

Interestingly I ran into the very same issue while doing basic validation of 
Kafka 0.8.1.1.  Note that unlike Jason Rosenberg's case I was only using 
code/tools/scripts that are shipped with Kafka (e.g. console producer and 
console consumer).

Here is an example log message, which was repeated indefinitely:

{code}
[2014-04-29 09:48:27,207] INFO conflict in /controller data: 
{"version":1,"brokerid":0,"timestamp":"1398764901156"} stored data: 
{"version":1,"brokerid":0,"timestamp":"1398764894941"} (kafka.utils.ZkUtils$)
[2014-04-29 09:48:27,218] INFO I wrote this conflicted ephemeral node 
[{"version":1,"brokerid":0,"timestamp":"1398764901156"}] at /controller a while 
back in a different session, hence I will backoff for this node to be deleted 
by Zookeeper and retry (kafka.utils.ZkUtils$)
{code}

*How to reproduce (unsolved)*

Unfortunately I cannot consistently reproduce the issue and, to be honest, I am 
still not sure what actually triggers the bug.  As such I can only summarize 
what I did before and around the time when this bug was triggered, which I 
could observe through errors in log files and through errors being displayed 
after running certain commands.  So yes, it's a bit like shooting in the dark.

Here's an overview of my test setup:

- I deployed Kafka 0.8.1.1 to one machine {{kafka1}} and ZooKeeper 3.4.5 to a 
second machine {{zookeeper1}}.
- I followed the Kafka 0.8.1 quick start guide to create a topic "test" with 1 
partition and a replication factor of 1.
- I sent test messages to the topic "test" via the console producer.
- I read test messages from the topic "test" via the console consumer.
- Apart from producing and consuming a handful of test messages I also ran some 
supposedly read-only admin commands such as "describing" the topic, and running 
the consumer offset checker tool.
- At "some" point, Kafka was caught in an indefinite loop complaining about 
"conflicted ephemeral node".


The following paragraphs list in more detail what I did before the error popped 
up.

Producer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# This command returned no topics at this point = worked as expected
$ bin/kafka-topics.sh --list --zookeeper zookeeper1

# I created a topic, worked as expected
$ bin/kafka-topics.sh --create --zookeeper zookeeper1 --replication-factor 1 
--partitions 1 --topic test

# I requested details of topic "test", worked as expected
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic test

# I started a console producer and manually send a handful of test messages, 
worked as expected (see consumer below)
$ bin/kafka-console-producer.sh --broker-list localhost:9092 --topic test
{code}

Consumer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# I started a console consumer, worked as expected (i.e. could read messages 
sent by producer, see above)
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning
{code}

Up to that point, everything worked.  But then the Kafka broker went the way of 
the dodo.  As I said I can't pinpoint the cause, and re-running the same 
commands on a fresh Kafka/ZooKeeper deployment (fresh VMs etc.) didn't 
consistently trigger the issue like I hoped.

Here's what I did after the commands above, and at some point I eventually did 
observe the original error described in this JIRA ticket.  Again, at the moment 
I cannot tell what actually triggered the bug.

Producer:

{code}
# Test-driving the consumer offset checker
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1
# At this point consumer "foo" was not expected to exist.
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1 --broker-info --group foo

#
# I then re-started the console producer (see below), now configured to use the 
group id "foo".
#
{code}

Consumer:

{code}
# I re-started the console consumer, now configured to use the group id "foo".
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning --group foo
{code}

At this point "describing" the topic gave the following info, indicating that 
there was a problem (e.g. no leader for partition):

{code}
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic testTopic:test  
PartitionCount:1ReplicationFactor:1 Configs:
Topic: test Partition: 0Leader: -1  Replicas: 0 Isr:
{code}

Log files such as {{state-change.log}} showed these error messages:

{code}
[2014-04-29 07:44:47,816] TRACE Broker 0 cached leader info 
(LeaderAndIsrInfo:(Leader:0,ISR:0,LeaderEpoch:0,ControllerEp
och:1),ReplicationFactor:1),AllReplicas:0) for partition [test,0] in response 
to UpdateMet

[jira] [Comment Edited] (KAFKA-1029) Zookeeper leader election stuck in ephemeral node retry loop

2014-04-29 Thread Michael Noll (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1029?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984265#comment-13984265
 ] 

Michael Noll edited comment on KAFKA-1029 at 4/29/14 1:12 PM:
--

Interestingly I ran into the very same issue while doing basic validation of 
Kafka 0.8.1.1.  Note that unlike Jason Rosenberg's case I was only using 
code/tools/scripts that are shipped with Kafka (e.g. console producer and 
console consumer).

Here is an example log message, which was repeated indefinitely:

{code}
[2014-04-29 09:48:27,207] INFO conflict in /controller data: 
{"version":1,"brokerid":0,"timestamp":"1398764901156"} stored data: 
{"version":1,"brokerid":0,"timestamp":"1398764894941"} (kafka.utils.ZkUtils$)
[2014-04-29 09:48:27,218] INFO I wrote this conflicted ephemeral node 
[{"version":1,"brokerid":0,"timestamp":"1398764901156"}] at /controller a while 
back in a different session, hence I will backoff for this node to be deleted 
by Zookeeper and retry (kafka.utils.ZkUtils$)
{code}

*How to reproduce (unsolved)*

Unfortunately I cannot consistently reproduce the issue and, to be honest, I am 
still not sure what actually triggers the bug.  As such I can only summarize 
what I did before and around the time when this bug was triggered, which I 
could observe through errors in log files and through errors being displayed 
after running certain commands.  So yes, it's a bit like shooting in the dark.

Here's an overview of my test setup:

- I deployed Kafka 0.8.1.1 to one machine {{kafka1}} and ZooKeeper 3.4.5 to a 
second machine {{zookeeper1}}.
- I followed the Kafka 0.8.1 quick start guide to create a topic "test" with 1 
partition and a replication factor of 1.
- I sent test messages to the topic "test" via the console producer.
- I read test messages from the topic "test" via the console consumer.
- Apart from producing and consuming a handful of test messages I also ran some 
supposedly read-only admin commands such as "describing" the topic, and running 
the consumer offset checker tool.
- At "some" point, Kafka was caught in an indefinite loop complaining about 
"conflicted ephemeral node".


The following paragraphs list in more detail what I did before the error popped 
up.

Producer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# This command returned no topics at this point = worked as expected
$ bin/kafka-topics.sh --list --zookeeper zookeeper1

# I created a topic, worked as expected
$ bin/kafka-topics.sh --create --zookeeper zookeeper1 --replication-factor 1 
--partitions 1 --topic test

# I requested details of topic "test", worked as expected
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic test

# I started a console producer and manually send a handful of test messages, 
worked as expected (see consumer below)
$ bin/kafka-console-producer.sh --broker-list localhost:9092 --topic test
{code}

Consumer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# I started a console consumer, worked as expected (i.e. could read messages 
sent by producer, see above)
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning
{code}

Up to that point, everything worked.  But then the Kafka broker went the way of 
the dodo.  As I said I can't pinpoint the cause, and re-running the same 
commands on a fresh Kafka/ZooKeeper deployment (fresh VMs etc.) didn't 
consistently trigger the issue like I hoped.

Here's what I did after the commands above, and at some point I eventually did 
observe the original error described in this JIRA ticket.  Again, at the moment 
I cannot tell what actually triggered the bug.

Producer:

{code}
# Test-driving the consumer offset checker
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1
# At this point consumer "foo" was not expected to exist.
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1 --broker-info --group foo

#
# I then re-started the console producer (see below), now configured to use the 
group id "foo".
#
{code}

Consumer:

{code}
# I re-started the console consumer, now configured to use the group id "foo".
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning --group foo
{code}

At this point "describing" the topic gave the following info, indicating that 
there was a problem (e.g. no leader for partition):

{code}
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic testTopic:test  
PartitionCount:1ReplicationFactor:1 Configs:
Topic: test Partition: 0Leader: -1  Replicas: 0 Isr:
{code}

Log files such as {{state-change.log}} showed these error messages:

{code}
[2014-04-29 07:44:47,816] TRACE Broker 0 cached leader info 
(LeaderAndIsrInfo:(Leader:0,ISR:0,LeaderEpoch:0,ControllerEp
och:1),ReplicationFactor:1),AllReplicas:0) for partition [test,0] in response 
to UpdateMet

[jira] [Comment Edited] (KAFKA-1029) Zookeeper leader election stuck in ephemeral node retry loop

2014-04-29 Thread Michael Noll (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1029?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984265#comment-13984265
 ] 

Michael Noll edited comment on KAFKA-1029 at 4/29/14 1:22 PM:
--

Interestingly I ran into a very similar issue while doing basic validation of 
Kafka 0.8.1.1.  Note that unlike Jason Rosenberg's case I was only using 
code/tools/scripts that are shipped with Kafka (e.g. console producer and 
console consumer).

Here is an example log message, which was repeated indefinitely:

{code}
[2014-04-29 09:48:27,207] INFO conflict in /controller data: 
{"version":1,"brokerid":0,"timestamp":"1398764901156"} stored data: 
{"version":1,"brokerid":0,"timestamp":"1398764894941"} (kafka.utils.ZkUtils$)
[2014-04-29 09:48:27,218] INFO I wrote this conflicted ephemeral node 
[{"version":1,"brokerid":0,"timestamp":"1398764901156"}] at /controller a while 
back in a different session, hence I will backoff for this node to be deleted 
by Zookeeper and retry (kafka.utils.ZkUtils$)
{code}

*How to reproduce (unsolved)*

Unfortunately I cannot consistently reproduce the issue and, to be honest, I am 
still not sure what actually triggers the bug.  As such I can only summarize 
what I did before and around the time when this bug was triggered, which I 
could observe through errors in log files and through errors being displayed 
after running certain commands.  So yes, it's a bit like shooting in the dark.

Here's an overview of my test setup:

- I deployed Kafka 0.8.1.1 to one machine {{kafka1}} and ZooKeeper 3.4.5 to a 
second machine {{zookeeper1}}.
- I followed the Kafka 0.8.1 quick start guide to create a topic "test" with 1 
partition and a replication factor of 1.
- I sent test messages to the topic "test" via the console producer.
- I read test messages from the topic "test" via the console consumer.
- Apart from producing and consuming a handful of test messages I also ran some 
supposedly read-only admin commands such as "describing" the topic, and running 
the consumer offset checker tool.
- At "some" point, Kafka was caught in an indefinite loop complaining about 
"conflicted ephemeral node".


The following paragraphs list in more detail what I did before the error popped 
up.

Producer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# This command returned no topics at this point = worked as expected
$ bin/kafka-topics.sh --list --zookeeper zookeeper1

# I created a topic, worked as expected
$ bin/kafka-topics.sh --create --zookeeper zookeeper1 --replication-factor 1 
--partitions 1 --topic test

# I requested details of topic "test", worked as expected
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic test

# I started a console producer and manually send a handful of test messages, 
worked as expected (see consumer below)
$ bin/kafka-console-producer.sh --broker-list localhost:9092 --topic test
{code}

Consumer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# I started a console consumer, worked as expected (i.e. could read messages 
sent by producer, see above)
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning
{code}

Up to that point, everything worked.  But then the Kafka broker went the way of 
the dodo.  As I said I can't pinpoint the cause, and re-running the same 
commands on a fresh Kafka/ZooKeeper deployment (fresh VMs etc.) didn't 
consistently trigger the issue like I hoped.

Here's what I did after the commands above, and at some point I eventually did 
observe the original error described in this JIRA ticket.  Again, at the moment 
I cannot tell what actually triggered the bug.

Producer:

{code}
# Test-driving the consumer offset checker
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1
# At this point consumer "foo" was not expected to exist.
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1 --broker-info --group foo

#
# I then re-started the console producer (see below), now configured to use the 
group id "foo".
#
{code}

Consumer:

{code}
# I re-started the console consumer, now configured to use the group id "foo".
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning --group foo
{code}

At this point "describing" the topic gave the following info, indicating that 
there was a problem (e.g. no leader for partition):

{code}
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic testTopic:test  
PartitionCount:1ReplicationFactor:1 Configs:
Topic: test Partition: 0Leader: -1  Replicas: 0 Isr:
{code}

Log files such as {{state-change.log}} showed these error messages:

{code}
[2014-04-29 07:44:47,816] TRACE Broker 0 cached leader info 
(LeaderAndIsrInfo:(Leader:0,ISR:0,LeaderEpoch:0,ControllerEp
och:1),ReplicationFactor:1),AllReplicas:0) for partition [test,0] in response 
to UpdateMe

[jira] [Comment Edited] (KAFKA-1029) Zookeeper leader election stuck in ephemeral node retry loop

2014-04-29 Thread Michael Noll (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1029?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984265#comment-13984265
 ] 

Michael Noll edited comment on KAFKA-1029 at 4/29/14 1:27 PM:
--

Interestingly I ran into a very similar issue while doing basic validation of 
Kafka 0.8.1.1.  Note that unlike Jason Rosenberg's case I was only using 
code/tools/scripts that are shipped with Kafka (e.g. console producer and 
console consumer).

Here is an example log message, which was repeated indefinitely:

{code}
[2014-04-29 09:48:27,207] INFO conflict in /controller data: 
{"version":1,"brokerid":0,"timestamp":"1398764901156"} stored data: 
{"version":1,"brokerid":0,"timestamp":"1398764894941"} (kafka.utils.ZkUtils$)
[2014-04-29 09:48:27,218] INFO I wrote this conflicted ephemeral node 
[{"version":1,"brokerid":0,"timestamp":"1398764901156"}] at /controller a while 
back in a different session, hence I will backoff for this node to be deleted 
by Zookeeper and retry (kafka.utils.ZkUtils$)
{code}

*How to reproduce (unsolved)*

Unfortunately I cannot consistently reproduce the issue and, to be honest, I am 
still not sure what actually triggers the bug.  As such I can only summarize 
what I did before and around the time when this bug was triggered, which I 
could observe through errors in log files and through errors being displayed 
after running certain commands.  So yes, it's a bit like shooting in the dark.

Here's an overview of my test setup:

- I deployed Kafka 0.8.1.1 to one machine {{kafka1}} and ZooKeeper 3.4.5 to a 
second machine {{zookeeper1}}.
- I followed the Kafka 0.8.1 quick start guide to create a topic "test" with 1 
partition and a replication factor of 1.
- I sent test messages to the topic "test" via the console producer.
- I read test messages from the topic "test" via the console consumer.
- Apart from producing and consuming a handful of test messages I also ran some 
supposedly read-only admin commands such as "describing" the topic, and running 
the consumer offset checker tool.
- At "some" point, Kafka was caught in an indefinite loop complaining about 
"conflicted ephemeral node".


The following paragraphs list in more detail what I did before the error popped 
up.

Producer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# This command returned no topics at this point = worked as expected
$ bin/kafka-topics.sh --list --zookeeper zookeeper1

# I created a topic, worked as expected
$ bin/kafka-topics.sh --create --zookeeper zookeeper1 --replication-factor 1 
--partitions 1 --topic test

# I requested details of topic "test", worked as expected
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic test

# I started a console producer and manually send a handful of test messages, 
worked as expected (see consumer below)
$ bin/kafka-console-producer.sh --broker-list localhost:9092 --topic test
{code}

Consumer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# I started a console consumer, worked as expected (i.e. could read messages 
sent by producer, see above)
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning
{code}

Up to that point, everything worked.  But then the Kafka broker went the way of 
the dodo.  As I said I can't pinpoint the cause, and re-running the same 
commands on a fresh Kafka/ZooKeeper deployment (fresh VMs etc.) didn't 
consistently trigger the issue like I hoped.

Here's what I did after the commands above, and at some point I eventually did 
observe the original error described in this JIRA ticket.  Again, at the moment 
I cannot tell what actually triggered the bug.

Producer:

{code}
# Test-driving the consumer offset checker
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1
# At this point consumer "foo" was not expected to exist.
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1 --broker-info --group foo

#
# I then re-started the console producer (see below), now configured to use the 
group id "foo".
#
{code}

Consumer:

{code}
# I re-started the console consumer, now configured to use the group id "foo".
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning --group foo
{code}

At this point "describing" the topic gave the following info, indicating that 
there was a problem (e.g. no leader for partition):

{code}
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic testTopic:test  
PartitionCount:1ReplicationFactor:1 Configs:
Topic: test Partition: 0Leader: -1  Replicas: 0 Isr:
{code}

Log files such as {{state-change.log}} showed these error messages:

{code}
[2014-04-29 07:44:47,816] TRACE Broker 0 cached leader info 
(LeaderAndIsrInfo:(Leader:0,ISR:0,LeaderEpoch:0,ControllerEp
och:1),ReplicationFactor:1),AllReplicas:0) for partition [test,0] in response 
to UpdateMe

[jira] [Comment Edited] (KAFKA-1029) Zookeeper leader election stuck in ephemeral node retry loop

2014-04-29 Thread Michael Noll (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1029?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984265#comment-13984265
 ] 

Michael Noll edited comment on KAFKA-1029 at 4/29/14 1:27 PM:
--

Interestingly I ran into a very similar issue while doing basic validation of 
Kafka 0.8.1.1.  Note that unlike Jason Rosenberg's case I was only using 
code/tools/scripts that are shipped with Kafka (e.g. console producer and 
console consumer).

Here is an example log message, which was repeated indefinitely:

{code}
[2014-04-29 09:48:27,207] INFO conflict in /controller data: 
{"version":1,"brokerid":0,"timestamp":"1398764901156"} stored data: 
{"version":1,"brokerid":0,"timestamp":"1398764894941"} (kafka.utils.ZkUtils$)
[2014-04-29 09:48:27,218] INFO I wrote this conflicted ephemeral node 
[{"version":1,"brokerid":0,"timestamp":"1398764901156"}] at /controller a while 
back in a different session, hence I will backoff for this node to be deleted 
by Zookeeper and retry (kafka.utils.ZkUtils$)
{code}

*How to reproduce (unsolved)*

Unfortunately I cannot consistently reproduce the issue and, to be honest, I am 
still not sure what actually triggers the bug.  As such I can only summarize 
what I did before and around the time when this bug was triggered, which I 
could observe through errors in log files and through errors being displayed 
after running certain commands.  So yes, it's a bit like shooting in the dark.

Here's an overview of my test setup:

- I deployed Kafka 0.8.1.1 to one machine {{kafka1}} and ZooKeeper 3.4.5 to a 
second machine {{zookeeper1}}.
- I followed the Kafka 0.8.1 quick start guide to create a topic "test" with 1 
partition and a replication factor of 1.
- I sent test messages to the topic "test" via the console producer.
- I read test messages from the topic "test" via the console consumer.
- Apart from producing and consuming a handful of test messages I also ran some 
supposedly read-only admin commands such as "describing" the topic, and running 
the consumer offset checker tool.
- At "some" point, Kafka was caught in an indefinite loop complaining about 
"conflicted ephemeral node".


The following paragraphs list in more detail what I did before the error popped 
up.

Producer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# This command returned no topics at this point = worked as expected
$ bin/kafka-topics.sh --list --zookeeper zookeeper1

# I created a topic, worked as expected
$ bin/kafka-topics.sh --create --zookeeper zookeeper1 --replication-factor 1 
--partitions 1 --topic test

# I requested details of topic "test", worked as expected
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic test

# I started a console producer and manually send a handful of test messages, 
worked as expected (see consumer below)
$ bin/kafka-console-producer.sh --broker-list localhost:9092 --topic test
{code}

Consumer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# I started a console consumer, worked as expected (i.e. could read messages 
sent by producer, see above)
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning
{code}

Up to that point, everything worked.  But then the Kafka broker went the way of 
the dodo.  As I said I can't pinpoint the cause, and re-running the same 
commands on a fresh Kafka/ZooKeeper deployment (fresh VMs etc.) didn't 
consistently trigger the issue like I hoped.

Here's what I did after the commands above, and at some point I eventually did 
observe the original error described in this JIRA ticket.  Again, at the moment 
I cannot tell what actually triggered the bug.

Producer:

{code}
# Test-driving the consumer offset checker
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1
# At this point consumer "foo" was not expected to exist.
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1 --broker-info --group foo

#
# I then re-started the console producer (see below), now configured to use the 
group id "foo".
#
{code}

Consumer:

{code}
# I re-started the console consumer, now configured to use the group id "foo".
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning --group foo
{code}

At this point "describing" the topic gave the following info, indicating that 
there was a problem (e.g. no leader for partition):

{code}
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic testTopic:test  
PartitionCount:1ReplicationFactor:1 Configs:
Topic: test Partition: 0Leader: -1  Replicas: 0 Isr:
{code}

Log files such as {{state-change.log}} showed these error messages:

{code}
[2014-04-29 07:44:47,816] TRACE Broker 0 cached leader info 
(LeaderAndIsrInfo:(Leader:0,ISR:0,LeaderEpoch:0,ControllerEp
och:1),ReplicationFactor:1),AllReplicas:0) for partition [test,0] in response 
to UpdateMe

[jira] [Comment Edited] (KAFKA-1029) Zookeeper leader election stuck in ephemeral node retry loop

2014-04-29 Thread Michael Noll (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1029?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984265#comment-13984265
 ] 

Michael Noll edited comment on KAFKA-1029 at 4/29/14 1:30 PM:
--

Interestingly I ran into a very similar issue while doing basic validation of 
Kafka 0.8.1.1.  Note that unlike Jason Rosenberg's case I was only using 
code/tools/scripts that are shipped with Kafka (e.g. console producer and 
console consumer).

Here is an example log message, which was repeated indefinitely:

{code}
[2014-04-29 09:48:27,207] INFO conflict in /controller data: 
{"version":1,"brokerid":0,"timestamp":"1398764901156"} stored data: 
{"version":1,"brokerid":0,"timestamp":"1398764894941"} (kafka.utils.ZkUtils$)
[2014-04-29 09:48:27,218] INFO I wrote this conflicted ephemeral node 
[{"version":1,"brokerid":0,"timestamp":"1398764901156"}] at /controller a while 
back in a different session, hence I will backoff for this node to be deleted 
by Zookeeper and retry (kafka.utils.ZkUtils$)
{code}

*How to reproduce (unsolved)*

Unfortunately I cannot consistently reproduce the issue and, to be honest, I am 
still not sure what actually triggers the bug.  As such I can only summarize 
what I did before and around the time when this bug was triggered, which I 
could observe through errors in log files and through errors being displayed 
after running certain commands.  So yes, it's a bit like shooting in the dark.

Here's an overview of my test setup:

- I deployed Kafka 0.8.1.1 to one machine {{kafka1}} and ZooKeeper 3.4.5 to a 
second machine {{zookeeper1}}.
- I followed the Kafka 0.8.1 quick start guide to create a topic "test" with 1 
partition and a replication factor of 1.
- I sent test messages to the topic "test" via the console producer.
- I read test messages from the topic "test" via the console consumer.
- Apart from producing and consuming a handful of test messages I also ran some 
supposedly read-only admin commands such as "describing" the topic, and running 
the consumer offset checker tool.
- At "some" point, Kafka was caught in an indefinite loop complaining about 
"conflict in {{/controller}} data".


The following paragraphs list in more detail what I did before the error popped 
up.

Producer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# This command returned no topics at this point = worked as expected
$ bin/kafka-topics.sh --list --zookeeper zookeeper1

# I created a topic, worked as expected
$ bin/kafka-topics.sh --create --zookeeper zookeeper1 --replication-factor 1 
--partitions 1 --topic test

# I requested details of topic "test", worked as expected
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic test

# I started a console producer and manually send a handful of test messages, 
worked as expected (see consumer below)
$ bin/kafka-console-producer.sh --broker-list localhost:9092 --topic test
{code}

Consumer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# I started a console consumer, worked as expected (i.e. could read messages 
sent by producer, see above)
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning
{code}

Up to that point, everything worked.  But then the Kafka broker went the way of 
the dodo.  As I said I can't pinpoint the cause, and re-running the same 
commands on a fresh Kafka/ZooKeeper deployment (fresh VMs etc.) didn't 
consistently trigger the issue like I hoped.

Here's what I did after the commands above, and at some point I eventually did 
observe the original error described in this JIRA ticket.  Again, at the moment 
I cannot tell what actually triggered the bug.

Producer:

{code}
# Test-driving the consumer offset checker
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1
# At this point consumer "foo" was not expected to exist.
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1 --broker-info --group foo

#
# I then re-started the console producer (see below), now configured to use the 
group id "foo".
#
{code}

Consumer:

{code}
# I re-started the console consumer, now configured to use the group id "foo".
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning --group foo
{code}

At this point "describing" the topic gave the following info, indicating that 
there was a problem (e.g. no leader for partition):

{code}
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic testTopic:test  
PartitionCount:1ReplicationFactor:1 Configs:
Topic: test Partition: 0Leader: -1  Replicas: 0 Isr:
{code}

Log files such as {{state-change.log}} showed these error messages:

{code}
[2014-04-29 07:44:47,816] TRACE Broker 0 cached leader info 
(LeaderAndIsrInfo:(Leader:0,ISR:0,LeaderEpoch:0,ControllerEp
och:1),ReplicationFactor:1),AllReplicas:0) for partition [test,0] in response 
to U

[jira] [Comment Edited] (KAFKA-1029) Zookeeper leader election stuck in ephemeral node retry loop

2014-04-29 Thread Michael Noll (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1029?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984265#comment-13984265
 ] 

Michael Noll edited comment on KAFKA-1029 at 4/29/14 1:35 PM:
--

Interestingly I ran into a very similar issue while doing basic validation of 
Kafka 0.8.1.1.  Note that unlike Jason Rosenberg's case I was only using 
code/tools/scripts that are shipped with Kafka (e.g. console producer and 
console consumer).

Here is an example log message, which was repeated indefinitely:

{code}
[2014-04-29 09:48:27,207] INFO conflict in /controller data: 
{"version":1,"brokerid":0,"timestamp":"1398764901156"} stored data: 
{"version":1,"brokerid":0,"timestamp":"1398764894941"} (kafka.utils.ZkUtils$)
[2014-04-29 09:48:27,218] INFO I wrote this conflicted ephemeral node 
[{"version":1,"brokerid":0,"timestamp":"1398764901156"}] at /controller a while 
back in a different session, hence I will backoff for this node to be deleted 
by Zookeeper and retry (kafka.utils.ZkUtils$)
{code}

*How to reproduce (unsolved)*

Unfortunately I cannot consistently reproduce the issue and, to be honest, I am 
still not sure what actually triggers the bug.  As such I can only summarize 
what I did before and around the time when this bug was triggered, which I 
could observe through errors in log files and through errors being displayed 
after running certain commands.  So yes, it's a bit like shooting in the dark.

Here's an overview of my test setup:

- I deployed Kafka 0.8.1.1 to one machine {{kafka1}} and ZooKeeper 3.4.5 to a 
second machine {{zookeeper1}}.
- I followed the Kafka 0.8.1 quick start guide to create a topic "test" with 1 
partition and a replication factor of 1.
- I sent test messages to the topic "test" via the console producer.
- I read test messages from the topic "test" via the console consumer.
- Apart from producing and consuming a handful of test messages I also ran some 
supposedly read-only admin commands such as "describing" the topic, and running 
the consumer offset checker tool.
- At "some" point, Kafka was caught in an indefinite loop complaining about 
"conflict in {{/controller}} data".


The following paragraphs list in more detail what I did before the error popped 
up.

Producer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# This command returned no topics at this point = worked as expected
$ bin/kafka-topics.sh --list --zookeeper zookeeper1

# I created a topic, worked as expected
$ bin/kafka-topics.sh --create --zookeeper zookeeper1 --replication-factor 1 
--partitions 1 --topic test

# I requested details of topic "test", worked as expected
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic test
Topic:test  PartitionCount:1ReplicationFactor:1 Configs:
Topic: test Partition: 0Leader: 0   Replicas: 0 Isr: 0

# I started a console producer and manually send a handful of test messages, 
worked as expected (see consumer below)
$ bin/kafka-console-producer.sh --broker-list localhost:9092 --topic test
{code}

Consumer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# I started a console consumer, worked as expected (i.e. could read messages 
sent by producer, see above)
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning
{code}

Up to that point, everything worked.  But then the Kafka broker went the way of 
the dodo.  As I said I can't pinpoint the cause, and re-running the same 
commands on a fresh Kafka/ZooKeeper deployment (fresh VMs etc.) didn't 
consistently trigger the issue like I hoped.

Here's what I did after the commands above, and at some point I eventually did 
observe the original error described in this JIRA ticket.  Again, at the moment 
I cannot tell what actually triggered the bug.

Producer:

{code}
# Test-driving the consumer offset checker
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1
# At this point consumer "foo" was not expected to exist.
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1 --broker-info --group foo

#
# I then re-started the console producer (see below), now configured to use the 
group id "foo".
#
{code}

Consumer:

{code}
# I re-started the console consumer, now configured to use the group id "foo".
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning --group foo
{code}

At this point "describing" the topic gave the following info, indicating that 
there was a problem (e.g. no leader for partition):

{code}
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic testTopic:test  
PartitionCount:1ReplicationFactor:1 Configs:
Topic: test Partition: 0Leader: -1  Replicas: 0 Isr:
{code}

Log files such as {{state-change.log}} showed these error messages:

{code}
[2014-04-29 07:44:47,816] TRACE Broker 0 cached leade

[jira] [Comment Edited] (KAFKA-1029) Zookeeper leader election stuck in ephemeral node retry loop

2014-04-29 Thread Michael Noll (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1029?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984265#comment-13984265
 ] 

Michael Noll edited comment on KAFKA-1029 at 4/29/14 1:35 PM:
--

Interestingly I ran into a very similar issue while doing basic validation of 
Kafka 0.8.1.1.  Note that unlike Jason Rosenberg's case I was only using 
code/tools/scripts that are shipped with Kafka (e.g. console producer and 
console consumer).

Here is an example log message, which was repeated indefinitely:

{code}
[2014-04-29 09:48:27,207] INFO conflict in /controller data: 
{"version":1,"brokerid":0,"timestamp":"1398764901156"} stored data: 
{"version":1,"brokerid":0,"timestamp":"1398764894941"} (kafka.utils.ZkUtils$)
[2014-04-29 09:48:27,218] INFO I wrote this conflicted ephemeral node 
[{"version":1,"brokerid":0,"timestamp":"1398764901156"}] at /controller a while 
back in a different session, hence I will backoff for this node to be deleted 
by Zookeeper and retry (kafka.utils.ZkUtils$)
{code}

*How to reproduce (unsolved)*

Unfortunately I cannot consistently reproduce the issue and, to be honest, I am 
still not sure what actually triggers the bug.  As such I can only summarize 
what I did before and around the time when this bug was triggered, which I 
could observe through errors in log files and through errors being displayed 
after running certain commands.  So yes, it's a bit like shooting in the dark.

Here's an overview of my test setup:

- I deployed Kafka 0.8.1.1 to one machine {{kafka1}} and ZooKeeper 3.4.5 to a 
second machine {{zookeeper1}}.
- I followed the Kafka 0.8.1 quick start guide to create a topic "test" with 1 
partition and a replication factor of 1.
- I sent test messages to the topic "test" via the console producer.
- I read test messages from the topic "test" via the console consumer.
- Apart from producing and consuming a handful of test messages I also ran some 
supposedly read-only admin commands such as "describing" the topic, and running 
the consumer offset checker tool.
- At "some" point, Kafka was caught in an indefinite loop complaining about 
"conflict in {{/controller}} data".


The following paragraphs list in more detail what I did before the error popped 
up.

Producer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# This command returned no topics at this point = worked as expected
$ bin/kafka-topics.sh --list --zookeeper zookeeper1

# I created a topic, worked as expected
$ bin/kafka-topics.sh --create --zookeeper zookeeper1 --replication-factor 1 
--partitions 1 --topic test

# I requested details of topic "test", worked as expected
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic test
Topic:test  PartitionCount:1ReplicationFactor:1 Configs:
Topic: test Partition: 0Leader: 0   Replicas: 0 Isr: 0

# I started a console producer and manually send a handful of test messages, 
worked as expected (see consumer below)
$ bin/kafka-console-producer.sh --broker-list localhost:9092 --topic test
{code}

Consumer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# I started a console consumer, worked as expected (i.e. could read messages 
sent by producer, see above)
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning
{code}

Up to that point, everything worked.  But then the Kafka broker went the way of 
the dodo.  As I said I can't pinpoint the cause, and re-running the same 
commands on a fresh Kafka/ZooKeeper deployment (fresh VMs etc.) didn't 
consistently trigger the issue like I hoped.

Here's what I did after the commands above, and at some point I eventually did 
observe the original error described in this JIRA ticket.  Again, at the moment 
I cannot tell what actually triggered the bug.

Producer:

{code}
# Test-driving the consumer offset checker
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1
# At this point consumer "foo" was not expected to exist.
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1 --broker-info --group foo

#
# I then re-started the console producer (see below), now configured to use the 
group id "foo".
#
{code}

Consumer:

{code}
# I re-started the console consumer, now configured to use the group id "foo".
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning --group foo
{code}

At this point "describing" the topic gave the following info, indicating that 
there was a problem (e.g. no leader for partition):

{code}
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic test
Topic:test  PartitionCount:1ReplicationFactor:1 Configs:
Topic: test Partition: 0Leader: -1  Replicas: 0 Isr:
{code}

Log files such as {{state-change.log}} showed these error messages:

{code}
[2014-04-29 07:44:47,816] TRACE Broker 0 cached l

[jira] [Comment Edited] (KAFKA-1029) Zookeeper leader election stuck in ephemeral node retry loop

2014-04-29 Thread Michael Noll (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1029?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984265#comment-13984265
 ] 

Michael Noll edited comment on KAFKA-1029 at 4/29/14 1:36 PM:
--

Interestingly I ran into a very similar issue while doing basic validation of 
Kafka 0.8.1.1.  Note that unlike Jason Rosenberg's case I was only using 
code/tools/scripts that are shipped with Kafka (e.g. console producer and 
console consumer).

Here is an example log message, which was repeated indefinitely:

{code}
[2014-04-29 09:48:27,207] INFO conflict in /controller data: 
{"version":1,"brokerid":0,"timestamp":"1398764901156"} stored data: 
{"version":1,"brokerid":0,"timestamp":"1398764894941"} (kafka.utils.ZkUtils$)
[2014-04-29 09:48:27,218] INFO I wrote this conflicted ephemeral node 
[{"version":1,"brokerid":0,"timestamp":"1398764901156"}] at /controller a while 
back in a different session, hence I will backoff for this node to be deleted 
by Zookeeper and retry (kafka.utils.ZkUtils$)
{code}

*How to reproduce (unsolved)*

Unfortunately I cannot consistently reproduce the issue and, to be honest, I am 
still not sure what actually triggers the bug.  As such I can only summarize 
what I did before and around the time when this bug was triggered, which I 
could observe through errors in log files and through errors being displayed 
after running certain commands.  So yes, it's a bit like shooting in the dark.

Here's an overview of my test setup:

- I deployed Kafka 0.8.1.1 to one machine {{kafka1}} and ZooKeeper 3.4.5 to a 
second machine {{zookeeper1}}.
- I followed the Kafka 0.8.1 quick start guide to create a topic "test" with 1 
partition and a replication factor of 1.
- I sent test messages to the topic "test" via the console producer.
- I read test messages from the topic "test" via the console consumer.
- Apart from producing and consuming a handful of test messages I also ran some 
supposedly read-only admin commands such as "describing" the topic, and running 
the consumer offset checker tool.
- At "some" point, Kafka was caught in an indefinite loop complaining about 
"conflict in {{/controller}} data".


The following paragraphs list in more detail what I did before the error popped 
up.

Producer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# This command returned no topics at this point = worked as expected
$ bin/kafka-topics.sh --list --zookeeper zookeeper1

# I created a topic, worked as expected
$ bin/kafka-topics.sh --create --zookeeper zookeeper1 --replication-factor 1 
--partitions 1 --topic test

# I requested details of topic "test", worked as expected
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic test
Topic:test  PartitionCount:1ReplicationFactor:1 Configs:
Topic: test Partition: 0Leader: 0   Replicas: 0 Isr: 0

# I started a console producer and manually send a handful of test messages, 
worked as expected (see consumer below)
$ bin/kafka-console-producer.sh --broker-list localhost:9092 --topic test
{code}

Consumer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# I started a console consumer, worked as expected (i.e. could read messages 
sent by producer, see above)
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning
{code}

Up to that point, everything worked.  But then the Kafka broker went the way of 
the dodo.  As I said I can't pinpoint the cause, and re-running the same 
commands on a fresh Kafka/ZooKeeper deployment (fresh VMs etc.) didn't 
consistently trigger the issue like I hoped.

Here's what I did after the commands above, and at some point I eventually did 
observe the original error described in this JIRA ticket.  Again, at the moment 
I cannot tell what actually triggered the bug.

Producer:

{code}
# Test-driving the consumer offset checker
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1
# At this point consumer "foo" was not expected to exist.
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1 --broker-info --group foo

#
# I then re-started the console producer (see below), now configured to use the 
group id "foo".
#
{code}

Consumer:

{code}
# I re-started the console consumer, now configured to use the group id "foo".
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning --group foo
{code}

At this point "describing" the topic gave the following info, indicating that 
there was a problem (e.g. no leader for partition, no ISR available):

{code}
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic test
Topic:test  PartitionCount:1ReplicationFactor:1 Configs:
Topic: test Partition: 0Leader: -1  Replicas: 0 Isr:
{code}

Log files such as {{state-change.log}} showed these error messages:

{code}
[2014-04-29 07:44:47,816] TRACE

[jira] [Commented] (KAFKA-1429) Yet another deadlock in controller shutdown

2014-04-29 Thread Dmitry Bugaychenko (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1429?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984295#comment-13984295
 ] 

Dmitry Bugaychenko commented on KAFKA-1429:
---

Updated libs to 0.8.1.1, will see how it goes.

> Yet another deadlock in controller shutdown
> ---
>
> Key: KAFKA-1429
> URL: https://issues.apache.org/jira/browse/KAFKA-1429
> Project: Kafka
>  Issue Type: Bug
>  Components: controller
>Affects Versions: 0.8.1
>Reporter: Dmitry Bugaychenko
>Assignee: Neha Narkhede
>
> Found one more case of deadlock in controller during shutdown:
> {code}
> ZkClient-EventThread-57-192.168.41.148:2181,192.168.36.250:2181,192.168.41.207:2181
>  id=57 state=TIMED_WAITING
> - waiting on <0x288a66ec> (a 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
> - locked <0x288a66ec> (a 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
> at sun.misc.Unsafe.park(Native Method)
> at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:226)
> at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2082)
> at 
> java.util.concurrent.ThreadPoolExecutor.awaitTermination(ThreadPoolExecutor.java:1468)
> at kafka.utils.KafkaScheduler.shutdown(KafkaScheduler.scala:88)
> at 
> kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply$mcV$sp(KafkaController.scala:339)
> at 
> kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337)
> at 
> kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337)
> at kafka.utils.Utils$.inLock(Utils.scala:538)
> at 
> kafka.controller.KafkaController.onControllerResignation(KafkaController.scala:337)
> at 
> kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply$mcZ$sp(KafkaController.scala:1068)
> at 
> kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1067)
> at 
> kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1067)
> at kafka.utils.Utils$.inLock(Utils.scala:538)
> at 
> kafka.controller.KafkaController$SessionExpirationListener.handleNewSession(KafkaController.scala:1067)
> at org.I0Itec.zkclient.ZkClient$4.run(ZkClient.java:472)
> at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71)
> Locked synchronizers: count = 1
>   - java.util.concurrent.locks.ReentrantLock$NonfairSync@22b9b31a
> kafka-scheduler-0 id=172 state=WAITING
> - waiting on <0x22b9b31a> (a 
> java.util.concurrent.locks.ReentrantLock$NonfairSync)
> - locked <0x22b9b31a> (a 
> java.util.concurrent.locks.ReentrantLock$NonfairSync)
>  owned by 
> ZkClient-EventThread-57-192.168.41.148:2181,192.168.36.250:2181,192.168.41.207:2181
>  id=57
> at sun.misc.Unsafe.park(Native Method)
> at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
> at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:834)
> at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:867)
> at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1197)
> at 
> java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java:214)
> at java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:290)
> at kafka.utils.Utils$.inLock(Utils.scala:536)
> at 
> kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$17.apply(KafkaController.scala:1110)
> at 
> kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$17.apply(KafkaController.scala:1108)
> at 
> scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98)
> at 
> scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98)
> at 
> scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:226)
> at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:39)
> at scala.collection.mutable.HashMap.foreach(HashMap.scala:98)
> at 
> kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4.apply(KafkaController.scala:1108)
> at 
> kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4.apply(KafkaController.scala:1087)
> at scala.collection.immutable.Map$Map4

[jira] [Comment Edited] (KAFKA-1029) Zookeeper leader election stuck in ephemeral node retry loop

2014-04-29 Thread Michael Noll (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1029?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984265#comment-13984265
 ] 

Michael Noll edited comment on KAFKA-1029 at 4/29/14 1:45 PM:
--

Interestingly I ran into a very similar issue while doing basic validation of 
Kafka 0.8.1.1.  Note that unlike Jason Rosenberg's case I was only using 
code/tools/scripts that are shipped with Kafka (e.g. console producer and 
console consumer).

Here is an example log message, which was repeated indefinitely:

{code}
[2014-04-29 09:48:27,207] INFO conflict in /controller data: 
{"version":1,"brokerid":0,"timestamp":"1398764901156"} stored data: 
{"version":1,"brokerid":0,"timestamp":"1398764894941"} (kafka.utils.ZkUtils$)
[2014-04-29 09:48:27,218] INFO I wrote this conflicted ephemeral node 
[{"version":1,"brokerid":0,"timestamp":"1398764901156"}] at /controller a while 
back in a different session, hence I will backoff for this node to be deleted 
by Zookeeper and retry (kafka.utils.ZkUtils$)
{code}

*How to reproduce (unsolved)*

Unfortunately I cannot consistently reproduce the issue and, to be honest, I am 
still not sure what actually triggers the bug.  As such I can only summarize 
what I did before and around the time when this bug was triggered, which I 
could observe through errors in log files and through errors being displayed 
after running certain commands.  So yes, it's a bit like shooting in the dark.

Here's an overview of my test setup:

- I deployed Kafka 0.8.1.1 to one machine {{kafka1}} and ZooKeeper 3.4.5 to a 
second machine {{zookeeper1}}.
- I followed the Kafka 0.8.1 quick start guide to create a topic "test" with 1 
partition and a replication factor of 1.
- I sent test messages to the topic "test" via the console producer.
- I read test messages from the topic "test" via the console consumer.
- Apart from producing and consuming a handful of test messages I also ran some 
supposedly read-only admin commands such as "describing" the topic, and running 
the consumer offset checker tool.
- At "some" point, Kafka was caught in an indefinite loop complaining about 
"conflict in {{/controller}} data".


The following paragraphs list in more detail what I did before the error popped 
up.

Producer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# This command returned no topics at this point = worked as expected
$ bin/kafka-topics.sh --list --zookeeper zookeeper1

# I created a topic, worked as expected
$ bin/kafka-topics.sh --create --zookeeper zookeeper1 --replication-factor 1 
--partitions 1 --topic test

# I requested details of topic "test", worked as expected
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic test
Topic:test  PartitionCount:1ReplicationFactor:1 Configs:
Topic: test Partition: 0Leader: 0   Replicas: 0 Isr: 0

# I started a console producer and manually send a handful of test messages, 
worked as expected (see consumer below)
$ bin/kafka-console-producer.sh --broker-list localhost:9092 --topic test
{code}

Consumer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# I started a console consumer, worked as expected (i.e. could read messages 
sent by producer, see above)
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning
{code}

Up to that point, everything worked.  But then the Kafka broker went the way of 
the dodo.  As I said I can't pinpoint the cause, and re-running the same 
commands on a fresh Kafka/ZooKeeper deployment (fresh VMs etc.) didn't 
consistently trigger the issue like I hoped.

Here's what I did after the commands above, and at some point I eventually did 
observe the original error described in this JIRA ticket.  Again, at the moment 
I cannot tell what actually triggered the bug.

Producer:

{code}
# Test-driving the consumer offset checker
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1
# At this point consumer "foo" was not expected to exist.
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1 --broker-info --group foo

#
# I then re-started the console producer (see below), now configured to use the 
group id "foo".
#
{code}

Consumer:

{code}
# I re-started the console consumer, now configured to use the group id "foo".
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning --group foo
{code}

At this point "describing" the topic gave the following info, indicating that 
there was a problem (e.g. no leader for partition, no ISR available):

{code}
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic test
Topic:test  PartitionCount:1ReplicationFactor:1 Configs:
Topic: test Partition: 0Leader: -1  Replicas: 0 Isr:
{code}

Log files such as {{state-change.log}} showed these error messages:

{code}


[2014-04-29 07:44:47,673] TRA

[jira] [Comment Edited] (KAFKA-1029) Zookeeper leader election stuck in ephemeral node retry loop

2014-04-29 Thread Michael Noll (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1029?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984265#comment-13984265
 ] 

Michael Noll edited comment on KAFKA-1029 at 4/29/14 1:45 PM:
--

Interestingly I ran into a very similar issue while doing basic validation of 
Kafka 0.8.1.1.  Note that unlike Jason Rosenberg's case I was only using 
code/tools/scripts that are shipped with Kafka (e.g. console producer and 
console consumer).

Here is an example log message, which was repeated indefinitely:

{code}
[2014-04-29 09:48:27,207] INFO conflict in /controller data: 
{"version":1,"brokerid":0,"timestamp":"1398764901156"} stored data: 
{"version":1,"brokerid":0,"timestamp":"1398764894941"} (kafka.utils.ZkUtils$)
[2014-04-29 09:48:27,218] INFO I wrote this conflicted ephemeral node 
[{"version":1,"brokerid":0,"timestamp":"1398764901156"}] at /controller a while 
back in a different session, hence I will backoff for this node to be deleted 
by Zookeeper and retry (kafka.utils.ZkUtils$)
{code}

*How to reproduce (unsolved)*

Unfortunately I cannot consistently reproduce the issue and, to be honest, I am 
still not sure what actually triggers the bug.  As such I can only summarize 
what I did before and around the time when this bug was triggered, which I 
could observe through errors in log files and through errors being displayed 
after running certain commands.  So yes, it's a bit like shooting in the dark.

Here's an overview of my test setup:

- I deployed Kafka 0.8.1.1 to one machine {{kafka1}} and ZooKeeper 3.4.5 to a 
second machine {{zookeeper1}}.
- I followed the Kafka 0.8.1 quick start guide to create a topic "test" with 1 
partition and a replication factor of 1.
- I sent test messages to the topic "test" via the console producer.
- I read test messages from the topic "test" via the console consumer.
- Apart from producing and consuming a handful of test messages I also ran some 
supposedly read-only admin commands such as "describing" the topic, and running 
the consumer offset checker tool.
- At "some" point, Kafka was caught in an indefinite loop complaining about 
"conflict in {{/controller}} data".


The following paragraphs list in more detail what I did before the error popped 
up.

Producer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# This command returned no topics at this point = worked as expected
$ bin/kafka-topics.sh --list --zookeeper zookeeper1

# I created a topic, worked as expected
$ bin/kafka-topics.sh --create --zookeeper zookeeper1 --replication-factor 1 
--partitions 1 --topic test

# I requested details of topic "test", worked as expected
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic test
Topic:test  PartitionCount:1ReplicationFactor:1 Configs:
Topic: test Partition: 0Leader: 0   Replicas: 0 Isr: 0

# I started a console producer and manually send a handful of test messages, 
worked as expected (see consumer below)
$ bin/kafka-console-producer.sh --broker-list localhost:9092 --topic test
{code}

Consumer:

{code}
$ sudo su - kafka
$ cd /opt/kafka

# I started a console consumer, worked as expected (i.e. could read messages 
sent by producer, see above)
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning
{code}

Up to that point, everything worked.  But then the Kafka broker went the way of 
the dodo.  As I said I can't pinpoint the cause, and re-running the same 
commands on a fresh Kafka/ZooKeeper deployment (fresh VMs etc.) didn't 
consistently trigger the issue like I hoped.

Here's what I did after the commands above, and at some point I eventually did 
observe the original error described in this JIRA ticket.  Again, at the moment 
I cannot tell what actually triggered the bug.

Producer:

{code}
# Test-driving the consumer offset checker
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1
# At this point consumer "foo" was not expected to exist.
$ bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zkconnect 
zookeeper1 --broker-info --group foo

#
# I then re-started the console producer (see below), now configured to use the 
group id "foo".
#
{code}

Consumer:

{code}
# I re-started the console consumer, now configured to use the group id "foo".
$ bin/kafka-console-consumer.sh --zookeeper zookeeper1 --topic test 
--from-beginning --group foo
{code}

At this point "describing" the topic gave the following info, indicating that 
there was a problem (e.g. no leader for partition, no ISR available):

{code}
$ bin/kafka-topics.sh --zookeeper zookeeper1 --describe --topic test
Topic:test  PartitionCount:1ReplicationFactor:1 Configs:
Topic: test Partition: 0Leader: -1  Replicas: 0 Isr:
{code}

Log files such as {{state-change.log}} showed these error messages:

{code}


[2014-04-29 07:44:47,673] TRA

[jira] [Commented] (KAFKA-1426) Rebalance issue on 0.8.1

2014-04-29 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1426?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984356#comment-13984356
 ] 

Jun Rao commented on KAFKA-1426:


Hmm, that's weird. Did you explicitly specify the consumer id? If so, each 
consumer in the same consumer group needs to have a different id.

> Rebalance issue on 0.8.1
> 
>
> Key: KAFKA-1426
> URL: https://issues.apache.org/jira/browse/KAFKA-1426
> Project: Kafka
>  Issue Type: Bug
>  Components: consumer
>Affects Versions: 0.8.1
>Reporter: Jeremy A Laycock
>Assignee: Neha Narkhede
>Priority: Minor
>
> When running up 2 console consumers reading from the same group the zookeeper 
> path  /consumers//ids only contains the latest consumer id - then 
> re-balance fails.
> Looking at the rebalance code the only deletion is the 
> /consumers//owners// for each partition a 
> consumer "owns"



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1429) Yet another deadlock in controller shutdown

2014-04-29 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1429?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984372#comment-13984372
 ] 

Jun Rao commented on KAFKA-1429:


Did you have auto.leader.rebalance.enable set to true? If so, there is known 
bug https://issues.apache.org/jira/browse/KAFKA-1305 that may be related to 
this.

> Yet another deadlock in controller shutdown
> ---
>
> Key: KAFKA-1429
> URL: https://issues.apache.org/jira/browse/KAFKA-1429
> Project: Kafka
>  Issue Type: Bug
>  Components: controller
>Affects Versions: 0.8.1
>Reporter: Dmitry Bugaychenko
>Assignee: Neha Narkhede
>
> Found one more case of deadlock in controller during shutdown:
> {code}
> ZkClient-EventThread-57-192.168.41.148:2181,192.168.36.250:2181,192.168.41.207:2181
>  id=57 state=TIMED_WAITING
> - waiting on <0x288a66ec> (a 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
> - locked <0x288a66ec> (a 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
> at sun.misc.Unsafe.park(Native Method)
> at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:226)
> at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2082)
> at 
> java.util.concurrent.ThreadPoolExecutor.awaitTermination(ThreadPoolExecutor.java:1468)
> at kafka.utils.KafkaScheduler.shutdown(KafkaScheduler.scala:88)
> at 
> kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply$mcV$sp(KafkaController.scala:339)
> at 
> kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337)
> at 
> kafka.controller.KafkaController$$anonfun$onControllerResignation$1.apply(KafkaController.scala:337)
> at kafka.utils.Utils$.inLock(Utils.scala:538)
> at 
> kafka.controller.KafkaController.onControllerResignation(KafkaController.scala:337)
> at 
> kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply$mcZ$sp(KafkaController.scala:1068)
> at 
> kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1067)
> at 
> kafka.controller.KafkaController$SessionExpirationListener$$anonfun$handleNewSession$1.apply(KafkaController.scala:1067)
> at kafka.utils.Utils$.inLock(Utils.scala:538)
> at 
> kafka.controller.KafkaController$SessionExpirationListener.handleNewSession(KafkaController.scala:1067)
> at org.I0Itec.zkclient.ZkClient$4.run(ZkClient.java:472)
> at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71)
> Locked synchronizers: count = 1
>   - java.util.concurrent.locks.ReentrantLock$NonfairSync@22b9b31a
> kafka-scheduler-0 id=172 state=WAITING
> - waiting on <0x22b9b31a> (a 
> java.util.concurrent.locks.ReentrantLock$NonfairSync)
> - locked <0x22b9b31a> (a 
> java.util.concurrent.locks.ReentrantLock$NonfairSync)
>  owned by 
> ZkClient-EventThread-57-192.168.41.148:2181,192.168.36.250:2181,192.168.41.207:2181
>  id=57
> at sun.misc.Unsafe.park(Native Method)
> at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
> at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:834)
> at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:867)
> at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1197)
> at 
> java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java:214)
> at java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:290)
> at kafka.utils.Utils$.inLock(Utils.scala:536)
> at 
> kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$17.apply(KafkaController.scala:1110)
> at 
> kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$17.apply(KafkaController.scala:1108)
> at 
> scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98)
> at 
> scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98)
> at 
> scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:226)
> at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:39)
> at scala.collection.mutable.HashMap.foreach(HashMap.scala:98)
> at 
> kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4.apply(KafkaController.scala:1108)
> at 
> kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartiti

[jira] [Commented] (KAFKA-1029) Zookeeper leader election stuck in ephemeral node retry loop

2014-04-29 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1029?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984377#comment-13984377
 ] 

Jun Rao commented on KAFKA-1029:


Interesting, did you see any ZK session expiration in the broker log (search 
for Expired)?

> Zookeeper leader election stuck in ephemeral node retry loop
> 
>
> Key: KAFKA-1029
> URL: https://issues.apache.org/jira/browse/KAFKA-1029
> Project: Kafka
>  Issue Type: Bug
>  Components: controller
>Affects Versions: 0.8.0
>Reporter: Sam Meder
>Assignee: Sam Meder
>Priority: Blocker
> Fix For: 0.8.0
>
> Attachments: 
> 0002-KAFKA-1029-Use-brokerId-instead-of-leaderId-when-tri.patch
>
>
> We're seeing the following log statements (over and over):
> [2013-08-27 07:21:49,538] INFO conflict in /controller data: { "brokerid":3, 
> "timestamp":"1377587945206", "version":1 } stored data: { "brokerid":2, 
> "timestamp":"1377587460904", "version":1 } (kafka.utils.ZkUtils$)
> [2013-08-27 07:21:49,559] INFO I wrote this conflicted ephemeral node [{ 
> "brokerid":3, "timestamp":"1377587945206", "version":1 }] at /controller a 
> while back in a different session, hence I will backoff for this node to be 
> deleted by Zookeeper and retry (kafka.utils.ZkUtils$)
> where the broker is essentially stuck in the loop that is trying to deal with 
> left-over ephemeral nodes. The code looks a bit racy to me. In particular:
> ZookeeperLeaderElector:
>   def elect: Boolean = {
> controllerContext.zkClient.subscribeDataChanges(electionPath, 
> leaderChangeListener)
> val timestamp = SystemTime.milliseconds.toString
> val electString = ...
> try {
>   
> createEphemeralPathExpectConflictHandleZKBug(controllerContext.zkClient, 
> electionPath, electString, leaderId,
> (controllerString : String, leaderId : Any) => 
> KafkaController.parseControllerId(controllerString) == 
> leaderId.asInstanceOf[Int],
> controllerContext.zkSessionTimeout)
> leaderChangeListener is registered before the create call (by the way, it 
> looks like a new registration will be added every elect call - shouldn't it 
> register in startup()?) so can update leaderId to the current leader before 
> the call to create. If that happens then we will continuously get node exists 
> exceptions and the checker function will always return true, i.e. we will 
> never get out of the while(true) loop.
> I think the right fix here is to pass brokerId instead of leaderId when 
> calling create, i.e.
> createEphemeralPathExpectConflictHandleZKBug(controllerContext.zkClient, 
> electionPath, electString, brokerId,
> (controllerString : String, leaderId : Any) => 
> KafkaController.parseControllerId(controllerString) == 
> leaderId.asInstanceOf[Int],
> controllerContext.zkSessionTimeout)
> The loop dealing with the ephemeral node bug is now only triggered for the 
> broker that owned the node previously, although I am still not 100% sure if 
> that is sufficient.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1382) Update zkVersion on partition state update failures

2014-04-29 Thread James Blackburn (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1382?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984416#comment-13984416
 ] 

James Blackburn commented on KAFKA-1382:


The other thing about this is that it can quickly churn through log.  There are 
log lines every few milliseconds at INFO level leading to  GBs of log in a very 
short time, e.g.:

{code}
[2014-04-28 00:01:37,010] INFO Partition [RSF_OPTIONS,10] on broker 1: Cached 
zkVersion [21] not equal to that in zookeeper, ski
p updating ISR (kafka.cluster.Partition)
[2014-04-28 00:01:37,017] INFO Partition [RSF_OPTIONS,10] on broker 1: 
Expanding ISR for partition [RSF_OPTIONS,10] from 1 to 1,
0 (kafka.cluster.Partition)
[2014-04-28 00:01:37,019] ERROR Conditional update of path 
/brokers/topics/RSF_OPTIONS/partitions/10/state with data {"controlle
r_epoch":19,"leader":1,"version":1,"leader_epoch":6,"isr":[1,0]} and expected 
version 21 failed due to org.apache.zookeeper.Keep
erException$BadVersionException: KeeperErrorCode = BadVersion for 
/brokers/topics/RSF_OPTIONS/partitions/10/state (kafka.utils.Z
kUtils$)
[2014-04-28 00:01:37,019] INFO Partition [RSF_OPTIONS,10] on broker 1: Cached 
zkVersion [21] not equal to that in zookeeper, ski
p updating ISR (kafka.cluster.Partition)
[2014-04-28 00:01:37,019] INFO Partition [RSF,14] on broker 1: Expanding ISR 
for partition [RSF,14] from 1 to 1,0 (kafka.cluster
.Partition)
[2014-04-28 00:01:37,020] ERROR Conditional update of path 
/brokers/topics/RSF/partitions/14/state with data {"controller_epoch"
:19,"leader":1,"version":1,"leader_epoch":6,"isr":[1,0]} and expected version 
21 failed due to org.apache.zookeeper.KeeperExcept
ion$BadVersionException: KeeperErrorCode = BadVersion for 
/brokers/topics/RSF/partitions/14/state (kafka.utils.ZkUtils$)
[2014-04-28 00:01:37,020] INFO Partition [RSF,14] on broker 1: Cached zkVersion 
[21] not equal to that in zookeeper, skip updati
ng ISR (kafka.cluster.Partition)
[2014-04-28 00:01:37,035] INFO Partition [RSF_OPTIONS,10] on broker 1: 
Expanding ISR for partition [RSF_OPTIONS,10] from 1 to 1,
0 (kafka.cluster.Partition)
[2014-04-28 00:01:37,037] ERROR Conditional update of path 
/brokers/topics/RSF_OPTIONS/partitions/10/state with data {"controlle
r_epoch":19,"leader":1,"version":1,"leader_epoch":6,"isr":[1,0]} and expected 
version 21 failed due to org.apache.zookeeper.Keep
erException$BadVersionException: KeeperErrorCode = BadVersion for 
/brokers/topics/RSF_OPTIONS/partitions/10/state (kafka.utils.Z
kUtils$)
[2014-04-28 00:01:37,037] INFO Partition [RSF_OPTIONS,10] on broker 1: Cached 
zkVersion [21] not equal to that in zookeeper, ski
p updating ISR (kafka.cluster.Partition)
[2014-04-28 00:01:37,037] INFO Partition [RSF,14] on broker 1: Expanding ISR 
for partition [RSF,14] from 1 to 1,0 (kafka.cluster
.Partition)
{code}

> Update zkVersion on partition state update failures
> ---
>
> Key: KAFKA-1382
> URL: https://issues.apache.org/jira/browse/KAFKA-1382
> Project: Kafka
>  Issue Type: Bug
>Reporter: Joel Koshy
> Fix For: 0.8.2
>
>
> Our updateIsr code is currently:
>   private def updateIsr(newIsr: Set[Replica]) {
> debug("Updated ISR for partition [%s,%d] to %s".format(topic, 
> partitionId, newIsr.mkString(",")))
> val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, 
> newIsr.map(r => r.brokerId).toList, zkVersion)
> // use the epoch of the controller that made the leadership decision, 
> instead of the current controller epoch
> val (updateSucceeded, newVersion) = 
> ZkUtils.conditionalUpdatePersistentPath(zkClient,
>   ZkUtils.getTopicPartitionLeaderAndIsrPath(topic, partitionId),
>   ZkUtils.leaderAndIsrZkData(newLeaderAndIsr, controllerEpoch), zkVersion)
> if (updateSucceeded){
>   inSyncReplicas = newIsr
>   zkVersion = newVersion
>   trace("ISR updated to [%s] and zkVersion updated to 
> [%d]".format(newIsr.mkString(","), zkVersion))
> } else {
>   info("Cached zkVersion [%d] not equal to that in zookeeper, skip 
> updating ISR".format(zkVersion))
> }
> We encountered an interesting scenario recently when a large producer fully
> saturated the broker's NIC for over an hour. The large volume of data led to
> a number of ISR shrinks (and subsequent expands). The NIC saturation
> affected the zookeeper client heartbeats and led to a session timeout. The
> timeline was roughly as follows:
> - Attempt to expand ISR
> - Expansion written to zookeeper (confirmed in zookeeper transaction logs)
> - Session timeout after around 13 seconds (the configured timeout is 20
>   seconds) so that lines up.
> - zkclient reconnects to zookeeper (with the same session ID) and retries
>   the write - but uses the old zkVersion. This fails because the zkVersion
>   has already been updated (above

[ANNOUNCEMENT] Apache Kafka 0.8.1.1 Released

2014-04-29 Thread Joe Stein
The Apache Kafka community is pleased to announce the general availability 
release of Apache Kafka 0.8.1.1.

The 0.8.1.1 is a maintenance release with bug fixes for the 0.8.1 release.
Changes in this release: 
https://archive.apache.org/dist/kafka/0.8.1.1/RELEASE_NOTES.html.

Apache Kafka is high-throughput, publish-subscribe messaging system rethought 
of as a distributed commit log.

** Fast => A single Kafka broker can handle hundreds of megabytes of reads and 
writes per second from thousands of clients.

** Scalable => Kafka is designed to allow a single cluster to serve as the 
central data backbone 
for a large organization. It can be elastically and transparently expanded 
without downtime. 
Data streams are partitioned and spread over a cluster of machines to allow 
data streams 
larger than the capability of any single machine and to allow clusters of 
co-ordinated consumers.

** Durable => Messages are persisted on disk and replicated within the cluster 
to prevent 
data loss. Each broker can handle terabytes of messages without performance 
impact.

** Distributed by Design => Kafka has a modern cluster-centric design that 
offers 
strong durability and fault-tolerance guarantees.

You can download the release from: http://kafka.apache.org/downloads.html

We welcome your help and feedback. For more information on how to
report problems, and to get involved, visit the project website at 
http://kafka.apache.org/



[jira] [Commented] (KAFKA-1354) Failed to load class "org.slf4j.impl.StaticLoggerBinder"

2014-04-29 Thread Sharmarke Aden (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1354?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984431#comment-13984431
 ] 

Sharmarke Aden commented on KAFKA-1354:
---

As long as the client library uses slf4j api to log messages it only needs to 
include the slf4j-api jar. 

Broker Dependencies:
slf4j-api.jar
slf4j-log4j12.jar
log4j.jar

Client Dependencies:
slf4j-api

For third party dependencies that use other logging frameworks both the client 
or the broker will also need to include slf4j bridge dependencies. For example, 
if a third party library used by broker/client uses commons logging then you 
should also distribute jcl-over-slf4j.jar

> Failed to load class "org.slf4j.impl.StaticLoggerBinder"
> 
>
> Key: KAFKA-1354
> URL: https://issues.apache.org/jira/browse/KAFKA-1354
> Project: Kafka
>  Issue Type: Bug
>  Components: log
>Affects Versions: 0.8.1
> Environment: RHEL
>Reporter: RakeshAcharya
>Assignee: Jay Kreps
>  Labels: newbie, patch, usability
> Fix For: 0.8.2
>
>   Original Estimate: 672h
>  Remaining Estimate: 672h
>
> Getting below errors during Kafka startup
> SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
> SLF4J: Defaulting to no-operation (NOP) logger implementation
> SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further 
> details.
> [2014-03-31 18:55:36,488] INFO Will not load MX4J, mx4j-tools.jar is not in 
> the classpath (kafka.utils.Mx4jLoader$)



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Comment Edited] (KAFKA-1354) Failed to load class "org.slf4j.impl.StaticLoggerBinder"

2014-04-29 Thread Sharmarke Aden (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1354?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984431#comment-13984431
 ] 

Sharmarke Aden edited comment on KAFKA-1354 at 4/29/14 4:00 PM:


As long as the client library uses slf4j api to log messages it only needs to 
include the slf4j-api jar. This way the user has the freedom to choose their 
own logging framework and add their slf4j bridges or bindings to their 
classpath.

Broker Dependencies:
slf4j-api.jar
slf4j-log4j12.jar
log4j.jar

Client Dependencies:
slf4j-api

For third party dependencies that use other logging frameworks the client and 
the broker will also need to include slf4j bridge dependencies. For example, if 
a third party library used by broker/client uses commons logging then you 
should also distribute jcl-over-slf4j.jar.


was (Author: saden1):
As long as the client library uses slf4j api to log messages it only needs to 
include the slf4j-api jar. This way the user has the freedom to choose their 
own logging framework and add their slf4j bridges or bindings to their 
classpath.

Broker Dependencies:
slf4j-api.jar
slf4j-log4j12.jar
log4j.jar

Client Dependencies:
slf4j-api

For third party dependencies that use other logging frameworks client and the 
broker will also need to include slf4j bridge dependencies. For example, if a 
third party library used by broker/client uses commons logging then you should 
also distribute jcl-over-slf4j.jar.

> Failed to load class "org.slf4j.impl.StaticLoggerBinder"
> 
>
> Key: KAFKA-1354
> URL: https://issues.apache.org/jira/browse/KAFKA-1354
> Project: Kafka
>  Issue Type: Bug
>  Components: log
>Affects Versions: 0.8.1
> Environment: RHEL
>Reporter: RakeshAcharya
>Assignee: Jay Kreps
>  Labels: newbie, patch, usability
> Fix For: 0.8.2
>
>   Original Estimate: 672h
>  Remaining Estimate: 672h
>
> Getting below errors during Kafka startup
> SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
> SLF4J: Defaulting to no-operation (NOP) logger implementation
> SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further 
> details.
> [2014-03-31 18:55:36,488] INFO Will not load MX4J, mx4j-tools.jar is not in 
> the classpath (kafka.utils.Mx4jLoader$)



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Comment Edited] (KAFKA-1354) Failed to load class "org.slf4j.impl.StaticLoggerBinder"

2014-04-29 Thread Sharmarke Aden (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1354?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984431#comment-13984431
 ] 

Sharmarke Aden edited comment on KAFKA-1354 at 4/29/14 3:59 PM:


As long as the client library uses slf4j api to log messages it only needs to 
include the slf4j-api jar. This way the user has the freedom to choose their 
own logging framework and add their slf4j bridges or bindings to their 
classpath.

Broker Dependencies:
slf4j-api.jar
slf4j-log4j12.jar
log4j.jar

Client Dependencies:
slf4j-api

For third party dependencies that use other logging frameworks client and the 
broker will also need to include slf4j bridge dependencies. For example, if a 
third party library used by broker/client uses commons logging then you should 
also distribute jcl-over-slf4j.jar.


was (Author: saden1):
As long as the client library uses slf4j api to log messages it only needs to 
include the slf4j-api jar. 

Broker Dependencies:
slf4j-api.jar
slf4j-log4j12.jar
log4j.jar

Client Dependencies:
slf4j-api

For third party dependencies that use other logging frameworks both the client 
or the broker will also need to include slf4j bridge dependencies. For example, 
if a third party library used by broker/client uses commons logging then you 
should also distribute jcl-over-slf4j.jar

> Failed to load class "org.slf4j.impl.StaticLoggerBinder"
> 
>
> Key: KAFKA-1354
> URL: https://issues.apache.org/jira/browse/KAFKA-1354
> Project: Kafka
>  Issue Type: Bug
>  Components: log
>Affects Versions: 0.8.1
> Environment: RHEL
>Reporter: RakeshAcharya
>Assignee: Jay Kreps
>  Labels: newbie, patch, usability
> Fix For: 0.8.2
>
>   Original Estimate: 672h
>  Remaining Estimate: 672h
>
> Getting below errors during Kafka startup
> SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
> SLF4J: Defaulting to no-operation (NOP) logger implementation
> SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further 
> details.
> [2014-03-31 18:55:36,488] INFO Will not load MX4J, mx4j-tools.jar is not in 
> the classpath (kafka.utils.Mx4jLoader$)



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Comment Edited] (KAFKA-1354) Failed to load class "org.slf4j.impl.StaticLoggerBinder"

2014-04-29 Thread Sharmarke Aden (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1354?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984431#comment-13984431
 ] 

Sharmarke Aden edited comment on KAFKA-1354 at 4/29/14 4:02 PM:


As long as the client library uses slf4j api to log messages it only needs to 
include the slf4j-api jar. This way the user has the freedom to choose their 
own logging framework and add their slf4j bridges or bindings to their 
classpath.

Broker Dependencies:
slf4j-api.jar
slf4j-log4j12.jar
log4j.jar

Client Dependencies:
slf4j-api

For third party dependencies that use other logging frameworks the client and 
the broker will also need to include slf4j bridge dependencies. For example, if 
a third party library broker/client dependency uses commons logging then you 
should also distribute jcl-over-slf4j.jar.


was (Author: saden1):
As long as the client library uses slf4j api to log messages it only needs to 
include the slf4j-api jar. This way the user has the freedom to choose their 
own logging framework and add their slf4j bridges or bindings to their 
classpath.

Broker Dependencies:
slf4j-api.jar
slf4j-log4j12.jar
log4j.jar

Client Dependencies:
slf4j-api

For third party dependencies that use other logging frameworks the client and 
the broker will also need to include slf4j bridge dependencies. For example, if 
a third party library used by broker/client uses commons logging then you 
should also distribute jcl-over-slf4j.jar.

> Failed to load class "org.slf4j.impl.StaticLoggerBinder"
> 
>
> Key: KAFKA-1354
> URL: https://issues.apache.org/jira/browse/KAFKA-1354
> Project: Kafka
>  Issue Type: Bug
>  Components: log
>Affects Versions: 0.8.1
> Environment: RHEL
>Reporter: RakeshAcharya
>Assignee: Jay Kreps
>  Labels: newbie, patch, usability
> Fix For: 0.8.2
>
>   Original Estimate: 672h
>  Remaining Estimate: 672h
>
> Getting below errors during Kafka startup
> SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
> SLF4J: Defaulting to no-operation (NOP) logger implementation
> SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further 
> details.
> [2014-03-31 18:55:36,488] INFO Will not load MX4J, mx4j-tools.jar is not in 
> the classpath (kafka.utils.Mx4jLoader$)



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Comment Edited] (KAFKA-1354) Failed to load class "org.slf4j.impl.StaticLoggerBinder"

2014-04-29 Thread Sharmarke Aden (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1354?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984431#comment-13984431
 ] 

Sharmarke Aden edited comment on KAFKA-1354 at 4/29/14 4:03 PM:


As long as the client library uses slf4j api to log messages it only needs to 
include the slf4j-api jar. This way the user has the freedom to choose their 
own logging framework and add their slf4j bridges or bindings to their 
classpath.

Broker Dependencies:
slf4j-api.jar
slf4j-log4j12.jar
log4j.jar

Client Dependencies:
slf4j-api

For third party dependencies that use other logging frameworks the client and 
the broker will also need to include slf4j bridge dependencies. For example, if 
a third party library broker/client dependency uses commons logging then you 
should also distribute jcl-over-slf4j.jar.




was (Author: saden1):
As long as the client library uses slf4j api to log messages it only needs to 
include the slf4j-api jar. This way the user has the freedom to choose their 
own logging framework and add their slf4j bridges or bindings to their 
classpath.

Broker Dependencies:
slf4j-api.jar
slf4j-log4j12.jar
log4j.jar

Client Dependencies:
slf4j-api

For third party dependencies that use other logging frameworks the client and 
the broker will also need to include slf4j bridge dependencies. For example, if 
a third party library broker/client dependency uses commons logging then you 
should also distribute jcl-over-slf4j.jar.

> Failed to load class "org.slf4j.impl.StaticLoggerBinder"
> 
>
> Key: KAFKA-1354
> URL: https://issues.apache.org/jira/browse/KAFKA-1354
> Project: Kafka
>  Issue Type: Bug
>  Components: log
>Affects Versions: 0.8.1
> Environment: RHEL
>Reporter: RakeshAcharya
>Assignee: Jay Kreps
>  Labels: newbie, patch, usability
> Fix For: 0.8.2
>
>   Original Estimate: 672h
>  Remaining Estimate: 672h
>
> Getting below errors during Kafka startup
> SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
> SLF4J: Defaulting to no-operation (NOP) logger implementation
> SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further 
> details.
> [2014-03-31 18:55:36,488] INFO Will not load MX4J, mx4j-tools.jar is not in 
> the classpath (kafka.utils.Mx4jLoader$)



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1426) Rebalance issue on 0.8.1

2014-04-29 Thread Jeremy A Laycock (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1426?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984463#comment-13984463
 ] 

Jeremy A Laycock commented on KAFKA-1426:
-

Ahh  think I have the reason - it's the inter-play between specifying a groupId 
and --from-beginning option on the console consumer.  Setting that option 
clears the /consumers zk directory. In turn that then affects re-balance and 
all other connected consumers.

> Rebalance issue on 0.8.1
> 
>
> Key: KAFKA-1426
> URL: https://issues.apache.org/jira/browse/KAFKA-1426
> Project: Kafka
>  Issue Type: Bug
>  Components: consumer
>Affects Versions: 0.8.1
>Reporter: Jeremy A Laycock
>Assignee: Neha Narkhede
>Priority: Minor
>
> When running up 2 console consumers reading from the same group the zookeeper 
> path  /consumers//ids only contains the latest consumer id - then 
> re-balance fails.
> Looking at the rebalance code the only deletion is the 
> /consumers//owners// for each partition a 
> consumer "owns"



--
This message was sent by Atlassian JIRA
(v6.2#6252)


Re: Review Request 20745: Patch for KAFKA-1397

2014-04-29 Thread Jun Rao

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/20745/#review41714
---


Some comments:

1. Should we restore the commented out test in ServerShutdownTest.scala?


core/src/main/scala/kafka/controller/ControllerChannelManager.scala


Is it better to do the check here or in the caller?



core/src/main/scala/kafka/controller/TopicDeletionManager.scala


The shutdown sequence is a bit complicated now. Could you add some comments 
to describe the sequence of actions?



core/src/main/scala/kafka/controller/TopicDeletionManager.scala


Do we need to check here at all since the caller of doWork() already does 
the check?



core/src/main/scala/kafka/controller/TopicDeletionManager.scala


Do we need to introduce isAnyReplicaInState? replicasInState() seems more 
general and we can just check the output to implement isAnyReplicaInState.



core/src/main/scala/kafka/server/KafkaApis.scala


It's better to do this in the ReplicaManager when handling the 
StopReplicaRequest. Then we don't have to expose LogManager to KafkaApis.



core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala


There seems to be no guarantee that the delete topic process is completed 
before the controller was shutdown. So, I am not sure how reliable the test is.



core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala


The comment is wrong. We are shutting down the leader, not the controller.



core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala


Could we explicitly specify whether leaderIdOpt is for the old leader or 
the new leader in waitUntilLeaderIsElectedOrChanged()?



core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala


I am not sure how reliable the test is since since by the time delete topic 
is started, the preferred leader election could have completed.



core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala


Again, this test is not reliable. There are multiple possible outcomes.

1. Delete topic completes before partition reassignment is started. In this 
case partition reassignment should fail. Then, we shouldn't verify the 
reassigned replicas since the topic doesn't exist.

2. Reassignment is started/completed before the topic deletion completes.

I am not sure which case this test is for.



core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala


Same comment as the above. I am not sure how we guarantee whether partition 
reassignment or topic deletion completes first.



core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala


Not sure if this adds any value than the previous send.


- Jun Rao


On April 29, 2014, 12:08 a.m., Timothy Chen wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/20745/
> ---
> 
> (Updated April 29, 2014, 12:08 a.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1397
> https://issues.apache.org/jira/browse/KAFKA-1397
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> Fix delete topic tests and deadlock
> 
> 
> Diffs
> -
> 
>   core/src/main/scala/kafka/controller/ControllerChannelManager.scala 
> 919aeb26f93d2fc34d873cfb3dbfab7235a9d635 
>   core/src/main/scala/kafka/controller/KafkaController.scala 
> 933de9dd324c7086efe6aa610335ef370d9e9c12 
>   core/src/main/scala/kafka/controller/ReplicaStateMachine.scala 
> 0e47dac8cbf65a86d053a3371a18af467afd70ae 
>   core/src/main/scala/kafka/controller/TopicDeletionManager.scala 
> e4bc2439ce1933c7c7571d255464ee678226a6cb 
>   core/src/main/scala/kafka/log/LogManager.scala 
> ac67f081e6219fd2181479e7a2bb88ea6044e6cc 
>   core/src/main/scala/kafka/server/KafkaApis.scala 
> 0b668f230c8556fdf08654ce522a11847d0bf39b 
>   core/src/main/scala/kafka/server/KafkaServer.scala 
> c208f83bed7fb91f07fae42f2b66892e6d46fecc 
>   core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala 
> 9c29e144bba2c9bafa91941b6ca5c263490693b3 
>   core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 
> b1c4ce9f66aa86c68f2e6988f67546fa63ed1f9b 
> 
> Diff: https://reviews.apache.org/r/20745/diff/
> 
> 
> Testing
> ---
> 
>

Re: Review Request 20745: Patch for KAFKA-1397

2014-04-29 Thread Jun Rao


> On April 28, 2014, 7:43 p.m., Neha Narkhede wrote:
> > core/src/main/scala/kafka/controller/ControllerChannelManager.scala, line 
> > 210
> > 
> >
> > better to not leak the logic from delete topic manager here. It is 
> > worth exposing the check as an API in DeleteTopicManager. In any case, can 
> > you explain the motivation behind your change?
> 
> Timothy Chen wrote:
> Sorry forgot to add more comments to the rb, have been chatting mostly 
> with Jun about my fixes.
> One test that was failing in delete topic test was a test that was 
> handling request while topic being deleted, and it made sure it can never 
> finish by shutting down one of the follower broker. Sending a fetch request 
> to a topic being deleted returns a NotLeaderForPartition error code instead 
> of expected UnknownTopicOrPartition.
>  
> The cause of this is that as part of the metadata cache changes in Kafka 
> api, we removed the check if a topic exists in the cache when serving 
> fetching request and now it tries to read message set from the replica 
> manager, and even though a stop replica request with delete partition is 
> already been sent and processed, the partition was somehow still in replica 
> manager and it returns NotLeaderForPartition exception since there is no 
> leader for this topic.
> 
> I found out that the partition was being recreated in the replica 
> manager, because in the process of retrying deleting the same topic it change 
> the failed replica state into OfflineReplica again that triggers a 
> LeaderAndIsrRequest to all brokers for shrinking the Isr for that replica. 
> 
> The brokers that received the LeaderAndIsrRequest recreates the partition 
> with getOrCreatePartition, and therefore future fetch requests hits it and 
> throws that error code.
> 
> I think the correct thing to do is to not send LeaderAndIsrRequest if the 
> topic is being deleted when bring replica to an offline state, as it is not 
> necessary anymore.
>

Could you explain the situation in a bit more details? It would be good to 
include the sequence of LeaderIsr and StopReplica requests that the broker 
sees. In particular, I am not sure why a partition still exists in replica 
manager if a stop replica request with delete partition has already been 
received.


- Jun


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/20745/#review41626
---


On April 29, 2014, 12:08 a.m., Timothy Chen wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/20745/
> ---
> 
> (Updated April 29, 2014, 12:08 a.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1397
> https://issues.apache.org/jira/browse/KAFKA-1397
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> Fix delete topic tests and deadlock
> 
> 
> Diffs
> -
> 
>   core/src/main/scala/kafka/controller/ControllerChannelManager.scala 
> 919aeb26f93d2fc34d873cfb3dbfab7235a9d635 
>   core/src/main/scala/kafka/controller/KafkaController.scala 
> 933de9dd324c7086efe6aa610335ef370d9e9c12 
>   core/src/main/scala/kafka/controller/ReplicaStateMachine.scala 
> 0e47dac8cbf65a86d053a3371a18af467afd70ae 
>   core/src/main/scala/kafka/controller/TopicDeletionManager.scala 
> e4bc2439ce1933c7c7571d255464ee678226a6cb 
>   core/src/main/scala/kafka/log/LogManager.scala 
> ac67f081e6219fd2181479e7a2bb88ea6044e6cc 
>   core/src/main/scala/kafka/server/KafkaApis.scala 
> 0b668f230c8556fdf08654ce522a11847d0bf39b 
>   core/src/main/scala/kafka/server/KafkaServer.scala 
> c208f83bed7fb91f07fae42f2b66892e6d46fecc 
>   core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala 
> 9c29e144bba2c9bafa91941b6ca5c263490693b3 
>   core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 
> b1c4ce9f66aa86c68f2e6988f67546fa63ed1f9b 
> 
> Diff: https://reviews.apache.org/r/20745/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Timothy Chen
> 
>



[jira] [Commented] (KAFKA-1426) Rebalance issue on 0.8.1

2014-04-29 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1426?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984513#comment-13984513
 ] 

Jun Rao commented on KAFKA-1426:


Thanks for finding this out. It's confusing that we auto delete the consumer 
path if --from-beginning is specified. Perhaps it's better to add a separate 
option to clean the consumer path in ZK. Could you close this jira and file a 
separate jira to fix console consumer?

> Rebalance issue on 0.8.1
> 
>
> Key: KAFKA-1426
> URL: https://issues.apache.org/jira/browse/KAFKA-1426
> Project: Kafka
>  Issue Type: Bug
>  Components: consumer
>Affects Versions: 0.8.1
>Reporter: Jeremy A Laycock
>Assignee: Neha Narkhede
>Priority: Minor
>
> When running up 2 console consumers reading from the same group the zookeeper 
> path  /consumers//ids only contains the latest consumer id - then 
> re-balance fails.
> Looking at the rebalance code the only deletion is the 
> /consumers//owners// for each partition a 
> consumer "owns"



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1354) Failed to load class "org.slf4j.impl.StaticLoggerBinder"

2014-04-29 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1354?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984523#comment-13984523
 ] 

Jun Rao commented on KAFKA-1354:


That makes sense. The problem is that currently the kafka jar is for both the 
sever and the client. We are creating the new clients in a separate jar. 
Perhaps we can revisit this when the old clients are phased out.

> Failed to load class "org.slf4j.impl.StaticLoggerBinder"
> 
>
> Key: KAFKA-1354
> URL: https://issues.apache.org/jira/browse/KAFKA-1354
> Project: Kafka
>  Issue Type: Bug
>  Components: log
>Affects Versions: 0.8.1
> Environment: RHEL
>Reporter: RakeshAcharya
>Assignee: Jay Kreps
>  Labels: newbie, patch, usability
> Fix For: 0.8.2
>
>   Original Estimate: 672h
>  Remaining Estimate: 672h
>
> Getting below errors during Kafka startup
> SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
> SLF4J: Defaulting to no-operation (NOP) logger implementation
> SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further 
> details.
> [2014-03-31 18:55:36,488] INFO Will not load MX4J, mx4j-tools.jar is not in 
> the classpath (kafka.utils.Mx4jLoader$)



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1029) Zookeeper leader election stuck in ephemeral node retry loop

2014-04-29 Thread Jason Rosenberg (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1029?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984631#comment-13984631
 ] 

Jason Rosenberg commented on KAFKA-1029:


I'll add that we still see this from time to time (we're still on 0.8.0), and 
it's usually after an abnormal event, such as a failed meta-data request for a 
large group of topics, that times out, etc.  But once this happens, it's very 
difficult to make it go away, other than restarting the consumer.

> Zookeeper leader election stuck in ephemeral node retry loop
> 
>
> Key: KAFKA-1029
> URL: https://issues.apache.org/jira/browse/KAFKA-1029
> Project: Kafka
>  Issue Type: Bug
>  Components: controller
>Affects Versions: 0.8.0
>Reporter: Sam Meder
>Assignee: Sam Meder
>Priority: Blocker
> Fix For: 0.8.0
>
> Attachments: 
> 0002-KAFKA-1029-Use-brokerId-instead-of-leaderId-when-tri.patch
>
>
> We're seeing the following log statements (over and over):
> [2013-08-27 07:21:49,538] INFO conflict in /controller data: { "brokerid":3, 
> "timestamp":"1377587945206", "version":1 } stored data: { "brokerid":2, 
> "timestamp":"1377587460904", "version":1 } (kafka.utils.ZkUtils$)
> [2013-08-27 07:21:49,559] INFO I wrote this conflicted ephemeral node [{ 
> "brokerid":3, "timestamp":"1377587945206", "version":1 }] at /controller a 
> while back in a different session, hence I will backoff for this node to be 
> deleted by Zookeeper and retry (kafka.utils.ZkUtils$)
> where the broker is essentially stuck in the loop that is trying to deal with 
> left-over ephemeral nodes. The code looks a bit racy to me. In particular:
> ZookeeperLeaderElector:
>   def elect: Boolean = {
> controllerContext.zkClient.subscribeDataChanges(electionPath, 
> leaderChangeListener)
> val timestamp = SystemTime.milliseconds.toString
> val electString = ...
> try {
>   
> createEphemeralPathExpectConflictHandleZKBug(controllerContext.zkClient, 
> electionPath, electString, leaderId,
> (controllerString : String, leaderId : Any) => 
> KafkaController.parseControllerId(controllerString) == 
> leaderId.asInstanceOf[Int],
> controllerContext.zkSessionTimeout)
> leaderChangeListener is registered before the create call (by the way, it 
> looks like a new registration will be added every elect call - shouldn't it 
> register in startup()?) so can update leaderId to the current leader before 
> the call to create. If that happens then we will continuously get node exists 
> exceptions and the checker function will always return true, i.e. we will 
> never get out of the while(true) loop.
> I think the right fix here is to pass brokerId instead of leaderId when 
> calling create, i.e.
> createEphemeralPathExpectConflictHandleZKBug(controllerContext.zkClient, 
> electionPath, electString, brokerId,
> (controllerString : String, leaderId : Any) => 
> KafkaController.parseControllerId(controllerString) == 
> leaderId.asInstanceOf[Int],
> controllerContext.zkSessionTimeout)
> The loop dealing with the ephemeral node bug is now only triggered for the 
> broker that owned the node previously, although I am still not 100% sure if 
> that is sufficient.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


fetch.wait.max.ms handling racy?

2014-04-29 Thread Sam Meder
Am I missing something or is the handling of fetch.wait.max.ms racy? In 0.8 
code I don't see any locking (might just be missing it) around the initial 
check to see if we can satisfy the event and enqueuing a delayed request in the 
purgatory, so if data arrives that satisfies the request it might be missed. Is 
that by design?

/Sam

[jira] [Commented] (KAFKA-1300) Added WaitForReplaction admin tool.

2014-04-29 Thread Brenden Matthews (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1300?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13984919#comment-13984919
 ] 

Brenden Matthews commented on KAFKA-1300:
-

Bump!

Anyone interested in this?  Presumably this would be valuable to others.

> Added WaitForReplaction admin tool.
> ---
>
> Key: KAFKA-1300
> URL: https://issues.apache.org/jira/browse/KAFKA-1300
> Project: Kafka
>  Issue Type: New Feature
>  Components: tools
>Affects Versions: 0.8.0
> Environment: Ubuntu 12.04
>Reporter: Brenden Matthews
>  Labels: patch
> Fix For: 0.8.1
>
> Attachments: 0001-Added-WaitForReplaction-admin-tool.patch
>
>
> I have created a tool similar to the broker shutdown tool for doing rolling 
> restarts of Kafka clusters.
> The tool watches the max replica lag of the specified broker, and waits until 
> the lag drops to 0 before exiting.
> To do a rolling restart, here's the process we use:
> for (broker <- brokers) {
>   run shutdown tool for broker
>   terminate broker
>   start new broker
>   run wait for replication tool on new broker
> }
> Here's an example command line use:
> ./kafka-run-class.sh kafka.admin.WaitForReplication --zookeeper 
> zk.host.com:2181 --num.retries 100 --retry.interval.ms 6 --broker 0



--
This message was sent by Atlassian JIRA
(v6.2#6252)


Build failed in Jenkins: Kafka-trunk #178

2014-04-29 Thread Apache Jenkins Server
See 

Changes:

[junrao] kafka-1400; transient unit test failure in SocketServerTest; patched 
by Neha Narkhede; reviewed by Guozhang and Jun Rao

--
[...truncated 332 lines...]

kafka.producer.AsyncProducerTest > testPartitionAndCollateEvents PASSED

kafka.producer.AsyncProducerTest > testSerializeEvents PASSED

kafka.producer.AsyncProducerTest > testInvalidPartition PASSED

kafka.producer.AsyncProducerTest > testNoBroker PASSED

kafka.producer.AsyncProducerTest > testIncompatibleEncoder PASSED

kafka.producer.AsyncProducerTest > testRandomPartitioner PASSED

kafka.producer.AsyncProducerTest > testFailedSendRetryLogic PASSED

kafka.producer.AsyncProducerTest > testJavaProducer PASSED

kafka.producer.AsyncProducerTest > testInvalidConfiguration PASSED

kafka.producer.ProducerTest > testUpdateBrokerPartitionInfo PASSED

kafka.producer.ProducerTest > testSendToNewTopic PASSED

kafka.producer.ProducerTest > testSendWithDeadBroker PASSED

kafka.producer.ProducerTest > testAsyncSendCanCorrectlyFailWithTimeout PASSED

kafka.producer.ProducerTest > testSendNullMessage PASSED

kafka.log4j.KafkaLog4jAppenderTest > testKafkaLog4jConfigs PASSED

kafka.log4j.KafkaLog4jAppenderTest > testLog4jAppends PASSED

kafka.admin.AdminTest > testReplicaAssignment PASSED

kafka.admin.AdminTest > testManualReplicaAssignment PASSED

kafka.admin.AdminTest > testTopicCreationInZK PASSED

kafka.admin.AdminTest > testPartitionReassignmentWithLeaderInNewReplicas PASSED

kafka.admin.AdminTest > testPartitionReassignmentWithLeaderNotInNewReplicas 
PASSED

kafka.admin.AdminTest > testPartitionReassignmentNonOverlappingReplicas PASSED

kafka.admin.AdminTest > testReassigningNonExistingPartition PASSED

kafka.admin.AdminTest > testResumePartitionReassignmentThatWasCompleted PASSED

kafka.admin.AdminTest > testPreferredReplicaJsonData PASSED

kafka.admin.AdminTest > testBasicPreferredReplicaElection PASSED

kafka.admin.AdminTest > testShutdownBroker PASSED

kafka.admin.AdminTest > testTopicConfigChange PASSED

kafka.admin.AddPartitionsTest > testTopicDoesNotExist PASSED

kafka.admin.AddPartitionsTest > testWrongReplicaCount PASSED

kafka.admin.AddPartitionsTest > testIncrementPartitions PASSED

kafka.admin.AddPartitionsTest > testManualAssignmentOfReplicas PASSED

kafka.admin.AddPartitionsTest > testReplicaPlacement PASSED

kafka.admin.DeleteTopicTest > testFake PASSED

kafka.api.ProducerFailureHandlingTest > testInvalidPartition PASSED

kafka.api.ProducerFailureHandlingTest > testTooLargeRecordWithAckZero PASSED

kafka.api.ProducerFailureHandlingTest > testTooLargeRecordWithAckOne PASSED

kafka.api.ProducerFailureHandlingTest > testNonExistTopic PASSED

kafka.api.ProducerFailureHandlingTest > testWrongBrokerList PASSED

kafka.api.ProducerFailureHandlingTest > testNoResponse PASSED

kafka.api.ProducerFailureHandlingTest > testSendAfterClosed PASSED

kafka.api.RequestResponseSerializationTest > 
testSerializationAndDeserialization PASSED

kafka.api.ApiUtilsTest > testShortStringNonASCII PASSED

kafka.api.ApiUtilsTest > testShortStringASCII PASSED

kafka.api.test.ProducerSendTest > testSendOffset PASSED

kafka.api.test.ProducerSendTest > testClose PASSED

kafka.api.test.ProducerSendTest > testSendToPartition PASSED

kafka.api.test.ProducerSendTest > testAutoCreateTopic PASSED

kafka.api.test.ProducerCompressionTest > testCompression[0] PASSED

kafka.api.test.ProducerCompressionTest > testCompression[1] PASSED

kafka.javaapi.consumer.ZookeeperConsumerConnectorTest > testBasic PASSED

kafka.javaapi.message.ByteBufferMessageSetTest > testWrittenEqualsRead PASSED

kafka.javaapi.message.ByteBufferMessageSetTest > testIteratorIsConsistent PASSED

kafka.javaapi.message.ByteBufferMessageSetTest > testSizeInBytes PASSED

kafka.javaapi.message.ByteBufferMessageSetTest > 
testIteratorIsConsistentWithCompression PASSED

kafka.javaapi.message.ByteBufferMessageSetTest > testSizeInBytesWithCompression 
PASSED

kafka.javaapi.message.ByteBufferMessageSetTest > testEquals PASSED

kafka.javaapi.message.ByteBufferMessageSetTest > testEqualsWithCompression 
PASSED

kafka.integration.UncleanLeaderElectionTest > testUncleanLeaderElectionEnabled 
PASSED

kafka.integration.UncleanLeaderElectionTest > testUncleanLeaderElectionDisabled 
PASSED

kafka.integration.UncleanLeaderElectionTest > 
testUncleanLeaderElectionEnabledByTopicOverride PASSED

kafka.integration.UncleanLeaderElectionTest > 
testCleanLeaderElectionDisabledByTopicOverride PASSED

kafka.integration.UncleanLeaderElectionTest > 
testUncleanLeaderElectionInvalidTopicOverride PASSED

kafka.integration.FetcherTest > testFetcher PASSED

kafka.integration.RollingBounceTest > testRollingBounce PASSED

kafka.integration.PrimitiveApiTest > testFetchRequestCanProperlySerialize PASSED

kafka.integration.PrimitiveApiTest > testEmptyFetchRequest PASSED

kafka.integration.PrimitiveApiTest > testDefaultEncoderProducerAndFetch PA

[jira] [Created] (KAFKA-1430) Purgatory redesign

2014-04-29 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-1430:
--

 Summary: Purgatory redesign
 Key: KAFKA-1430
 URL: https://issues.apache.org/jira/browse/KAFKA-1430
 Project: Kafka
  Issue Type: Improvement
  Components: core
Affects Versions: 0.8.2
Reporter: Jun Rao


We have seen 2 main issues with the Purgatory.

1. There is no atomic checkAndWatch functionality. So, a client typically first 
checks whether a request is satisfied or not and then register the watcher. 
However, by the time the watcher is registered, the registered item could 
already be satisfied. This item won't be satisfied until the next update 
happens or the delayed time expires, which means the watched item could be 
delayed. 

2. FetchRequestPurgatory doesn't quite work. This is because the current design 
tries to incrementally maintain the accumulated bytes ready for fetch. However, 
this is difficult since the right time to check whether a fetch (for regular 
consumer) request is satisfied is when the high watermark moves. At that point, 
it's hard to figure out how many bytes we should incrementally add to each 
pending fetch request.

The problem has been reported in KAFAK-1150 and KAFKA-703.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


Re: [jira] [Created] (KAFKA-1430) Purgatory redesign

2014-04-29 Thread Sam Meder
Thanks Jun, that answers my question. I'll see if we can find some time to work 
on these.

/Sam

On Apr 29, 2014, at 5:24 PM, Jun Rao (JIRA)  wrote:

> Jun Rao created KAFKA-1430:
> --
> 
> Summary: Purgatory redesign
> Key: KAFKA-1430
> URL: https://issues.apache.org/jira/browse/KAFKA-1430
> Project: Kafka
>  Issue Type: Improvement
>  Components: core
>Affects Versions: 0.8.2
>Reporter: Jun Rao
> 
> 
> We have seen 2 main issues with the Purgatory.
> 
> 1. There is no atomic checkAndWatch functionality. So, a client typically 
> first checks whether a request is satisfied or not and then register the 
> watcher. However, by the time the watcher is registered, the registered item 
> could already be satisfied. This item won't be satisfied until the next 
> update happens or the delayed time expires, which means the watched item 
> could be delayed. 
> 
> 2. FetchRequestPurgatory doesn't quite work. This is because the current 
> design tries to incrementally maintain the accumulated bytes ready for fetch. 
> However, this is difficult since the right time to check whether a fetch (for 
> regular consumer) request is satisfied is when the high watermark moves. At 
> that point, it's hard to figure out how many bytes we should incrementally 
> add to each pending fetch request.
> 
> The problem has been reported in KAFAK-1150 and KAFKA-703.
> 
> 
> 
> --
> This message was sent by Atlassian JIRA
> (v6.2#6252)



[jira] [Commented] (KAFKA-1430) Purgatory redesign

2014-04-29 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1430?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13985159#comment-13985159
 ] 

Jun Rao commented on KAFKA-1430:


The following is a proposed redesign of Purgatory.

1. In RequestPurgatory, we replace 
   watch(delayedRequest: T)
with 
   checkAndMaybeWatch(delayedRequest: T,  request: R): Boolean
checkAndMaybeWatch() will use request to check if delayedRequest can be 
satisfied immediately. If so, it will return true and will not add 
delayedRequest to the watcher. Otherwise, it will return false and add 
delayedRequest to the watcher. The check and add will be done atomically inside 
the synchronization point in Watchers.add.

2. Log
2.1 Change Log.append() so that it additionally returns the end file position 
of the last appended message.
2.2 Change Log.read() so that it additionally returns the file position for the 
offset used in the fetch request.

3. Partition/Replica
3.1 We maintain the logEndPosition in each replica.
3.2 In appendMessage(), we additionally pass in the end file position returned 
from Log.append() and save it in the leader replica.
3.3. We pass in the file position returned from Log.read() from 
KafkaApis.handleFetchRequest() all the way to 
Partition.updateLeaderHWAndMaybeExpandIsr() and save it in the follower replica.

4. To unblock a pending regular consumer fetch request:
4.1 When creating a delayed fetch request, we pass in its fetchOfffsetPosition.
4.2 In Partition.maybeIncrementLeaderHW(), maintain a highWatermarkPosition. 
Every time the high watermark moves, we move highWatermarkPosition as well. We 
then call FetchPurgatory.update() to unblock regular consumer fetch requests. 
The check can now be done by just comparing the difference btw 
highWatermarkPosition and fetchOfffsetPosition.

5. To unblock a pending follower fetch request:
5.1 When creating a delayed fetch request, we pass in its fetchOfffsetPosition.
5.2 After each local log append in the leader, we call FetchPurgatory.update() 
to unblock follower consumer fetch requests, by passing in the end file 
position returned from Log.append(). The check can now be done by just 
comparing the difference btw end file position and fetchOfffsetPosition.

We need to handle fetch requests not on the last log segment. One solution is 
to maintain file position as the accumulated byte position since the first 
segment.  We can store a startPosition value in each log segment. Note that 
value just needs to be maintained in memory. We can initialize them again after 
broker restart.

> Purgatory redesign
> --
>
> Key: KAFKA-1430
> URL: https://issues.apache.org/jira/browse/KAFKA-1430
> Project: Kafka
>  Issue Type: Improvement
>  Components: core
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>
> We have seen 2 main issues with the Purgatory.
> 1. There is no atomic checkAndWatch functionality. So, a client typically 
> first checks whether a request is satisfied or not and then register the 
> watcher. However, by the time the watcher is registered, the registered item 
> could already be satisfied. This item won't be satisfied until the next 
> update happens or the delayed time expires, which means the watched item 
> could be delayed. 
> 2. FetchRequestPurgatory doesn't quite work. This is because the current 
> design tries to incrementally maintain the accumulated bytes ready for fetch. 
> However, this is difficult since the right time to check whether a fetch (for 
> regular consumer) request is satisfied is when the high watermark moves. At 
> that point, it's hard to figure out how many bytes we should incrementally 
> add to each pending fetch request.
> The problem has been reported in KAFAK-1150 and KAFKA-703.



--
This message was sent by Atlassian JIRA
(v6.2#6252)