Skip to content

Instantly share code, notes, and snippets.

@djspiewak
Last active March 29, 2024 16:35
Show Gist options
  • Star 112 You must be signed in to star a gist
  • Fork 6 You must be signed in to fork a gist
  • Save djspiewak/f4cfc08e0827088f17032e0e9099d292 to your computer and use it in GitHub Desktop.
Save djspiewak/f4cfc08e0827088f17032e0e9099d292 to your computer and use it in GitHub Desktop.

Understanding Comparative Benchmarks

I'm going to do something that I don't normally do, which is to say I'm going to talk about comparative benchmarks. In general, I try to confine performance discussion to absolute metrics as much as possible, or comparisons to other well-defined neutral reference points. This is precisely why Cats Effect's readme mentions a comparison to a fixed thread pool, rather doing comparisons with other asynchronous runtimes like Akka or ZIO. Comparisons in general devolve very quickly into emotional marketing.

But, just once, today we're going to talk about the emotional marketing. In particular, we're going to look at Cats Effect 3 and ZIO 2. Now, for context, as of this writing ZIO 2 has released their first milestone; they have not released a final 2.0 version. This implies straight off the bat that we're comparing apples to oranges a bit, since Cats Effect 3 has been out and in production for months. However, there has been a post going around which cites various comparative benchmarks between the two runtimes, so that's exactly what we're going to examine here.

This post will go through a series of comparative benchmarks drawn from both ZIO's and Cats Effect's code bases. For each one, we'll look at the sources, look at the results, and discuss in detail why the results are the way they are and speculate about what it means for realistic applications. All benchmarks are artificial measurements, so we're going to dig into what exactly is being measured. The point of this post isn't really to convince you of any particular opinion, but rather to equip you with the tools to examine performance claims on asynchronous runtimes and draw your own conclusions, rather than having to just trust the judgment of someone like me.

The intent is also to make it as easy as possible for you to run the benchmarks for yourself. To that end, please feel free to follow along at https://github.com/djspiewak/async-runtime-benchmarks The sources can be found in the Benchmarks.scala file, and they're organized to try to make things as easy-to-read and as easy-to-run as possible. Specifically, if you run sbt 'Jmh / run' in the project root, the results will appear in about an hour. JMH takes various parameters which allow you to get results more quickly at the cost of some accuracy (e.g. try -f 1 -i 5 for something that can give you reasonable results in about 10-15 minutes).

Each benchmark is a single method, annotated with @Benchmark. JMH works by running that method repeatedly under very controlled circumstances, with very precise timing, and measuring the outcome.

In all benchmarks, higher numbers are better.

Special thanks to Vasil Vasilev (the author of the Cats Effect work stealing pool) for working with me on this post and all associated analysis, benchmarks, and general testing.

Tldr Results

The whole point of this post is to not be a tldr. However, if you want a concise relative graphic to stare at, here it is:

All graphics in this post will be plotted on a linear relative x-axis, precisely because the absolute differences are generally not relevant.

Leveling the Playing Field

val catsEffectRuntime = cats.effect.unsafe.implicits.global

val zioPlatform = zio.internal.Platform
  .makeDefault(1024)
  .withReportFailure(_ => ())
  .withTracing(zio.internal.Tracing.disabled)
val zioRuntime = zio.Runtime.unsafeFromLayer(zio.ZEnv.live, zioPlatform)

private[this] def runCatsEffect3[A](io: cats.effect.IO[A]): A =
  (cats.effect.IO.cede.flatMap(_ => io)).unsafeRunSync()(catsEffectRuntime)

private[this] def runZIO[A](io: zio.UIO[A]): A =
  zioRuntime.unsafeRun(zio.UIO.yieldNow.flatMap(_ => io))

For starters, we need to make sure that both runtimes are being evaluated on the same terms. This means that features like tracing are disabled for both runtimes, since that's not really what we're trying to measure here. Additionally, both runtimes support various tuning parameters such as number of worker threads and frequency of preemptive yielding. Preemptive yields (or "auto-yields") in particular are interesting since they are a feature of the runtime which trades throughput performance for fairness. This is extremely important in practical applications, but it also hurts microbenchmarks which are only measuring very small-scale throughput.

By default, ZIO runs their benchmarks with auto-yielding disabled, though in an apparent oversight, they only apply this configuration to their own runtime. All of their invocations of Cats Effect are using its default runtime configuration, which includes an auto-yield once every 1024 steps. These benchmarks produce the numbers which the ZIO team has been quoting publicly.

By default, ZIO auto-yields once every 2048 steps (the exact number is somewhat arbitrary; Tokio yields once every 128 steps). In the interest of exactly even measurements, the benchmarks in this post set ZIO's auto-yield threshold to 1024, matching Cats Effect's. It would have also been possible and interesting to disable auto-yielding altogether on both runtimes.

One additional tricky difference between Cats Effect and ZIO is the fact that ZIO keeps synchronous fiber prefixes on the calling thread when run. This basically means that if you write something like UIO.foreach(0.until(100))(_ => UIO(println("hi"))) and call unsafeRun on that value, it will run entirely on the calling thread, and won't be shifted into the normal async runtime. It's only once you hit an "async boundary" (like a yield or a Promise) that the normal runtime will kick in.

This implementation detail has a number of implications, but by and large it mostly just affects benchmarks. In steady-state operation, most applications are well-beyond their first async boundary all of the time (e.g. serving an HTTP request is, by definition, beyond an async boundary). However, microbenchmarks do a lot of running of these types of programs, meaning that ZIO is very well optimized... for the benchmarks.

Note that this is very similar to what Cats Effect 2's IO used to do, an optimization which was not retained in Cats Effect 3 for exactly this reason.

It's also worth noting that this optimization is not entirely without cost. Hogging the calling thread in this fashion and preventing it from sleeping can result in poor thread affinity in applications which call unsafeRun at many points (i.e. which aren't using IOApp, or which are using it in conjunction with legacy code). This is why Cats Effect doesn't bother optimizing for the microbenchmark case. When you call unsafeRunSync() on an IO, it immediately and proactively shifts to the runtime thread pool. This shift has a one-time performance cost when your application starts, or when a benchmark runs.

In this case, it's relatively straightforward to adjust the benchmarks to compare ZIO and Cats Effect more evenly: all benchmarks are modified to start with a IO.cede or UIO.yieldNow. This ensures that both runtimes pay the penalty of the leading yield. It's still not technically an even comparison, since Cats Effect is paying that penalty twice (thus, all benchmarks in this post are slightly and systematically biased in favor of ZIO), but it's a lot more even than if things were left in their default configuration.

Beyond the above changes, the benchmarks you find in this repository are effectively identical to the ones you would find in the ZIO and Cats Effect repositories, and they allow both runtimes to be directly compared as accurately as possible.

Benchmarks

Again, always remember that every benchmark is measuring a specific thing. Benchmarks are very carefully calibrated experiments, and they only answer exactly the question they were designed to answer, nothing more. JMH, the standard microbenchmark framework on the JVM, prints the following warning with every result:

[info] REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on
[info] why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial
[info] experiments, perform baseline and negative tests that provide experimental control, make sure
[info] the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts.
[info] Do not assume the numbers tell you what you want them to tell.

This is excellent advice.

Each benchmark below is annotated with its origination, which is to say, what project originally created the benchmark. This is helpful context since it lends some insight into the choices and biases of those particular projects, helping us understand the cases in which one runtime or another might be better optimized.

Deep Bind

(origin: Monix)

def catsEffect3DeepBind(): Unit = {
  import cats.effect.IO

  def loop(i: Int): IO[Unit] =
    IO.unit.flatMap { _ =>
      if (i > 10000)
        IO.unit
      else
        loop(i + 1)
    }

  runCatsEffect3(loop(0))
}

def zio2DeepBind(): Unit = {
  import zio.UIO

  def loop(i: Int): UIO[Unit] =
    UIO.unit.flatMap { _ =>
      if (i > 10000)
        UIO.unit
      else
        loop(i + 1)
    }

  runZIO(loop(0))
}

This is a great benchmark because it's totally artificial and means almost nothing. No one has a practical application which is just a giant series of flatMaps. Really all this benchmark measures is the fundamental overhead of the callback stack and almost nothing more. It's a good introduction to our methodology though.

The idea here is simple: build an IO which runs flatMap 10,000 times, with each flatMap nested within the previous one (i.e. left-associated). Measure how long this takes. The answer, as it turns out, is about 7.86 nanoseconds per flatMap on Cats Effect, and about 8.31 nanoseconds per flatMap on ZIO. Always remember: a nanosecond saved is a nanosecond earned!

Yeah, it's silly. This used to be the gold standard of IO benchmarking though, so look at how far we've come! The only reason this would matter is if the numbers were wildly apart (like, orders of magnitude). In that case, it might result in a noticeable difference, but we're just no where near that.

