Re: bootstrap.servers for the new Producer
Are you seeing this in practice or is this just a concern about the way the code currently works? If the broker is actually down and the host is rejecting connections, the situation you describe shouldn't be a problem. It's true that the NetworkClient chooses a fixed nodeIndexOffset, but the expectation is that if we run one iteration of leastLoadedNode and select a node, we'll try to connect and any failure will be handled by putting that node into a blackout period during which subsequent calls to leastLoadedNode will give priority to other options. If your server is *not* explicitly rejecting connections, I think it could be possible that we end up hanging for a long while just waiting for that connection. If this is the case (e.g., if you are running on EC2 and it has this behavior -- I believe default firewall rules will not kill the connection), this would be useful to know. A couple of bugs you might want to be aware of: https://issues.apache.org/jira/browse/KAFKA-1843 is meant to generally address the fact that there are a lot of states that we could be in, and the way we handle them (especially with leastLoadedNode), may not work well in all cases. It's very difficult to be comprehensive here, so if there is a scenario that is not failing for you, the more information you can give about the state of the system and the producer, the better. https://issues.apache.org/jira/browse/KAFKA-1842 might also be relevant -- right now we rely on the underlying TCP connection timeouts, but this is definitely not ideal. They can be quite long by default, and we might want to consider connections failed much sooner. I also could have sworn there was a JIRA filed about the fact that the bootstrap servers are never reused, but I can't find it at the moment -- in some cases, if you have no better option then it would be best to revert back to the original set of bootstrap servers for loading metadata. This can especially become a problem in some cases where your only producing to one or a small number of topics and therefore only have metadata for a couple of servers. If anything happens to those servers too quickly (within the metadata refresh period) you might potentially get stuck with only references to dead nodes. -Ewen On Fri, Aug 21, 2015 at 6:56 PM, Kishore Senji kse...@gmail.com wrote: If one of the broker we specify in the bootstrap servers list is down, there is a chance that the Producer (a brand new instance with no prior metadata) will never be able to publish anything to Kafka until that broker is up. Because the logic for getting the initial metadata is based on some random index to the set of bootstrap nodes and if it happens to be the down node, Kafka producer keeps on trying to get the metadata on that node only. It is never switched to another node. Without metadata, the Producer can never send anything. The nodeIndexOffset is chosen at the creation of the NetworkClient (and this offset is not changed when we fail to get a new connection) and so for getting the metadata for the first time, there is a possibility that we keep on trying on the broker that is down. This can be a problem if a broker goes down and also a Producer is restarted or a new instance is brought up. Is this a known issue? -- Thanks, Ewen
Re: is SSL support feature ready to use in kafka-truck branch
Hi Qi Trunk seems fairly stable. There are guidelines here which includes how to generate keys https://cwiki.apache.org/confluence/display/KAFKA/Deploying+SSL+for+Kafka https://cwiki.apache.org/confluence/display/KAFKA/Deploying+SSL+for+Kafka Your server config needs these properties (also on the webpage): listeners=PLAINTEXT://:9092,SSL://:9093 ssl.protocol = TLS ssl.keystore.type = JKS ssl.keystore.location = path/keystore.jks ssl.keystore.password = pass ssl.key.password = pass ssl.truststore.type = JKS ssl.truststore.location = path/truststore.jks ssl.truststore.password = pass To get yourself going it’s easiest to just generate a set of certs locally and spark up the console producer/consumer pair. You’ll need the latest cut from trunk (from today) to get a console consumer that works. Hope that helps Ben On 21 Aug 2015, at 07:10, Qi Xu shkir...@gmail.com wrote: Hi folks, I tried to clone the latest version of kafka truck and try to enable the SSL. The server.properties seems not having any security related settings, and it seems there's no other config file relevant to SSL either. So may I know is this feature ready to use now in truck branch? BTW, we're using the SSL feature from the branch : https://github.com/relango/kafka/tree/0.8.2. Is there any significant difference between Kafka-truck and relango's branch? Thanks, Qi
Painfully slow kafka recovery
Hey everyone, here's my crosspost from irc. Our setup: 3 kafka 0.8.2 brokers with zookeeper, powerful hardware (20 cores, 27 logdisks each). We use a handful of topics, but only one topic is utilized heavily. It features a replication of 2 and 600 partitions. Our issue: If one kafka was down, it takes very long ( from 1 to 10 hours) to show that all partitions have all isr again. This seems to heavily depend on the amount of data which is in the log.dirs (I have configured 27 threads - one for each dir featuring a own drive). This all takes this long while there is NO data flowing into kafka. We seem to be missing something critical here. It might be some option set wrong, or are we thinking wrong and it's not critical to have the replicas in sync. Any pointers would be great. Cheers Jörg
Re: Any drawbacks of running Kafka consumer in a web container
I've done this (and still do) in production, works fine. On Thu, Aug 20, 2015 at 6:21 PM, Venkat K venkatk...@gmail.com wrote: Hello guys, Does anyone run Kafka consumer in a web container like Tomcat in their production environments (processing millions of messages per hour). I am wondering if the web container could have any adverse impact on the performance of consumer. Is it advisable to NOT run the consumer in a web container ? I am thinking about any restrictions imposed by the web container on file handlers, etc. Thanks for the insights. - Venkatesh -- Hisham Mardam-Bey -=[ CTO ]-=-[ Mate1 Inc. ]=- A: Because it messes up the order in which people normally read text. Q: Why is top-posting such a bad thing? A: Top-posting. Q: What is the most annoying thing in e-mail? -=[ Codito Ergo Sum ]=-
Re: Painfully slow kafka recovery
By default, num.replica.fetchers = 1. This means only one thread per broker is fetching data from leaders. This means it make take a while for the recovering machine to catch up and rejoin the ISR. If you have bandwidth to spare, try increasing this value. Regarding no data flowing into kafka - If you have 3 replicas and only one is down, I'd expect writes to continue to the new leader even if one replica is not in the ISR yet. Can you see that a new leader is elected? Gwen On Fri, Aug 21, 2015 at 6:50 AM, Jörg Wagner joerg.wagn...@1und1.de wrote: Hey everyone, here's my crosspost from irc. Our setup: 3 kafka 0.8.2 brokers with zookeeper, powerful hardware (20 cores, 27 logdisks each). We use a handful of topics, but only one topic is utilized heavily. It features a replication of 2 and 600 partitions. Our issue: If one kafka was down, it takes very long ( from 1 to 10 hours) to show that all partitions have all isr again. This seems to heavily depend on the amount of data which is in the log.dirs (I have configured 27 threads - one for each dir featuring a own drive). This all takes this long while there is NO data flowing into kafka. We seem to be missing something critical here. It might be some option set wrong, or are we thinking wrong and it's not critical to have the replicas in sync. Any pointers would be great. Cheers Jörg
Re: kafka-producer-perf-test.sh - No visible difference between request-num-acks 1 and -1
Hi Prabhjot, Do you intent to use the old producer performance microbenchmark? Thanks, -Tao On Fri, Aug 21, 2015 at 2:41 AM, Prabhjot Bharaj prabhbha...@gmail.com wrote: Hi, I'm using Kafka 0.8.2.1 with the default zookeeper build that comes along the bundle I have setup a 5 machine cluster and on the same 5 machines, I'm also running zookeeper as well I am trying to see what is the maximum produce throughput I can get on this 5 node cluster I have created only 1 topic - tops1 in the cluster:- root@x.x.x.x:~# kafka-topics.sh --describe --topic tops1 --zookeeper localhost:2182 Topic:tops1 PartitionCount:4 ReplicationFactor:3 Configs: Topic: tops1 Partition: 0 Leader: 4 Replicas: 4,1,2 Isr: 4,1,2 Topic: tops1 Partition: 1 Leader: 5 Replicas: 5,2,3 Isr: 5,3,2 Topic: tops1 Partition: 2 Leader: 1 Replicas: 1,3,4 Isr: 4,1,3 Topic: tops1 Partition: 3 Leader: 2 Replicas: 2,4,5 Isr: 4,2,5 This is the output of the kafka-producer-perf-test.sh for request-num-acks 1 and request-num-acks -1:- root@x.x.x.x:~# date;time kafka-producer-perf-test.sh --broker-list x.x.x.x:9092,x.x.x.y:9092,x.x.x.z:9092,x.x.x.a:9092,x.x.x.b:9092 --messages 100 --message-size 500 --topics tops1 --show-detailed-stats --threads 30 --request-num-acks 1 --batch-size 1000 --request-timeout-ms 1 start.time, end.time, compression, message.size, batch.size, total.data.sent.in.MB, MB.sec, total.data.sent.in.nMsg, nMsg.sec 2015-08-21 09:36:01:021, 2015-08-21 09:36:49:351, 0, 500, 1000, 476.83, 9.8662, 90, 20690.8752 real 0m49.375s user 0m38.777s sys 0m5.951s root@x.x.x.x:~# date;time kafka-producer-perf-test.sh --broker-list x.x.x.x:9092,x.x.x.y:9092,x.x.x.z:9092,x.x.x.a:9092,x.x.x.b:9092 --messages 100 --message-size 500 --topics tops1 --show-detailed-stats --threads 30 --request-num-acks -1 --batch-size 1000 --request-timeout-ms 1 start.time, end.time, compression, message.size, batch.size, total.data.sent.in.MB, MB.sec, total.data.sent.in.nMsg, nMsg.sec 2015-08-21 09:31:19:329, 2015-08-21 09:32:08:627, 0, 500, 1000, 476.83, 9.6724, 90, 20284.5957 real 0m50.341s user 0m39.029s sys 0m6.051s I'll continue to test with more number of messages, to see if I get any difference. But, I just want to be sure whatever I'm doing is right. Please share your thoughts on the results. Regards, prabcs
Raid vs individual disks
Hi, I've gone through the details mentioned about Raid and individual disks in the ops section of the documentation But, I would like to know what performance boost we can get with individual disks. Is anybody using Kafka with multiple disks or all are raid into 1 big disk ? Regards, Prabcs
Re: My emails don't seem to go through
I got your email from the list? On Fri, Aug 21, 2015 at 1:56 PM, Rajiv Kurian ra...@signalfuse.com wrote: Wondering why my emails to the mailing list don't go through. -- David Luu Member of Technical Staff Mist Systems, Inc. 1601 S. De Anza Blvd. #248 Cupertino, CA 95014
Re: Painfully slow kafka recovery
We are seeing same behavior in 5 broker cluster when losing one broker. In our case, we are losing broker as well as kafka data dir. Jörg Wagner, Are you losing just broker or kafka data dir as well? Gwen, We have also observed that latency of messages arriving at consumers goes up by 10x when we lose a broker. Is it because the broker is busy with handling failed fetch requests and loaded with more data thats slowing down the writes ? Also, if we had simply lost the broker not the data dir, impact would have been minimal? Thanks, Raja. On Fri, Aug 21, 2015 at 12:31 PM, Gwen Shapira g...@confluent.io wrote: By default, num.replica.fetchers = 1. This means only one thread per broker is fetching data from leaders. This means it make take a while for the recovering machine to catch up and rejoin the ISR. If you have bandwidth to spare, try increasing this value. Regarding no data flowing into kafka - If you have 3 replicas and only one is down, I'd expect writes to continue to the new leader even if one replica is not in the ISR yet. Can you see that a new leader is elected? Gwen On Fri, Aug 21, 2015 at 6:50 AM, Jörg Wagner joerg.wagn...@1und1.de wrote: Hey everyone, here's my crosspost from irc. Our setup: 3 kafka 0.8.2 brokers with zookeeper, powerful hardware (20 cores, 27 logdisks each). We use a handful of topics, but only one topic is utilized heavily. It features a replication of 2 and 600 partitions. Our issue: If one kafka was down, it takes very long ( from 1 to 10 hours) to show that all partitions have all isr again. This seems to heavily depend on the amount of data which is in the log.dirs (I have configured 27 threads - one for each dir featuring a own drive). This all takes this long while there is NO data flowing into kafka. We seem to be missing something critical here. It might be some option set wrong, or are we thinking wrong and it's not critical to have the replicas in sync. Any pointers would be great. Cheers Jörg -- Thanks, Raja.
Re: Painfully slow kafka recovery
I suspect that in general the broker may be busier since it needs to handle more partitions now, and the extra replication. It could be good to dig into the specifics of the latency - there's a request log that you can turn on, I believe. On Fri, Aug 21, 2015 at 10:18 AM, Rajasekar Elango rela...@salesforce.com wrote: We are seeing same behavior in 5 broker cluster when losing one broker. In our case, we are losing broker as well as kafka data dir. Jörg Wagner, Are you losing just broker or kafka data dir as well? Gwen, We have also observed that latency of messages arriving at consumers goes up by 10x when we lose a broker. Is it because the broker is busy with handling failed fetch requests and loaded with more data thats slowing down the writes ? Also, if we had simply lost the broker not the data dir, impact would have been minimal? Thanks, Raja. On Fri, Aug 21, 2015 at 12:31 PM, Gwen Shapira g...@confluent.io wrote: By default, num.replica.fetchers = 1. This means only one thread per broker is fetching data from leaders. This means it make take a while for the recovering machine to catch up and rejoin the ISR. If you have bandwidth to spare, try increasing this value. Regarding no data flowing into kafka - If you have 3 replicas and only one is down, I'd expect writes to continue to the new leader even if one replica is not in the ISR yet. Can you see that a new leader is elected? Gwen On Fri, Aug 21, 2015 at 6:50 AM, Jörg Wagner joerg.wagn...@1und1.de wrote: Hey everyone, here's my crosspost from irc. Our setup: 3 kafka 0.8.2 brokers with zookeeper, powerful hardware (20 cores, 27 logdisks each). We use a handful of topics, but only one topic is utilized heavily. It features a replication of 2 and 600 partitions. Our issue: If one kafka was down, it takes very long ( from 1 to 10 hours) to show that all partitions have all isr again. This seems to heavily depend on the amount of data which is in the log.dirs (I have configured 27 threads - one for each dir featuring a own drive). This all takes this long while there is NO data flowing into kafka. We seem to be missing something critical here. It might be some option set wrong, or are we thinking wrong and it's not critical to have the replicas in sync. Any pointers would be great. Cheers Jörg -- Thanks, Raja.
Re: Higher CPU going from 0.8.1 to 0.8.2.1
The only thing I notice in the logs which is a bit unsettling is about a once a second rate of messages of the type Closing socket connection to some-ip-address. I used to see these messages before but it seems like its more often than usual. Also all the clients that it seems to close connections with are running the Java wrapper over the Scala SimpleConsumer. Is there any logging I can enable to understand why exactly these connections are being closed so often? Thanks, Rajiv On Fri, Aug 21, 2015 at 3:50 PM, Rajiv Kurian ra...@signalfuse.com wrote: We upgraded a 9 broker cluster from version 0.8.1 to version 0.8.2.1. Actually we cherry-picked the commit at 41ba26273b497e4cbcc947c742ff6831b7320152 to get zkClient 0.5 because we ran into a bug described at https://issues.apache.org/jira/browse/KAFKA-824 Right after the update the CPU spiked quite a bit but I am guessing that is because the brokers were pulling in log segments from other brokers right after restart. The CPU remained elevated for a while and I thought it would come down after things settled down but the CPU has remained higher even after a day. Our steady state CPU on the brokers went from about 28% (0.8.1) to 34% (0.8.2.1). We do not use compression on any topic or partition. Our incoming traffic (number of messages/sec) has not increased at all. Our incoming bytes/sec has actually decreased because we managed to reduce the size of one our message types from 256 bytes to 32 bytes. The message size change was made hours after the Kafka version update and didn't seem to harm or help the cpu. The bytes-in/sec and bytes-out/sec metrics have definitely gone down after the message size reduction. Here is a link to the graph showing how the CPU went up - http://i.imgur.com/KVJLzsX.png?1 The restarts were done from 18:00 to 19:00 and I'd expect the CPU to go up at that time but I can't explain the steady state CPU rise. Are there any known performance regressions after 0.8.1? Any hints on what I should investigate if you think that this is not normal? Thanks, Rajiv
Re: Raid vs individual disks
We are running with a JBOD configuration, and it is not recommended for the following reasons: - any volume failure causes an unclean shutdown and requires lengthy recovery - data is not distributed consistently across volumes, so you could have skew within a broker We are planning to switch to a RAID-10 implementation. Chi On Fri, Aug 21, 2015 at 1:59 PM, Todd Palino tpal...@gmail.com wrote: At LinkedIn, we are using a RAID-10 of 14 disks. This is using software RAID. I recently did some performance testing with RAID 0, 5, and 6. I found that 5 and 6 underperformed significantly, possibly due to the parity calculations. RAID 0 had a sizable performance gain over 10, and I would expect single disks to perform similarly. I didn't test it because it lacks some balancing ability that We would need. -Todd On Friday, August 21, 2015, Prabhjot Bharaj prabhbha...@gmail.com wrote: Hi, I've gone through the details mentioned about Raid and individual disks in the ops section of the documentation But, I would like to know what performance boost we can get with individual disks. Is anybody using Kafka with multiple disks or all are raid into 1 big disk ? Regards, Prabcs
Higher CPU going from 0.8.1 to 0.8.2.1
We upgraded a 9 broker cluster from version 0.8.1 to version 0.8.2.1. Actually we cherry-picked the commit at 41ba26273b497e4cbcc947c742ff6831b7320152 to get zkClient 0.5 because we ran into a bug described at https://issues.apache.org/jira/browse/KAFKA-824 Right after the update the CPU spiked quite a bit but I am guessing that is because the brokers were pulling in log segments from other brokers right after restart. The CPU remained elevated for a while and I thought it would come down after things settled down but the CPU has remained higher even after a day. Our steady state CPU on the brokers went from about 28% (0.8.1) to 34% (0.8.2.1). We do not use compression on any topic or partition. Our incoming traffic (number of messages/sec) has not increased at all. Our incoming bytes/sec has actually decreased because we managed to reduce the size of one our message types from 256 bytes to 32 bytes. The message size change was made hours after the Kafka version update and didn't seem to harm or help the cpu. The bytes-in/sec and bytes-out/sec metrics have definitely gone down after the message size reduction. Here is a link to the graph showing how the CPU went up - http://i.imgur.com/KVJLzsX.png?1 The restarts were done from 18:00 to 19:00 and I'd expect the CPU to go up at that time but I can't explain the steady state CPU rise. Are there any known performance regressions after 0.8.1? Any hints on what I should investigate if you think that this is not normal? Thanks, Rajiv
Re: Higher CPU going from 0.8.1 to 0.8.2.1
Have you done a profiling on your broker process? Any hot code path differences between these two versions? Thanks, -Tao On Fri, Aug 21, 2015 at 3:59 PM, Rajiv Kurian ra...@signalfuse.com wrote: The only thing I notice in the logs which is a bit unsettling is about a once a second rate of messages of the type Closing socket connection to some-ip-address. I used to see these messages before but it seems like its more often than usual. Also all the clients that it seems to close connections with are running the Java wrapper over the Scala SimpleConsumer. Is there any logging I can enable to understand why exactly these connections are being closed so often? Thanks, Rajiv On Fri, Aug 21, 2015 at 3:50 PM, Rajiv Kurian ra...@signalfuse.com wrote: We upgraded a 9 broker cluster from version 0.8.1 to version 0.8.2.1. Actually we cherry-picked the commit at 41ba26273b497e4cbcc947c742ff6831b7320152 to get zkClient 0.5 because we ran into a bug described at https://issues.apache.org/jira/browse/KAFKA-824 Right after the update the CPU spiked quite a bit but I am guessing that is because the brokers were pulling in log segments from other brokers right after restart. The CPU remained elevated for a while and I thought it would come down after things settled down but the CPU has remained higher even after a day. Our steady state CPU on the brokers went from about 28% (0.8.1) to 34% (0.8.2.1). We do not use compression on any topic or partition. Our incoming traffic (number of messages/sec) has not increased at all. Our incoming bytes/sec has actually decreased because we managed to reduce the size of one our message types from 256 bytes to 32 bytes. The message size change was made hours after the Kafka version update and didn't seem to harm or help the cpu. The bytes-in/sec and bytes-out/sec metrics have definitely gone down after the message size reduction. Here is a link to the graph showing how the CPU went up - http://i.imgur.com/KVJLzsX.png?1 The restarts were done from 18:00 to 19:00 and I'd expect the CPU to go up at that time but I can't explain the steady state CPU rise. Are there any known performance regressions after 0.8.1? Any hints on what I should investigate if you think that this is not normal? Thanks, Rajiv
bootstrap.servers for the new Producer
If one of the broker we specify in the bootstrap servers list is down, there is a chance that the Producer (a brand new instance with no prior metadata) will never be able to publish anything to Kafka until that broker is up. Because the logic for getting the initial metadata is based on some random index to the set of bootstrap nodes and if it happens to be the down node, Kafka producer keeps on trying to get the metadata on that node only. It is never switched to another node. Without metadata, the Producer can never send anything. The nodeIndexOffset is chosen at the creation of the NetworkClient (and this offset is not changed when we fail to get a new connection) and so for getting the metadata for the first time, there is a possibility that we keep on trying on the broker that is down. This can be a problem if a broker goes down and also a Producer is restarted or a new instance is brought up. Is this a known issue?
Re: Higher CPU going from 0.8.1 to 0.8.2.1
All the new brokers are running 0.8.2.1 so I can only profile the new version and not the old one any more without reverting the change on some of the brokers. The restart of brokers causes clients to lose a select few messages so its not very desirable. Profiling the new brokers using jVisualVm (don' have a better profiler in production) doesn't show anything very odd AFAICT. Here is a link to a particular profile: http://i.imgur.com/8T7jrTw.png Most of the time is spent on read calls. Thanks, Rajiv On Fri, Aug 21, 2015 at 4:22 PM, Tao Feng fengta...@gmail.com wrote: Have you done a profiling on your broker process? Any hot code path differences between these two versions? Thanks, -Tao On Fri, Aug 21, 2015 at 3:59 PM, Rajiv Kurian ra...@signalfuse.com wrote: The only thing I notice in the logs which is a bit unsettling is about a once a second rate of messages of the type Closing socket connection to some-ip-address. I used to see these messages before but it seems like its more often than usual. Also all the clients that it seems to close connections with are running the Java wrapper over the Scala SimpleConsumer. Is there any logging I can enable to understand why exactly these connections are being closed so often? Thanks, Rajiv On Fri, Aug 21, 2015 at 3:50 PM, Rajiv Kurian ra...@signalfuse.com wrote: We upgraded a 9 broker cluster from version 0.8.1 to version 0.8.2.1. Actually we cherry-picked the commit at 41ba26273b497e4cbcc947c742ff6831b7320152 to get zkClient 0.5 because we ran into a bug described at https://issues.apache.org/jira/browse/KAFKA-824 Right after the update the CPU spiked quite a bit but I am guessing that is because the brokers were pulling in log segments from other brokers right after restart. The CPU remained elevated for a while and I thought it would come down after things settled down but the CPU has remained higher even after a day. Our steady state CPU on the brokers went from about 28% (0.8.1) to 34% (0.8.2.1). We do not use compression on any topic or partition. Our incoming traffic (number of messages/sec) has not increased at all. Our incoming bytes/sec has actually decreased because we managed to reduce the size of one our message types from 256 bytes to 32 bytes. The message size change was made hours after the Kafka version update and didn't seem to harm or help the cpu. The bytes-in/sec and bytes-out/sec metrics have definitely gone down after the message size reduction. Here is a link to the graph showing how the CPU went up - http://i.imgur.com/KVJLzsX.png?1 The restarts were done from 18:00 to 19:00 and I'd expect the CPU to go up at that time but I can't explain the steady state CPU rise. Are there any known performance regressions after 0.8.1? Any hints on what I should investigate if you think that this is not normal? Thanks, Rajiv
Re: SocketTimeoutException with kafka-producer-perf-test.sh
Hi, Never mind. I have solved this problem by referring to an earlier question, where Neha had suggested to use a higher value for request-timeout-ms Regards, prabcs On Fri, Aug 21, 2015 at 12:22 PM, Prabhjot Bharaj prabhbha...@gmail.com wrote: Hello Folks, I'm using Kafka 0.8.2.1 with the default zookeeper build that comes along the bundle I have setup a 5 machine cluster and on the same 5 machines, I'm also running zookeeper as well I am trying to see what is the maximum produce throughput I can get on this 5 node cluster I have created only 1 topic - tops1 in the cluster:- root@x.x.x.x:~# kafka-topics.sh --describe --topic tops1 --zookeeper localhost:2182 Topic:tops1 PartitionCount:4 ReplicationFactor:3 Configs: Topic: tops1 Partition: 0 Leader: 4 Replicas: 4,1,2 Isr: 4,1,2 Topic: tops1 Partition: 1 Leader: 5 Replicas: 5,2,3 Isr: 5,3,2 Topic: tops1 Partition: 2 Leader: 1 Replicas: 1,3,4 Isr: 4,1,3 Topic: tops1 Partition: 3 Leader: 2 Replicas: 2,4,5 Isr: 4,2,5 When I run this command in parallel from 2 different machines, I get SocketTimeoutException:- time kafka-producer-perf-test.sh --broker-list x.x.x.x:9092,x.x.x.y:9092,x.x.x.z:9092,x.x.x.a:9092,x.x.x.b:9092 --messages 100 --message-size 500 --topics tops1 --show-detailed-stats --threads 5 --request-num-acks -1 --batch-size 1000 My server.properties has these details:- broker.id=0 port=9092 num.network.threads=3 num.io.threads=8 socket.send.buffer.bytes=1048576 socket.receive.buffer.bytes=1048576 socket.request.max.bytes=104857600 log.dirs=/kafka-logs num.partitions=1 num.recovery.threads.per.data.dir=1 log.retention.hours=168 log.segment.bytes=1073741824 log.retention.check.interval.ms=30 log.cleaner.enable=false zookeeper.connect=localhost:2181 zookeeper.connection.timeout.ms=6000 num.replica.fetchers=4 2 major changes in the server.properties, which gave a performance boost on this test were:- num.replica.fetchers=4 socket.send.buffer.bytes=1048576 socket.receive.buffer.bytes=1048576 I think I'm giving enough requests on the network, but not sure how I should correct it I am particularly interested in knowing how much max throughput per topic can I get, with the highest level of durability (i.e. request-num-acks = -1) Request you to share your thoughts on this. Thanks, Prabhjot -- - There are only 10 types of people in the world: Those who understand binary, and those who don't
kafka-producer-perf-test.sh - No visible difference between request-num-acks 1 and -1
Hi, I'm using Kafka 0.8.2.1 with the default zookeeper build that comes along the bundle I have setup a 5 machine cluster and on the same 5 machines, I'm also running zookeeper as well I am trying to see what is the maximum produce throughput I can get on this 5 node cluster I have created only 1 topic - tops1 in the cluster:- root@x.x.x.x:~# kafka-topics.sh --describe --topic tops1 --zookeeper localhost:2182 Topic:tops1 PartitionCount:4 ReplicationFactor:3 Configs: Topic: tops1 Partition: 0 Leader: 4 Replicas: 4,1,2 Isr: 4,1,2 Topic: tops1 Partition: 1 Leader: 5 Replicas: 5,2,3 Isr: 5,3,2 Topic: tops1 Partition: 2 Leader: 1 Replicas: 1,3,4 Isr: 4,1,3 Topic: tops1 Partition: 3 Leader: 2 Replicas: 2,4,5 Isr: 4,2,5 This is the output of the kafka-producer-perf-test.sh for request-num-acks 1 and request-num-acks -1:- root@x.x.x.x:~# date;time kafka-producer-perf-test.sh --broker-list x.x.x.x:9092,x.x.x.y:9092,x.x.x.z:9092,x.x.x.a:9092,x.x.x.b:9092 --messages 100 --message-size 500 --topics tops1 --show-detailed-stats --threads 30 --request-num-acks 1 --batch-size 1000 --request-timeout-ms 1 start.time, end.time, compression, message.size, batch.size, total.data.sent.in.MB, MB.sec, total.data.sent.in.nMsg, nMsg.sec 2015-08-21 09:36:01:021, 2015-08-21 09:36:49:351, 0, 500, 1000, 476.83, 9.8662, 90, 20690.8752 real 0m49.375s user 0m38.777s sys 0m5.951s root@x.x.x.x:~# date;time kafka-producer-perf-test.sh --broker-list x.x.x.x:9092,x.x.x.y:9092,x.x.x.z:9092,x.x.x.a:9092,x.x.x.b:9092 --messages 100 --message-size 500 --topics tops1 --show-detailed-stats --threads 30 --request-num-acks -1 --batch-size 1000 --request-timeout-ms 1 start.time, end.time, compression, message.size, batch.size, total.data.sent.in.MB, MB.sec, total.data.sent.in.nMsg, nMsg.sec 2015-08-21 09:31:19:329, 2015-08-21 09:32:08:627, 0, 500, 1000, 476.83, 9.6724, 90, 20284.5957 real 0m50.341s user 0m39.029s sys 0m6.051s I'll continue to test with more number of messages, to see if I get any difference. But, I just want to be sure whatever I'm doing is right. Please share your thoughts on the results. Regards, prabcs
Re: SocketTimeoutException with kafka-producer-perf-test.sh
This link already had the answer I was searching for: http://grokbase.com/t/kafka/users/145vmz70cb/java-net-sockettimeoutexception-in-broker On Fri, Aug 21, 2015 at 3:03 PM, Prabhjot Bharaj prabhbha...@gmail.com wrote: Hi, Never mind. I have solved this problem by referring to an earlier question, where Neha had suggested to use a higher value for request-timeout-ms Regards, prabcs On Fri, Aug 21, 2015 at 12:22 PM, Prabhjot Bharaj prabhbha...@gmail.com wrote: Hello Folks, I'm using Kafka 0.8.2.1 with the default zookeeper build that comes along the bundle I have setup a 5 machine cluster and on the same 5 machines, I'm also running zookeeper as well I am trying to see what is the maximum produce throughput I can get on this 5 node cluster I have created only 1 topic - tops1 in the cluster:- root@x.x.x.x:~# kafka-topics.sh --describe --topic tops1 --zookeeper localhost:2182 Topic:tops1 PartitionCount:4 ReplicationFactor:3 Configs: Topic: tops1 Partition: 0 Leader: 4 Replicas: 4,1,2 Isr: 4,1,2 Topic: tops1 Partition: 1 Leader: 5 Replicas: 5,2,3 Isr: 5,3,2 Topic: tops1 Partition: 2 Leader: 1 Replicas: 1,3,4 Isr: 4,1,3 Topic: tops1 Partition: 3 Leader: 2 Replicas: 2,4,5 Isr: 4,2,5 When I run this command in parallel from 2 different machines, I get SocketTimeoutException:- time kafka-producer-perf-test.sh --broker-list x.x.x.x:9092,x.x.x.y:9092,x.x.x.z:9092,x.x.x.a:9092,x.x.x.b:9092 --messages 100 --message-size 500 --topics tops1 --show-detailed-stats --threads 5 --request-num-acks -1 --batch-size 1000 My server.properties has these details:- broker.id=0 port=9092 num.network.threads=3 num.io.threads=8 socket.send.buffer.bytes=1048576 socket.receive.buffer.bytes=1048576 socket.request.max.bytes=104857600 log.dirs=/kafka-logs num.partitions=1 num.recovery.threads.per.data.dir=1 log.retention.hours=168 log.segment.bytes=1073741824 log.retention.check.interval.ms=30 log.cleaner.enable=false zookeeper.connect=localhost:2181 zookeeper.connection.timeout.ms=6000 num.replica.fetchers=4 2 major changes in the server.properties, which gave a performance boost on this test were:- num.replica.fetchers=4 socket.send.buffer.bytes=1048576 socket.receive.buffer.bytes=1048576 I think I'm giving enough requests on the network, but not sure how I should correct it I am particularly interested in knowing how much max throughput per topic can I get, with the highest level of durability (i.e. request-num-acks = -1) Request you to share your thoughts on this. Thanks, Prabhjot -- - There are only 10 types of people in the world: Those who understand binary, and those who don't -- - There are only 10 types of people in the world: Those who understand binary, and those who don't
Re: Possible DEAD LOCK for one day at broker controller?
Hi Zhao, Do you see any other errors regarding checkpoint file? Is this reproducible by you and if you can you enable debug log level to get more info. On Thu, Aug 20, 2015 at 7:44 AM, Zhao Weinan zhaow...@gmail.com wrote: Hi Kishore Senji, I've been busy recovering some data these two days... and found that I maybe hit more serious problem than I thought. I lost almost all data on one broker at least at some time, here is some log from server.log pasted below, and very like the situation described by Jason and Thunder here. http://mail-archives.apache.org/mod_mbox/kafka-users/201504.mbox/%3CCAA%2BBczTUBqg1-tpcUjwfZgZYZyOXC-Myuhd_2EaGkeKWkrCVUQ%40mail.gmail.com%3E Any idea about this? Why Kafka got segments with no-zero LEO then said No checkpointed highwatermark? From broker's logs/server.log: [2015-08-16 17:14:35,204] INFO Completed load of log xx-1 with log end offset 863967227 (kafka.log.Log) [2015-08-16 17:15:29,648] WARN Partition [xx,1] on broker 1: No checkpointed highwatermark is found for partition [xx,1] (kafka.cluster.Partition) [2015-08-16 17:15:36,887] INFO Truncating log xx-1 to offset 0. (kafka.log.Log) [2015-08-16 17:15:36,887] INFO Scheduling log segment 763102206 for log xx-1 for deletion. (kafka.log.Log) [2015-08-16 17:15:36,888] INFO Scheduling log segment 768984638 for log xx-1 for deletion. (kafka.log.Log) [2015-08-16 17:15:36,888] INFO Scheduling log segment 773712058 for log xx-1 for deletion. (kafka.log.Log) [2015-08-16 17:15:36,888] INFO Scheduling log segment 778002546 for log xx-1 for deletion. (kafka.log.Log) . [2015-08-16 17:34:18,168] INFO Scheduling log segment 0 for log xx-1 for deletion. (kafka.log.Log) [2015-08-16 17:36:37,811] WARN [ReplicaFetcherThread-0-0], Replica 1 for partition [xx,1] reset its fetch offset from 791913697 to current leader 0's start offset 791913697 (kafka.server.ReplicaFetcherThread) [2015-08-16 17:36:37,811] ERROR [ReplicaFetcherThread-0-0], Current offset 0 for partition [xx,1] out of range; reset offset to 791913697 (kafka.server.ReplicaFetcherThread) From broker's logs/controller.log: [2015-08-16 17:14:41,444] INFO [Controller 1]: Controller starting up (kafka.controller.KafkaController) [2015-08-16 17:14:41,492] INFO [Controller 1]: Controller startup complete (kafka.controller.KafkaController) [2015-08-16 17:16:24,850] INFO [SessionExpirationListener on 1], ZK expired; shut down all controller components and try to re-elect (kafka.controller.KafkaController$SessionExpirationListener) 2015-08-18 23:43 GMT+08:00 Kishore Senji kse...@gmail.com: Yes you are right. I misread the code. So the only thing that can explain the behavior you are seeing is that may be there are many segments that need to be deleted all at once. Can you try may be reducing the retention.ms in smaller intervals - like reduce it to 9 days from 10 days and see if the brokers are fine. On Tue, Aug 18, 2015 at 12:48 AM Zhao Weinan zhaow...@gmail.com wrote: Hi Kishore Senji, Did you constantly send messages to your test topic? Or just one time send? I've just did some test, the log.lastModified is updated with every message received (or every flush to disk at least). So I think if your interval between two neibouring messages is never smaller than retention.ms, then your current segments should never be deleted. The wired thing is the offset of one partition in my data-loss-topic became to be zero, while offsets in other partition are normally minimum... 2015-08-18 14:45 GMT+08:00 Kishore Senji kse...@gmail.com: It is log.deleteOldSegments(startMs - _.lastModified log.config.retentionMs) You might have missed the startMs. I have tested it myself. I created a test topic with retention.ms equal to 20 minutes and added some messages. Later I changed the retention.ms to 2 min. I can see whenever the delete thread runs (every five min), it deletes even the latest Segment because that Segment age is older than retention.ms On Mon, Aug 17, 2015 at 11:30 PM, Zhao Weinan zhaow...@gmail.com wrote: Hi Kishore Senji, The size of segement file is default 1GB. According to the LogManager.scala#cleanupExpiredSegments, Kafka will only delete segments whose lastModTime is older than retention.ms, so I dont think this is the reason for my data loss. Actually I lost some data in topic other than the topic I reduced the retention... I dont know whether destage these several GB files will cause this kind of system chattering, though we do use not very fancy hardwares. 2015-08-18 7:48 GMT+08:00 Kishore Senji kse...@gmail.com: What is the size of the segment file? You are reducing the retention from 10 days to 1 day.
SocketTimeoutException with kafka-producer-perf-test.sh
Hello Folks, I'm using Kafka 0.8.2.1 with the default zookeeper build that comes along the bundle I have setup a 5 machine cluster and on the same 5 machines, I'm also running zookeeper as well I am trying to see what is the maximum produce throughput I can get on this 5 node cluster I have created only 1 topic - tops1 in the cluster:- root@x.x.x.x:~# kafka-topics.sh --describe --topic tops1 --zookeeper localhost:2182 Topic:tops1 PartitionCount:4 ReplicationFactor:3 Configs: Topic: tops1 Partition: 0 Leader: 4 Replicas: 4,1,2 Isr: 4,1,2 Topic: tops1 Partition: 1 Leader: 5 Replicas: 5,2,3 Isr: 5,3,2 Topic: tops1 Partition: 2 Leader: 1 Replicas: 1,3,4 Isr: 4,1,3 Topic: tops1 Partition: 3 Leader: 2 Replicas: 2,4,5 Isr: 4,2,5 When I run this command in parallel from 2 different machines, I get SocketTimeoutException:- time kafka-producer-perf-test.sh --broker-list x.x.x.x:9092,x.x.x.y:9092,x.x.x.z:9092,x.x.x.a:9092,x.x.x.b:9092 --messages 100 --message-size 500 --topics tops1 --show-detailed-stats --threads 5 --request-num-acks -1 --batch-size 1000 My server.properties has these details:- broker.id=0 port=9092 num.network.threads=3 num.io.threads=8 socket.send.buffer.bytes=1048576 socket.receive.buffer.bytes=1048576 socket.request.max.bytes=104857600 log.dirs=/kafka-logs num.partitions=1 num.recovery.threads.per.data.dir=1 log.retention.hours=168 log.segment.bytes=1073741824 log.retention.check.interval.ms=30 log.cleaner.enable=false zookeeper.connect=localhost:2181 zookeeper.connection.timeout.ms=6000 num.replica.fetchers=4 2 major changes in the server.properties, which gave a performance boost on this test were:- num.replica.fetchers=4 socket.send.buffer.bytes=1048576 socket.receive.buffer.bytes=1048576 I think I'm giving enough requests on the network, but not sure how I should correct it I am particularly interested in knowing how much max throughput per topic can I get, with the highest level of durability (i.e. request-num-acks = -1) Request you to share your thoughts on this. Thanks, Prabhjot