kafka_2.8.0-0.8.0-beta1.jar doesn't include kafka.utils.TestUtils

2013-08-19 Thread Wu, Jiang2
Hi,

In Kafka source code,  kafka.utils.TestUtils is defined in 
src/core/src/test/scala/unit/kafka/utils/TestUtils.scala, but the 
kafka_2.8.0-0.8.0-beta1.jar doesn't include kafka.utils.TestUtils. I'm 
wondering how to write java unit test using kafka.utils.TestUtils? Should I use 
some other jars or to compile one from the scala source code? Sorry if it's a 
rudimentary question; I know little about scala, just know it can be compiled 
to java class.

Thanks,
Jiang



re: questtion about log.retention.bytes

2013-08-19 Thread Paul Christian
Hi Jun,

Thank you for your reply. I'm still a little fuzzy on the concept.

Are you saying I can have topic A, B and C and with

log.retention.bytes.per.topic.A = 15MB
log.retention.bytes.per.topic.B = 20MB
log.retention.bytes = 30MB

And thus topic C will get the value 30MB? Since it's not defined like the
others' 'per topic'?

log.retention.bytes is for all topics that are not included in
log.retention.bytes.per.topic
(which defines a map of topic -> size).

Otherwise, log.retention.bytes.per.topic and log.retention.bytes seem very
similar to me.

Additionally, we've experimented with this value on our test cluster where
we set the log.retention.bytes to 11MB as a test. Below is a snippet from
our server.properties.

# A size-based retention policy for logs. Segments are pruned from the log
as long as the remaining
# segments don't drop below log.retention.bytes.
log.retention.bytes=11534336

Here is a ls -lh from one of the topics

-rw-r--r-- 1 kafka service  10M Aug 19 15:45 07021913.index
-rw-r--r-- 1 kafka service 114M Aug 19 15:45 07021913.log

The index file appears to be reflected in the property
log.index.size.max.bytes, but the log just keeps going.


Re: kafka_2.8.0-0.8.0-beta1.jar doesn't include kafka.utils.TestUtils

2013-08-19 Thread Neha Narkhede
You raised a good point. The same is true for all the tools that are
under src/core/src/test/scala/other/kafka.
I think we need to fix the packaging under src/core/test. Probably move the
tools under core/src/main/scala/kafka/tools so that it goes in the main
Kafka jar. And either include the test utils there or publish a test jar.
Please can you file a JIRA for this?

Thanks,
Neha


On Mon, Aug 19, 2013 at 9:08 AM, Wu, Jiang2  wrote:

> Hi,
>
> In Kafka source code,  kafka.utils.TestUtils is defined in
> src/core/src/test/scala/unit/kafka/utils/TestUtils.scala, but the
> kafka_2.8.0-0.8.0-beta1.jar doesn't include kafka.utils.TestUtils. I'm
> wondering how to write java unit test using kafka.utils.TestUtils? Should I
> use some other jars or to compile one from the scala source code? Sorry if
> it's a rudimentary question; I know little about scala, just know it can be
> compiled to java class.
>
> Thanks,
> Jiang
>
>


Re: kafka_2.8.0-0.8.0-beta1.jar doesn't include kafka.utils.TestUtils

2013-08-19 Thread Jay Kreps
These are really tests, not something we should necessarily ship. They live
in the kafka-test jar, which I think is right. You can build them by
running
 ./sbt test:package
I would prefer adding this to the main package target rather than moving
our test code into the main source directory.

-Jay


On Mon, Aug 19, 2013 at 9:25 AM, Neha Narkhede wrote:

> You raised a good point. The same is true for all the tools that are
> under src/core/src/test/scala/other/kafka.
> I think we need to fix the packaging under src/core/test. Probably move the
> tools under core/src/main/scala/kafka/tools so that it goes in the main
> Kafka jar. And either include the test utils there or publish a test jar.
> Please can you file a JIRA for this?
>
> Thanks,
> Neha
>
>
> On Mon, Aug 19, 2013 at 9:08 AM, Wu, Jiang2  wrote:
>
> > Hi,
> >
> > In Kafka source code,  kafka.utils.TestUtils is defined in
> > src/core/src/test/scala/unit/kafka/utils/TestUtils.scala, but the
> > kafka_2.8.0-0.8.0-beta1.jar doesn't include kafka.utils.TestUtils. I'm
> > wondering how to write java unit test using kafka.utils.TestUtils?
> Should I
> > use some other jars or to compile one from the scala source code? Sorry
> if
> > it's a rudimentary question; I know little about scala, just know it can
> be
> > compiled to java class.
> >
> > Thanks,
> > Jiang
> >
> >
>


slf4j bindings

2013-08-19 Thread Paul Mackles
Hi – Has anyone figured out a clean way to ignore/exclude the "simple" slf4j 
bindings that get included in the kafka-assembly jar for 0.8? I would like all 
of the libaries in my app to log through log4j but for those libraries using 
slf4j, the "simple" bindings in the kafka-assembly jar are getting in the way. 
Ironically, kafka uses log4j directly so that's working fine ;)

Thanks,
Paul



Failed to start preferred replica election

2013-08-19 Thread Vadim Keylis
I have a cluster of 3 kafka servers. Replication factor is 3. Two out of 3
servers were shutdown and traffic was sent to only one server that was up.
I brought second host up and it says according to logs that server has
started.

I ran ./kafka-list-topic.sh --zookeeper  Still was showing leaders
are not distributed. Then ran
kafka-preferred-replica-election.sh which exited with error:

kafka.common.AdminCommandFailedException: Admin command failed
at
kafka.admin.PreferredReplicaLeaderElectionCommand.moveLeaderToPreferredReplica(PreferredReplicaLeaderElectionCommand.scala:119)
at
kafka.admin.PreferredReplicaLeaderElectionCommand$.main(PreferredReplicaLeaderElectionCommand.scala:60)
at
kafka.admin.PreferredReplicaLeaderElectionCommand.main(PreferredReplicaLeaderElectionCommand.scala)

Would you please give suggestion what have caused the exception and how to
recover from it?

Thanks so much in advance,
Vadim


Re: Kafka startup/restart process

2013-08-19 Thread Vadim Keylis
What is preferred method for control shutdown using admin tool or setting
as flag "controlled.shutdown.enable" to true? What is the advantage of
using one verses the other?

Thanks,
Vadim


On Sun, Aug 18, 2013 at 11:05 PM, Vadim Keylis wrote:

> thanks so much. Greatly appreciated.
>
>
> On Sun, Aug 18, 2013 at 10:00 PM, Neha Narkhede 
> wrote:
>
>> It is exposed on every leader through the
>> "kafka.server.UnderReplicatedPartitions" jmx bean. It is independent of
>> the
>> controlled shutdown functionality.
>>
>> Thanks,
>> Neha
>>
>>
>> On Sun, Aug 18, 2013 at 8:33 PM, Vadim Keylis 
>> wrote:
>>
>> > Neha. Thanks so much for response. How can I get under replicated
>> partition
>> > count during control shutdown that is configured in the property file?
>> >
>> > Thanks,
>> > Vadim
>> >
>> >
>> > On Sun, Aug 18, 2013 at 6:11 PM, Neha Narkhede > > >wrote:
>> >
>> > > Vadim,
>> > >
>> > > Controlled shutdown takes 2 parameters - number of retries and
>> shutdown
>> > > timeout. In every retry, controlled shutdown attempts to move leaders
>> off
>> > > of the broker that needs to be shutdown. If the controlled shutdown
>> runs
>> > > out of retries, it proceeds to shutting down the broker even if it
>> still
>> > > hosts a few leaders. At LinkedIn, the script to bounce Kafka brokers
>> > waits
>> > > for the under replicated partition count to drop to 0 before invoking
>> > > controlled shutdown on the next broker. The aim is to avoid data loss
>> > that
>> > > occurs if you shut down a broker that still has some leaders. If the
>> > under
>> > > replicated count never drops to 0, it indicates a bug in Kafka code
>> and
>> > the
>> > > script does not proceed to bouncing any more brokers in a cluster. We
>> > > measure the time it takes to move "n" leaders off of some broker, and
>> > > configure the shutdown timeout accordingly. We also configure the
>> retries
>> > > to a small number (2 or 3). If the controlled shutdown fails the
>> retries,
>> > > the broker shuts itself down anyways. In general, you want to avoid
>> hard
>> > > killing (kill -9) a broker since that means the broker will run a long
>> > > running log recovery process on startup. That significantly delays the
>> > time
>> > > the broker takes to rejoin the cluster.
>> > >
>> > > Thanks,
>> > > Neha
>> > >
>> > >
>> > > On Sun, Aug 18, 2013 at 3:33 PM, Vadim Keylis 
>> > > wrote:
>> > >
>> > > > Good afternoon. We are running kafka on centos linux. I enabled
>> > > controlled
>> > > > shutdown in the property file. We are starting/stopping kafka using
>> > init
>> > > > script. The init script will issue term signal first followed 3
>> seconds
>> > > > later by kill signal. Is that right process to shutdown kafka? Which
>> > > > startup/shutdown/restart script you guys use? What shutdown process
>> > > > linkedin uses? What side effects could be after kafka service is
>> killed
>> > > > uncleanly using kill -9 signal?
>> > > >
>> > > > Thanks,
>> > > > Vadim
>> > > >
>> > >
>> >
>>
>
>


Re: Failed to start preferred replica election

2013-08-19 Thread Neha Narkhede
I think the error message can be improved to at least print which
partitions it couldn't move the leader for. What could be happening is that
the 2 brokers that were down might not have entered the ISR yet. So the
tool will not be able to move any leaders to them. You can run
kafka-list-topics with the --under-replicated-count option to print the
list of under replicated partitions.

Please can you file a bug to improve the error reporting of this tool?

Thanks,
Neha


On Mon, Aug 19, 2013 at 12:26 PM, Vadim Keylis wrote:

> I have a cluster of 3 kafka servers. Replication factor is 3. Two out of 3
> servers were shutdown and traffic was sent to only one server that was up.
> I brought second host up and it says according to logs that server has
> started.
>
> I ran ./kafka-list-topic.sh --zookeeper  Still was showing leaders
> are not distributed. Then ran
> kafka-preferred-replica-election.sh which exited with error:
>
> kafka.common.AdminCommandFailedException: Admin command failed
> at
>
> kafka.admin.PreferredReplicaLeaderElectionCommand.moveLeaderToPreferredReplica(PreferredReplicaLeaderElectionCommand.scala:119)
> at
>
> kafka.admin.PreferredReplicaLeaderElectionCommand$.main(PreferredReplicaLeaderElectionCommand.scala:60)
> at
>
> kafka.admin.PreferredReplicaLeaderElectionCommand.main(PreferredReplicaLeaderElectionCommand.scala)
>
> Would you please give suggestion what have caused the exception and how to
> recover from it?
>
> Thanks so much in advance,
> Vadim
>


Re: Failed to start preferred replica election

2013-08-19 Thread Vadim Keylis
It does print partitions. I just did not include them in the bug.

How can I monitor replica resync progress as well as know when resync
process completed using script? That should allow me to better predict when
the tool would run successfully.

Thanks so much.


On Mon, Aug 19, 2013 at 12:59 PM, Neha Narkhede wrote:

> I think the error message can be improved to at least print which
> partitions it couldn't move the leader for. What could be happening is that
> the 2 brokers that were down might not have entered the ISR yet. So the
> tool will not be able to move any leaders to them. You can run
> kafka-list-topics with the --under-replicated-count option to print the
> list of under replicated partitions.
>
> Please can you file a bug to improve the error reporting of this tool?
>
> Thanks,
> Neha
>
>
> On Mon, Aug 19, 2013 at 12:26 PM, Vadim Keylis  >wrote:
>
> > I have a cluster of 3 kafka servers. Replication factor is 3. Two out of
> 3
> > servers were shutdown and traffic was sent to only one server that was
> up.
> > I brought second host up and it says according to logs that server has
> > started.
> >
> > I ran ./kafka-list-topic.sh --zookeeper  Still was showing leaders
> > are not distributed. Then ran
> > kafka-preferred-replica-election.sh which exited with error:
> >
> > kafka.common.AdminCommandFailedException: Admin command failed
> > at
> >
> >
> kafka.admin.PreferredReplicaLeaderElectionCommand.moveLeaderToPreferredReplica(PreferredReplicaLeaderElectionCommand.scala:119)
> > at
> >
> >
> kafka.admin.PreferredReplicaLeaderElectionCommand$.main(PreferredReplicaLeaderElectionCommand.scala:60)
> > at
> >
> >
> kafka.admin.PreferredReplicaLeaderElectionCommand.main(PreferredReplicaLeaderElectionCommand.scala)
> >
> > Would you please give suggestion what have caused the exception and how
> to
> > recover from it?
> >
> > Thanks so much in advance,
> > Vadim
> >
>


RE: kafka_2.8.0-0.8.0-beta1.jar doesn't include kafka.utils.TestUtils

2013-08-19 Thread Wu, Jiang2
Thanks Jay and Neha. I built kafka_2.8.0-0.8.0-beta1-test.jar, which contains 
the TestUtils class. Is there any sample java code for writing kafka test cases?

Regards,
Jiang

-Original Message-
From: Jay Kreps [mailto:jay.kr...@gmail.com]
Sent: Monday, August 19, 2013 12:33 PM
To: users@kafka.apache.org
Subject: Re: kafka_2.8.0-0.8.0-beta1.jar doesn't include kafka.utils.TestUtils

These are really tests, not something we should necessarily ship. They live in 
the kafka-test jar, which I think is right. You can build them by running  
./sbt test:package I would prefer adding this to the main package target rather 
than moving our test code into the main source directory.

-Jay


On Mon, Aug 19, 2013 at 9:25 AM, Neha Narkhede wrote:

> You raised a good point. The same is true for all the tools that are
> under src/core/src/test/scala/other/kafka.
> I think we need to fix the packaging under src/core/test. Probably
> move the tools under core/src/main/scala/kafka/tools so that it goes
> in the main Kafka jar. And either include the test utils there or publish a 
> test jar.
> Please can you file a JIRA for this?
>
> Thanks,
> Neha
>
>
> On Mon, Aug 19, 2013 at 9:08 AM, Wu, Jiang2  wrote:
>
> > Hi,
> >
> > In Kafka source code,  kafka.utils.TestUtils is defined in
> > src/core/src/test/scala/unit/kafka/utils/TestUtils.scala, but the
> > kafka_2.8.0-0.8.0-beta1.jar doesn't include kafka.utils.TestUtils.
> > I'm wondering how to write java unit test using kafka.utils.TestUtils?
> Should I
> > use some other jars or to compile one from the scala source code?
> > Sorry
> if
> > it's a rudimentary question; I know little about scala, just know it
> > can
> be
> > compiled to java class.
> >
> > Thanks,
> > Jiang
> >
> >
>


Re: Failed to start preferred replica election

2013-08-19 Thread Neha Narkhede
You can monitor the under replicated partition count through the
"kafka.server.UnderReplicatedPartitions" jmx bean on every leader. Another
way, which is heavy weight is to run kafka-list-topics, but I would
recommend running that only for diagnostic purposes, not for monitoring.

Thanks,
Neha


On Mon, Aug 19, 2013 at 1:07 PM, Vadim Keylis  wrote:

> It does print partitions. I just did not include them in the bug.
>
> How can I monitor replica resync progress as well as know when resync
> process completed using script? That should allow me to better predict when
> the tool would run successfully.
>
> Thanks so much.
>
>
> On Mon, Aug 19, 2013 at 12:59 PM, Neha Narkhede  >wrote:
>
> > I think the error message can be improved to at least print which
> > partitions it couldn't move the leader for. What could be happening is
> that
> > the 2 brokers that were down might not have entered the ISR yet. So the
> > tool will not be able to move any leaders to them. You can run
> > kafka-list-topics with the --under-replicated-count option to print the
> > list of under replicated partitions.
> >
> > Please can you file a bug to improve the error reporting of this tool?
> >
> > Thanks,
> > Neha
> >
> >
> > On Mon, Aug 19, 2013 at 12:26 PM, Vadim Keylis  > >wrote:
> >
> > > I have a cluster of 3 kafka servers. Replication factor is 3. Two out
> of
> > 3
> > > servers were shutdown and traffic was sent to only one server that was
> > up.
> > > I brought second host up and it says according to logs that server has
> > > started.
> > >
> > > I ran ./kafka-list-topic.sh --zookeeper  Still was showing
> leaders
> > > are not distributed. Then ran
> > > kafka-preferred-replica-election.sh which exited with error:
> > >
> > > kafka.common.AdminCommandFailedException: Admin command failed
> > > at
> > >
> > >
> >
> kafka.admin.PreferredReplicaLeaderElectionCommand.moveLeaderToPreferredReplica(PreferredReplicaLeaderElectionCommand.scala:119)
> > > at
> > >
> > >
> >
> kafka.admin.PreferredReplicaLeaderElectionCommand$.main(PreferredReplicaLeaderElectionCommand.scala:60)
> > > at
> > >
> > >
> >
> kafka.admin.PreferredReplicaLeaderElectionCommand.main(PreferredReplicaLeaderElectionCommand.scala)
> > >
> > > Would you please give suggestion what have caused the exception and how
> > to
> > > recover from it?
> > >
> > > Thanks so much in advance,
> > > Vadim
> > >
> >
>


Re: kafka_2.8.0-0.8.0-beta1.jar doesn't include kafka.utils.TestUtils

2013-08-19 Thread Neha Narkhede
You can find simple producer/consumer demo code under the examples/ sub
project. For unit test cases, you can look under
core/src/test/scala/unit/kafka.

Thanks,
Neha


On Mon, Aug 19, 2013 at 1:53 PM, Wu, Jiang2  wrote:

> Thanks Jay and Neha. I built kafka_2.8.0-0.8.0-beta1-test.jar, which
> contains the TestUtils class. Is there any sample java code for writing
> kafka test cases?
>
> Regards,
> Jiang
>
> -Original Message-
> From: Jay Kreps [mailto:jay.kr...@gmail.com]
> Sent: Monday, August 19, 2013 12:33 PM
> To: users@kafka.apache.org
> Subject: Re: kafka_2.8.0-0.8.0-beta1.jar doesn't include
> kafka.utils.TestUtils
>
> These are really tests, not something we should necessarily ship. They
> live in the kafka-test jar, which I think is right. You can build them by
> running  ./sbt test:package I would prefer adding this to the main package
> target rather than moving our test code into the main source directory.
>
> -Jay
>
>
> On Mon, Aug 19, 2013 at 9:25 AM, Neha Narkhede  >wrote:
>
> > You raised a good point. The same is true for all the tools that are
> > under src/core/src/test/scala/other/kafka.
> > I think we need to fix the packaging under src/core/test. Probably
> > move the tools under core/src/main/scala/kafka/tools so that it goes
> > in the main Kafka jar. And either include the test utils there or
> publish a test jar.
> > Please can you file a JIRA for this?
> >
> > Thanks,
> > Neha
> >
> >
> > On Mon, Aug 19, 2013 at 9:08 AM, Wu, Jiang2  wrote:
> >
> > > Hi,
> > >
> > > In Kafka source code,  kafka.utils.TestUtils is defined in
> > > src/core/src/test/scala/unit/kafka/utils/TestUtils.scala, but the
> > > kafka_2.8.0-0.8.0-beta1.jar doesn't include kafka.utils.TestUtils.
> > > I'm wondering how to write java unit test using kafka.utils.TestUtils?
> > Should I
> > > use some other jars or to compile one from the scala source code?
> > > Sorry
> > if
> > > it's a rudimentary question; I know little about scala, just know it
> > > can
> > be
> > > compiled to java class.
> > >
> > > Thanks,
> > > Jiang
> > >
> > >
> >
>


RE: kafka_2.8.0-0.8.0-beta1.jar doesn't include kafka.utils.TestUtils

2013-08-19 Thread Wu, Jiang2
Thanks Neha, it's really helpful.

Jiang

-Original Message-
From: Neha Narkhede [mailto:neha.narkh...@gmail.com]
Sent: Monday, August 19, 2013 5:44 PM
To: users@kafka.apache.org
Subject: Re: kafka_2.8.0-0.8.0-beta1.jar doesn't include kafka.utils.TestUtils

You can find simple producer/consumer demo code under the examples/ sub 
project. For unit test cases, you can look under core/src/test/scala/unit/kafka.

Thanks,
Neha


On Mon, Aug 19, 2013 at 1:53 PM, Wu, Jiang2  wrote:

> Thanks Jay and Neha. I built kafka_2.8.0-0.8.0-beta1-test.jar, which
> contains the TestUtils class. Is there any sample java code for
> writing kafka test cases?
>
> Regards,
> Jiang
>
> -Original Message-
> From: Jay Kreps [mailto:jay.kr...@gmail.com]
> Sent: Monday, August 19, 2013 12:33 PM
> To: users@kafka.apache.org
> Subject: Re: kafka_2.8.0-0.8.0-beta1.jar doesn't include
> kafka.utils.TestUtils
>
> These are really tests, not something we should necessarily ship. They
> live in the kafka-test jar, which I think is right. You can build them
> by running  ./sbt test:package I would prefer adding this to the main
> package target rather than moving our test code into the main source 
> directory.
>
> -Jay
>
>
> On Mon, Aug 19, 2013 at 9:25 AM, Neha Narkhede
>  >wrote:
>
> > You raised a good point. The same is true for all the tools that are
> > under src/core/src/test/scala/other/kafka.
> > I think we need to fix the packaging under src/core/test. Probably
> > move the tools under core/src/main/scala/kafka/tools so that it goes
> > in the main Kafka jar. And either include the test utils there or
> publish a test jar.
> > Please can you file a JIRA for this?
> >
> > Thanks,
> > Neha
> >
> >
> > On Mon, Aug 19, 2013 at 9:08 AM, Wu, Jiang2  wrote:
> >
> > > Hi,
> > >
> > > In Kafka source code,  kafka.utils.TestUtils is defined in
> > > src/core/src/test/scala/unit/kafka/utils/TestUtils.scala, but the
> > > kafka_2.8.0-0.8.0-beta1.jar doesn't include kafka.utils.TestUtils.
> > > I'm wondering how to write java unit test using kafka.utils.TestUtils?
> > Should I
> > > use some other jars or to compile one from the scala source code?
> > > Sorry
> > if
> > > it's a rudimentary question; I know little about scala, just know
> > > it can
> > be
> > > compiled to java class.
> > >
> > > Thanks,
> > > Jiang
> > >
> > >
> >
>


Re: Offset committing on rebalance

2013-08-19 Thread Ian Friedman
Jun, I read that FAQ entry you linked, but I am not seeing any Zookeeper 
connection loss in the logs. It's rebalancing multiple times per minute, 
though. Any idea what else could cause this? We're running kafka 0.7.2 on 
approx 400 consumers against a topic with 400 partitions * 3 brokers.   

--  
Ian Friedman


On Thursday, August 15, 2013 at 11:52 AM, Jun Rao wrote:

> Yes, during rebalances, messages could be re-delievered since the new owner
> of a partition starts fetching from the last checkpointed offset in ZK.
>  
> For reasons on why rebalances happen a lot, see
> https://cwiki.apache.org/confluence/display/KAFKA/FAQ#FAQ-Whyaretheremanyrebalancesinmyconsumerlog%3F
>  
> Thanks,
>  
> Jun
>  
>  
> On Thu, Aug 15, 2013 at 8:36 AM, Ian Friedman  (mailto:i...@flurry.com)> wrote:
>  
> > It's a simple enough patch, but wouldn't this mean that messages still in
> > process when a rebalance happens could get delivered to another consumer if
> > we end up losing the partition? Rebalances seem to happen very frequently
> > with a lot of consumers for some reason… And it doesn't seem like a
> > consumer is guaranteed or likely to retain ownership of a partition it's in
> > the middle of consuming after a rebalance.
> >  
> > --
> > Ian Friedman
> >  
> >  
> > On Thursday, August 15, 2013 at 10:53 AM, Jun Rao wrote:
> >  
> > > We are only patching blocker issues in 0.7. 0.8 beta1 has been released
> > and
> > > most dev effort will be on 0.8 and beyond. That said. This particular
> >  
> > case
> > > is easy to fix. If you can port the patch in
> > > https://issues.apache.org/jira/browse/KAFKA-919 o the 0.7 branch , we
> > >  
> >  
> > can
> > > commit that to the 0.7 branch.
> > >  
> > > Thanks,
> > >  
> > > Jun
> > >  
> > >  
> > > On Wed, Aug 14, 2013 at 9:30 PM, Ian Friedman  > > (mailto:i...@flurry.com) (mailto:
> > i...@flurry.com (mailto:i...@flurry.com))> wrote:
> > >  
> > > > Ugh.
> > > >  
> > > > Is there any way to make this work in 0.7, or is transitioning to 0.8
> > the
> > > > only way? My operations engineers spent a lot of effort in configuring
> > >  
> >  
> > and
> > > > hardening our 0.7 production install, and 0.8 isn't released yet. Not
> > >  
> >  
> > to
> > > > mention having to integrate the new client side code.
> > > >  
> > > > Either way, thanks for all your help Jun.
> > > >  
> > > > --
> > > > Ian Friedman
> > > >  
> > > >  
> > > > On Thursday, August 15, 2013 at 12:21 AM, Jun Rao wrote:
> > > >  
> > > > > Yes, this is an issue and has been fixed in 0.8.
> > > > >  
> > > > > Thanks,
> > > > >  
> > > > > Jun
> > > > >  
> > > > >  
> > > > > On Wed, Aug 14, 2013 at 5:21 PM, Ian Friedman  > > > > (mailto:i...@flurry.com)(mailto:
> > i...@flurry.com (mailto:i...@flurry.com)) (mailto:
> > > > i...@flurry.com (mailto:i...@flurry.com))> wrote:
> > > > >  
> > > > > > Hey guys,
> > > > > >  
> > > > > > I designed my consumer app (running on 0.7) to run with autocommit
> > off
> > > > and
> > > > > > commit manually once it was done processing a record. The intent
> > > > >  
> > > >  
> > > >  
> > >  
> >  
> > was so
> > > > > > that if a consumer died while processing a message, the offset
> > > > >  
> > > >  
> > >  
> >  
> > would
> > > > > >  
> > > > >  
> > > >  
> > > >  
> > > > not be
> > > > > > committed, and another box would pick up the partition and
> > > > >  
> > > >  
> > > >  
> > >  
> >  
> > reprocess
> > > > >  
> > > >  
> > > >  
> > > > the
> > > > > > message. This seemed to work fine with small numbers of consumers
> > > > >  
> > > >  
> > > >  
> > > > (~10).
> > > > > > But now that I'm scaling it out, I'm running into a problem where
> > > > >  
> > > >  
> > > >  
> > >  
> >  
> > it
> > > > >  
> > > >  
> > > >  
> > > > looks
> > > > > > like messages that consumers picked up and then errored on are not
> > > > >  
> > > >  
> > > >  
> > > > getting
> > > > > > processed on another machine.
> > > > > >  
> > > > > > After investigating the logs and the partition offsets in
> > zookeeper, I
> > > > > > found that in ZookeeperConsumerConnector.scala
> > > > >  
> > > >  
> > >  
> >  
> > closeFetchersForQueues,
> > > > > > called during the rebalance process, will commit the offset
> > > > >  
> > > >  
> > >  
> >  
> > regardless
> > > > > >  
> > > > >  
> > > >  
> > > >  
> > > > of
> > > > > > the autocommit status. So it looks like even if my consumer is in
> > > > >  
> > > >  
> > > >  
> > >  
> >  
> > the
> > > > > > middle of processing a message, the offset will be committed, and
> > > > >  
> > > >  
> > >  
> >  
> > even
> > > > > >  
> > > > >  
> > > >  
> > > >  
> > > > if
> > > > > > the processing fails, it will never be picked up again. Now that I
> > > > >  
> > > >  
> > > >  
> > > > have a
> > > > > > lot of consumer nodes, the rebalancer is going off a lot more often
> > > > >  
> > > >  
> > > >  
> > > > and I'm
> > > > > > running into this constantly.
> > > > > >  
> > > > > > Were my assumpt

Re: questtion about log.retention.bytes

2013-08-19 Thread Jun Rao
For the first question, yes, topic C will get the value of 30MB.

For the second question, log.retention.bytes only controls the segment log
file size, not the index. Typically, index file size is much smaller than
the log file. The index file of the last (active) segment is presized to
the max index size (defaults to 10MB). However, the size is trimmed as soon
as the segment rolls.

Thanks,

Jun


On Mon, Aug 19, 2013 at 9:22 AM, Paul Christian
wrote:

> Hi Jun,
>
> Thank you for your reply. I'm still a little fuzzy on the concept.
>
> Are you saying I can have topic A, B and C and with
>
> log.retention.bytes.per.topic.A = 15MB
> log.retention.bytes.per.topic.B = 20MB
> log.retention.bytes = 30MB
>
> And thus topic C will get the value 30MB? Since it's not defined like the
> others' 'per topic'?
>
> log.retention.bytes is for all topics that are not included in
> log.retention.bytes.per.topic
> (which defines a map of topic -> size).
>
> Otherwise, log.retention.bytes.per.topic and log.retention.bytes seem very
> similar to me.
>
> Additionally, we've experimented with this value on our test cluster where
> we set the log.retention.bytes to 11MB as a test. Below is a snippet from
> our server.properties.
>
> # A size-based retention policy for logs. Segments are pruned from the log
> as long as the remaining
> # segments don't drop below log.retention.bytes.
> log.retention.bytes=11534336
>
> Here is a ls -lh from one of the topics
>
> -rw-r--r-- 1 kafka service  10M Aug 19 15:45 07021913.index
> -rw-r--r-- 1 kafka service 114M Aug 19 15:45 07021913.log
>
> The index file appears to be reflected in the property
> log.index.size.max.bytes, but the log just keeps going.
>


Re: slf4j bindings

2013-08-19 Thread Jun Rao
In our binary release, you can just remove slf4j-simple-1.6.4.jar and
add a slf4j-log4j12
jar.

Thanks,

Jun


On Mon, Aug 19, 2013 at 10:13 AM, Paul Mackles  wrote:

> Hi – Has anyone figured out a clean way to ignore/exclude the "simple"
> slf4j bindings that get included in the kafka-assembly jar for 0.8? I would
> like all of the libaries in my app to log through log4j but for those
> libraries using slf4j, the "simple" bindings in the kafka-assembly jar are
> getting in the way. Ironically, kafka uses log4j directly so that's working
> fine ;)
>
> Thanks,
> Paul
>
>


Re: Failed to start preferred replica election

2013-08-19 Thread Jun Rao
We also have a jmx bean that tracks the lag in messages per partition in
the follower broker.

Thanks,

Jun


On Mon, Aug 19, 2013 at 1:07 PM, Vadim Keylis  wrote:

> It does print partitions. I just did not include them in the bug.
>
> How can I monitor replica resync progress as well as know when resync
> process completed using script? That should allow me to better predict when
> the tool would run successfully.
>
> Thanks so much.
>
>
> On Mon, Aug 19, 2013 at 12:59 PM, Neha Narkhede  >wrote:
>
> > I think the error message can be improved to at least print which
> > partitions it couldn't move the leader for. What could be happening is
> that
> > the 2 brokers that were down might not have entered the ISR yet. So the
> > tool will not be able to move any leaders to them. You can run
> > kafka-list-topics with the --under-replicated-count option to print the
> > list of under replicated partitions.
> >
> > Please can you file a bug to improve the error reporting of this tool?
> >
> > Thanks,
> > Neha
> >
> >
> > On Mon, Aug 19, 2013 at 12:26 PM, Vadim Keylis  > >wrote:
> >
> > > I have a cluster of 3 kafka servers. Replication factor is 3. Two out
> of
> > 3
> > > servers were shutdown and traffic was sent to only one server that was
> > up.
> > > I brought second host up and it says according to logs that server has
> > > started.
> > >
> > > I ran ./kafka-list-topic.sh --zookeeper  Still was showing
> leaders
> > > are not distributed. Then ran
> > > kafka-preferred-replica-election.sh which exited with error:
> > >
> > > kafka.common.AdminCommandFailedException: Admin command failed
> > > at
> > >
> > >
> >
> kafka.admin.PreferredReplicaLeaderElectionCommand.moveLeaderToPreferredReplica(PreferredReplicaLeaderElectionCommand.scala:119)
> > > at
> > >
> > >
> >
> kafka.admin.PreferredReplicaLeaderElectionCommand$.main(PreferredReplicaLeaderElectionCommand.scala:60)
> > > at
> > >
> > >
> >
> kafka.admin.PreferredReplicaLeaderElectionCommand.main(PreferredReplicaLeaderElectionCommand.scala)
> > >
> > > Would you please give suggestion what have caused the exception and how
> > to
> > > recover from it?
> > >
> > > Thanks so much in advance,
> > > Vadim
> > >
> >
>


Re: Offset committing on rebalance

2013-08-19 Thread Jun Rao
Any failure/restart of a consumer or a broker can also trigger a rebalance.

Thanks,

Jun


On Mon, Aug 19, 2013 at 6:00 PM, Ian Friedman  wrote:

> Jun, I read that FAQ entry you linked, but I am not seeing any Zookeeper
> connection loss in the logs. It's rebalancing multiple times per minute,
> though. Any idea what else could cause this? We're running kafka 0.7.2 on
> approx 400 consumers against a topic with 400 partitions * 3 brokers.
>
> --
> Ian Friedman
>
>
> On Thursday, August 15, 2013 at 11:52 AM, Jun Rao wrote:
>
> > Yes, during rebalances, messages could be re-delievered since the new
> owner
> > of a partition starts fetching from the last checkpointed offset in ZK.
> >
> > For reasons on why rebalances happen a lot, see
> >
> https://cwiki.apache.org/confluence/display/KAFKA/FAQ#FAQ-Whyaretheremanyrebalancesinmyconsumerlog%3F
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Thu, Aug 15, 2013 at 8:36 AM, Ian Friedman  i...@flurry.com)> wrote:
> >
> > > It's a simple enough patch, but wouldn't this mean that messages still
> in
> > > process when a rebalance happens could get delivered to another
> consumer if
> > > we end up losing the partition? Rebalances seem to happen very
> frequently
> > > with a lot of consumers for some reason… And it doesn't seem like a
> > > consumer is guaranteed or likely to retain ownership of a partition
> it's in
> > > the middle of consuming after a rebalance.
> > >
> > > --
> > > Ian Friedman
> > >
> > >
> > > On Thursday, August 15, 2013 at 10:53 AM, Jun Rao wrote:
> > >
> > > > We are only patching blocker issues in 0.7. 0.8 beta1 has been
> released
> > > and
> > > > most dev effort will be on 0.8 and beyond. That said. This particular
> > >
> > > case
> > > > is easy to fix. If you can port the patch in
> > > > https://issues.apache.org/jira/browse/KAFKA-919 o the 0.7 branch ,
> we
> > > >
> > >
> > > can
> > > > commit that to the 0.7 branch.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > >
> > > > On Wed, Aug 14, 2013 at 9:30 PM, Ian Friedman  i...@flurry.com) (mailto:
> > > i...@flurry.com (mailto:i...@flurry.com))> wrote:
> > > >
> > > > > Ugh.
> > > > >
> > > > > Is there any way to make this work in 0.7, or is transitioning to
> 0.8
> > > the
> > > > > only way? My operations engineers spent a lot of effort in
> configuring
> > > >
> > >
> > > and
> > > > > hardening our 0.7 production install, and 0.8 isn't released yet.
> Not
> > > >
> > >
> > > to
> > > > > mention having to integrate the new client side code.
> > > > >
> > > > > Either way, thanks for all your help Jun.
> > > > >
> > > > > --
> > > > > Ian Friedman
> > > > >
> > > > >
> > > > > On Thursday, August 15, 2013 at 12:21 AM, Jun Rao wrote:
> > > > >
> > > > > > Yes, this is an issue and has been fixed in 0.8.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > >
> > > > > > On Wed, Aug 14, 2013 at 5:21 PM, Ian Friedman 
> > > > > >  i...@flurry.com)(mailto:
> > > i...@flurry.com (mailto:i...@flurry.com)) (mailto:
> > > > > i...@flurry.com (mailto:i...@flurry.com))> wrote:
> > > > > >
> > > > > > > Hey guys,
> > > > > > >
> > > > > > > I designed my consumer app (running on 0.7) to run with
> autocommit
> > > off
> > > > > and
> > > > > > > commit manually once it was done processing a record. The
> intent
> > > > > >
> > > > >
> > > > >
> > > >
> > >
> > > was so
> > > > > > > that if a consumer died while processing a message, the offset
> > > > > >
> > > > >
> > > >
> > >
> > > would
> > > > > > >
> > > > > >
> > > > >
> > > > >
> > > > > not be
> > > > > > > committed, and another box would pick up the partition and
> > > > > >
> > > > >
> > > > >
> > > >
> > >
> > > reprocess
> > > > > >
> > > > >
> > > > >
> > > > > the
> > > > > > > message. This seemed to work fine with small numbers of
> consumers
> > > > > >
> > > > >
> > > > >
> > > > > (~10).
> > > > > > > But now that I'm scaling it out, I'm running into a problem
> where
> > > > > >
> > > > >
> > > > >
> > > >
> > >
> > > it
> > > > > >
> > > > >
> > > > >
> > > > > looks
> > > > > > > like messages that consumers picked up and then errored on are
> not
> > > > > >
> > > > >
> > > > >
> > > > > getting
> > > > > > > processed on another machine.
> > > > > > >
> > > > > > > After investigating the logs and the partition offsets in
> > > zookeeper, I
> > > > > > > found that in ZookeeperConsumerConnector.scala
> > > > > >
> > > > >
> > > >
> > >
> > > closeFetchersForQueues,
> > > > > > > called during the rebalance process, will commit the offset
> > > > > >
> > > > >
> > > >
> > >
> > > regardless
> > > > > > >
> > > > > >
> > > > >
> > > > >
> > > > > of
> > > > > > > the autocommit status. So it looks like even if my consumer is
> in
> > > > > >
> > > > >
> > > > >
> > > >
> > >
> > > the
> > > > > > > middle of processing a message, the offset will be committed,
> and
> > > > > >
> > > > >
> > > >
> > >
> > > even
> > > > > > >
> > > > > >
> > > > >
> > > > >
> > > > > if
> > > > > >

Re: Kafka startup/restart process

2013-08-19 Thread Neha Narkhede
It depends on how much flexibility you need during the controlled shutdown
and whether you have remote jmx operations enabled in your production Kafka
cluster. The jmx controlled shutdown method will offer more flexibility as
your script will have the retry logic, you don't need to make config
changes to Kafka brokers to change the timeout or the # of retries for
controlled shutdown. On the other hand, the jmx controlled shutdown method
requires access to remote jmx on the broker. At LinkedIn, we do not have
the ability to invoke jmx operations remotely on Kafka brokers in
production. So we prefer using the controlled.shutdown.enable method.

Thanks,
Neha


On Mon, Aug 19, 2013 at 12:34 PM, Vadim Keylis wrote:

> What is preferred method for control shutdown using admin tool or setting
> as flag "controlled.shutdown.enable" to true? What is the advantage of
> using one verses the other?
>
> Thanks,
> Vadim
>
>
> On Sun, Aug 18, 2013 at 11:05 PM, Vadim Keylis  >wrote:
>
> > thanks so much. Greatly appreciated.
> >
> >
> > On Sun, Aug 18, 2013 at 10:00 PM, Neha Narkhede  >wrote:
> >
> >> It is exposed on every leader through the
> >> "kafka.server.UnderReplicatedPartitions" jmx bean. It is independent of
> >> the
> >> controlled shutdown functionality.
> >>
> >> Thanks,
> >> Neha
> >>
> >>
> >> On Sun, Aug 18, 2013 at 8:33 PM, Vadim Keylis 
> >> wrote:
> >>
> >> > Neha. Thanks so much for response. How can I get under replicated
> >> partition
> >> > count during control shutdown that is configured in the property file?
> >> >
> >> > Thanks,
> >> > Vadim
> >> >
> >> >
> >> > On Sun, Aug 18, 2013 at 6:11 PM, Neha Narkhede <
> neha.narkh...@gmail.com
> >> > >wrote:
> >> >
> >> > > Vadim,
> >> > >
> >> > > Controlled shutdown takes 2 parameters - number of retries and
> >> shutdown
> >> > > timeout. In every retry, controlled shutdown attempts to move
> leaders
> >> off
> >> > > of the broker that needs to be shutdown. If the controlled shutdown
> >> runs
> >> > > out of retries, it proceeds to shutting down the broker even if it
> >> still
> >> > > hosts a few leaders. At LinkedIn, the script to bounce Kafka brokers
> >> > waits
> >> > > for the under replicated partition count to drop to 0 before
> invoking
> >> > > controlled shutdown on the next broker. The aim is to avoid data
> loss
> >> > that
> >> > > occurs if you shut down a broker that still has some leaders. If the
> >> > under
> >> > > replicated count never drops to 0, it indicates a bug in Kafka code
> >> and
> >> > the
> >> > > script does not proceed to bouncing any more brokers in a cluster.
> We
> >> > > measure the time it takes to move "n" leaders off of some broker,
> and
> >> > > configure the shutdown timeout accordingly. We also configure the
> >> retries
> >> > > to a small number (2 or 3). If the controlled shutdown fails the
> >> retries,
> >> > > the broker shuts itself down anyways. In general, you want to avoid
> >> hard
> >> > > killing (kill -9) a broker since that means the broker will run a
> long
> >> > > running log recovery process on startup. That significantly delays
> the
> >> > time
> >> > > the broker takes to rejoin the cluster.
> >> > >
> >> > > Thanks,
> >> > > Neha
> >> > >
> >> > >
> >> > > On Sun, Aug 18, 2013 at 3:33 PM, Vadim Keylis <
> vkeylis2...@gmail.com>
> >> > > wrote:
> >> > >
> >> > > > Good afternoon. We are running kafka on centos linux. I enabled
> >> > > controlled
> >> > > > shutdown in the property file. We are starting/stopping kafka
> using
> >> > init
> >> > > > script. The init script will issue term signal first followed 3
> >> seconds
> >> > > > later by kill signal. Is that right process to shutdown kafka?
> Which
> >> > > > startup/shutdown/restart script you guys use? What shutdown
> process
> >> > > > linkedin uses? What side effects could be after kafka service is
> >> killed
> >> > > > uncleanly using kill -9 signal?
> >> > > >
> >> > > > Thanks,
> >> > > > Vadim
> >> > > >
> >> > >
> >> >
> >>
> >
> >
>


Re: questtion about log.retention.bytes

2013-08-19 Thread Neha Narkhede
Paul,

I'm trying to understand the 2nd problem you reported. Are you saying that
you set the log.retention.bytes=11534336 (11MB) but nevertheless your log
grew to 114MB. Which means the config option didn't really work as expected?

Thanks,
Neha


On Mon, Aug 19, 2013 at 8:46 PM, Jun Rao  wrote:

> For the first question, yes, topic C will get the value of 30MB.
>
> For the second question, log.retention.bytes only controls the segment log
> file size, not the index. Typically, index file size is much smaller than
> the log file. The index file of the last (active) segment is presized to
> the max index size (defaults to 10MB). However, the size is trimmed as soon
> as the segment rolls.
>
> Thanks,
>
> Jun
>
>
> On Mon, Aug 19, 2013 at 9:22 AM, Paul Christian
> wrote:
>
> > Hi Jun,
> >
> > Thank you for your reply. I'm still a little fuzzy on the concept.
> >
> > Are you saying I can have topic A, B and C and with
> >
> > log.retention.bytes.per.topic.A = 15MB
> > log.retention.bytes.per.topic.B = 20MB
> > log.retention.bytes = 30MB
> >
> > And thus topic C will get the value 30MB? Since it's not defined like the
> > others' 'per topic'?
> >
> > log.retention.bytes is for all topics that are not included in
> > log.retention.bytes.per.topic
> > (which defines a map of topic -> size).
> >
> > Otherwise, log.retention.bytes.per.topic and log.retention.bytes seem
> very
> > similar to me.
> >
> > Additionally, we've experimented with this value on our test cluster
> where
> > we set the log.retention.bytes to 11MB as a test. Below is a snippet from
> > our server.properties.
> >
> > # A size-based retention policy for logs. Segments are pruned from the
> log
> > as long as the remaining
> > # segments don't drop below log.retention.bytes.
> > log.retention.bytes=11534336
> >
> > Here is a ls -lh from one of the topics
> >
> > -rw-r--r-- 1 kafka service  10M Aug 19 15:45 07021913.index
> > -rw-r--r-- 1 kafka service 114M Aug 19 15:45 07021913.log
> >
> > The index file appears to be reflected in the property
> > log.index.size.max.bytes, but the log just keeps going.
> >
>


Re: Kafka startup/restart process

2013-08-19 Thread Vadim Keylis
Neha. Thanks so much for explaining. That leaves only one open question.
How do you validate  that shutdown was successful if you do not have remote
jmx access unless besides setting timeout reasonable high?

Thanks so much again,
Vadim


On Mon, Aug 19, 2013 at 9:11 PM, Neha Narkhede wrote:

> It depends on how much flexibility you need during the controlled shutdown
> and whether you have remote jmx operations enabled in your production Kafka
> cluster. The jmx controlled shutdown method will offer more flexibility as
> your script will have the retry logic, you don't need to make config
> changes to Kafka brokers to change the timeout or the # of retries for
> controlled shutdown. On the other hand, the jmx controlled shutdown method
> requires access to remote jmx on the broker. At LinkedIn, we do not have
> the ability to invoke jmx operations remotely on Kafka brokers in
> production. So we prefer using the controlled.shutdown.enable method.
>
> Thanks,
> Neha
>
>
> On Mon, Aug 19, 2013 at 12:34 PM, Vadim Keylis  >wrote:
>
> > What is preferred method for control shutdown using admin tool or setting
> > as flag "controlled.shutdown.enable" to true? What is the advantage of
> > using one verses the other?
> >
> > Thanks,
> > Vadim
> >
> >
> > On Sun, Aug 18, 2013 at 11:05 PM, Vadim Keylis  > >wrote:
> >
> > > thanks so much. Greatly appreciated.
> > >
> > >
> > > On Sun, Aug 18, 2013 at 10:00 PM, Neha Narkhede <
> neha.narkh...@gmail.com
> > >wrote:
> > >
> > >> It is exposed on every leader through the
> > >> "kafka.server.UnderReplicatedPartitions" jmx bean. It is independent
> of
> > >> the
> > >> controlled shutdown functionality.
> > >>
> > >> Thanks,
> > >> Neha
> > >>
> > >>
> > >> On Sun, Aug 18, 2013 at 8:33 PM, Vadim Keylis 
> > >> wrote:
> > >>
> > >> > Neha. Thanks so much for response. How can I get under replicated
> > >> partition
> > >> > count during control shutdown that is configured in the property
> file?
> > >> >
> > >> > Thanks,
> > >> > Vadim
> > >> >
> > >> >
> > >> > On Sun, Aug 18, 2013 at 6:11 PM, Neha Narkhede <
> > neha.narkh...@gmail.com
> > >> > >wrote:
> > >> >
> > >> > > Vadim,
> > >> > >
> > >> > > Controlled shutdown takes 2 parameters - number of retries and
> > >> shutdown
> > >> > > timeout. In every retry, controlled shutdown attempts to move
> > leaders
> > >> off
> > >> > > of the broker that needs to be shutdown. If the controlled
> shutdown
> > >> runs
> > >> > > out of retries, it proceeds to shutting down the broker even if it
> > >> still
> > >> > > hosts a few leaders. At LinkedIn, the script to bounce Kafka
> brokers
> > >> > waits
> > >> > > for the under replicated partition count to drop to 0 before
> > invoking
> > >> > > controlled shutdown on the next broker. The aim is to avoid data
> > loss
> > >> > that
> > >> > > occurs if you shut down a broker that still has some leaders. If
> the
> > >> > under
> > >> > > replicated count never drops to 0, it indicates a bug in Kafka
> code
> > >> and
> > >> > the
> > >> > > script does not proceed to bouncing any more brokers in a cluster.
> > We
> > >> > > measure the time it takes to move "n" leaders off of some broker,
> > and
> > >> > > configure the shutdown timeout accordingly. We also configure the
> > >> retries
> > >> > > to a small number (2 or 3). If the controlled shutdown fails the
> > >> retries,
> > >> > > the broker shuts itself down anyways. In general, you want to
> avoid
> > >> hard
> > >> > > killing (kill -9) a broker since that means the broker will run a
> > long
> > >> > > running log recovery process on startup. That significantly delays
> > the
> > >> > time
> > >> > > the broker takes to rejoin the cluster.
> > >> > >
> > >> > > Thanks,
> > >> > > Neha
> > >> > >
> > >> > >
> > >> > > On Sun, Aug 18, 2013 at 3:33 PM, Vadim Keylis <
> > vkeylis2...@gmail.com>
> > >> > > wrote:
> > >> > >
> > >> > > > Good afternoon. We are running kafka on centos linux. I enabled
> > >> > > controlled
> > >> > > > shutdown in the property file. We are starting/stopping kafka
> > using
> > >> > init
> > >> > > > script. The init script will issue term signal first followed 3
> > >> seconds
> > >> > > > later by kill signal. Is that right process to shutdown kafka?
> > Which
> > >> > > > startup/shutdown/restart script you guys use? What shutdown
> > process
> > >> > > > linkedin uses? What side effects could be after kafka service is
> > >> killed
> > >> > > > uncleanly using kill -9 signal?
> > >> > > >
> > >> > > > Thanks,
> > >> > > > Vadim
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >
> > >
> >
>


Re: Kafka startup/restart process

2013-08-19 Thread Tejas Patil
Not that I am some expert on this subject, but I can see that broker logs
indicate the shutdown progress:
https://github.com/tejasapatil/kafka/blob/0.8.0-beta1-candidate1/core/src/main/scala/kafka/server/KafkaServer.scala#L165


On Mon, Aug 19, 2013 at 10:19 PM, Vadim Keylis wrote:

> Neha. Thanks so much for explaining. That leaves only one open question.
> How do you validate  that shutdown was successful if you do not have remote
> jmx access unless besides setting timeout reasonable high?
>
> Thanks so much again,
> Vadim
>
>
> On Mon, Aug 19, 2013 at 9:11 PM, Neha Narkhede  >wrote:
>
> > It depends on how much flexibility you need during the controlled
> shutdown
> > and whether you have remote jmx operations enabled in your production
> Kafka
> > cluster. The jmx controlled shutdown method will offer more flexibility
> as
> > your script will have the retry logic, you don't need to make config
> > changes to Kafka brokers to change the timeout or the # of retries for
> > controlled shutdown. On the other hand, the jmx controlled shutdown
> method
> > requires access to remote jmx on the broker. At LinkedIn, we do not have
> > the ability to invoke jmx operations remotely on Kafka brokers in
> > production. So we prefer using the controlled.shutdown.enable method.
> >
> > Thanks,
> > Neha
> >
> >
> > On Mon, Aug 19, 2013 at 12:34 PM, Vadim Keylis  > >wrote:
> >
> > > What is preferred method for control shutdown using admin tool or
> setting
> > > as flag "controlled.shutdown.enable" to true? What is the advantage of
> > > using one verses the other?
> > >
> > > Thanks,
> > > Vadim
> > >
> > >
> > > On Sun, Aug 18, 2013 at 11:05 PM, Vadim Keylis  > > >wrote:
> > >
> > > > thanks so much. Greatly appreciated.
> > > >
> > > >
> > > > On Sun, Aug 18, 2013 at 10:00 PM, Neha Narkhede <
> > neha.narkh...@gmail.com
> > > >wrote:
> > > >
> > > >> It is exposed on every leader through the
> > > >> "kafka.server.UnderReplicatedPartitions" jmx bean. It is independent
> > of
> > > >> the
> > > >> controlled shutdown functionality.
> > > >>
> > > >> Thanks,
> > > >> Neha
> > > >>
> > > >>
> > > >> On Sun, Aug 18, 2013 at 8:33 PM, Vadim Keylis <
> vkeylis2...@gmail.com>
> > > >> wrote:
> > > >>
> > > >> > Neha. Thanks so much for response. How can I get under replicated
> > > >> partition
> > > >> > count during control shutdown that is configured in the property
> > file?
> > > >> >
> > > >> > Thanks,
> > > >> > Vadim
> > > >> >
> > > >> >
> > > >> > On Sun, Aug 18, 2013 at 6:11 PM, Neha Narkhede <
> > > neha.narkh...@gmail.com
> > > >> > >wrote:
> > > >> >
> > > >> > > Vadim,
> > > >> > >
> > > >> > > Controlled shutdown takes 2 parameters - number of retries and
> > > >> shutdown
> > > >> > > timeout. In every retry, controlled shutdown attempts to move
> > > leaders
> > > >> off
> > > >> > > of the broker that needs to be shutdown. If the controlled
> > shutdown
> > > >> runs
> > > >> > > out of retries, it proceeds to shutting down the broker even if
> it
> > > >> still
> > > >> > > hosts a few leaders. At LinkedIn, the script to bounce Kafka
> > brokers
> > > >> > waits
> > > >> > > for the under replicated partition count to drop to 0 before
> > > invoking
> > > >> > > controlled shutdown on the next broker. The aim is to avoid data
> > > loss
> > > >> > that
> > > >> > > occurs if you shut down a broker that still has some leaders. If
> > the
> > > >> > under
> > > >> > > replicated count never drops to 0, it indicates a bug in Kafka
> > code
> > > >> and
> > > >> > the
> > > >> > > script does not proceed to bouncing any more brokers in a
> cluster.
> > > We
> > > >> > > measure the time it takes to move "n" leaders off of some
> broker,
> > > and
> > > >> > > configure the shutdown timeout accordingly. We also configure
> the
> > > >> retries
> > > >> > > to a small number (2 or 3). If the controlled shutdown fails the
> > > >> retries,
> > > >> > > the broker shuts itself down anyways. In general, you want to
> > avoid
> > > >> hard
> > > >> > > killing (kill -9) a broker since that means the broker will run
> a
> > > long
> > > >> > > running log recovery process on startup. That significantly
> delays
> > > the
> > > >> > time
> > > >> > > the broker takes to rejoin the cluster.
> > > >> > >
> > > >> > > Thanks,
> > > >> > > Neha
> > > >> > >
> > > >> > >
> > > >> > > On Sun, Aug 18, 2013 at 3:33 PM, Vadim Keylis <
> > > vkeylis2...@gmail.com>
> > > >> > > wrote:
> > > >> > >
> > > >> > > > Good afternoon. We are running kafka on centos linux. I
> enabled
> > > >> > > controlled
> > > >> > > > shutdown in the property file. We are starting/stopping kafka
> > > using
> > > >> > init
> > > >> > > > script. The init script will issue term signal first followed
> 3
> > > >> seconds
> > > >> > > > later by kill signal. Is that right process to shutdown kafka?
> > > Which
> > > >> > > > startup/shutdown/restart script you guys use? What shutdown
> > > process
> > > >> > > > linkedin uses? What side effects c

Re: Kafka startup/restart process

2013-08-19 Thread Vadim Keylis
Tejas. I saw that too, but was hoping to avoid old grandpa approach:):).
That will work as well


On Mon, Aug 19, 2013 at 10:41 PM, Tejas Patil wrote:

> Not that I am some expert on this subject, but I can see that broker logs
> indicate the shutdown progress:
>
> https://github.com/tejasapatil/kafka/blob/0.8.0-beta1-candidate1/core/src/main/scala/kafka/server/KafkaServer.scala#L165
>
>
> On Mon, Aug 19, 2013 at 10:19 PM, Vadim Keylis  >wrote:
>
> > Neha. Thanks so much for explaining. That leaves only one open question.
> > How do you validate  that shutdown was successful if you do not have
> remote
> > jmx access unless besides setting timeout reasonable high?
> >
> > Thanks so much again,
> > Vadim
> >
> >
> > On Mon, Aug 19, 2013 at 9:11 PM, Neha Narkhede  > >wrote:
> >
> > > It depends on how much flexibility you need during the controlled
> > shutdown
> > > and whether you have remote jmx operations enabled in your production
> > Kafka
> > > cluster. The jmx controlled shutdown method will offer more flexibility
> > as
> > > your script will have the retry logic, you don't need to make config
> > > changes to Kafka brokers to change the timeout or the # of retries for
> > > controlled shutdown. On the other hand, the jmx controlled shutdown
> > method
> > > requires access to remote jmx on the broker. At LinkedIn, we do not
> have
> > > the ability to invoke jmx operations remotely on Kafka brokers in
> > > production. So we prefer using the controlled.shutdown.enable method.
> > >
> > > Thanks,
> > > Neha
> > >
> > >
> > > On Mon, Aug 19, 2013 at 12:34 PM, Vadim Keylis  > > >wrote:
> > >
> > > > What is preferred method for control shutdown using admin tool or
> > setting
> > > > as flag "controlled.shutdown.enable" to true? What is the advantage
> of
> > > > using one verses the other?
> > > >
> > > > Thanks,
> > > > Vadim
> > > >
> > > >
> > > > On Sun, Aug 18, 2013 at 11:05 PM, Vadim Keylis <
> vkeylis2...@gmail.com
> > > > >wrote:
> > > >
> > > > > thanks so much. Greatly appreciated.
> > > > >
> > > > >
> > > > > On Sun, Aug 18, 2013 at 10:00 PM, Neha Narkhede <
> > > neha.narkh...@gmail.com
> > > > >wrote:
> > > > >
> > > > >> It is exposed on every leader through the
> > > > >> "kafka.server.UnderReplicatedPartitions" jmx bean. It is
> independent
> > > of
> > > > >> the
> > > > >> controlled shutdown functionality.
> > > > >>
> > > > >> Thanks,
> > > > >> Neha
> > > > >>
> > > > >>
> > > > >> On Sun, Aug 18, 2013 at 8:33 PM, Vadim Keylis <
> > vkeylis2...@gmail.com>
> > > > >> wrote:
> > > > >>
> > > > >> > Neha. Thanks so much for response. How can I get under
> replicated
> > > > >> partition
> > > > >> > count during control shutdown that is configured in the property
> > > file?
> > > > >> >
> > > > >> > Thanks,
> > > > >> > Vadim
> > > > >> >
> > > > >> >
> > > > >> > On Sun, Aug 18, 2013 at 6:11 PM, Neha Narkhede <
> > > > neha.narkh...@gmail.com
> > > > >> > >wrote:
> > > > >> >
> > > > >> > > Vadim,
> > > > >> > >
> > > > >> > > Controlled shutdown takes 2 parameters - number of retries and
> > > > >> shutdown
> > > > >> > > timeout. In every retry, controlled shutdown attempts to move
> > > > leaders
> > > > >> off
> > > > >> > > of the broker that needs to be shutdown. If the controlled
> > > shutdown
> > > > >> runs
> > > > >> > > out of retries, it proceeds to shutting down the broker even
> if
> > it
> > > > >> still
> > > > >> > > hosts a few leaders. At LinkedIn, the script to bounce Kafka
> > > brokers
> > > > >> > waits
> > > > >> > > for the under replicated partition count to drop to 0 before
> > > > invoking
> > > > >> > > controlled shutdown on the next broker. The aim is to avoid
> data
> > > > loss
> > > > >> > that
> > > > >> > > occurs if you shut down a broker that still has some leaders.
> If
> > > the
> > > > >> > under
> > > > >> > > replicated count never drops to 0, it indicates a bug in Kafka
> > > code
> > > > >> and
> > > > >> > the
> > > > >> > > script does not proceed to bouncing any more brokers in a
> > cluster.
> > > > We
> > > > >> > > measure the time it takes to move "n" leaders off of some
> > broker,
> > > > and
> > > > >> > > configure the shutdown timeout accordingly. We also configure
> > the
> > > > >> retries
> > > > >> > > to a small number (2 or 3). If the controlled shutdown fails
> the
> > > > >> retries,
> > > > >> > > the broker shuts itself down anyways. In general, you want to
> > > avoid
> > > > >> hard
> > > > >> > > killing (kill -9) a broker since that means the broker will
> run
> > a
> > > > long
> > > > >> > > running log recovery process on startup. That significantly
> > delays
> > > > the
> > > > >> > time
> > > > >> > > the broker takes to rejoin the cluster.
> > > > >> > >
> > > > >> > > Thanks,
> > > > >> > > Neha
> > > > >> > >
> > > > >> > >
> > > > >> > > On Sun, Aug 18, 2013 at 3:33 PM, Vadim Keylis <
> > > > vkeylis2...@gmail.com>
> > > > >> > > wrote:
> > > > >> > >
> > > > >> > > > Good afternoon. We are running 

Correlation id

2013-08-19 Thread Timothy Chen
Hi,

This is probably a very obvious questoin, but I cannot find the answer for
this.

What does the correlation id mean in a producer request?

Tim


Re: Offset committing on rebalance

2013-08-19 Thread Ian Friedman
That's not it either. I just had all my consumers shut down on me with this:  

INFO  21:51:13,948 () ZkUtils$ - conflict in 
/consumers/flurry1/owners/dataLogPaths/1-183 data: 
flurry1_hs1030-1376964634130-dcc9192a-0 stored data: 
flurry1_hs1061-1376964609207-4b7f348b-0
INFO  21:51:13,948 () ZookeeperConsumerConnector - 
flurry1_hs1030-1376964634130-dcc9192a waiting for the partition ownership to be 
deleted: 1-183
INFO  21:51:13,950 () ZookeeperConsumerConnector - 
flurry1_hs1030-1376964634130-dcc9192a flurry1_hs1030-1376964634130-dcc9192a-0 
successfully owned partition 1-180 for topic dataLogPaths



and I've also been seeing:

INFO  21:51:15,971 () ZookeeperConsumerConnector - 
flurry1_hs1030-1376964634130-dcc9192a begin rebalancing consumer 
flurry1_hs1030-1376964634130-dcc9192a try #3
INFO  21:51:16,038 () ZookeeperConsumerConnector - 
flurry1_hs1030-1376964634130-dcc9192a exception during rebalance  
org.I0Itec.zkclient.exception.ZkNoNodeException: 
org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode 
for /consumers/flurry1/ids/flurry1_hs676-1376964612747-6f532caa
at org.I0Itec.zkclient.exception.ZkException.create(ZkException.java:47)
at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:685)
at org.I0Itec.zkclient.ZkClient.readData(ZkClient.java:766)
at org.I0Itec.zkclient.ZkClient.readData(ZkClient.java:761)
at org.I0Itec.zkclient.ZkClient.readData(ZkClient.java:750)
at org.I0Itec.zkclient.ZkClient.readData(ZkClient.java:744)
at kafka.utils.ZkUtils$.readData(ZkUtils.scala:162)
at kafka.consumer.TopicCount$.constructTopicCount(TopicCount.scala:66)
at 
kafka.utils.ZkUtils$$anonfun$getConsumersPerTopic$1.apply(ZkUtils.scala:259)
at 
kafka.utils.ZkUtils$$anonfun$getConsumersPerTopic$1.apply(ZkUtils.scala:258)
at scala.collection.Iterator$class.foreach(Iterator.scala:631)
at 
scala.collection.JavaConversions$JIteratorWrapper.foreach(JavaConversions.scala:474)
at scala.collection.IterableLike$class.foreach(IterableLike.scala:79)
at 
scala.collection.JavaConversions$JListWrapper.foreach(JavaConversions.scala:521)
at kafka.utils.ZkUtils$.getConsumersPerTopic(ZkUtils.scala:258)
at 
kafka.consumer.ZookeeperConsumerConnector$ZKRebalancerListener.kafka$consumer$ZookeeperConsumerConnector$ZKRebalancerListener$$rebalance(ZookeeperConsumerConnector.scala:478)
at 
kafka.consumer.ZookeeperConsumerConnector$ZKRebalancerListener$$anonfun$syncedRebalance$1.apply$mcVI$sp(ZookeeperConsumerConnector.scala:449)
at 
scala.collection.immutable.Range$ByOne$class.foreach$mVc$sp(Range.scala:285)
at 
scala.collection.immutable.Range$$anon$2.foreach$mVc$sp(Range.scala:265)
at 
kafka.consumer.ZookeeperConsumerConnector$ZKRebalancerListener.syncedRebalance(ZookeeperConsumerConnector.scala:444)
at 
kafka.consumer.ZookeeperConsumerConnector$ZKRebalancerListener$$anon$1.run(ZookeeperConsumerConnector.scala:401)
Caused by: org.apache.zookeeper.KeeperException$NoNodeException: 
KeeperErrorCode = NoNode for 
/consumers/flurry1/ids/flurry1_hs676-1376964612747-6f532caa
at org.apache.zookeeper.KeeperException.create(KeeperException.java:111)
at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:1151)
at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:1180)
at org.I0Itec.zkclient.ZkConnection.readData(ZkConnection.java:103)
at org.I0Itec.zkclient.ZkClient$9.call(ZkClient.java:770)
at org.I0Itec.zkclient.ZkClient$9.call(ZkClient.java:766)
at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
... 19 more
INFO  21:51:16,039 () ZookeeperConsumerConnector - 
flurry1_hs1030-1376964634130-dcc9192a end rebalancing consumer 
flurry1_hs1030-1376964634130-dcc9192a try #3


any ideas?  

--  
Ian Friedman


On Monday, August 19, 2013 at 11:58 PM, Jun Rao wrote:

> Any failure/restart of a consumer or a broker can also trigger a rebalance.
>  
> Thanks,
>  
> Jun
>  
>  
> On Mon, Aug 19, 2013 at 6:00 PM, Ian Friedman  (mailto:i...@flurry.com)> wrote:
>  
> > Jun, I read that FAQ entry you linked, but I am not seeing any Zookeeper
> > connection loss in the logs. It's rebalancing multiple times per minute,
> > though. Any idea what else could cause this? We're running kafka 0.7.2 on
> > approx 400 consumers against a topic with 400 partitions * 3 brokers.
> >  
> > --
> > Ian Friedman
> >  
> >  
> > On Thursday, August 15, 2013 at 11:52 AM, Jun Rao wrote:
> >  
> > > Yes, during rebalances, messages could be re-delievered since the new
> > owner
> > > of a partition starts fetching from the last checkpointed offset in ZK.
> > >  
> > > For reasons on why rebalances happen a lot, see
> > https://cwiki.apache.org/confluence/display/KAFKA/FAQ#FAQ-Whyaretheremanyrebalancesinmyconsume

Re: Offset committing on rebalance

2013-08-19 Thread Ian Friedman
Sorry, ignore that first exception, I believe that was caused by an actual 
manual shutdown. The NoNode exception though, has been popping up a lot, and I 
am not sure if it's relevant, but it seems to show up a bunch when the 
consumers decide it's time to rebalance continuously.  

--  
Ian Friedman


On Tuesday, August 20, 2013 at 2:17 AM, Ian Friedman wrote:

> That's not it either. I just had all my consumers shut down on me with this:  
>  
> INFO  21:51:13,948 () ZkUtils$ - conflict in 
> /consumers/flurry1/owners/dataLogPaths/1-183 data: 
> flurry1_hs1030-1376964634130-dcc9192a-0 stored data: 
> flurry1_hs1061-1376964609207-4b7f348b-0
> INFO  21:51:13,948 () ZookeeperConsumerConnector - 
> flurry1_hs1030-1376964634130-dcc9192a waiting for the partition ownership to 
> be deleted: 1-183
> INFO  21:51:13,950 () ZookeeperConsumerConnector - 
> flurry1_hs1030-1376964634130-dcc9192a flurry1_hs1030-1376964634130-dcc9192a-0 
> successfully owned partition 1-180 for topic dataLogPaths
>  
>  
>  
> and I've also been seeing:
>  
> INFO  21:51:15,971 () ZookeeperConsumerConnector - 
> flurry1_hs1030-1376964634130-dcc9192a begin rebalancing consumer 
> flurry1_hs1030-1376964634130-dcc9192a try #3
> INFO  21:51:16,038 () ZookeeperConsumerConnector - 
> flurry1_hs1030-1376964634130-dcc9192a exception during rebalance  
> org.I0Itec.zkclient.exception.ZkNoNodeException: 
> org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = 
> NoNode for /consumers/flurry1/ids/flurry1_hs676-1376964612747-6f532caa
> at 
> org.I0Itec.zkclient.exception.ZkException.create(ZkException.java:47)
> at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:685)
> at org.I0Itec.zkclient.ZkClient.readData(ZkClient.java:766)
> at org.I0Itec.zkclient.ZkClient.readData(ZkClient.java:761)
> at org.I0Itec.zkclient.ZkClient.readData(ZkClient.java:750)
> at org.I0Itec.zkclient.ZkClient.readData(ZkClient.java:744)
> at kafka.utils.ZkUtils$.readData(ZkUtils.scala:162)
> at kafka.consumer.TopicCount$.constructTopicCount(TopicCount.scala:66)
> at 
> kafka.utils.ZkUtils$$anonfun$getConsumersPerTopic$1.apply(ZkUtils.scala:259)
> at 
> kafka.utils.ZkUtils$$anonfun$getConsumersPerTopic$1.apply(ZkUtils.scala:258)
> at scala.collection.Iterator$class.foreach(Iterator.scala:631)
> at 
> scala.collection.JavaConversions$JIteratorWrapper.foreach(JavaConversions.scala:474)
> at scala.collection.IterableLike$class.foreach(IterableLike.scala:79)
> at 
> scala.collection.JavaConversions$JListWrapper.foreach(JavaConversions.scala:521)
> at kafka.utils.ZkUtils$.getConsumersPerTopic(ZkUtils.scala:258)
> at 
> kafka.consumer.ZookeeperConsumerConnector$ZKRebalancerListener.kafka$consumer$ZookeeperConsumerConnector$ZKRebalancerListener$$rebalance(ZookeeperConsumerConnector.scala:478)
> at 
> kafka.consumer.ZookeeperConsumerConnector$ZKRebalancerListener$$anonfun$syncedRebalance$1.apply$mcVI$sp(ZookeeperConsumerConnector.scala:449)
> at 
> scala.collection.immutable.Range$ByOne$class.foreach$mVc$sp(Range.scala:285)
> at 
> scala.collection.immutable.Range$$anon$2.foreach$mVc$sp(Range.scala:265)
> at 
> kafka.consumer.ZookeeperConsumerConnector$ZKRebalancerListener.syncedRebalance(ZookeeperConsumerConnector.scala:444)
> at 
> kafka.consumer.ZookeeperConsumerConnector$ZKRebalancerListener$$anon$1.run(ZookeeperConsumerConnector.scala:401)
> Caused by: org.apache.zookeeper.KeeperException$NoNodeException: 
> KeeperErrorCode = NoNode for 
> /consumers/flurry1/ids/flurry1_hs676-1376964612747-6f532caa
> at 
> org.apache.zookeeper.KeeperException.create(KeeperException.java:111)
> at 
> org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
> at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:1151)
> at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:1180)
> at org.I0Itec.zkclient.ZkConnection.readData(ZkConnection.java:103)
> at org.I0Itec.zkclient.ZkClient$9.call(ZkClient.java:770)
> at org.I0Itec.zkclient.ZkClient$9.call(ZkClient.java:766)
> at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
> ... 19 more
> INFO  21:51:16,039 () ZookeeperConsumerConnector - 
> flurry1_hs1030-1376964634130-dcc9192a end rebalancing consumer 
> flurry1_hs1030-1376964634130-dcc9192a try #3
>  
>  
> any ideas?  
>  
> --  
> Ian Friedman
>  
>  
> On Monday, August 19, 2013 at 11:58 PM, Jun Rao wrote:
>  
> > Any failure/restart of a consumer or a broker can also trigger a rebalance.
> >  
> > Thanks,
> >  
> > Jun
> >  
> >  
> > On Mon, Aug 19, 2013 at 6:00 PM, Ian Friedman  > (mailto:i...@flurry.com)> wrote:
> >  
> > > Jun, I read that FAQ entry you linked, but I am not seeing any Zookeeper
> > > connection loss in the logs. It's rebalancing multiple times per minute,
> > > though.

Re: Correlation id

2013-08-19 Thread Tejas Patil
Multiple produce requests are sent asynchronously over the same socket.
Suppose you send 2 requests and get back single response, how do you figure
out which one it corresponds to of those 2 requests ? Correlation Id helps
here.
AFAIK, correlation Id is added to produce requests and broker uses the same
id in its response so that the producer can keep track of its requests.
Correlation Id also helps in debugging issues as now you can uniquely
identify requests across producer and broker logs.


On Mon, Aug 19, 2013 at 11:01 PM, Timothy Chen  wrote:

> Hi,
>
> This is probably a very obvious questoin, but I cannot find the answer for
> this.
>
> What does the correlation id mean in a producer request?
>
> Tim
>


use kafka as dependency in sbt from central maven repository

2013-08-19 Thread Lorenz Knies
Hi,

i am having problems using kafka as a dependency in sbt.

with this simple build.sbt:

 name := "kafka-dependency-test"

 scalaVersion := "2.9.2"

 libraryDependencies += "org.apache.kafka" % "kafka_2.9.2" % "0.8.0-beta1"


when i do

 sbt update

i get the following error:

 sbt.ResolveException: unresolved dependency: 
org.apache.kafka#kafka_2.9.2;0.8.0-beta1: null name not allowed

i am using sbt 0.12.4 but also tried other versions with no luck

looks like the ivy implementation ignores the org and name attributes for the 
zookeeper dependency in the custom inline xml included in kafkas Build.scala

anybody else having these problems and maybe some advice for a 
solution/workaround?


regards

lorenz