The only reason I'm listing this benchmark here is to counter a misconception I have heard repeated (and even encouraged) in various contexts, which is the notion that Cats Effect is slower than ZIO. It simply isn't, and the answer to which one is slower or faster depends considerably on what precise question you're asking. This post (and the accompanying repository) asks several such questions and examines the answers closely, but it is by no means a comprehensive comparative analysis.

Fork Many

(origin: ZIO)

def catsEffect3RuntimeForkMany(): Int = {
  import cats.effect.IO

  def catsEffectForkMany(): Int = {

    def catsEffectRepeat[A](n: Int)(io: IO[A]): IO[A] =
      if (n <= 1) io
      else io.flatMap(_ => catsEffectRepeat(n - 1)(io))

    val io = for {
      deferred <- IO.deferred[Unit]
      ref <- IO.ref(10000)
      effect = ref
        .modify(n => (n - 1, if (n == 1) deferred.complete(()) else IO.unit))
        .flatten
      _ <- catsEffectRepeat(10000)(effect.start)
      _ <- deferred.get
    } yield 0

    runCatsEffect3(io)
  }

  catsEffectForkMany()
}

def zio2SchedulerForkMany(): Int = {
  import zio.{Promise, Ref, ZIO}

  def zioForkMany(): Int = {

    def repeat[R, E, A](n: Int)(zio: ZIO[R, E, A]): ZIO[R, E, A] =
      if (n <= 1) zio
      else zio *> repeat(n - 1)(zio)

    val io = for {
      promise <- Promise.make[Nothing, Unit]
      ref <- Ref.make(10000)
      effect = ref
        .modify(n => (if (n == 1) promise.succeed(()) else ZIO.unit, n - 1))
        .flatten
      _ <- repeat(10000)(effect.forkDaemon)
      _ <- promise.await
    } yield 0

    runZIO(io)
  }

  zioForkMany()
}

And as you might expect from looking at the benchmark, the difference is extremely marginal. Cats Effect is slightly faster, but not because of its fiber runtime, but rather because its fibers are simply lighter weight. Cats Effect Fiber is considerably smaller in terms of memory footprint than ZIO's Fiber (roughly 3x smaller). This isn't surprising, since ZIO carries a lot more context around in its runtime, but the difference adds up in all those allocations.

Now, as we'll see in a moment, ZIO does have a hidden advantage in this benchmark, but it's mostly washed out by the way in which it is constructed. Most of the time spent in this benchmark is just in failed modify operations on the Ref, since all of the fibers will be aggressively contending with each other. The rest of it is just the raw overhead of start/forkDaemon, which is very low on both runtimes.

All in all, this benchmark taken together with the Chained Fork results (below) is mostly just saying that Cats Effect fibers are noticeably lighter than ZIO's fibers, but likely not be enough to matter much in practice. It doesn't say anything about the scheduler itself.

Chained Fork

(origin: ZIO)

def catsEffect3RuntimeChainedFork(): Int = {
  import cats.effect.{Deferred, IO}
  import cats.effect.unsafe.IORuntime

  def catsChainedFork(): Int = {

    def iterate(deferred: Deferred[IO, Unit], n: Int): IO[Any] =
      if (n <= 0) deferred.complete(())
      else IO.unit.flatMap(_ => iterate(deferred, n - 1).start)

    val io = for {
      deferred <- IO.deferred[Unit]
      _ <- iterate(deferred, 1000).start
      _ <- deferred.get
    } yield 0

    runCatsEffect3(io)
  }

  catsChainedFork()
}

def zio2SchedulerChainedFork(): Int = {
  import zio.{Promise, UIO, ZIO}

  def zioChainedFork(): Int = {

    def iterate(promise: Promise[Nothing, Unit], n: Int): UIO[Any] =
      if (n <= 0) promise.succeed(())
      else ZIO.unit.flatMap(_ => iterate(promise, n - 1).forkDaemon)

    val io = for {
      promise <- Promise.make[Nothing, Unit]
      _ <- iterate(promise, 1000).forkDaemon
      _ <- promise.await
    } yield 0

    runZIO(io)
  }

  zioChainedFork()
}

Here is a much more definitive win for ZIO! Let's look at what's going on. Basically, this is a very sequential chain of fibers. Each one starts up, checks to see if it's the last in the chain, and if not creates and starts a new copy of itself, then shuts down. This kind of thing is more than a little strange because there's no parallelism here at all, and if you look at a CPU monitor while you run this benchmark, you'll notice that it's mostly not using any processors except the primary one.

At least, ZIO is mostly only using one processor. Cats Effect uses multiple processors more aggressively, and that actually hurts it here. What is happening is Cats Effect proactively wakes up sleeping worker threads whenever start runs, basically asking them to steal the new fiber. This is different from ZIO, which simply leaves the new fiber on the forking thread until some other thread comes along to take it.

ZIO's semantics are slightly better here. When the runtime is fully loaded and all of the workers are busy, there is virtually no difference in how forkDaemon works vs how start works. In either runtime, the new fiber will remain on the current thread, maintaining a high degree of cache affinity and avoiding (most) unnecessary memory barriers. However, when there are any idle workers, Cats Effect works much harder to get them into the game, meaning that the newly-started fiber ends up moving to another thread before it executes when that thread was idle. Which is exactly the situation that most of the threads will be in on this benchmark, since we're really only running one fiber at a time (the whole thing is sequential).

The main difference in a realistic application (where all of the workers are busy) would be that ZIO's forkDaemon has to cross one fewer memory barrier, which does matter in many workloads. This optimization is implementable in Cats Effect's runtime, and probably should be implemented. Doing so flips the advantage on this benchmark by a pretty sizable margin, though it costs Cats Effect about 10-15% in the Scheduler Stress benchmark below.

This is a good example of a benchmark which is not measuring anything particularly useful in and of itself (another measurement of the overhead of start, really), but through careful analysis, we can tease apart why it is producing its outcomes, and this in turn can result in a clearer understanding of one aspect of the problem space.

Ping Pong

(origin: ZIO)

def catsEffect3RuntimePingPong(): Int = {
  import cats.effect.{Deferred, IO}
  import cats.effect.std.Queue

  def catsEffectPingPong(): Int = {

    def catsEffectRepeat[A](n: Int)(io: IO[A]): IO[A] =
      if (n <= 1) io
      else io.flatMap(_ => catsEffectRepeat(n - 1)(io))

    def iterate(deferred: Deferred[IO, Unit], n: Int): IO[Any] =
      for {
        ref <- IO.ref(n)
        queue <- Queue.bounded[IO, Unit](1)
        effect = queue.offer(()).start >>
          queue.take >>
          ref
            .modify(n =>
              (n - 1, if (n == 1) deferred.complete(()) else IO.unit)
            )
            .flatten
        _ <- catsEffectRepeat(1000)(effect.start)
      } yield ()

    val io = for {
      deferred <- IO.deferred[Unit]
      _ <- iterate(deferred, 1000).start
      _ <- deferred.get
    } yield 0

    runCatsEffect3(io)
  }

  catsEffectPingPong()
}

def zio2SchedulerPingPong(): Int = {
  import zio.{Promise, Queue, Ref, UIO, ZIO}

  def zioPingPong(): Int = {

    def repeat[R, E, A](n: Int)(zio: ZIO[R, E, A]): ZIO[R, E, A] =
      if (n <= 1) zio
      else zio *> repeat(n - 1)(zio)

    def iterate(promise: Promise[Nothing, Unit], n: Int): UIO[Any] =
      for {
        ref <- Ref.make(n)
        queue <- Queue.bounded[Unit](1)
        effect = queue.offer(()).forkDaemon *>
          queue.take *>
          ref
            .modify(n =>
              (if (n == 1) promise.succeed(()) else ZIO.unit, n - 1)
            )
            .flatten
        _ <- repeat(1000)(effect.forkDaemon)
      } yield ()

    val io = for {
      promise <- Promise.make[Nothing, Unit]
      _ <- iterate(promise, 1000).forkDaemon
      _ <- promise.await
    } yield 0

    runZIO(io)
  }

  zioPingPong()
}

NB: This benchmark gets a special asterisk because the ZIO implementation deadlocks nondeterministically. A few of the other benchmarks also exhibit this behavior, but this one appears to be the most common. As mentioned at the top, ZIO 2 has only released its first milestone, meaning these kinds of issues are to be expected. Since we couldn't get a reliable final score from JMH, we instead averaged a set of late (non-first) warmup iterations. Thus, you should consider the error bars on ZIO's score here to be relatively high.

And… here we have our first example of a really deceptive benchmark! At first glance, this seems like both a reasonable test and a resounding win for ZIO. It's more than 25% faster than Cats Effect on this test, which is quite significant. However, the benchmark itself is not measuring the runtime: it is measuring Queue.

In this test, 1000 fibers are spawned. Each of which enqueues a single value then dequeues it again immediately. After that, it checks to see if the test is over (note: the use of modify here is really unnecessary and means that a good chunk, though not the majority, of this test is once again just measuring compare-and-swap contention on Ref) and fulfills the Promise/Deferred if so. The problem is that ZIO and Cats Effect have their own Queue data structures with very different implementations. This is a real difference between ZIO and Cats Effect, but it has nothing to do with the runtime.

We can see this relatively clearly if we write a benchmark which compares these queues without involving fibers or Refs:

def catsEffect3EnqueueDequeue(): Unit = {
  import cats.effect.IO
  import cats.effect.std.Queue

  def loop(q: Queue[IO, Unit], i: Int): IO[Unit] =
    if (i >= 10000)
      IO.unit
    else
      q.offer(()).flatMap(_ => q.take.flatMap(_ => loop(q, i + 1)))

  runCatsEffect3(Queue.bounded[IO, Unit](1).flatMap(loop(_, 0)))
}

def zio2EnqueueDequeue(): Unit = {
  import zio.{UIO, Queue}

  def loop(q: Queue[Unit], i: Int): UIO[Unit] =
    if (i >= 10000)
      UIO.unit
    else
      q.offer(()).flatMap(_ => q.take.flatMap(_ => loop(q, i + 1)))

  runZIO(Queue.bounded[Unit](1).flatMap(loop(_, 0)))
}

This performs the same offer >> take sequence as in the Ping Pong test, but without spawning fibers. And as we can see in the results, ZIO's Queue is almost exactly twice as fast as Cats Effect's Queue.

To be clear, this is a meaningful result. Queues are important. Cats Effect's Queue is slower because it is a purely functional implementation, done solely in terms of Ref and Deferred, and despite this its performance is still high enough that a TCP echo server backed by exactly this queue is able to rival raw hand-optimized Netty (which has its own internal mutable queue). But with all that, ZIO's queue is faster because it internally uses a mutable data structure based on jctools. Cats Effect's Queue will probably be modified in the future to take advantage of this optimization when the underlying F[_] forms an Async (instead of just a Concurrent), but it hasn't been a high priority due to its relatively low impact.

Coming back to Ping Pong though, the real question is not "why is ZIO winning?" but more "why isn't ZIO winning more?" Its Queue is almost twice as fast as Cats Effect's at the operations which dominate the majority of the benchmark, so why isn't its advantage even more pronounced? The Scheduler Stress benchmark hints at an answer to this question, but we'll get to that in a bit.

Yield Many

(origin: ZIO)

def catsEffect3RuntimeYieldMany(): Int = {
  import cats.effect.IO

  def catsEffectYieldMany(): Int = {

    def catsEffectRepeat[A](n: Int)(io: IO[A]): IO[A] =
      if (n <= 1) io
      else io.flatMap(_ => catsEffectRepeat(n - 1)(io))

    val io = for {
      deferred <- IO.deferred[Unit]
      ref <- IO.ref(200)
      effect =
        catsEffectRepeat(1000)(IO.cede) >> ref
          .modify(n =>
            (n - 1, if (n == 1) deferred.complete(()) else IO.unit)
          )
          .flatten
      _ <- catsEffectRepeat(200)(effect.start)
      _ <- deferred.get
    } yield 0

    runCatsEffect3(io)
  }

  catsEffectYieldMany()
}

def zio2SchedulerYieldMany(): Int = {
  import zio.{Promise, Ref, ZIO}

  def zioYieldMany(): Int = {

    def repeat[R, E, A](n: Int)(zio: ZIO[R, E, A]): ZIO[R, E, A] =
      if (n <= 1) zio
      else zio *> repeat(n - 1)(zio)

    val io = for {
      promise <- Promise.make[Nothing, Unit]
      ref <- Ref.make(200)
      effect =
        repeat(1000)(ZIO.yieldNow) *> ref
          .modify(n => (if (n == 1) promise.succeed(()) else ZIO.unit, n - 1))
          .flatten
      _ <- repeat(200)(effect.forkDaemon)
      _ <- promise.await
    } yield 0

    runZIO(io)
  }

  zioYieldMany()
}

