Re: could new java producer miss callbacks after successful send?

2015-02-10 Thread Jay Kreps
Cool, let us know if you can uncover anything as that would be a serious issue if we are dropping callbacks. -Jay On Tue, Feb 10, 2015 at 6:44 PM, Steven Wu wrote: > couldn't reproduce/confirm the issue with my test. send 6 million msgs from > 6 instances. got 6 million callbacks. > > this coul

Re: createMessageStreams vs createMessageStreamsByFilter

2015-02-10 Thread tao xiao
Do you know when the new consumer API will be publicly available? On Wed, Feb 11, 2015 at 10:43 AM, Guozhang Wang wrote: > Yes, it can get stuck. For example, AC and BC are processed by two > different processes and AC processors gets stuck, hence AC messages will > fill up in the consumer's buf

Re: could new java producer miss callbacks after successful send?

2015-02-10 Thread Steven Wu
couldn't reproduce/confirm the issue with my test. send 6 million msgs from 6 instances. got 6 million callbacks. this could be some metric issues. On Mon, Feb 9, 2015 at 8:23 PM, Steven Wu wrote: > I don't have strong evidence that this is a bug yet. let me write some > test program and see if

Re: createMessageStreams vs createMessageStreamsByFilter

2015-02-10 Thread Guozhang Wang
Yes, it can get stuck. For example, AC and BC are processed by two different processes and AC processors gets stuck, hence AC messages will fill up in the consumer's buffer and eventually prevents the fetcher thread to put more data into it; the fetcher thread will be blocked on that and not be abl

Re: createMessageStreams vs createMessageStreamsByFilter

2015-02-10 Thread tao xiao
Thank you Guozhang for your detailed explanation. In your example createMessageStreamsByFilter("*C" => 3) since threads are shared among topics there may be situation where all 3 threads threads get stuck with topic AC e.g. topic is empty which will be holding the connecting threads (setting consu

Re: Lack of JMX LogCleaner and LogCleanerManager metrics

2015-02-10 Thread Gwen Shapira
I'm counting this as three committer +1 :) Opened: KAFKA-1944 - Rename LogCleaner and related classes to LogCompactor On Tue, Feb 10, 2015 at 4:31 PM, Guozhang Wang wrote: > +1 > > On Tue, Feb 10, 2015 at 2:23 PM, Joel Koshy wrote: > > > +1

Re: Lack of JMX LogCleaner and LogCleanerManager metrics

2015-02-10 Thread Guozhang Wang
+1 On Tue, Feb 10, 2015 at 2:23 PM, Joel Koshy wrote: > +1 > > On Tue, Feb 10, 2015 at 01:32:13PM -0800, Jay Kreps wrote: > > I agree that would be a better name. We could rename it if everyone likes > > Compactor better. > > > > -Jay > > > > On Tue, Feb 10, 2015 at 9:33 AM, Gwen Shapira > wrot

Kafka rebalancing causes Zookeeper to fail

2015-02-10 Thread Zhang, Tian
Jun and Ahmed, I am using Kafka 0.8.1.1. and Zookeeper 3.3.6. and I am hitting exactly the same error occasionally. What is the final solution worked for you? Thanks. Tian

Re: Lack of JMX LogCleaner and LogCleanerManager metrics

2015-02-10 Thread Joel Koshy
+1 On Tue, Feb 10, 2015 at 01:32:13PM -0800, Jay Kreps wrote: > I agree that would be a better name. We could rename it if everyone likes > Compactor better. > > -Jay > > On Tue, Feb 10, 2015 at 9:33 AM, Gwen Shapira wrote: > > > btw. the name LogCleaner is seriously misleading. Its more of a

Re: Issue with auto topic creation as well

2015-02-10 Thread Sumit Rangwala
On Thu, Feb 5, 2015 at 12:37 PM, Sumit Rangwala wrote: > > > On Wed, Feb 4, 2015 at 9:23 PM, Jun Rao wrote: > >> Could you try the 0.8.2.0 release? It fixed one issue related to topic >> creation. >> > Jun, If you need more info let me know. Seems like TopicMetadataResponse is expecting more fi

Re: Lack of JMX LogCleaner and LogCleanerManager metrics

2015-02-10 Thread Jay Kreps
I agree that would be a better name. We could rename it if everyone likes Compactor better. -Jay On Tue, Feb 10, 2015 at 9:33 AM, Gwen Shapira wrote: > btw. the name LogCleaner is seriously misleading. Its more of a log > compacter. > Deleting old logs happens elsewhere from what I've seen. > >

Re: What ports does a Kafka Producer/Consumer use other than 2180's and 9090's?

2015-02-10 Thread Su She
Okay got it, thanks! On Tue, Feb 10, 2015 at 12:45 PM, Zijing Guo wrote: > The kafka's port can be configured through server.properties, and if you > use kafka's zk, then you can modify the zookeeper.properties file, for > clientPort. > > On Tuesday, February 10, 2015 2:11 PM, Su She > wro

Re: High Latency in Kafka

2015-02-10 Thread Andrey Yegorov
I am not familiar with logstash, but in custom log replay tool (used to replay messages logged locally in case if e.g. kafka was not available and useful in some other scenarios) I've seen it reaching 30,000 messages/sec with avg message size of 4.5 kilobytes, all with regular production load on ka

Re: What ports does a Kafka Producer/Consumer use other than 2180's and 9090's?

2015-02-10 Thread Zijing Guo
The kafka's port can be configured through server.properties, and if you use kafka's zk, then you can modify the zookeeper.properties file, for clientPort. On Tuesday, February 10, 2015 2:11 PM, Su She wrote: I was looking at the documentation and I see that the broker/server/consum

What ports does a Kafka Producer/Consumer use other than 2180's and 9090's?

2015-02-10 Thread Su She
I was looking at the documentation and I see that the broker/server/consumer listen to ports 2181 and 9092, but can be configured for other ports in that range. Are there any other ports that a kafka producer/consumer use in the back-end to publish/consume messages? Thank you!

Recommended way of handling brokers coming up / down in the SimpleConsumer

2015-02-10 Thread Rajiv Kurian
I am using the SimpleConsumer to consume specific partitions on specific processes. The workflow is kind of like this: i) An external arbiter assigns partitions to a specific processes. It provides the guarantees of: a) All partitions are consumed by the cluster. b) A single partition is onl

