From 3c954b04d237b73f55b93f617c5b3bb9a649b29c Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 10 Mar 2015 09:32:49 -0700 Subject: [PATCH] purgatory micro benchmark --- .../other/kafka/TestPurgatoryPerformance.scala | 199 +++++++++++++++++++++ 1 file changed, 199 insertions(+) create mode 100644 core/src/test/scala/other/kafka/TestPurgatoryPerformance.scala diff --git a/core/src/test/scala/other/kafka/TestPurgatoryPerformance.scala b/core/src/test/scala/other/kafka/TestPurgatoryPerformance.scala new file mode 100644 index 0000000..28c7ad3 --- /dev/null +++ b/core/src/test/scala/other/kafka/TestPurgatoryPerformance.scala @@ -0,0 +1,199 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka + +import java.util.Random +import java.util.concurrent.CountDownLatch + +import joptsimple._ +import kafka.server.{DelayedOperationPurgatory, DelayedOperation} +import kafka.utils._ + +/** + * This is a benchmark test of the purgatory. + */ +object TestPurgatoryPerformance { + + def main(args: Array[String]): Unit = { + val parser = new OptionParser + val numRequestsOpt = parser.accepts("num", "The number of requests") + .withRequiredArg + .describedAs("num_requests") + .ofType(classOf[java.lang.Double]) + val requestRateOpt = parser.accepts("rate", "The request rate") + .withRequiredArg + .describedAs("request_per_second") + .ofType(classOf[java.lang.Double]) + val requestDataSizeOpt = parser.accepts("size", "The request data size") + .withRequiredArg + .describedAs("num_bytes") + .ofType(classOf[java.lang.Long]) + val timeoutOpt = parser.accepts("timeout", "The request timeout") + .withRequiredArg + .describedAs("timeout_milliseconds") + .ofType(classOf[java.lang.Long]) + val pct75Opt = parser.accepts("pct75", "75th percentile of request latency (log-normal distribution)") + .withRequiredArg + .describedAs("75th_percentile") + .ofType(classOf[java.lang.Double]) + val pct50Opt = parser.accepts("pct50", "50th percentile of request latency (log-normal distribution)") + .withRequiredArg + .describedAs("50th_percentile") + .ofType(classOf[java.lang.Double]) + val verboseOpt = parser.accepts("verbose", "show additional information") + .withRequiredArg + .describedAs("true|false") + .ofType(classOf[java.lang.Boolean]) + .defaultsTo(true) + + val options = parser.parse(args : _*) + + CommandLineUtils.checkRequiredArgs(parser, options, numRequestsOpt, requestRateOpt, requestDataSizeOpt, pct75Opt, pct50Opt) + + val numRequests = options.valueOf(numRequestsOpt).intValue + val requestRate = options.valueOf(requestRateOpt).doubleValue + val requestDataSize = options.valueOf(requestDataSizeOpt).intValue + val timeout = options.valueOf(timeoutOpt).longValue + val pct75 = options.valueOf(pct75Opt).doubleValue + val pct50 = options.valueOf(pct50Opt).doubleValue + val verbose = options.valueOf(verboseOpt).booleanValue + + val latencySamples = new LatencySamples(1000000, pct75, pct50) + val intervalSamples = new IntervalSamples(1000000, requestRate) + + val purgatory = new DelayedOperationPurgatory[FakeOperation]("fake purgatory") + + val latch = new CountDownLatch(numRequests) + val start = System.currentTimeMillis + var qtime = start + var end = 0L + var due = start + val generator = new Runnable { + def run(): Unit = { + var i = numRequests + while (i > 0) { + i -= 1 + val interval = intervalSamples.next() + val latency = latencySamples.next() + val now = System.currentTimeMillis + qtime = qtime + interval + due = math.max(due, qtime + math.min(latency, timeout)) + + if (qtime > now) Thread.sleep(qtime - now) + + val request = new FakeOperation(timeout, requestDataSize, latency, latch) + purgatory.tryCompleteElseWatch(request, Seq("fakeKey1", "fakeKey2", "fakeKey3")) + } + end = System.currentTimeMillis + } + } + val generatorThread = new Thread(generator) + generatorThread.start() + generatorThread.join() + latch.await() + val done = System.currentTimeMillis + + if (verbose) { + latencySamples.printlnStats() + intervalSamples.printStats() + println(s"# enqueue rate ($numRequests requests):") + println(s"# target actual") + } + + val targetRate = numRequests.toDouble * 1000d / (qtime - start).toDouble + val actualRate = numRequests.toDouble * 1000d / (end - start).toDouble + + println(s"$targetRate $actualRate") + + purgatory.shutdown() + } + + class LogNormalDistribution(mu: Double, sigma: Double) { + val rand = new Random + def next(): Double = { + val n = rand.nextGaussian() * sigma + mu + math.exp(n) + } + } + + class ExponentialDistribution(lambda: Double) { + val rand = new Random + def next(): Double = { + math.log(1d - rand.nextDouble()) / (- lambda) + } + } + + class LatencySamples(sampleSize: Int, pct75: Double, pct50: Double) { + private[this] val rand = new Random + private[this] val samples = { + val normalMean = math.log(pct50) + val normalStDev = (math.log(pct75) - normalMean) / 0.674490d + val dist = new LogNormalDistribution(normalMean, normalStDev) + (0 until sampleSize).map { _ => dist.next().toLong }.toArray + } + def next() = samples(rand.nextInt(sampleSize)) + + def printlnStats(): Unit = { + val p75 = samples.sorted.apply((sampleSize.toDouble * 0.75d).toInt) + val p50 = samples.sorted.apply((sampleSize.toDouble * 0.5d).toInt) + + println(s"# latency samples: pct75 = $p75, pct50 = $p50, min = ${samples.min}, max = ${samples.max}") + } + } + + class IntervalSamples(sampleSize: Int, requestPerSecond: Double) { + private[this] val rand = new Random + private[this] val samples = { + val dist = new ExponentialDistribution(requestPerSecond / 1000d) + var residue = 0.0 + (0 until sampleSize).map { _ => + val interval = dist.next() + residue + val roundedInterval = interval.toLong + residue = interval - roundedInterval.toDouble + roundedInterval + }.toArray + } + + def next() = samples(rand.nextInt(sampleSize)) + + def printStats(): Unit = { + println(s"# interval samples: rate = ${1000d / (samples.map(_.toDouble).sum / sampleSize.toDouble)}, min = ${samples.min}, max = ${samples.max}") + } + } + + class FakeOperation(delayMs: Long, size: Int, latencyMs: Long, latch: CountDownLatch) extends DelayedOperation(delayMs) { + private[this] val data = new Array[Byte](size) + private[this] val completesAt = System.currentTimeMillis + latencyMs + + def onExpiration(): Unit = {} + def onComplete(): Unit = { + latch.countDown() + } + def tryComplete(): Boolean = { + if (System.currentTimeMillis >= completesAt) + forceComplete() + else + false + } + override def isCompleted(): Boolean = { + if (System.currentTimeMillis >= completesAt) forceComplete() + super.isCompleted() + } + } + +} -- 2.3.0