By now you should be picking up on a pattern in ZIO's scheduler benchmarks: they create a Ref (or similar, like a Queue) and a Promise/Deferred, then they hammer on the Ref across many fibers using different forking patterns until some limit is reached. These tests are interesting in some sense, but most of what they're measuring is how expensive it is to contend on a single Ref. So literally, a compare-and-swap loop (in the modify implementation).

This particular benchmark is theoretically measuring the overhead of yieldNow/cede, but it ends up being dominated by the overhead on modify. You can see this relatively clearly by increasing the number of forked fibers. The benchmark spawns 200 of them, but if you increase this to something much higher, the results are completely different. In particular, we tested with a value of 2000 forked fibers and a Ref limit of 2000, which resulted in Cats Effect coming out 15% faster than ZIO.

The reasons for this are probably somewhat complex. Without deeper profiling and analysis it's hard to say exactly, but my best guess for why there is such a dramatic swing in the results is the fact that the Cats Effect runtime is much more efficient when dealing with a very large set of fibers, particularly when using a much larger set of physical threads (i.e. CPUs). This efficiency advantage starts to dominate over the Ref overhead as the number of fibers increases.

In other words, it appears that ZIO either has a faster Ref or its runtime schedules a small set of fibers in such a way that the contention on that Ref is lower. Given what we learned in the Chained Fork demo, this could be a consequence of worker threads stealing less aggressively, meaning that the physical parallelism is lower and thus the contention is happening less frequently. That's pure speculation, but the theory at least fits the evidence, particularly given that the Ref implementations in ZIO and Cats Effect are functionally identical (they're both just simple wrappers around AtomicReference). However, regardless of ZIO Ref contention is lower in this case, the overhead of their fiber runtime specifically is much higher than Cats Effect's, which is demonstrated by the fact that Cats Effect easily catches up and even pulls into the lead as the number of fibers (and thus, the associated scheduler overhead) increases. The following shows what happens if we increase the number of fibers (and the ref limit) to 2000:

Even more than the Ping Pong benchmark, this is a very subtle and deceptive test. It really isn't measuring the runtime per se, but rather measuring an incidental contention pattern created by a quirk of thread scheduling. The overhead of the runtime can be measured if you increase the number of fibers, and it specifically demonstrates a very different result from the one initially shown.

Scheduler Stress

(origin: Cats Effect)

def catsEffect3Scheduling(): Int = {
  import cats.effect.IO
  import cats.syntax.traverse._

  def schedulingBenchmark(): Int = {
    def fiber(i: Int): IO[Int] =
      IO.cede.flatMap { _ =>
        IO(i).flatMap { j =>
          IO.cede.flatMap { _ =>
            if (j > 10000)
              IO.cede.flatMap(_ => IO(j))
            else
              IO.cede.flatMap(_ => fiber(j + 1))
          }
        }
      }

    val io = List
      .range(0, 1000000)
      .traverse(fiber(_).start)
      .flatMap(_.traverse(_.joinWithNever))
      .map(_.sum)

    runCatsEffect3(io)
  }

  schedulingBenchmark()
}

def zio2Scheduling(): Int = {
  import zio.UIO

  def schedulingBenchmark(): Int = {
    def fiber(i: Int): UIO[Int] =
      UIO.yieldNow.flatMap { _ =>
        UIO(i).flatMap { j =>
          UIO.yieldNow.flatMap { _ =>
            if (j > 10000)
              UIO.yieldNow.flatMap(_ => UIO(j))
            else
              UIO.yieldNow.flatMap(_ => fiber(j + 1))
          }
        }
      }

    val io = UIO
      .foreach(List.range(0, 1000000))(n => fiber(n).forkDaemon)
      .flatMap { list =>
        UIO.foreach(list)(_.join)
      }
      .map(_.sum)

    runZIO(io)
  }

  schedulingBenchmark()
}

By now, you've probably been sensing a bit of a pattern. Most of the benchmarks spawn a large number of fibers which do a small amount of work and then shut down. These are interesting things to measure, as noted extensively above, but it doesn't really tell the whole story. Since only a small number of fibers are executing at once, and those fibers are running over a small subset of the worker threads, the scheduler itself operates throughout the test far below its maximum capacity. A very important element of a fiber runtime is the behavior under load, when a large number of fibers are contending for resources simultaneously. That is what this benchmark is designed to examine.

More specifically, what happens in this test is 1,000,000 fibers are created, the first 10,000 of which iterate rapidly and repeatedly yield until they reach the bound. Thus, different fibers have different workloads, and all of them are pressuring the internal structures within the runtime. This test is meant to emulate a very condensed version of a scatter/gather type workload, but really it's just measuring the overhead of the thread scheduling system itself.

In this test, Cats Effect consistently comes out dramatically ahead of ZIO. This result get more pronounced with larger numbers of processors. We've run this test on machines with an extremely high number of CPUs and the results scale even more dramatically.

The reason for the difference between Cats Effect and ZIO on this test comes from the contention points within the work-stealing scheduler itself. Cats Effect's runtime has been very aggressively optimized to remove contention points between worker threads, ensuring that the overhead of scheduling is almost entirely eliminated. This is one area in which Cats Effect has advanced the state of the art considerably relative to older designs (like Tokio's work-stealing runtime, or the ForkJoinPool in the Java Standard Library), which generally put a considerable amount of load on singular contention points during stealing and local queue overflow, among other cases.

To be clear, this is definitely not a "perfect" benchmark. It is measuring a very specific aspect of the runtime, which is to say, the scheduling overhead under high load. It is certainly not representative of all cases, but it does demonstrate behaviors which are not well-captured by other benchmarks, but which are nevertheless extremely important in production environments at scale. In particular, it suggests that the Cats Effect runtime has lower overhead and is able to more effectively amortize its overhead across available worker threads.

Allocation

(source: Cats Effect)

def catsEffect3Alloc(): Int = {
  import cats.effect.IO
  import cats.syntax.traverse._

  def alloc(): Int = {
    def allocation(n: Int): IO[Array[AnyRef]] =
      IO {
        val size = math.max(100, math.min(n, 2000))
        val array = new Array[AnyRef](size)
        for (i <- (0 until size)) {
          array(i) = new AnyRef()
        }
        array
      }

    def sum(array: Array[AnyRef]): IO[Int] =
      IO {
        array.map(_.hashCode()).sum
      }

    def fiber(i: Int): IO[Int] =
      IO.cede.flatMap { _ =>
        allocation(i).flatMap { arr =>
          IO.cede.flatMap(_ => sum(arr)).flatMap { _ =>
            if (i > 1000)
              IO.cede.flatMap(_ => IO(i))
            else
              IO.cede.flatMap(_ => fiber(i + 1))
          }
        }
      }

    val io = List
      .range(0, 2500)
      .traverse(_ => fiber(0).start)
      .flatMap(_.traverse(_.joinWithNever))
      .map(_.sum)

    runCatsEffect3(io)
  }

  alloc()
}

def zio2Alloc(): Int = {
  import zio.UIO

  def alloc(): Int = {
    def allocation(n: Int): UIO[Array[AnyRef]] =
      UIO {
        val size = math.max(100, math.min(n, 2000))
        val array = new Array[AnyRef](size)
        for (i <- (0 until size)) {
          array(i) = new AnyRef()
        }
        array
      }

    def sum(array: Array[AnyRef]): UIO[Int] =
      UIO {
        array.map(_.hashCode()).sum
      }

    def fiber(i: Int): UIO[Int] =
      UIO.yieldNow.flatMap { _ =>
        allocation(i).flatMap { arr =>
          UIO.yieldNow.flatMap(_ => sum(arr)).flatMap { _ =>
            if (i > 1000)
              UIO.yieldNow.flatMap(_ => UIO(i))
            else
              UIO.yieldNow.flatMap(_ => fiber(i + 1))
          }
        }
      }

    val io = UIO
      .foreach(List.range(0, 2500))(_ => fiber(0).forkDaemon)
      .flatMap { list =>
        UIO.foreach(list)(_.join)
      }
      .map(_.sum)

    runZIO(io)
  }

  alloc()
}

This is a fun one! Teasing it apart a bit, this benchmark allocates 2,500 fibers. Each fiber allocates an array sized between 100 and 2000 (the exact size intentionally varies between fibers), filling every index of the array with a newly-allocated instance of java.lang.Object. The fiber then explicitly yields control to the runtime before traversing the array inefficiently using the map and sum functions, adding up the hash codes for the objects. Then, for any fibers with an index less than 1000, they do it again with successively larger array sizes until 1000 is reached.

There's a lot going on here, but the main intention behind this kind of test is to show how the scheduler interacts with heavy memory pressure. Most JVM applications spend a lot of time allocating objects of varying life spans, traversing those objects, then allowing them to go out of scope and allocating some more. These kinds of patterns are very seldom identical across all running fibers, particularly once the application has been running for a while, thus it is important to set up a test in which the fibers are performing different tasks depending on which fiber it is.

