What's replica.fetch.min.bytes? Is it the default value of 1? If not, could
you try to set it to 1? If yes, could you try using a smaller value of
replica.fetch.wait.max.ms? You can then re-run the test and report back the
results.

Thanks,

Jun


On Mon, Feb 3, 2014 at 9:26 AM, Michael Popov <[email protected]> wrote:

> Hi Jun,
>
> Based on visual review of the log files, most of the time is spent in
> remoteTime.
>
> Example:
> > grep Completed kafka-request.log.2014-01-30-01 | sed
> 's/^.*totalTime://' | less
> ...
> 502,queueTime:1,localTime:1,remoteTime:499,sendTime:1
> (kafka.request.logger)
> 502,queueTime:0,localTime:1,remoteTime:500,sendTime:1
> (kafka.request.logger)
> 502,queueTime:1,localTime:0,remoteTime:500,sendTime:1
> (kafka.request.logger)
> 502,queueTime:1,localTime:0,remoteTime:500,sendTime:1
> (kafka.request.logger)
> 502,queueTime:1,localTime:1,remoteTime:500,sendTime:0
> (kafka.request.logger)
> 503,queueTime:1,localTime:0,remoteTime:501,sendTime:1
> (kafka.request.logger)
> 503,queueTime:1,localTime:1,remoteTime:501,sendTime:0
> (kafka.request.logger)
> 502,queueTime:1,localTime:0,remoteTime:501,sendTime:0
> (kafka.request.logger)
> 502,queueTime:1,localTime:0,remoteTime:501,sendTime:0
> (kafka.request.logger)
> ...
>
> Some of the tests were executed with recommended configuration settings
> copied from the product webpage. These configuration settings included
> following lines for "replica":
>
> num.replica.fetchers=4
> replica.fetch.max.bytes=1048576
> replica.fetch.wait.max.ms=500
> replica.high.watermark.checkpoint.interval.ms=5000
> replica.socket.timeout.ms=30000
> replica.socket.receive.buffer.bytes=65536
> replica.lag.time.max.ms=10000
> replica.lag.max.messages=4000
>
> In most cases the tests were executed with "out-of-box" settings, which
> don't change "replica" configuration.
>
> We are running these tests on very weak machines. If absolute throughput
> numbers are not as high as in other people's tests, that's understandable.
> The main concern is why throughput drops 4-10 times when a number of
> expected acks is not 1.
>
> Should we wait for newer versions of Kafka and settle with acks=1 for now?
>
> Thank you,
> Michael Popov
>
> -----Original Message-----
> From: Jun Rao [mailto:[email protected]]
> Sent: Friday, January 31, 2014 8:57 PM
> To: [email protected]
> Subject: Re: Kafka performance test: "--request-num-acks -1" kills
> throughput
>
> The request log shows you the breakdown of the request time. Where is most
> of the time spent? Also, did you change replica.fetch.wait.max.ms in the
> broker?
>
> Thanks,
>
> Jun
>
>
> On Fri, Jan 31, 2014 at 10:14 AM, Michael Popov <[email protected]
> >wrote:
>
> > Hi Jun,
> >
> > The usage output of bin/kafka-producer-perf-test.sh shows
> >
> > --request-timeout-ms <Integer>          The produce request timeout in ms
> >
> >   (default: 3000)
> >
> > In my test runs I used the following command-line arguments:
> >
> >    bin/kafka-producer-perf-test.sh --broker-list 10.0.0.8:9092,
> > 10.0.0.10:9092 --compression-codec 0 --message-size 1024
> > --request-num-acks -1 --sync --messages 100000 -threads 1
> > --show-detailed-stats --reporting-interval 1000 --topics d2111 | grep
> > -v "at "
> >
> > I assume producer uses default timeout of 3000ms in my tests.
> >
> > I ran a few data processing operations on kafka-request.log.*
> >
> > grep Completed kafka-request.log.2014-01-30-17 | sed 's/^.*totalTime://'
> > | sed 's/,.*$//' > /tmp/times.log
> > cat /tmp/times.log | sort -k 1n | uniq > /tmp/result.log tail
> > /tmp/result.log
> > 5643
> > 5761
> > 5834
> > 6319
> > 6320
> > 6535
> > 8407
> > 8554
> > 9422
> > 9502
> >
> > I guess that means some of Fetch requests from the leader broker to
> > the follower broker took much longer than 3000ms. That explains
> > expired producer's requests and probably the overall decrease of the
> > throughput in tests with acksNum!=1.
> >
> > What may cause such long responses? Is there a way to address this issue?
> >
> > Thank you,
> > Michael Popov
> >
> >
> >
> > -----Original Message-----
> > From: Jun Rao [mailto:[email protected]]
> > Sent: Friday, January 31, 2014 7:52 AM
> > To: [email protected]
> > Subject: Re: Kafka performance test: "--request-num-acks -1" kills
> > throughput
> >
> > Michael,
> >
> > Those SocketTimeoutExceptions meas that the producer didn't receive
> > the response from the broker in time. Could you check the request log
> > in the broker and see what the request completion time is and how it
> > compares with the request socket timeout?
> >
> > I did some testing a while back. Latency wise, ack=-1 is about twice
> > that of ack=1 since it requires an extra network round trip.
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Thu, Jan 30, 2014 at 11:23 AM, Michael Popov <[email protected]
> > >wrote:
> >
> > > Hi Jun,
> > >
> > > I ran a new new tests today.
> > >
> > > Settings: 2 Kafka brokers, 1 Zookeeper node, 4 client machines.
> > > Creating a new topic with 1 partition for each client process.
> > > Commands to create a topic looked like this one:
> > >
> > >    bin/kafka-create-topic.sh --partition 1 --replica 2 --zookeeper
> > > 10.0.0.4:2181 --topic d1
> > >
> > > Commands to run a test producer looked like this:
> > >
> > >     bin/kafka-producer-perf-test.sh --broker-list 10.0.0.8:9092,
> > > 10.0.0.10:9092 --compression-codec 0 --message-size 1024
> > > --request-num-acks 1 --sync --messages 100000 -threads 10
> > > --show-detailed-stats --reporting-interval 1000 --topics d1 | grep
> > > -v
> > "at "
> > >
> > > Results of 4 test runs:
> > > ======================================
> > > --request-num-acks  -1
> > > =====================================
> > > 2014-01-30 17:55:01:576, 2014-01-30 17:59:23:294, 0, 1024, 200,
> > > 97.66, 0.3731, 100000, 382.0906
> > > 2014-01-30 17:55:01:921, 2014-01-30 18:20:56:952, 0, 1024, 200,
> > > 97.66, 0.0628, 99999, 64.3068
> > > 2014-01-30 17:55:00:037, 2014-01-30 18:20:57:164, 0, 1024, 200,
> > > 97.66, 0.0627, 100000, 64.2208
> > > 2014-01-30 17:54:59:723, 2014-01-30 18:20:34:633, 0, 1024, 200,
> > > 97.66, 0.0636, 100000, 65.1504
> > >
> > > ======================================
> > > --request-num-acks  1
> > > =====================================
> > > 2014-01-30 18:24:54:588, 2014-01-30 18:27:42:408, 0, 1024, 200,
> > > 97.66, 0.5819, 100000, 595.8765
> > > 2014-01-30 18:24:54:830, 2014-01-30 18:27:24:080, 0, 1024, 200,
> > > 97.66, 0.6543, 100000, 670.0168
> > > 2014-01-30 18:24:52:855, 2014-01-30 18:25:48:119, 0, 1024, 200,
> > > 97.66, 1.7671, 100000, 1809.4962
> > > 2014-01-30 18:24:52:763, 2014-01-30 18:27:45:675, 0, 1024, 200,
> > > 97.66, 0.5648, 100000, 578.3289
> > >
> > > ======================================
> > > --request-num-acks  -1
> > > =====================================
> > > 2014-01-30 18:31:08:756, 2014-01-30 18:57:12:123, 0, 1024, 200,
> > > 97.66, 0.0625, 99998, 63.9632
> > > 2014-01-30 18:31:08:860, 2014-01-30 18:37:18:634, 0, 1024, 200,
> > > 97.66, 0.2641, 100000, 270.4355
> > > 2014-01-30 18:31:06:968, 2014-01-30 18:57:28:389, 0, 1024, 200,
> > > 97.66, 0.0618, 99999, 63.2336
> > > 2014-01-30 18:31:06:819, 2014-01-30 18:37:19:703, 0, 1024, 200,
> > > 97.66, 0.2619, 100000, 268.1799
> > >
> > > ======================================
> > > --request-num-acks  1
> > > =====================================
> > > 2014-01-30 19:02:35:171, 2014-01-30 19:05:21:650, 0, 1024, 200,
> > > 97.66, 0.5866, 100000, 600.6764
> > > 2014-01-30 19:02:40:264, 2014-01-30 19:05:01:863, 0, 1024, 200,
> > > 97.66, 0.6897, 100000, 706.2197
> > > 2014-01-30 19:02:38:537, 2014-01-30 19:03:27:687, 0, 1024, 200,
> > > 97.66, 1.9869, 100000, 2034.5880
> > > 2014-01-30 19:02:33:384, 2014-01-30 19:04:51:134, 0, 1024, 200,
> > > 97.66, 0.7089, 100000, 725.9528
> > >
> > >
> > > There is something seriously wrong with Kafka throughput in "2acks"
> > tests.
> > > Anything I can do to resolve this issue?
> > >
> > > One more thing. Producers kept printing error messages like this
> > > during the test runs:
> > > [2014-01-30 18:32:39,777] WARN Failed to send producer request with
> > > correlation id 51676 to broker 1 with data for partitions [d4111,0]
> > > (kafka.producer.async.DefaultEventHandler)
> > > java.net.SocketTimeoutException
> > > [2014-01-30 18:32:53,308] WARN Produce request with correlation id
> > > 57999 failed due to [d4111,0]: kafka.common.RequestTimedOutException
> > > (kafka.producer.async.DefaultEventHandler)
> > > [2014-01-30 18:33:22,350] WARN Failed to send producer request with
> > > correlation id 72096 to broker 1 with data for partitions [d4111,0]
> > > (kafka.producer.async.DefaultEventHandler)
> > > java.net.SocketTimeoutException
> > > [2014-01-30 18:33:39,740] WARN Failed to send producer request with
> > > correlation id 81147 to broker 1 with data for partitions [d4111,0]
> > > (kafka.producer.async.DefaultEventHandler)
> > > java.net.SocketTimeoutException
> > >
> > > Thank you,
> > > Michael Popov
> > >
> > >
> > > -----Original Message-----
> > > From: Jun Rao [mailto:[email protected]]
> > > Sent: Wednesday, January 29, 2014 9:01 PM
> > > To: [email protected]
> > > Subject: Re: Kafka performance test: "--request-num-acks -1" kills
> > > throughput
> > >
> > > Does the result change with just 1 partition?
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Wed, Jan 29, 2014 at 4:06 PM, Michael Popov
> > > <[email protected]>
> > > wrote:
> > >
> > > > Hi,
> > > >
> > > > We need a reliable low-latency message queue that can scale. Kafka
> > > > looks like a right system for this role.
> > > >
> > > > I am running performance tests on multiple platforms: Linux and
> > Windows.
> > > > For test purposes I create topics with 2 replicas and multiple
> > > partitions.
> > > > In all deployments running test producers that wait for both
> replicas'
> > > > acks practically kills Kafka throughput. For example, on the
> > > > following deployment on Linux machines: 2 Kafka brokers, 1
> > > > Zookeeper node, 4 client hosts to create load, 4 topics with 10
> > > > partitions each and 2 replicas
> > > >
> > > > -          running tests with "--request-num-acks 1" produces ~3,600
> > > > msgs/sec
> > > >
> > > > -          running tests with "--request-num-acks -1" produces ~348
> > > > msgs/sec
> > > >
> > > >
> > > > Here is output of one of the four concurrent processes:
> > > >
> > > > [User@Client2 kafka_2.8.0-0.8.0]$  bin/kafka-producer-perf-test.sh
> > > > --broker-list 10.0.0.8:9092,10.0.0.10:9092 --compression-codec 0
> > > > --message-size 1024 --request-num-acks -1 --sync --messages 100000
> > > > -threads
> > > > 10 --show-detailed-stats --reporting-interval 1000 --topics c12 |
> > > > grep -v "at "
> > > > start.time, end.time, compression, message.size, batch.size,
> > > > total.data.sent.in.MB, MB.sec, total.data.sent.in.nMsg, nMsg.sec
> > > > [2014-01-29 23:21:16,720] WARN Property reconnect.interval is not
> > > > valid
> > > > (kafka.utils.VerifiableProperties)
> > > > [2014-01-29 23:21:16,825] WARN Property reconnect.interval is not
> > > > valid
> > > > (kafka.utils.VerifiableProperties)
> > > > [2014-01-29 23:21:16,830] WARN Property reconnect.interval is not
> > > > valid
> > > > (kafka.utils.VerifiableProperties)
> > > > [2014-01-29 23:21:16,831] WARN Property reconnect.interval is not
> > > > valid
> > > > (kafka.utils.VerifiableProperties)
> > > > [2014-01-29 23:21:16,839] WARN Property reconnect.interval is not
> > > > valid
> > > > (kafka.utils.VerifiableProperties)
> > > > [2014-01-29 23:21:16,841] WARN Property reconnect.interval is not
> > > > valid
> > > > (kafka.utils.VerifiableProperties)
> > > > [2014-01-29 23:21:16,847] WARN Property reconnect.interval is not
> > > > valid
> > > > (kafka.utils.VerifiableProperties)
> > > > [2014-01-29 23:21:16,858] WARN Property reconnect.interval is not
> > > > valid
> > > > (kafka.utils.VerifiableProperties)
> > > > [2014-01-29 23:21:16,862] WARN Property reconnect.interval is not
> > > > valid
> > > > (kafka.utils.VerifiableProperties)
> > > > [2014-01-29 23:21:16,867] WARN Property reconnect.interval is not
> > > > valid
> > > > (kafka.utils.VerifiableProperties)
> > > > [2014-01-29 23:32:03,830] WARN Produce request with correlation id
> > > > 11467 failed due to [c12,2]: kafka.common.RequestTimedOutException
> > > > (kafka.producer.async.DefaultEventHandler)
> > > > [2014-01-29 23:32:03,831] WARN Produce request with correlation id
> > > > 11859 failed due to [c12,8]: kafka.common.RequestTimedOutException
> > > > (kafka.producer.async.DefaultEventHandler)
> > > > [2014-01-29 23:32:03,831] WARN Failed to send producer request
> > > > with correlation id 11819 to broker 0 with data for partitions
> > > > [c12,8]
> > > > (kafka.producer.async.DefaultEventHandler)
> > > > java.net.SocketTimeoutException
> > > > [2014-01-29 23:32:03,834] WARN Failed to send producer request
> > > > with correlation id 11315 to broker 0 with data for partitions
> > > > [c12,6]
> > > > (kafka.producer.async.DefaultEventHandler)
> > > > java.net.SocketTimeoutException
> > > > [2014-01-29 23:32:03,834] WARN Failed to send producer request
> > > > with correlation id 11191 to broker 0 with data for partitions
> > > > [c12,4]
> > > > (kafka.producer.async.DefaultEventHandler)
> > > > java.net.SocketTimeoutException
> > > > [2014-01-29 23:32:03,834] WARN Failed to send producer request
> > > > with correlation id 11791 to broker 0 with data for partitions
> > > > [c12,4]
> > > > (kafka.producer.async.DefaultEventHandler)
> > > > java.net.SocketTimeoutException
> > > > [2014-01-29 23:32:03,834] WARN Failed to send producer request
> > > > with correlation id 11395 to broker 0 with data for partitions
> > > > [c12,6]
> > > > (kafka.producer.async.DefaultEventHandler)
> > > > java.net.SocketTimeoutException
> > > > [2014-01-29 23:32:03,834] WARN Failed to send producer request
> > > > with correlation id 11631 to broker 0 with data for partitions
> > > > [c12,4]
> > > > (kafka.producer.async.DefaultEventHandler)
> > > > java.net.SocketTimeoutException
> > > > [2014-01-29 23:32:03,834] WARN Failed to send producer request
> > > > with correlation id 10563 to broker 0 with data for partitions
> > > > [c12,0]
> > > > (kafka.producer.async.DefaultEventHandler)
> > > > java.net.SocketTimeoutException
> > > > [2014-01-29 23:32:03,834] WARN Failed to send producer request
> > > > with correlation id 10907 to broker 0 with data for partitions
> > > > [c12,2]
> > > > (kafka.producer.async.DefaultEventHandler)
> > > > java.net.SocketTimeoutException
> > > > 2014-01-29 23:21:16:562, 2014-01-29 23:40:15:886, 0, 1024, 200,
> > > > 97.66, 0.0857, 100000, 87.7713
> > > >
> > > > The test result is consistent and reproducible in all deployments:
> > > > numbers can vary but changing acks setting consistently reduces
> > > > Kafka throughput
> > > > 4-10 times.
> > > >
> > > > Is it expected system behavior? Any tuning options to resolve the
> > > problem?
> > > >
> > > > Thank you,
> > > > Michael Popov
> > > >
> > > >
> > >
> >
>

Reply via email to