Re: Reassigning Partition Failing

2014-10-02 Thread Joe Stein
What version of zookeeper are you running?

First check to see if there is a znode for the /admin/reassign_partitions in
zookeeper.

If so, you could try a graceful shutdown of the controller broker.

Once the new controller leader elects on another broker look at zk the
znode /admin/reassign_partitions for the reassignment operation should be
gone.

If not, you are going to have to remove it through the shell or executor or
however you reliably and safely do that.

Re-run the command again.

I created a JIRA for this https://issues.apache.org/jira/browse/KAFKA-1665
issue once we can reproduce it we should fix it.

On Wed, Oct 1, 2014 at 6:43 PM, Lung, Paul pl...@ebay.com wrote:

 Hi All,

 I had a 0.8.1.1 Kafka Broker go down, and I was trying to use the reassign
 partition script to move topics off that broker. When I describe the
 topics, I see the following:

 Topic: mini__022active_120__33__mini Partition: 0 Leader: 2131118
 Replicas: 2131118,2166601,2163421 Isr: 2131118,2166601

 This shows that the broker “2163421” is down. So I create the following
 file /tmp/move_topic.json:
 {
 version: 1,
 partitions: [
 {
 topic: mini__022active_120__33__mini,
 partition: 0,
 replicas: [
 2131118, 2166601,  2156998
 ]
 }
 ]
 }

 And then do this:

 ./kafka-reassign-partitions.sh --execute --reassignment-json-file
 /tmp/move_topic.json
 Successfully started reassignment of partitions
 {version:1,partitions:[{topic:mini__022active_120__33__mini,partition:0,replicas:[2131118,2166601,2156998]}]}

 However, when I try to verify this, I get the following error:
 ./kafka-reassign-partitions.sh --verify --reassignment-json-file
 /tmp/move_topic.json
 Status of partition reassignment:
 ERROR: Assigned replicas (2131118,2166601,2156998,2163421) don't match the
 list of replicas for reassignment (2131118,2166601,2156998) for partition
 [mini__022active_120__33__mini,0]
 Reassignment of partition [mini__022active_120__33__mini,0] failed

 If I describe the topics, I now see there are 4 replicas. This has been
 like this for many hours now, so it seems to have permanently moved to 4
 replicas for some reason.
 Topic:mini__022active_120__33__mini PartitionCount:1
 ReplicationFactor:4 Configs:
 Topic: mini__022active_120__33__mini Partition: 0 Leader: 2131118
 Replicas: 2131118,2166601,2156998,2163421 Isr: 2131118,2166601

 If I re-execute and re-verify, I get the same error. So it seems to be
 wedged.

 Can someone help?

 Paul Lung





Re: Reassigning Partition Failing

2014-10-02 Thread Jun Rao
The reassign partition process only completes after the new replicas are
fully caught up and the old replicas are deleted. So, if the old replica is
down, the process can never complete, which is what you observed. In your
case, if you just want to replace a broker host with a new one, instead of
using the reassign partition tool, simply start a new broker with the same
broker id as the old one, the new broker will replicate all the data
automatically.

Thanks,

Jun

On Wed, Oct 1, 2014 at 3:43 PM, Lung, Paul pl...@ebay.com wrote:

 Hi All,

 I had a 0.8.1.1 Kafka Broker go down, and I was trying to use the reassign
 partition script to move topics off that broker. When I describe the
 topics, I see the following:

 Topic: mini__022active_120__33__mini Partition: 0 Leader: 2131118
 Replicas: 2131118,2166601,2163421 Isr: 2131118,2166601

 This shows that the broker “2163421” is down. So I create the following
 file /tmp/move_topic.json:
 {
 version: 1,
 partitions: [
 {
 topic: mini__022active_120__33__mini,
 partition: 0,
 replicas: [
 2131118, 2166601,  2156998
 ]
 }
 ]
 }

 And then do this:

 ./kafka-reassign-partitions.sh --execute --reassignment-json-file
 /tmp/move_topic.json
 Successfully started reassignment of partitions
 {version:1,partitions:[{topic:mini__022active_120__33__mini,partition:0,replicas:[2131118,2166601,2156998]}]}

 However, when I try to verify this, I get the following error:
 ./kafka-reassign-partitions.sh --verify --reassignment-json-file
 /tmp/move_topic.json
 Status of partition reassignment:
 ERROR: Assigned replicas (2131118,2166601,2156998,2163421) don't match the
 list of replicas for reassignment (2131118,2166601,2156998) for partition
 [mini__022active_120__33__mini,0]
 Reassignment of partition [mini__022active_120__33__mini,0] failed

 If I describe the topics, I now see there are 4 replicas. This has been
 like this for many hours now, so it seems to have permanently moved to 4
 replicas for some reason.
 Topic:mini__022active_120__33__mini PartitionCount:1
 ReplicationFactor:4 Configs:
 Topic: mini__022active_120__33__mini Partition: 0 Leader: 2131118
 Replicas: 2131118,2166601,2156998,2163421 Isr: 2131118,2166601

 If I re-execute and re-verify, I get the same error. So it seems to be
 wedged.

 Can someone help?

 Paul Lung





Connection Reset By Peer

2014-10-02 Thread Aniket Kulkarni
Thank you, Neha. I appreciate your help.

-- 
*Have a nice day.*
Regards,
Aniket Kulkarni.


Different free space and log files between nodes

2014-10-02 Thread Dayo Oliyide
Hi,

I've noticed an interesting behaviour which I hope someone can fully
explain.
I have 3 Kafka Node cluster with a setting of log.retention.hours=168 (7
days) and log.segment.bytes=536870912.

I recently restarted one of the nodes and it's uptime is now 3 days behind
than the other 2.

After about 7 days I noticed that the other 2 nodes cleared out an equal
amount of stale logs/data,
but the restarted node didn't cleared out the same amount. The restarted
node only cleared out a
similar amount 3 days later. Generally now the restarted node seems to be 3
days (free space wise) behind
the other 2.

I noticed that certain partition log and index files on the restarted node
are different to the other 2.
Below is an example (Node C is the restarted one)


Node A - Topic Z Partition 12
[488 Sep 26 11:47]  00046460.index
[1781829  Sep 24 12:09]  00046460.log
[   10485760 Sep 28 22:23]  00046522.index
[1536693  Sep 28 22:23]  00046522.log

Node B - Topic Z Partition 12
[488 Sep 26 11:47]  00046460.index
[1781829  Sep 24 12:09]  00046460.log
[   10485760 Sep 28 22:23]  00046522.index
[1536693  Sep 28 22:23]  00046522.log

Node C - Topic Z Partition 12
[   10485760 Sep 28 22:23]  00046485.index
[2277311  Sep 28 22:23]  00046485.log

I can see that Node C's base offset (log prefix) is in between the offsets
of the other logs on Node A  B, and that
suggests to me some partition 12 messages are on Node A  B but not on Node
C?

I was hoping someone could help me figure out what's happening.

Thanks
Dayo


Re: kafka producer performance test

2014-10-02 Thread Sa Li
Thanks, Jay,

Here is what I did this morning, I git clone the latest version of kafka
from git, (I am currently using kafka 8.0) now it is 8.1.1, and it use
gradle to build project. I am having trouble to build it. I installed
gradle, and run ./gradlew jar in kafka root directory, it comes out:
Error: Could not find or load main class
org.gradle.wrapper.GradleWrapperMain

Any idea about this.

Thanks

Alec

On Wed, Oct 1, 2014 at 9:21 PM, Jay Kreps jay.kr...@gmail.com wrote:

 Hi Sa,

 That script was developed with the new producer that is included on
 trunk. Checkout trunk and build and it should be there.

 -Jay

 On Wed, Oct 1, 2014 at 7:55 PM, Sa Li sal...@gmail.com wrote:
  Hi, All
 
  I built a 3-node kafka cluster, I want to make performance test, I found
 someone post following thread, that is exactly the problem I have:
  -
  While testing kafka producer performance, I found 2 testing scripts.
 
  1) performance testing script in kafka distribution
 
  bin/kafka-producer-perf-test.sh --broker-list localhost:9092 --messages
  1000 --topic test --threads 10 --message-size 100 --batch-size 1
  --compression-codec 1
 
  2) performance testing script mentioned in
 
 
 https://engineering.linkedin.com/kafka/benchmarking-apache-kafka-2-million-writes-second-three-cheap-machines
 
  bin/kafka-run-class.sh
  org.apache.kafka.clients.tools.ProducerPerformance test6 5000 100
  -1 acks=1 bootstrap.servers=esv4-hcl198.grid.linkedin.com:9092
  buffer.memory=67108864 batch.size=8196
 
  based on org.apache.kafka.clients.producer.Producer.
 
  ——
 
 
  I was unable to duplicate either of above method, I figure the commands
 are outdated, anyone point me how to do such test with new command?
 
 
  thanks
 
  Alec




-- 

Alec Li


Re: [DISCUSS] 0.8.1.2 Release

2014-10-02 Thread Kane Kane
Having the same question: what happened to 0.8.2 release, when it's
supposed to happen?

Thanks.

On Tue, Sep 30, 2014 at 12:49 PM, Jonathan Weeks
jonathanbwe...@gmail.com wrote:
 I was one asking for 0.8.1.2 a few weeks back, when 0.8.2 was at least 6-8 
 weeks out.

 If we truly believe that 0.8.2 will go “golden” and stable in 2-3 weeks, I, 
 for one, don’t need a 0.8.1.2, but it depends on the confidence in shipping 
 0.8.2 soonish.

 YMMV,

 -Jonathan


 On Sep 30, 2014, at 12:37 PM, Neha Narkhede neha.narkh...@gmail.com wrote:

 Can we discuss the need for 0.8.1.2? I'm wondering if it's related to the
 timeline of 0.8.2 in any way? For instance, if we can get 0.8.2 out in the
 next 2-3 weeks, do we still need to get 0.8.1.2 out or can people just
 upgrade to 0.8.2?

 On Tue, Sep 30, 2014 at 9:53 AM, Joe Stein joe.st...@stealth.ly wrote:

 Hi, I wanted to kick off a specific discussion on a 0.8.1.2 release.

 Here are the JIRAs I would like to propose to back port a patch (if not
 already done so) and apply them to the 0.8.1 branch for a 0.8.1.2 release

 https://issues.apache.org/jira/browse/KAFKA-1502 (source jar is empty)
 https://issues.apache.org/jira/browse/KAFKA-1419 (cross build for scala
 2.11)
 https://issues.apache.org/jira/browse/KAFKA-1382 (Update zkVersion on
 partition state update failures)
 https://issues.apache.org/jira/browse/KAFKA-1490 (remove gradlew initial
 setup output from source distribution)
 https://issues.apache.org/jira/browse/KAFKA-1645 (some more jars in our
 src
 release)

 If the community and committers can comment on the patches proposed that
 would be great. If I missed any bring them up or if you think any I have
 proposed shouldn't be int he release bring that up too please.

 Once we have consensus on this thread my thought was that I would apply and
 commit the agreed to tickets to the 0.8.1 branch. If any tickets don't
 apply of course a back port patch has to happen through our standard
 process (not worried about that we have some engineering cycles to
 contribute to making that happen). Once that is all done, I will build
 0.8.1.2 release artifacts and call a VOTE for RC1.

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




can't run kafka example code

2014-10-02 Thread Sa Li
Hi, all

Here I want to run example code associated with kafka package, I run as
readme says:

To run the demo using scripts:
+
+   1. Start Zookeeper and the Kafka server
+   2. For simple consumer demo, run bin/java-simple-consumer-demo.sh
+   3. For unlimited producer-consumer run, run
bin/java-producer-consumer-demo.sh

but I got such error,

:bin/../../project/boot/scala-2.8.0/lib/*.jar:bin/../../core/lib_managed/scala_2.8.0/compile/*.jar:bin/../../core/lib/*.jar:bin/../../core/target/scala_2.8.0/
*.jar:bin/../../examples/target/scala_2.8.0/*.jar

 Error: Could not find or load main class kafka.examples.SimpleConsumerDemo

But I already build the package under kafka directory, I can see the
class in examples/target/classes/kafka/examples


Any idea about this issue?


thanks




-- 

Alec Li


Re: kafka producer performance test

2014-10-02 Thread Guozhang Wang
Hello Sa,

KAFKA-1490 introduces a new step of downloading the wrapper, details are
included in the latest README file.

Guozhang

On Thu, Oct 2, 2014 at 11:00 AM, Sa Li sal...@gmail.com wrote:

 Thanks, Jay,

 Here is what I did this morning, I git clone the latest version of kafka
 from git, (I am currently using kafka 8.0) now it is 8.1.1, and it use
 gradle to build project. I am having trouble to build it. I installed
 gradle, and run ./gradlew jar in kafka root directory, it comes out:
 Error: Could not find or load main class
 org.gradle.wrapper.GradleWrapperMain

 Any idea about this.

 Thanks

 Alec

 On Wed, Oct 1, 2014 at 9:21 PM, Jay Kreps jay.kr...@gmail.com wrote:

  Hi Sa,
 
  That script was developed with the new producer that is included on
  trunk. Checkout trunk and build and it should be there.
 
  -Jay
 
  On Wed, Oct 1, 2014 at 7:55 PM, Sa Li sal...@gmail.com wrote:
   Hi, All
  
   I built a 3-node kafka cluster, I want to make performance test, I
 found
  someone post following thread, that is exactly the problem I have:
   -
   While testing kafka producer performance, I found 2 testing scripts.
  
   1) performance testing script in kafka distribution
  
   bin/kafka-producer-perf-test.sh --broker-list localhost:9092 --messages
   1000 --topic test --threads 10 --message-size 100 --batch-size
 1
   --compression-codec 1
  
   2) performance testing script mentioned in
  
  
 
 https://engineering.linkedin.com/kafka/benchmarking-apache-kafka-2-million-writes-second-three-cheap-machines
  
   bin/kafka-run-class.sh
   org.apache.kafka.clients.tools.ProducerPerformance test6 5000 100
   -1 acks=1 bootstrap.servers=esv4-hcl198.grid.linkedin.com:9092
   buffer.memory=67108864 batch.size=8196
  
   based on org.apache.kafka.clients.producer.Producer.
  
   ——
  
  
   I was unable to duplicate either of above method, I figure the commands
  are outdated, anyone point me how to do such test with new command?
  
  
   thanks
  
   Alec
 



 --

 Alec Li




-- 
-- Guozhang


Re: Different free space and log files between nodes

2014-10-02 Thread Guozhang Wang
Hello Dayo,

This is a known issue, since today Kafka's log rolling / cleaning policy
depends on the creation timestamp of the segment files, which could be
modified upon partition migration / broker restart, it can cause the server
to not honor the specified log cleaning config. Some more details ca be
found at

KAFKA-881
KAFKA-979

Guozhang


On Thu, Oct 2, 2014 at 10:48 AM, Dayo Oliyide dayo.oliy...@gmail.com
wrote:

 Hi,

 I've noticed an interesting behaviour which I hope someone can fully
 explain.
 I have 3 Kafka Node cluster with a setting of log.retention.hours=168 (7
 days) and log.segment.bytes=536870912.

 I recently restarted one of the nodes and it's uptime is now 3 days behind
 than the other 2.

 After about 7 days I noticed that the other 2 nodes cleared out an equal
 amount of stale logs/data,
 but the restarted node didn't cleared out the same amount. The restarted
 node only cleared out a
 similar amount 3 days later. Generally now the restarted node seems to be 3
 days (free space wise) behind
 the other 2.

 I noticed that certain partition log and index files on the restarted node
 are different to the other 2.
 Below is an example (Node C is the restarted one)


 Node A - Topic Z Partition 12
 [488 Sep 26 11:47]  00046460.index
 [1781829  Sep 24 12:09]  00046460.log
 [   10485760 Sep 28 22:23]  00046522.index
 [1536693  Sep 28 22:23]  00046522.log

 Node B - Topic Z Partition 12
 [488 Sep 26 11:47]  00046460.index
 [1781829  Sep 24 12:09]  00046460.log
 [   10485760 Sep 28 22:23]  00046522.index
 [1536693  Sep 28 22:23]  00046522.log

 Node C - Topic Z Partition 12
 [   10485760 Sep 28 22:23]  00046485.index
 [2277311  Sep 28 22:23]  00046485.log

 I can see that Node C's base offset (log prefix) is in between the offsets
 of the other logs on Node A  B, and that
 suggests to me some partition 12 messages are on Node A  B but not on Node
 C?

 I was hoping someone could help me figure out what's happening.

 Thanks
 Dayo




-- 
-- Guozhang


Re: kafka docker

2014-10-02 Thread Joe Stein
Yes, here is a vagrant virtual box setup
https://github.com/stealthly/scala-kafka

On Thu, Oct 2, 2014 at 3:51 PM, Mingtao Zhang mail2ming...@gmail.com
wrote:

 Thanks for the response!

 Any one has it working on Virtualbox? which is the case for Winddos/Mac?

 How do we configure the network adapter?

 Best Regards,
 Mingtao

 Best Regards,
 Mingtao

 On Tue, Sep 30, 2014 at 3:31 PM, Joe Stein joe.st...@stealth.ly wrote:

Is there a 'Kafka-HDFS with Camus' docker as well one can play
 around
  with?
 
  Not that I know of.  These folks
  http://blog.sequenceiq.com/blog/2014/09/15/hadoop-2-5-1-docker/ have
 nice
  Hadoop docker containers may be a good starting point.
 
   What's the story for persisting data with Docker? Do you use a data
  volume or do you just start fresh every time you start the Docker
 instance?
 
  We only use it for development  testing so starting fresh and
  bootstrapping data is how we use it when we do. Volumes should work fine
  though for persisting if need be.
 
  /***
   Joe Stein
   Founder, Principal Consultant
   Big Data Open Source Security LLC
   http://www.stealth.ly
   Twitter: @allthingshadoop http://www.twitter.com/allthingshadoop
  /
 
  On Tue, Sep 30, 2014 at 3:00 PM, Daniel Compton d...@danielcompton.net
  wrote:
 
   Hi Joe
  
   What's the story for persisting data with Docker? Do you use a data
  volume
   or do you just start fresh every time you start the Docker instance?
  
   Daniel.
  
On 1/10/2014, at 7:13 am, Buntu Dev buntu...@gmail.com wrote:
   
Thanks Joe.. seems quite handy. Is there a 'Kafka-HDFS with Camus'
   docker
as well one can play around with?
   
On Tue, Sep 30, 2014 at 9:00 AM, Joe Stein joe.st...@stealth.ly
   wrote:
   
You need to change the advertised hostname.
   
Take a look
 https://registry.hub.docker.com/u/stealthly/docker-kafka/
   and
https://registry.hub.docker.com/u/stealthly/docker-zookeeper/ we
 use
  it
often for local testing here is how to start
   
 https://github.com/stealthly/docker-kafka/blob/master/start-broker.sh
   e.g.
   
  https://github.com/stealthly/metrics-kafka/blob/master/bootstrap.sh#L13
   
/***
Joe Stein
Founder, Principal Consultant
Big Data Open Source Security LLC
http://www.stealth.ly
Twitter: @allthingshadoop http://www.twitter.com/allthingshadoop
/
   
On Tue, Sep 30, 2014 at 11:46 AM, Mingtao Zhang 
  mail2ming...@gmail.com
   
wrote:
   
Hi,
   
Any one has dockerized kafka working?
   
Should we specify the ip address?
   
I expected everything working on just localhost but saw this
 SEVERE:
Producer connection to 172.17.0.3:9092 unsuccessful.
   
Thanks in advance!
   
Best Regards,
Mingtao
   
  
 



Re: kafka producer performance test

2014-10-02 Thread Sa Li
Thanks Guozhang

I tried this as in KAFKA-1490:

git clone https://git-wip-us.apache.org/repos/asf/kafka.git

cd kafka

gradle


but fails to build:

FAILURE: Build failed with an exception.

* Where:

 Script '/home/stuser/trunk/gradle/license.gradle' line: 2

* What went wrong:

 A problem occurred evaluating script.

 Could not find method create() for arguments [downloadLicenses, class 
 nl.javadude.gradle.plugins.license.DownloadLicenses] on task set.

* Try:

Run with --stacktrace option to get the stack trace. Run with --info
or --debug option to get more log output.

BUILD FAILED




Seems it is really not that straightforward to build


thanks




On Thu, Oct 2, 2014 at 12:56 PM, Guozhang Wang wangg...@gmail.com wrote:

 Hello Sa,

 KAFKA-1490 introduces a new step of downloading the wrapper, details are
 included in the latest README file.

 Guozhang

 On Thu, Oct 2, 2014 at 11:00 AM, Sa Li sal...@gmail.com wrote:

  Thanks, Jay,
 
  Here is what I did this morning, I git clone the latest version of kafka
  from git, (I am currently using kafka 8.0) now it is 8.1.1, and it use
  gradle to build project. I am having trouble to build it. I installed
  gradle, and run ./gradlew jar in kafka root directory, it comes out:
  Error: Could not find or load main class
  org.gradle.wrapper.GradleWrapperMain
 
  Any idea about this.
 
  Thanks
 
  Alec
 
  On Wed, Oct 1, 2014 at 9:21 PM, Jay Kreps jay.kr...@gmail.com wrote:
 
   Hi Sa,
  
   That script was developed with the new producer that is included on
   trunk. Checkout trunk and build and it should be there.
  
   -Jay
  
   On Wed, Oct 1, 2014 at 7:55 PM, Sa Li sal...@gmail.com wrote:
Hi, All
   
I built a 3-node kafka cluster, I want to make performance test, I
  found
   someone post following thread, that is exactly the problem I have:
-
While testing kafka producer performance, I found 2 testing scripts.
   
1) performance testing script in kafka distribution
   
bin/kafka-producer-perf-test.sh --broker-list localhost:9092
 --messages
1000 --topic test --threads 10 --message-size 100 --batch-size
  1
--compression-codec 1
   
2) performance testing script mentioned in
   
   
  
 
 https://engineering.linkedin.com/kafka/benchmarking-apache-kafka-2-million-writes-second-three-cheap-machines
   
bin/kafka-run-class.sh
org.apache.kafka.clients.tools.ProducerPerformance test6 5000 100
-1 acks=1 bootstrap.servers=esv4-hcl198.grid.linkedin.com:9092
buffer.memory=67108864 batch.size=8196
   
based on org.apache.kafka.clients.producer.Producer.
   
——
   
   
I was unable to duplicate either of above method, I figure the
 commands
   are outdated, anyone point me how to do such test with new command?
   
   
thanks
   
Alec
  
 
 
 
  --
 
  Alec Li
 



 --
 -- Guozhang




-- 

Alec Li


can't gradle

2014-10-02 Thread Sa Li
I git clone the latest kafka package, why can't I build the package

gradle

FAILURE: Build failed with an exception.

* Where:
Script '/home/ubuntu/kafka/gradle/license.gradle' line: 2

* What went wrong:
A problem occurred evaluating script.
 Could not find method create() for arguments [downloadLicenses, class 
 nl.javadude.gradle.plugins.license.DownloadLicenses] on task set.

* Try:
Run with --stacktrace option to get the stack trace. Run with --info
or --debug option to get more log output.

BUILD FAILED


Thanks


-- 

Alec Li


Re: kafka producer performance test

2014-10-02 Thread Sa Li
I can't really gradle through, even clone the latest trunk, anyone having
same issue?



On Thu, Oct 2, 2014 at 1:55 PM, Sa Li sal...@gmail.com wrote:

 Thanks Guozhang

 I tried this as in KAFKA-1490:

 git clone https://git-wip-us.apache.org/repos/asf/kafka.git

 cd kafka

 gradle


 but fails to build:

 FAILURE: Build failed with an exception.

 * Where:

  Script '/home/stuser/trunk/gradle/license.gradle' line: 2

 * What went wrong:

  A problem occurred evaluating script.

  Could not find method create() for arguments [downloadLicenses, class 
  nl.javadude.gradle.plugins.license.DownloadLicenses] on task set.

 * Try:

 Run with --stacktrace option to get the stack trace. Run with --info or 
 --debug option to get more log output.

 BUILD FAILED


 

 Seems it is really not that straightforward to build


 thanks




 On Thu, Oct 2, 2014 at 12:56 PM, Guozhang Wang wangg...@gmail.com wrote:

 Hello Sa,

 KAFKA-1490 introduces a new step of downloading the wrapper, details are
 included in the latest README file.

 Guozhang

 On Thu, Oct 2, 2014 at 11:00 AM, Sa Li sal...@gmail.com wrote:

  Thanks, Jay,
 
  Here is what I did this morning, I git clone the latest version of kafka
  from git, (I am currently using kafka 8.0) now it is 8.1.1, and it use
  gradle to build project. I am having trouble to build it. I installed
  gradle, and run ./gradlew jar in kafka root directory, it comes out:
  Error: Could not find or load main class
  org.gradle.wrapper.GradleWrapperMain
 
  Any idea about this.
 
  Thanks
 
  Alec
 
  On Wed, Oct 1, 2014 at 9:21 PM, Jay Kreps jay.kr...@gmail.com wrote:
 
   Hi Sa,
  
   That script was developed with the new producer that is included on
   trunk. Checkout trunk and build and it should be there.
  
   -Jay
  
   On Wed, Oct 1, 2014 at 7:55 PM, Sa Li sal...@gmail.com wrote:
Hi, All
   
I built a 3-node kafka cluster, I want to make performance test, I
  found
   someone post following thread, that is exactly the problem I have:
-
While testing kafka producer performance, I found 2 testing scripts.
   
1) performance testing script in kafka distribution
   
bin/kafka-producer-perf-test.sh --broker-list localhost:9092
 --messages
1000 --topic test --threads 10 --message-size 100 --batch-size
  1
--compression-codec 1
   
2) performance testing script mentioned in
   
   
  
 
 https://engineering.linkedin.com/kafka/benchmarking-apache-kafka-2-million-writes-second-three-cheap-machines
   
bin/kafka-run-class.sh
org.apache.kafka.clients.tools.ProducerPerformance test6 5000
 100
-1 acks=1 bootstrap.servers=esv4-hcl198.grid.linkedin.com:9092
buffer.memory=67108864 batch.size=8196
   
based on org.apache.kafka.clients.producer.Producer.
   
——
   
   
I was unable to duplicate either of above method, I figure the
 commands
   are outdated, anyone point me how to do such test with new command?
   
   
thanks
   
Alec
  
 
 
 
  --
 
  Alec Li
 



 --
 -- Guozhang




 --

 Alec Li




-- 

Alec Li


Re: kafka producer performance test

2014-10-02 Thread Guozhang Wang
Did you installed gradle as the README stated?

You need to have [gradle](http://www.gradle.org/installation) installed.

Guozhang

On Thu, Oct 2, 2014 at 1:55 PM, Sa Li sal...@gmail.com wrote:

 Thanks Guozhang

 I tried this as in KAFKA-1490:

 git clone https://git-wip-us.apache.org/repos/asf/kafka.git

 cd kafka

 gradle


 but fails to build:

 FAILURE: Build failed with an exception.

 * Where:

  Script '/home/stuser/trunk/gradle/license.gradle' line: 2

 * What went wrong:

  A problem occurred evaluating script.

  Could not find method create() for arguments [downloadLicenses, class
 nl.javadude.gradle.plugins.license.DownloadLicenses] on task set.

 * Try:

 Run with --stacktrace option to get the stack trace. Run with --info
 or --debug option to get more log output.

 BUILD FAILED


 

 Seems it is really not that straightforward to build


 thanks




 On Thu, Oct 2, 2014 at 12:56 PM, Guozhang Wang wangg...@gmail.com wrote:

  Hello Sa,
 
  KAFKA-1490 introduces a new step of downloading the wrapper, details are
  included in the latest README file.
 
  Guozhang
 
  On Thu, Oct 2, 2014 at 11:00 AM, Sa Li sal...@gmail.com wrote:
 
   Thanks, Jay,
  
   Here is what I did this morning, I git clone the latest version of
 kafka
   from git, (I am currently using kafka 8.0) now it is 8.1.1, and it use
   gradle to build project. I am having trouble to build it. I installed
   gradle, and run ./gradlew jar in kafka root directory, it comes out:
   Error: Could not find or load main class
   org.gradle.wrapper.GradleWrapperMain
  
   Any idea about this.
  
   Thanks
  
   Alec
  
   On Wed, Oct 1, 2014 at 9:21 PM, Jay Kreps jay.kr...@gmail.com wrote:
  
Hi Sa,
   
That script was developed with the new producer that is included on
trunk. Checkout trunk and build and it should be there.
   
-Jay
   
On Wed, Oct 1, 2014 at 7:55 PM, Sa Li sal...@gmail.com wrote:
 Hi, All

 I built a 3-node kafka cluster, I want to make performance test, I
   found
someone post following thread, that is exactly the problem I have:
 -
 While testing kafka producer performance, I found 2 testing
 scripts.

 1) performance testing script in kafka distribution

 bin/kafka-producer-perf-test.sh --broker-list localhost:9092
  --messages
 1000 --topic test --threads 10 --message-size 100 --batch-size
   1
 --compression-codec 1

 2) performance testing script mentioned in


   
  
 
 https://engineering.linkedin.com/kafka/benchmarking-apache-kafka-2-million-writes-second-three-cheap-machines

 bin/kafka-run-class.sh
 org.apache.kafka.clients.tools.ProducerPerformance test6 5000
 100
 -1 acks=1 bootstrap.servers=esv4-hcl198.grid.linkedin.com:9092
 buffer.memory=67108864 batch.size=8196

 based on org.apache.kafka.clients.producer.Producer.

 ——


 I was unable to duplicate either of above method, I figure the
  commands
are outdated, anyone point me how to do such test with new command?


 thanks

 Alec
   
  
  
  
   --
  
   Alec Li
  
 
 
 
  --
  -- Guozhang
 



 --

 Alec Li




-- 
-- Guozhang


Re: multi-node and multi-broker kafka cluster setup

2014-10-02 Thread Sa Li
Daniel, thanks for reply

It is still the learn curve to me to setup the cluster, we finally want to
make connection between kafka cluster and storm cluster. As you mentioned,
seems 1 single broker per node is more efficient, is it good to handle
multiple topics? For my case, say I can build the 3-node kafka cluster, and
three brokers, and certainly that will limit the replica number, as far as
I understand, broker number should greater or equal to replica number.

For the zk Server, my understanding after play around is: I should run zk
Server server for each kafka node, I could zk.connect to single zk server
in kafka server.properties, and all the broker info will store in that
zkserver, But I may think it might be better to store each individual
broker info in local zkServer, then when zkCli,sh, we can see things under
/brokers/ids.

Is that good solution? I am using such architecture now.

thanks

On Tue, Sep 30, 2014 at 1:02 PM, Daniel Compton d...@danielcompton.net
wrote:

 Hi Sa

 While it's possible to run multiple brokers on a single machine, I would
 be interested to hear why you would want to. Kafka is very efficient and
 can use all of the system resources under load. Running multiple brokers
 would increase zookeeper load, force resource sharing between the Kafka
 processes, and require more admin overhead.

 Additionally, you almost certainly want to run three Zookeepers. Two
 Zookeepers gives you no more reliability than one because ZK voting is
 based on a majority vote. If neither ZK can reach a majority on its own
 then it will fail. More info at
 http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A7

 Daniel.

  On 1/10/2014, at 4:35 am, Guozhang Wang wangg...@gmail.com wrote:
 
  Hello,
 
  In general it is not required to have the kafka brokers installed on the
  same nodes of the zk servers, and each node can host multiple kafka
  brokers: you just need to make sure they do not share the same port and
 the
  same data dir.
 
  Guozhang
 
  On Mon, Sep 29, 2014 at 8:31 PM, Sa Li sal...@gmail.com wrote:
 
  Hi,
  I am kinda newbie to kafka, I plan to build a cluster with multiple
 nodes,
  and multiple brokers on each node, I can find tutorials for set multiple
  brokers cluster in single node, say
 
 
 http://www.michael-noll.com/blog/2013/03/13/running-a-multi-broker-apache-kafka-cluster-on-a-single-node/
  Also I can find some instructions for multiple node setup, but with
 single
  broker on each node. I have not seen any documents to teach me how to
 setup
  multiple nodes cluster and multiple brokers in each node. I notice some
  documents points out: we should install kafka on each node which makes
  sense, and all the brokers in each node should connect to same
 zookeeper. I
  am confused since I thought I could setup a zookeeper ensemble cluster
  separately, and all the brokers connecting to this zookeeper cluster and
  this zk cluster doesn’t have to be the server hosting the kafka, but
 some
  tutorial says I should install zookeeper on each kafka node.
 
  Here is my plan:
  - I have three nodes: kfServer1, kfserver2, kfserver3,
  - kfserver1 and kfserver2 are configured as the zookeeper ensemble,
 which
  i have done.
   zk.connect=kfserver1:2181,kfserver2:2181
  - broker1, broker2, broker3 are in kfserver1,
   broker4, broker5, broker6 are on kfserver2,
   broker7, broker8, broker9 are on kfserver3.
 
  When I am configuring, the zk DataDir is in local directory of each
 node,
  instead located at the zk ensemble directory, is that correct? So far, I
  couldnot make above scheme working, anyone have ever made multi-node and
  multi-broker kafka cluster setup?
 
  thanks
 
  Alec
 
 
  --
  -- Guozhang




-- 

Alec Li


Re: multi-node and multi-broker kafka cluster setup

2014-10-02 Thread Sa Li
Just clarify, I am using 3 zkServer ensemble, myid: 1, 2, 3. But in each
kafka node server.properties of each broker, I make zk.connect to
localhost, which means the broker info stored in local zkServer, I know it
is bit of weird, other than assign the broker info automatically by
zkServer leader.

On Thu, Oct 2, 2014 at 2:25 PM, Sa Li sal...@gmail.com wrote:

 Daniel, thanks for reply

 It is still the learn curve to me to setup the cluster, we finally want to
 make connection between kafka cluster and storm cluster. As you mentioned,
 seems 1 single broker per node is more efficient, is it good to handle
 multiple topics? For my case, say I can build the 3-node kafka cluster, and
 three brokers, and certainly that will limit the replica number, as far as
 I understand, broker number should greater or equal to replica number.

 For the zk Server, my understanding after play around is: I should run zk
 Server server for each kafka node, I could zk.connect to single zk server
 in kafka server.properties, and all the broker info will store in that
 zkserver, But I may think it might be better to store each individual
 broker info in local zkServer, then when zkCli,sh, we can see things under
 /brokers/ids.

 Is that good solution? I am using such architecture now.

 thanks

 On Tue, Sep 30, 2014 at 1:02 PM, Daniel Compton d...@danielcompton.net
 wrote:

 Hi Sa

 While it's possible to run multiple brokers on a single machine, I would
 be interested to hear why you would want to. Kafka is very efficient and
 can use all of the system resources under load. Running multiple brokers
 would increase zookeeper load, force resource sharing between the Kafka
 processes, and require more admin overhead.

 Additionally, you almost certainly want to run three Zookeepers. Two
 Zookeepers gives you no more reliability than one because ZK voting is
 based on a majority vote. If neither ZK can reach a majority on its own
 then it will fail. More info at
 http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A7

 Daniel.

  On 1/10/2014, at 4:35 am, Guozhang Wang wangg...@gmail.com wrote:
 
  Hello,
 
  In general it is not required to have the kafka brokers installed on the
  same nodes of the zk servers, and each node can host multiple kafka
  brokers: you just need to make sure they do not share the same port and
 the
  same data dir.
 
  Guozhang
 
  On Mon, Sep 29, 2014 at 8:31 PM, Sa Li sal...@gmail.com wrote:
 
  Hi,
  I am kinda newbie to kafka, I plan to build a cluster with multiple
 nodes,
  and multiple brokers on each node, I can find tutorials for set
 multiple
  brokers cluster in single node, say
 
 
 http://www.michael-noll.com/blog/2013/03/13/running-a-multi-broker-apache-kafka-cluster-on-a-single-node/
  Also I can find some instructions for multiple node setup, but with
 single
  broker on each node. I have not seen any documents to teach me how to
 setup
  multiple nodes cluster and multiple brokers in each node. I notice some
  documents points out: we should install kafka on each node which makes
  sense, and all the brokers in each node should connect to same
 zookeeper. I
  am confused since I thought I could setup a zookeeper ensemble cluster
  separately, and all the brokers connecting to this zookeeper cluster
 and
  this zk cluster doesn’t have to be the server hosting the kafka, but
 some
  tutorial says I should install zookeeper on each kafka node.
 
  Here is my plan:
  - I have three nodes: kfServer1, kfserver2, kfserver3,
  - kfserver1 and kfserver2 are configured as the zookeeper ensemble,
 which
  i have done.
   zk.connect=kfserver1:2181,kfserver2:2181
  - broker1, broker2, broker3 are in kfserver1,
   broker4, broker5, broker6 are on kfserver2,
   broker7, broker8, broker9 are on kfserver3.
 
  When I am configuring, the zk DataDir is in local directory of each
 node,
  instead located at the zk ensemble directory, is that correct? So far,
 I
  couldnot make above scheme working, anyone have ever made multi-node
 and
  multi-broker kafka cluster setup?
 
  thanks
 
  Alec
 
 
  --
  -- Guozhang




 --

 Alec Li




-- 

Alec Li


auto topic creation not working for attempts to consume non-existing topic

2014-10-02 Thread Stevo Slavić
Hello Apache Kafka community,

auto.create.topics.enable configuration option docs state:
Enable auto creation of topic on the server. If this is set to true then
attempts to produce, consume, or fetch metadata for a non-existent topic
will automatically create it with the default replication factor and number
of partitions.

I read this that topic should be created on any attempt to consume
non-existing topic.

With auto.create.topics.enable left at default or explicitly set to true,
attempts to consume non existing topic, using blocking consumer, or a
non-blocking consumer with positive consumer.timeout.ms configured, will
not result in topic creation (I cannot see one registered in ZooKeeper).

Additionally, for non-blocking consumer with timeout, no offset will be
recorded. This further means, if such consumer had auto.offset.reset set to
largest, that it will miss at least one message (initial one that when
published creates the topic), even though consumer attempted to read before
first message was published.

I'm using Kafka 0.8.1.1 but I see same issue exists in current trunk.

Is this a known issue? Or are my expectations/assumptions wrong and this is
expected behavior?

Kind regards,
Stevo Slavic.


Re: [DISCUSS] 0.8.1.2 Release

2014-10-02 Thread Jun Rao
We already cut an 0.8.2 release branch. The plan is to have the remaining
blockers resolved before releasing it. Hopefully this will just take a
couple of weeks.

https://issues.apache.org/jira/browse/KAFKA-1663?filter=-4jql=project%20%3D%20KAFKA%20AND%20status%20in%20(Open%2C%20%22In%20Progress%22%2C%20Reopened%2C%20%22Patch%20Available%22)%20AND%20priority%20%3D%20Blocker%20AND%20fixVersion%20%3D%200.8.2%20ORDER%20BY%20createdDate%20DESC

Thanks,

Jun

On Thu, Oct 2, 2014 at 11:28 AM, Kane Kane kane.ist...@gmail.com wrote:

 Having the same question: what happened to 0.8.2 release, when it's
 supposed to happen?

 Thanks.

 On Tue, Sep 30, 2014 at 12:49 PM, Jonathan Weeks
 jonathanbwe...@gmail.com wrote:
  I was one asking for 0.8.1.2 a few weeks back, when 0.8.2 was at least
 6-8 weeks out.
 
  If we truly believe that 0.8.2 will go “golden” and stable in 2-3 weeks,
 I, for one, don’t need a 0.8.1.2, but it depends on the confidence in
 shipping 0.8.2 soonish.
 
  YMMV,
 
  -Jonathan
 
 
  On Sep 30, 2014, at 12:37 PM, Neha Narkhede neha.narkh...@gmail.com
 wrote:
 
  Can we discuss the need for 0.8.1.2? I'm wondering if it's related to
 the
  timeline of 0.8.2 in any way? For instance, if we can get 0.8.2 out in
 the
  next 2-3 weeks, do we still need to get 0.8.1.2 out or can people just
  upgrade to 0.8.2?
 
  On Tue, Sep 30, 2014 at 9:53 AM, Joe Stein joe.st...@stealth.ly
 wrote:
 
  Hi, I wanted to kick off a specific discussion on a 0.8.1.2 release.
 
  Here are the JIRAs I would like to propose to back port a patch (if not
  already done so) and apply them to the 0.8.1 branch for a 0.8.1.2
 release
 
  https://issues.apache.org/jira/browse/KAFKA-1502 (source jar is empty)
  https://issues.apache.org/jira/browse/KAFKA-1419 (cross build for
 scala
  2.11)
  https://issues.apache.org/jira/browse/KAFKA-1382 (Update zkVersion on
  partition state update failures)
  https://issues.apache.org/jira/browse/KAFKA-1490 (remove gradlew
 initial
  setup output from source distribution)
  https://issues.apache.org/jira/browse/KAFKA-1645 (some more jars in
 our
  src
  release)
 
  If the community and committers can comment on the patches proposed
 that
  would be great. If I missed any bring them up or if you think any I
 have
  proposed shouldn't be int he release bring that up too please.
 
  Once we have consensus on this thread my thought was that I would
 apply and
  commit the agreed to tickets to the 0.8.1 branch. If any tickets don't
  apply of course a back port patch has to happen through our standard
  process (not worried about that we have some engineering cycles to
  contribute to making that happen). Once that is all done, I will build
  0.8.1.2 release artifacts and call a VOTE for RC1.
 
  /***
  Joe Stein
  Founder, Principal Consultant
  Big Data Open Source Security LLC
  http://www.stealth.ly
  Twitter: @allthingshadoop http://www.twitter.com/allthingshadoop
  /
 
 



Re: can't run kafka example code

2014-10-02 Thread Jun Rao
Can you follow the example in quickstart (
http://kafka.apache.org/documentation.html#quickstart)?

Thanks,

Jun

On Thu, Oct 2, 2014 at 12:01 PM, Sa Li sal...@gmail.com wrote:

 Hi, all

 Here I want to run example code associated with kafka package, I run as
 readme says:

 To run the demo using scripts:
 +
 +   1. Start Zookeeper and the Kafka server
 +   2. For simple consumer demo, run bin/java-simple-consumer-demo.sh
 +   3. For unlimited producer-consumer run, run
 bin/java-producer-consumer-demo.sh

 but I got such error,


 :bin/../../project/boot/scala-2.8.0/lib/*.jar:bin/../../core/lib_managed/scala_2.8.0/compile/*.jar:bin/../../core/lib/*.jar:bin/../../core/target/scala_2.8.0/
 *.jar:bin/../../examples/target/scala_2.8.0/*.jar

  Error: Could not find or load main class kafka.examples.SimpleConsumerDemo

 But I already build the package under kafka directory, I can see the
 class in examples/target/classes/kafka/examples


 Any idea about this issue?


 thanks




 --

 Alec Li



Re: can't gradle

2014-10-02 Thread Jun Rao
Hmm, not sure what the issue is. You can also just copy the following files
from the 0.8.1 branch.

gradle/wrapper/

gradle-wrapper.jar gradle-wrapper.properties

Thanks,

Jun

On Thu, Oct 2, 2014 at 2:05 PM, Sa Li sal...@gmail.com wrote:

 I git clone the latest kafka package, why can't I build the package

 gradle

 FAILURE: Build failed with an exception.

 * Where:
 Script '/home/ubuntu/kafka/gradle/license.gradle' line: 2

 * What went wrong:
 A problem occurred evaluating script.
  Could not find method create() for arguments [downloadLicenses, class
 nl.javadude.gradle.plugins.license.DownloadLicenses] on task set.

 * Try:
 Run with --stacktrace option to get the stack trace. Run with --info
 or --debug option to get more log output.

 BUILD FAILED


 Thanks


 --

 Alec Li



Re: can't gradle

2014-10-02 Thread Sa Li
Thank you all,  I am able to gradle now, here is my mistake, I install
gradle by apt-get, and from gradle web, but system automatically pick
apt-get gradle to run, and this version is quite outdated, what I did to
apt-get remove gradle, and add higher version gradle to /etc/environment,
now it works.

Hope this can help anyone who have the similar problem, always download and
install latest version.



On Thu, Oct 2, 2014 at 3:02 PM, Jun Rao jun...@gmail.com wrote:

 Hmm, not sure what the issue is. You can also just copy the following files
 from the 0.8.1 branch.

 gradle/wrapper/

 gradle-wrapper.jar gradle-wrapper.properties

 Thanks,

 Jun

 On Thu, Oct 2, 2014 at 2:05 PM, Sa Li sal...@gmail.com wrote:

  I git clone the latest kafka package, why can't I build the package
 
  gradle
 
  FAILURE: Build failed with an exception.
 
  * Where:
  Script '/home/ubuntu/kafka/gradle/license.gradle' line: 2
 
  * What went wrong:
  A problem occurred evaluating script.
   Could not find method create() for arguments [downloadLicenses, class
  nl.javadude.gradle.plugins.license.DownloadLicenses] on task set.
 
  * Try:
  Run with --stacktrace option to get the stack trace. Run with --info
  or --debug option to get more log output.
 
  BUILD FAILED
 
 
  Thanks
 
 
  --
 
  Alec Li
 




-- 

Alec Li


Re: auto topic creation not working for attempts to consume non-existing topic

2014-10-02 Thread Jun Rao
In general, only writers should trigger auto topic creation, but not the
readers. So, a topic can be auto created by the producer, but not the
consumer.

Thanks,

Jun

On Thu, Oct 2, 2014 at 2:44 PM, Stevo Slavić ssla...@gmail.com wrote:

 Hello Apache Kafka community,

 auto.create.topics.enable configuration option docs state:
 Enable auto creation of topic on the server. If this is set to true then
 attempts to produce, consume, or fetch metadata for a non-existent topic
 will automatically create it with the default replication factor and number
 of partitions.

 I read this that topic should be created on any attempt to consume
 non-existing topic.

 With auto.create.topics.enable left at default or explicitly set to true,
 attempts to consume non existing topic, using blocking consumer, or a
 non-blocking consumer with positive consumer.timeout.ms configured, will
 not result in topic creation (I cannot see one registered in ZooKeeper).

 Additionally, for non-blocking consumer with timeout, no offset will be
 recorded. This further means, if such consumer had auto.offset.reset set to
 largest, that it will miss at least one message (initial one that when
 published creates the topic), even though consumer attempted to read before
 first message was published.

 I'm using Kafka 0.8.1.1 but I see same issue exists in current trunk.

 Is this a known issue? Or are my expectations/assumptions wrong and this is
 expected behavior?

 Kind regards,
 Stevo Slavic.