Re: Performance issues
By increasing partitions and using kafka from master branch I was able to cut down the response times into half. But it still seems high and it looks like there still is a delay between a successful post and the first time message is seen by the consumers. There are plenty of resources available. Is there a way I can easily check breakdown of latency on every tier. For eg: producer - broker - consumer On Wed, Oct 22, 2014 at 2:37 PM, Neha Narkhede neha.narkh...@gmail.com wrote: the server.properties file doesn't have all the properties. You can add it there and try your test. On Wed, Oct 22, 2014 at 11:41 AM, Mohit Anchlia mohitanch...@gmail.com wrote: I can't find this property in server.properties file. Is that the right place to set this parameter? On Tue, Oct 21, 2014 at 6:27 PM, Jun Rao jun...@gmail.com wrote: Could you also set replica.fetch.wait.max.ms in the broker to sth much smaller? Thanks, Jun On Tue, Oct 21, 2014 at 2:15 PM, Mohit Anchlia mohitanch...@gmail.com wrote: I set the property to 1 in the consumer code that is passed to createJavaConsumerConnector code, but it didn't seem to help props.put(fetch.wait.max.ms, fetchMaxWait); On Tue, Oct 21, 2014 at 1:21 PM, Guozhang Wang wangg...@gmail.com wrote: This is a consumer config: fetch.wait.max.ms On Tue, Oct 21, 2014 at 11:39 AM, Mohit Anchlia mohitanch...@gmail.com wrote: Is this a parameter I need to set it in kafka server or on the client side? Also, can you help point out which one exactly is consumer max wait time from this list? https://kafka.apache.org/08/configuration.html On Tue, Oct 21, 2014 at 11:35 AM, Jay Kreps jay.kr...@gmail.com wrote: There was a bug that could lead to the fetch request from the consumer hitting it's timeout instead of being immediately triggered by the produce request. To see if you are effected by that set you consumer max wait time to 1 ms and see if the latency drops to 1 ms (or, alternately, try with trunk and see if that fixes the problem). The reason I suspect this problem is because the default timeout in the java consumer is 100ms. -Jay On Tue, Oct 21, 2014 at 11:06 AM, Mohit Anchlia mohitanch...@gmail.com wrote: This is the version I am using: kafka_2.10-0.8.1.1 I think this is fairly recent version On Tue, Oct 21, 2014 at 10:57 AM, Jay Kreps jay.kr...@gmail.com wrote: What version of Kafka is this? Can you try the same test against trunk? We fixed a couple of latency related bugs which may be the cause. -Jay On Tue, Oct 21, 2014 at 10:50 AM, Mohit Anchlia mohitanch...@gmail.com wrote: It's consistently close to 100ms which makes me believe that there are some settings that I might have to tweak, however, I am not sure how to confirm that assumption :) On Tue, Oct 21, 2014 at 8:53 AM, Mohit Anchlia mohitanch...@gmail.com wrote: I have a java test that produces messages and then consumer consumers it. Consumers are active all the time. There is 1 consumer for 1 producer. I am measuring the time between the message is successfully written to the queue and the time consumer picks it up. On Tue, Oct 21, 2014 at 8:32 AM, Neha Narkhede neha.narkh...@gmail.com wrote: Can you give more information about the performance test? Which test? Which queue? How did you measure the dequeue latency. On Mon, Oct 20, 2014 at 5:09 PM, Mohit Anchlia mohitanch...@gmail.com wrote: I am running a performance test and from what I am seeing is that messages are taking about 100ms to pop from the queue itself and hence making the test slow. I am looking for pointers of how I can troubleshoot this issue. There seems to be plenty of CPU and IO available. I am running 22 producers and 22 consumers in the same group. -- -- Guozhang
Re: Performance issues
I can't find this property in server.properties file. Is that the right place to set this parameter? On Tue, Oct 21, 2014 at 6:27 PM, Jun Rao jun...@gmail.com wrote: Could you also set replica.fetch.wait.max.ms in the broker to sth much smaller? Thanks, Jun On Tue, Oct 21, 2014 at 2:15 PM, Mohit Anchlia mohitanch...@gmail.com wrote: I set the property to 1 in the consumer code that is passed to createJavaConsumerConnector code, but it didn't seem to help props.put(fetch.wait.max.ms, fetchMaxWait); On Tue, Oct 21, 2014 at 1:21 PM, Guozhang Wang wangg...@gmail.com wrote: This is a consumer config: fetch.wait.max.ms On Tue, Oct 21, 2014 at 11:39 AM, Mohit Anchlia mohitanch...@gmail.com wrote: Is this a parameter I need to set it in kafka server or on the client side? Also, can you help point out which one exactly is consumer max wait time from this list? https://kafka.apache.org/08/configuration.html On Tue, Oct 21, 2014 at 11:35 AM, Jay Kreps jay.kr...@gmail.com wrote: There was a bug that could lead to the fetch request from the consumer hitting it's timeout instead of being immediately triggered by the produce request. To see if you are effected by that set you consumer max wait time to 1 ms and see if the latency drops to 1 ms (or, alternately, try with trunk and see if that fixes the problem). The reason I suspect this problem is because the default timeout in the java consumer is 100ms. -Jay On Tue, Oct 21, 2014 at 11:06 AM, Mohit Anchlia mohitanch...@gmail.com wrote: This is the version I am using: kafka_2.10-0.8.1.1 I think this is fairly recent version On Tue, Oct 21, 2014 at 10:57 AM, Jay Kreps jay.kr...@gmail.com wrote: What version of Kafka is this? Can you try the same test against trunk? We fixed a couple of latency related bugs which may be the cause. -Jay On Tue, Oct 21, 2014 at 10:50 AM, Mohit Anchlia mohitanch...@gmail.com wrote: It's consistently close to 100ms which makes me believe that there are some settings that I might have to tweak, however, I am not sure how to confirm that assumption :) On Tue, Oct 21, 2014 at 8:53 AM, Mohit Anchlia mohitanch...@gmail.com wrote: I have a java test that produces messages and then consumer consumers it. Consumers are active all the time. There is 1 consumer for 1 producer. I am measuring the time between the message is successfully written to the queue and the time consumer picks it up. On Tue, Oct 21, 2014 at 8:32 AM, Neha Narkhede neha.narkh...@gmail.com wrote: Can you give more information about the performance test? Which test? Which queue? How did you measure the dequeue latency. On Mon, Oct 20, 2014 at 5:09 PM, Mohit Anchlia mohitanch...@gmail.com wrote: I am running a performance test and from what I am seeing is that messages are taking about 100ms to pop from the queue itself and hence making the test slow. I am looking for pointers of how I can troubleshoot this issue. There seems to be plenty of CPU and IO available. I am running 22 producers and 22 consumers in the same group. -- -- Guozhang
Re: Performance issues
I have a java test that produces messages and then consumer consumers it. Consumers are active all the time. There is 1 consumer for 1 producer. I am measuring the time between the message is successfully written to the queue and the time consumer picks it up. On Tue, Oct 21, 2014 at 8:32 AM, Neha Narkhede neha.narkh...@gmail.com wrote: Can you give more information about the performance test? Which test? Which queue? How did you measure the dequeue latency. On Mon, Oct 20, 2014 at 5:09 PM, Mohit Anchlia mohitanch...@gmail.com wrote: I am running a performance test and from what I am seeing is that messages are taking about 100ms to pop from the queue itself and hence making the test slow. I am looking for pointers of how I can troubleshoot this issue. There seems to be plenty of CPU and IO available. I am running 22 producers and 22 consumers in the same group.
Re: Performance issues
This is the version I am using: kafka_2.10-0.8.1.1 I think this is fairly recent version On Tue, Oct 21, 2014 at 10:57 AM, Jay Kreps jay.kr...@gmail.com wrote: What version of Kafka is this? Can you try the same test against trunk? We fixed a couple of latency related bugs which may be the cause. -Jay On Tue, Oct 21, 2014 at 10:50 AM, Mohit Anchlia mohitanch...@gmail.com wrote: It's consistently close to 100ms which makes me believe that there are some settings that I might have to tweak, however, I am not sure how to confirm that assumption :) On Tue, Oct 21, 2014 at 8:53 AM, Mohit Anchlia mohitanch...@gmail.com wrote: I have a java test that produces messages and then consumer consumers it. Consumers are active all the time. There is 1 consumer for 1 producer. I am measuring the time between the message is successfully written to the queue and the time consumer picks it up. On Tue, Oct 21, 2014 at 8:32 AM, Neha Narkhede neha.narkh...@gmail.com wrote: Can you give more information about the performance test? Which test? Which queue? How did you measure the dequeue latency. On Mon, Oct 20, 2014 at 5:09 PM, Mohit Anchlia mohitanch...@gmail.com wrote: I am running a performance test and from what I am seeing is that messages are taking about 100ms to pop from the queue itself and hence making the test slow. I am looking for pointers of how I can troubleshoot this issue. There seems to be plenty of CPU and IO available. I am running 22 producers and 22 consumers in the same group.
Re: Performance issues
There was a bug that could lead to the fetch request from the consumer hitting it's timeout instead of being immediately triggered by the produce request. To see if you are effected by that set you consumer max wait time to 1 ms and see if the latency drops to 1 ms (or, alternately, try with trunk and see if that fixes the problem). The reason I suspect this problem is because the default timeout in the java consumer is 100ms. -Jay On Tue, Oct 21, 2014 at 11:06 AM, Mohit Anchlia mohitanch...@gmail.com wrote: This is the version I am using: kafka_2.10-0.8.1.1 I think this is fairly recent version On Tue, Oct 21, 2014 at 10:57 AM, Jay Kreps jay.kr...@gmail.com wrote: What version of Kafka is this? Can you try the same test against trunk? We fixed a couple of latency related bugs which may be the cause. -Jay On Tue, Oct 21, 2014 at 10:50 AM, Mohit Anchlia mohitanch...@gmail.com wrote: It's consistently close to 100ms which makes me believe that there are some settings that I might have to tweak, however, I am not sure how to confirm that assumption :) On Tue, Oct 21, 2014 at 8:53 AM, Mohit Anchlia mohitanch...@gmail.com wrote: I have a java test that produces messages and then consumer consumers it. Consumers are active all the time. There is 1 consumer for 1 producer. I am measuring the time between the message is successfully written to the queue and the time consumer picks it up. On Tue, Oct 21, 2014 at 8:32 AM, Neha Narkhede neha.narkh...@gmail.com wrote: Can you give more information about the performance test? Which test? Which queue? How did you measure the dequeue latency. On Mon, Oct 20, 2014 at 5:09 PM, Mohit Anchlia mohitanch...@gmail.com wrote: I am running a performance test and from what I am seeing is that messages are taking about 100ms to pop from the queue itself and hence making the test slow. I am looking for pointers of how I can troubleshoot this issue. There seems to be plenty of CPU and IO available. I am running 22 producers and 22 consumers in the same group.
Re: Performance issues
Is this a parameter I need to set it in kafka server or on the client side? Also, can you help point out which one exactly is consumer max wait time from this list? https://kafka.apache.org/08/configuration.html On Tue, Oct 21, 2014 at 11:35 AM, Jay Kreps jay.kr...@gmail.com wrote: There was a bug that could lead to the fetch request from the consumer hitting it's timeout instead of being immediately triggered by the produce request. To see if you are effected by that set you consumer max wait time to 1 ms and see if the latency drops to 1 ms (or, alternately, try with trunk and see if that fixes the problem). The reason I suspect this problem is because the default timeout in the java consumer is 100ms. -Jay On Tue, Oct 21, 2014 at 11:06 AM, Mohit Anchlia mohitanch...@gmail.com wrote: This is the version I am using: kafka_2.10-0.8.1.1 I think this is fairly recent version On Tue, Oct 21, 2014 at 10:57 AM, Jay Kreps jay.kr...@gmail.com wrote: What version of Kafka is this? Can you try the same test against trunk? We fixed a couple of latency related bugs which may be the cause. -Jay On Tue, Oct 21, 2014 at 10:50 AM, Mohit Anchlia mohitanch...@gmail.com wrote: It's consistently close to 100ms which makes me believe that there are some settings that I might have to tweak, however, I am not sure how to confirm that assumption :) On Tue, Oct 21, 2014 at 8:53 AM, Mohit Anchlia mohitanch...@gmail.com wrote: I have a java test that produces messages and then consumer consumers it. Consumers are active all the time. There is 1 consumer for 1 producer. I am measuring the time between the message is successfully written to the queue and the time consumer picks it up. On Tue, Oct 21, 2014 at 8:32 AM, Neha Narkhede neha.narkh...@gmail.com wrote: Can you give more information about the performance test? Which test? Which queue? How did you measure the dequeue latency. On Mon, Oct 20, 2014 at 5:09 PM, Mohit Anchlia mohitanch...@gmail.com wrote: I am running a performance test and from what I am seeing is that messages are taking about 100ms to pop from the queue itself and hence making the test slow. I am looking for pointers of how I can troubleshoot this issue. There seems to be plenty of CPU and IO available. I am running 22 producers and 22 consumers in the same group.
Re: Performance issues
This is a consumer config: fetch.wait.max.ms On Tue, Oct 21, 2014 at 11:39 AM, Mohit Anchlia mohitanch...@gmail.com wrote: Is this a parameter I need to set it in kafka server or on the client side? Also, can you help point out which one exactly is consumer max wait time from this list? https://kafka.apache.org/08/configuration.html On Tue, Oct 21, 2014 at 11:35 AM, Jay Kreps jay.kr...@gmail.com wrote: There was a bug that could lead to the fetch request from the consumer hitting it's timeout instead of being immediately triggered by the produce request. To see if you are effected by that set you consumer max wait time to 1 ms and see if the latency drops to 1 ms (or, alternately, try with trunk and see if that fixes the problem). The reason I suspect this problem is because the default timeout in the java consumer is 100ms. -Jay On Tue, Oct 21, 2014 at 11:06 AM, Mohit Anchlia mohitanch...@gmail.com wrote: This is the version I am using: kafka_2.10-0.8.1.1 I think this is fairly recent version On Tue, Oct 21, 2014 at 10:57 AM, Jay Kreps jay.kr...@gmail.com wrote: What version of Kafka is this? Can you try the same test against trunk? We fixed a couple of latency related bugs which may be the cause. -Jay On Tue, Oct 21, 2014 at 10:50 AM, Mohit Anchlia mohitanch...@gmail.com wrote: It's consistently close to 100ms which makes me believe that there are some settings that I might have to tweak, however, I am not sure how to confirm that assumption :) On Tue, Oct 21, 2014 at 8:53 AM, Mohit Anchlia mohitanch...@gmail.com wrote: I have a java test that produces messages and then consumer consumers it. Consumers are active all the time. There is 1 consumer for 1 producer. I am measuring the time between the message is successfully written to the queue and the time consumer picks it up. On Tue, Oct 21, 2014 at 8:32 AM, Neha Narkhede neha.narkh...@gmail.com wrote: Can you give more information about the performance test? Which test? Which queue? How did you measure the dequeue latency. On Mon, Oct 20, 2014 at 5:09 PM, Mohit Anchlia mohitanch...@gmail.com wrote: I am running a performance test and from what I am seeing is that messages are taking about 100ms to pop from the queue itself and hence making the test slow. I am looking for pointers of how I can troubleshoot this issue. There seems to be plenty of CPU and IO available. I am running 22 producers and 22 consumers in the same group. -- -- Guozhang
Re: Performance issues
I set the property to 1 in the consumer code that is passed to createJavaConsumerConnector code, but it didn't seem to help props.put(fetch.wait.max.ms, fetchMaxWait); On Tue, Oct 21, 2014 at 1:21 PM, Guozhang Wang wangg...@gmail.com wrote: This is a consumer config: fetch.wait.max.ms On Tue, Oct 21, 2014 at 11:39 AM, Mohit Anchlia mohitanch...@gmail.com wrote: Is this a parameter I need to set it in kafka server or on the client side? Also, can you help point out which one exactly is consumer max wait time from this list? https://kafka.apache.org/08/configuration.html On Tue, Oct 21, 2014 at 11:35 AM, Jay Kreps jay.kr...@gmail.com wrote: There was a bug that could lead to the fetch request from the consumer hitting it's timeout instead of being immediately triggered by the produce request. To see if you are effected by that set you consumer max wait time to 1 ms and see if the latency drops to 1 ms (or, alternately, try with trunk and see if that fixes the problem). The reason I suspect this problem is because the default timeout in the java consumer is 100ms. -Jay On Tue, Oct 21, 2014 at 11:06 AM, Mohit Anchlia mohitanch...@gmail.com wrote: This is the version I am using: kafka_2.10-0.8.1.1 I think this is fairly recent version On Tue, Oct 21, 2014 at 10:57 AM, Jay Kreps jay.kr...@gmail.com wrote: What version of Kafka is this? Can you try the same test against trunk? We fixed a couple of latency related bugs which may be the cause. -Jay On Tue, Oct 21, 2014 at 10:50 AM, Mohit Anchlia mohitanch...@gmail.com wrote: It's consistently close to 100ms which makes me believe that there are some settings that I might have to tweak, however, I am not sure how to confirm that assumption :) On Tue, Oct 21, 2014 at 8:53 AM, Mohit Anchlia mohitanch...@gmail.com wrote: I have a java test that produces messages and then consumer consumers it. Consumers are active all the time. There is 1 consumer for 1 producer. I am measuring the time between the message is successfully written to the queue and the time consumer picks it up. On Tue, Oct 21, 2014 at 8:32 AM, Neha Narkhede neha.narkh...@gmail.com wrote: Can you give more information about the performance test? Which test? Which queue? How did you measure the dequeue latency. On Mon, Oct 20, 2014 at 5:09 PM, Mohit Anchlia mohitanch...@gmail.com wrote: I am running a performance test and from what I am seeing is that messages are taking about 100ms to pop from the queue itself and hence making the test slow. I am looking for pointers of how I can troubleshoot this issue. There seems to be plenty of CPU and IO available. I am running 22 producers and 22 consumers in the same group. -- -- Guozhang
Re: Performance issues
Most of the consumer threads seems to be waiting: ConsumerFetcherThread-groupA_ip-10-38-19-230-1413925671158-3cc3e22f-0-0 prio=10 tid=0x7f0aa84db800 nid=0x5be9 runnable [0x7f0a5a618000] java.lang.Thread.State: RUNNABLE at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method) at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269) at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:79) at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:87) - locked 0x9515bec0 (a sun.nio.ch.Util$2) - locked 0x9515bea8 (a java.util.Collections$UnmodifiableSet) - locked 0x95511d00 (a sun.nio.ch.EPollSelectorImpl) at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:98) at sun.nio.ch.SocketAdaptor$SocketInputStream.read(SocketAdaptor.java:221) - locked 0x9515bd28 (a java.lang.Object) at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:103) - locked 0x95293828 (a sun.nio.ch.SocketAdaptor$SocketInputStream) at java.nio.channels.Channels$ReadableByteChannelImpl.read(Channels.java:385) - locked 0x9515bcb0 (a java.lang.Object) at kafka.utils.Utils$.read(Utils.scala:375) On Tue, Oct 21, 2014 at 2:15 PM, Mohit Anchlia mohitanch...@gmail.com wrote: I set the property to 1 in the consumer code that is passed to createJavaConsumerConnector code, but it didn't seem to help props.put(fetch.wait.max.ms, fetchMaxWait); On Tue, Oct 21, 2014 at 1:21 PM, Guozhang Wang wangg...@gmail.com wrote: This is a consumer config: fetch.wait.max.ms On Tue, Oct 21, 2014 at 11:39 AM, Mohit Anchlia mohitanch...@gmail.com wrote: Is this a parameter I need to set it in kafka server or on the client side? Also, can you help point out which one exactly is consumer max wait time from this list? https://kafka.apache.org/08/configuration.html On Tue, Oct 21, 2014 at 11:35 AM, Jay Kreps jay.kr...@gmail.com wrote: There was a bug that could lead to the fetch request from the consumer hitting it's timeout instead of being immediately triggered by the produce request. To see if you are effected by that set you consumer max wait time to 1 ms and see if the latency drops to 1 ms (or, alternately, try with trunk and see if that fixes the problem). The reason I suspect this problem is because the default timeout in the java consumer is 100ms. -Jay On Tue, Oct 21, 2014 at 11:06 AM, Mohit Anchlia mohitanch...@gmail.com wrote: This is the version I am using: kafka_2.10-0.8.1.1 I think this is fairly recent version On Tue, Oct 21, 2014 at 10:57 AM, Jay Kreps jay.kr...@gmail.com wrote: What version of Kafka is this? Can you try the same test against trunk? We fixed a couple of latency related bugs which may be the cause. -Jay On Tue, Oct 21, 2014 at 10:50 AM, Mohit Anchlia mohitanch...@gmail.com wrote: It's consistently close to 100ms which makes me believe that there are some settings that I might have to tweak, however, I am not sure how to confirm that assumption :) On Tue, Oct 21, 2014 at 8:53 AM, Mohit Anchlia mohitanch...@gmail.com wrote: I have a java test that produces messages and then consumer consumers it. Consumers are active all the time. There is 1 consumer for 1 producer. I am measuring the time between the message is successfully written to the queue and the time consumer picks it up. On Tue, Oct 21, 2014 at 8:32 AM, Neha Narkhede neha.narkh...@gmail.com wrote: Can you give more information about the performance test? Which test? Which queue? How did you measure the dequeue latency. On Mon, Oct 20, 2014 at 5:09 PM, Mohit Anchlia mohitanch...@gmail.com wrote: I am running a performance test and from what I am seeing is that messages are taking about 100ms to pop from the queue itself and hence making the test slow. I am looking for pointers of how I can troubleshoot this issue. There seems to be plenty of CPU and IO available. I am running 22 producers and 22 consumers in the same group. -- -- Guozhang