Re: createMessageStreams vs createMessageStreamsByFilter

2015-02-10 Thread Guozhang Wang
I was not clear before .. for createMessageStreamsByFilter each matched topic will have num-threads, but shared: i.e. there will be totally num-threads created, but each thread will be responsible for fetching all matched topics. A more concrete example: say you have topic AC: 3 partitions, topic

Re: Poor performance consuming multiple topics

2015-02-10 Thread Guozhang Wang
CJ, On the consumer side there is another two configs named "fetch.min.bytes" and "fetch.wait.max.ms": http://kafka.apache.org/documentation.html#consumerconfigs They controls how long the fetch request will wait on data. What are the values on your consumers? You may want to try tuning these t

Re: Ship Kafka in on prem product

2015-02-10 Thread Gwen Shapira
Thanks Joe :) We definitely found Kafka to be solid enough for enterprise. It does require taking care in shipping "safe" configurations. Whether its secure enough depends on requirements - with no authentication, authorization or encryptions, its definitely not secure enough for some use cases (

Re: ping kafka server

2015-02-10 Thread Joe Stein
One good canary is to have a topic you loop through every partition and write a timestamp on every broker for n in 1..P; do; echo $(datetime)>kafkacat -b host:port -t ops -p n done and trap any errors and output accordingly. You can get fancy and start validating reading and stuff but it is a cana

Re: Lack of JMX LogCleaner and LogCleanerManager metrics

2015-02-10 Thread Gwen Shapira
btw. the name LogCleaner is seriously misleading. Its more of a log compacter. Deleting old logs happens elsewhere from what I've seen. Gwen On Tue, Feb 10, 2015 at 8:07 AM, Jay Kreps wrote: > Probably you need to enable the log cleaner for those to show up? We > disable it by default and so I

Re: ping kafka server

2015-02-10 Thread Magnus Edenhill
Relying on just the TCP connection getting established seems a bit poor, the easiest non-intrusive approach is probably to query the broker for metadata, e.g.: kafkacat -b mybroker -L 2015-02-10 1:47 GMT+01:00 Koert Kuipers : > a simple nagios check_tcp works fine. as gwen indicated kafka closes

Re: Ship Kafka in on prem product

2015-02-10 Thread Joe Stein
Hey Geeta, have you taken a look at http://www.cloudera.com/content/cloudera/en/developers/home/cloudera-labs/apache-kafka.html ??? I really liked what they did with 0.8.2-beta and the changes for 0.8.2.0 are looking good https://github.com/cloudera-labs/kafka/tree/cdh5-0.8.2_1.1.0 I don't know wha

Re: Get Latest Offset for Specific Topic for All Partition

2015-02-10 Thread Gwen Shapira
Yes, its the size of the log in bytes. We can improve on documentation a bit :) Metric is defined in Log.scala: newGauge("Size", new Gauge[Long] { def value = size }, tags) so the value is "size", and: /** * The size of the log in bytes */ def size: Long = logSegmen

Re: regarding custom msg

2015-02-10 Thread Gaurav Agarwal
sorry for the late reply while sending the message i serialize the message and send the same across on topic . the message private static class TestMessage implements Serializable{ private String time; ublic String getTime() { return time; } public void setTime(String time) { this.time = time; }

Re: createMessageStreams vs createMessageStreamsByFilter

2015-02-10 Thread tao xiao
Guozhang, Do you mean that each regex matched topic owns number of threads that get passed in to createMessageStreamsByFilter ? For example in below code If I have 3 matched topics each of which has 2 partitions then I should have 3 * 2 = 6 threads in total with each topic owning 2 threads. Topic

Ship Kafka in on prem product

2015-02-10 Thread Geeta Gharpure
Hello I am looking for a reference about shipping kafka as part of Enterprise on prem product. Does anyone know if Kafka can be redistributed safely and/or is found secure enough to ship as part of enterprise product? Thanks in advance for your help. Thanks and regards, Geeta

Re: Lack of JMX LogCleaner and LogCleanerManager metrics

2015-02-10 Thread Jay Kreps
Probably you need to enable the log cleaner for those to show up? We disable it by default and so I think those metrics never get created. -Jay On Tue, Feb 10, 2015 at 3:33 AM, o...@sematext.com wrote: > Hello, > > I have a problem with some JMX metrics. In Kafka source code I see > LogCleaner

Re: createMessageStreams vs createMessageStreamsByFilter

2015-02-10 Thread Guozhang Wang
createMessageStreams is used for consuming from specific topic(s), where you can put a map of [topic-name, num-threads] as its input parameters; createMessageStreamsByFilter is used for consuming from wildcard topics, where you can put a (regex, num-threads) as its input parameters, and for each r

Lack of JMX LogCleaner and LogCleanerManager metrics

2015-02-10 Thread o...@sematext.com
Hello, I have a problem with some JMX metrics. In Kafka source code I see LogCleaner (has metrics: 'max-buffer-utilization-percent', 'cleaner-recopy-percent', 'max-clean-time-secs') and LogCleanerManager (has metric: 'max-dirty-percent') But I don't see this beans&metrics in jconsole when I att

Re: High Latency in Kafka

2015-02-10 Thread Vineet Mishra
Hi Gwen, Well I have gone through this link while trying to setup my Logstash Kafka handler, https://github.com/joekiller/logstash-kafka I could achieve what I was looking for but the performance is badly affected while trying to write a big file of GB's. I guess there should be some way so as t

createMessageStreams vs createMessageStreamsByFilter

2015-02-10 Thread tao xiao
Hi team, I am comparing the differences between ConsumerConnector.createMessageStreams and ConsumerConnector.createMessageStreamsByFilter. My understanding is that createMessageStreams creates x number of threads (x is the number of threads passed in to the method) dedicated to the specified topic