All in all, this is probably the most realistic test for a steady-state request/response service which has no parallelism within its request handling (i.e. imagine a service which parses its request body as JSON, performs some structural transformation, then writes the resulting JSON back as a response string). Services which delegate to other upstream services, particularly those which delegate to multiple upstreams in parallel (scatter/gather patterns), would have a load pattern closer to Scheduler Stress.

Tellingly, ZIO and Cats Effect are almost exactly neck and neck on this test. In fact, the confidence intervals for the results are over 50% overlapping, suggesting that there is almost no material difference. Based on the other benchmarks, it seems likely that ZIO is slightly more efficient at the initial forking, while Cats Effect is slightly more efficient at rebalancing the work during steady-state, but neither factor is all that meaningful. Ultimately, the overhead of the aggressive memory allocations dominates over the runtime overhead in this test.

Summary

It's important to always look at performance claims with a skeptical eye. Performance claims are an answer, and the meaning of that answer depends very much on the question being asked. This is part of why it's so vital to benchmark and load test your own application, with your own traffic patterns, to determine what works best in your environment.

As for the microbenchmarks themselves, ZIO's benchmarks tend to skew considerably towards single contention points, like a Ref or a shared Queue. Since this is where most of their load is in the measurement, this is what their measurements tend to bias towards. Ping Pong is a great example of this, where the benchmark is actually measuring the performance of Queue. These are certainly meaningful measurements, but they're not actually measuring the runtime. Ref#modify for example is basically just flatMap and AtomicReference#compareAndSet, for both ZIO and Cats Effect, which isn't an interesting thing to measure.

Conversely, Cats Effect's benchmarks skew very heavily towards a lot of fibers applying artificially high pressure to the runtime. This kind of measurement can be deceptive in its own right since most applications simply don't pressure the runtime that aggressively, even under load. Just as no application is a single long chain of flatMaps (which is what Deep Bind measures), so too is no application just a giant blob of cedeing fibers of varying lengths (which is what Scheduler Stress measures).

To be clear, both of these are very fast runtimes. Neither ZIO 1 nor Cats Effect 2 are being measured by these benchmarks, simply because they would almost break the scale with how slow they are by comparison. Both use a fixed thread pool by default, which is generally around 50-60x slower than Cats Effect 3's runtime on the scheduler stress test. Additionally, it's still quite early for ZIO 2, as its sources were only recently made publicly available, and the runtime still appears to be somewhat unreliable (at least judging by how it behaved in benchmark scenarios). The situation will undoubtedly shift as it moves towards a production-ready version.

Hopefully this post has made it clear why it is important to view performance claims based on microbenchmarks with a healthy dose of skepticism. Comparative numbers posted without any context, and without verification that the measurement environment was evenly configured and balanced, are at best confusing and at worst actively deceptive.

My best advice is simply this: rather than trusting their numbers or our numbers or anyone's numbers, run the tests for yourself, read the tests, and draw your own conclusions about what it means for your application. If you have any questions, or something just doesn't make sense, it's always safe to reach out and ask! This is certainly a very complicated space and all of us are still learning how best to navigate it.

@djspiewak
Copy link
Author

"Tease apart" is an English expression meaning "gently and carefully find separation points". Like if you imagine a slice of pizza, where you're trying to separate the piece you're about to eat from the rest of the pizza without losing all of the toppings, you might say you're trying to "tease apart the pizza". That'd be a weird thing to say though…

I do tend to use this expression a lot. :-)

@arbitrary-dev
Copy link

Where's your manhood, man? 😁

@objektwerks
Copy link

To paraphrase, Mark Twain, lies, damned lies and benchmarks. :)

@djspiewak
Copy link
Author

To paraphrase, Mark Twain, lies, damned lies and benchmarks. :)

This. Forever this.

@NicolaeNMV
Copy link

NicolaeNMV commented Dec 20, 2023

I'm wondering why there is so much emphasis on these micro benchmarks. I keep hearing people quoting how much more performant that vs this effect library is.
This will most certainly are not going to be the bottleneck of any application.
I cannot picture anyone profiling their application coming to the conclusion aha, the Future implementation is slow!
This sound very much like premature optimisation.

To paraphrase, Mark Twain, lies, damned lies and benchmarks. :)

100% agreeing with this.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment