[ https://issues.apache.org/jira/browse/KAFKA-200?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13168567#comment-13168567 ]
John Fung commented on KAFKA-200: --------------------------------- Hi Jay, Values greater than 64k is taking effect in a throughput test carried out recently. Please refer to the following comments for more details. =================== A throughput test had been carried out to observe the throughput gain by varying the socket buffer sizes at both the sending and receiving ends. The following is the layout of the components to test: | Sending Colo | Receiving Colo | |------------------ L.A. --------------------|-------- Chicago -------| | Producer (500MB data) --> Source Kafka <------- ConsoleConsumer | Environments ============ - Sending Colo : L.A. - Receiving Colo : Chicago - Send Buffer Sizes for Source Kafka server : 100K, 500K, 1M - Receive Buffer Sizes for ConsoleConsumer : 100K, 500K, 1M - Hardware for both colos: CPU (2 physical, 16 virtual), 24GB RAM, Linux x86_64 Testing Steps ============= 1. Add the property "socket.send.buffer=102400" (eg. 100K) in "kafka.properties" file for source kafka server 2. Start Zookeeper and Kafka in sending colo 3. Start ProducerPerformance class to produce 500MB of data to source Kafka server and wait until all data is produced. 4. ConsoleConsumer is modified to timeout after 5 sec without incoming messages to get the total time for consuming 500MB of data. 5. In receiving colo, specify the argument "--socket-buffer-size 102400" (100K in this case) for ConsoleConsumer 6. Start ConsoleConsumer to connect to the zookeeper in sending colo and consume the data Results ======= Kafka Cons Cons Cons Send Recv Fetch Time Buff Buff Size Taken Min ===== ===== ===== ===== ===== 100K 100K 1M 313 5.20 100K 500K 1M 305 5.10 100K 1M 1M 307 5.12 500K 100K 1M 311 5.20 500K 500K 1M 120 2.00 500K 1M 1M 121 2.00 1M 100K 1M 311 5.20 1M 500K 1M 121 2.00 1M 1M 1M 98 1.63 1st col: Kafka Send Buffer Size - ("socket.send.buffer") configured in kafka.properties file 2nd col: Consumer Receive Buffer Size "--socket-buffer-size" - which is a command line argument for ConsoleConsumer class 3rd col: Consumer Fetch Size 4th col: Seconds taken by ConsoleConsumer to consume all data 5th col: Corresponding minutes taken Observations ============ The results indicate that the overall throughput is related to the smaller socket buffer size at either the sending or the receiving end. In other words, the smaller socket buffer size acts as a "bottleneck" against the throughput of the pipeline. > Support configurable send / receive socket buffer size in server > ---------------------------------------------------------------- > > Key: KAFKA-200 > URL: https://issues.apache.org/jira/browse/KAFKA-200 > Project: Kafka > Issue Type: Improvement > Components: core > Affects Versions: 0.7 > Reporter: John Fung > Fix For: 0.8 > > Attachments: KAFKA-200.patch > > > * Make the send / receive socket buffer size configurable in server. > * KafkaConfig.scala already has the following existing variables to support > send / receive buffer: > socketSendBuffer > socketReceiveBuffer > * The patch attached to this ticket will read the following existing settings > in <kafka>/config/server.properties and set the corresponding socket buffers > . . . > # The send buffer (SO_SNDBUF) used by the socket server > socket.send.buffer=1048576 > # The receive buffer (SO_RCVBUF) used by the socket server > socket.receive.buffer=1048576 -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira