Re: Improving the Kafka client ecosystem

2014-07-18 Thread Philip O'Toole
Thanks Jay -- some good ideas there. I agree strongly that fewer, more solid, non-Java clients are better than many shallow ones. Interesting that you feel we could do some more work in this area, as I thought it was well served (even if they have proliferated). One area I would like see docume

Re: message loss for sync producer, acks=2, topic replicas=3

2014-07-18 Thread Guozhang Wang
Hi Jiang, One thing you can try is to set acks=-1, and set the replica.lag.max.messages properly such that it will not kicks all follower replicas immediately under your produce load. Then if one of the follower replica is lagging and the other is not, this one will be dropped out of ISR and when

Re: New Consumer Design

2014-07-18 Thread Robert Withers
Hi Guozhang, Thank you for considering my suggestions. The security layer sounds like the right facet to design for these sorts of capabilities. Have you considered a chained ocap security model for the broker using hash tokens? This would provide for per-partition read/write capabilities wi

Re: message loss for sync producer, acks=2, topic replicas=3

2014-07-18 Thread Jiang Wu (Pricehistory) (BLOOMBERG/ 731 LEX -)
Hi Jun, I had some discussion with Guozhang about this; just realize it's not post to the maillist, so I post them again. Setting replica.lag.max.messages=10^12 is to ensure a follower will not be removed from ISR because it's behind the leader, so larger value is safer. 10^12 here is used as

Re: How to recover from a disk full situation in Kafka cluster?

2014-07-18 Thread Neha Narkhede
One option is to reduce the value of topic.metadata.refresh.interval.ms but the concern is that may end up sending too many requests to the brokers causing overhead. I'd suggest you use the new producer under org.apache.kafka.clients.producer.KafkaProducer that does not have the problem. It is fair

Re: How to recover from a disk full situation in Kafka cluster?

2014-07-18 Thread Connie Yang
Sure, I will try to take a snapshot of the data distribution when it happens next time. Assuming the topic.metadata.refresh.interval.ms is the concern, how should we "unstuck" our producers? The important note from that the documentation seems to suggest that the metadata refresh will only happen

Re: How to recover from a disk full situation in Kafka cluster?

2014-07-18 Thread Neha Narkhede
Does this mean that we should set "auto.leader.rebalance.enable" to true? I wouldn't recommend that just yet since it is not known to be very stable. You mentioned that only 2 brokers ever took the traffic and the replication factor is 2, makes me think that the producer stuck to 1 or few partitio

Re: Improving the Kafka client ecosystem

2014-07-18 Thread Jay Kreps
Basically my thought with getting a separate mailing list was to have a place specifically to discuss issues around clients. I don't see a lot of discussion about them on the main list. I thought perhaps this was because people don't like to ask questions which are about adjacent projects/code base

Re: Improving the Kafka client ecosystem

2014-07-18 Thread Jun Rao
Another important part of eco-system could be around the adaptors of getting data from other systems into Kafka and vice versa. So, for the ingestion part, this can include things like getting data from mysql, syslog, apache server log, etc. For the egress part, this can include putting Kafka data

Re: Interested in contributing to Kafka?

2014-07-18 Thread Jay Kreps
Yeah that was more or less what I was proposing. I posted my random ideas in the other thread, let me know what you think. -Jay On Fri, Jul 18, 2014 at 3:20 PM, Philip O'Toole wrote: > That sounds great -- I do think documentation is always very important (I've > had some ideas, but that's for

Re: message loss for sync producer, acks=2, topic replicas=3

2014-07-18 Thread Jun Rao
You probably don't need to set replica.lag.max.messages that high. You can observe the max lag in jmx and set the value to be a bit higher than that. Thanks, Jun On Fri, Jul 18, 2014 at 11:20 AM, Jiang Wu (Pricehistory) (BLOOMBERG/ 731 LEX -) wrote: > We tested ack=-1 with replica.lag.max.mes

Re: Interested in contributing to Kafka?

2014-07-18 Thread Philip O'Toole
That sounds great -- I do think documentation is always very important (I've had some ideas, but that's for another time). I would be very interested in more ideas around what you think is missing from the eco-system. That way people get to contribute, but can deepen their understanding in thei

Improving the Kafka client ecosystem

2014-07-18 Thread Jay Kreps
A question was asked in another thread about what was an effective way to contribute to the Kafka project for people who weren't very enthusiastic about writing Java/Scala code. I wanted to kind of advocate for an area I think is really important and not as good as it could be--the client ecosyste

Some doubts regarding kafka config parameters

2014-07-18 Thread shweta khare
hi, I have the following doubts regarding some kafka config parameters: For example if I have a Throughput topic with replication factor 1 and a single partition 0,then i will see the following files under /tmp/kafka-logs/Throughput_0: .index .log 000

Re: Interested in contributing to Kafka?

2014-07-18 Thread Jay Kreps
Hey Philip, That is awesome! You actually don't have to write Java or Scala code to contribute. There is the usual thing that presentations (which you are already doing), improving website docs, and general community participation are all at least as valuable as new code. However in addition to

Re: message loss for sync producer, acks=2, topic replicas=3

2014-07-18 Thread Jiang Wu (Pricehistory) (BLOOMBERG/ 731 LEX -)
We tested ack=-1 with replica.lag.max.messages=1. In this config no message loss was found. This is the only config we found to satisfy 1. no message loss and 2. service keeps available when 1 single broker is down. Are there other configs that can achieve the same, or stronger cons

Re: Kafka 0.8.1.1 / Can't read latest message

2014-07-18 Thread Guozhang Wang
Hi Tanguy, Just updated the wording on this protocol wiki a bit: https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI Hope it is now clearer. Guozhang On Thu, Jul 17, 2014 at 11:48 PM, Tanguy tlrx wrote: > Hi Jay, Hi Guozhang, >

Re: New Consumer Design

2014-07-18 Thread Guozhang Wang
Hi Rob, Sorry for the late reply. If I understand your approach correctly, it requires all brokers to remember the partition assignment of each consumer in order to decide whether or not authorizing the fetch request, correct? If we are indeed going to do such authorization for the security proje

Re: [DISCUSS] Kafka Security Specific Features

2014-07-18 Thread Pramod Deshmukh
Thanks Joe, I don't see any Out of memory error. Now I get exception when Producer fetches metadata for a topic Here is how I created the topic and run producer pdeshmukh$ bin/kafka-topics.sh --create --zookeeper localhost:2181 --replication-factor 1 --partitions 1 --topic secureTopic Created top

Re: [DISCUSS] Kafka Security Specific Features

2014-07-18 Thread Joe Stein
Hi Pramod, Can you increase KAFKA_HEAP_OPTS to lets say -Xmx1G in the kafka-console-producer.sh to see if that gets you further along please in your testing? Thanks! /*** Joe Stein Founder, Principal Consultant Big Data Open Source Security LLC http://

Re: In the middle of a leadership election, there is currently no leader...

2014-07-18 Thread Guozhang Wang
Hi Josh, Will this be due to this cause? https://cwiki.apache.org/confluence/display/KAFKA/FAQ#FAQ-Whycan%27tmyconsumers/producersconnecttothebrokers ? Guozhang On Fri, Jul 18, 2014 at 12:45 AM, Josh J wrote: > > What is the Kafka version you are using? > > 0.8.1.1. > > > And can you describ

Re: [DISCUSS] Kafka Security Specific Features

2014-07-18 Thread Pramod Deshmukh
Hello Raja/Joe, When I turn on security, i still get out of memory error on producer. Is this something to do with keys? Is there any other way I can connect to broker? *producer log* [2014-07-17 15:38:14,186] ERROR OOME with size 352518400 (kafka.network. BoundedByteBufferReceive) java.lang.OutOf

Re: In the middle of a leadership election, there is currently no leader...

2014-07-18 Thread Josh J
> What is the Kafka version you are using? 0.8.1.1. > And can you describe the steps to re-produce this issue? I'm running three kafka brokers using Docker. I was using the docker IP 172.17.42.1 for the kafka host address. It looks like kafka doesn't work nicely with the same local IP. I ended u