From 72952a1aeb6c36ef64c67953e1bd80ed8564097b Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Sat, 25 Apr 2020 19:23:13 -0500 Subject: [PATCH 01/78] wip --- .../src/main/scala/cats/effect/IO.scala | 25 +++++-- .../cats/effect/internals/IOBracket.scala | 4 ++ .../cats/effect/internals/IORunLoop.scala | 70 +++++++++++++++---- .../scala/cats/effect/internals/IOStart.scala | 2 + .../cats/effect/internals/IOTracing.scala | 66 +++++++++++++++++ .../cats/effect/tracing/FiberTracing.scala | 46 ++++++++++++ .../cats/effect/tracing/TraceElement.scala | 24 +++++++ .../cats/effect/tracing/TracingStatus.scala | 23 ++++++ 8 files changed, 244 insertions(+), 16 deletions(-) create mode 100644 core/shared/src/main/scala/cats/effect/internals/IOTracing.scala create mode 100644 core/shared/src/main/scala/cats/effect/tracing/FiberTracing.scala create mode 100644 core/shared/src/main/scala/cats/effect/tracing/TraceElement.scala create mode 100644 core/shared/src/main/scala/cats/effect/tracing/TracingStatus.scala diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index eae58406cc..2b7f83d5da 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -26,6 +26,7 @@ import scala.concurrent.{ExecutionContext, Future, Promise, TimeoutException} import scala.util.control.NonFatal import scala.util.{Failure, Left, Right, Success, Try} import cats.data.Ior +import cats.effect.tracing.TraceElement /** * A pure abstraction representing the intention to perform a @@ -100,8 +101,8 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * failures would be completely silent and `IO` references would * never terminate on evaluation. */ - final def map[B](f: A => B): IO[B] = - this match { + final def map[B](f: A => B): IO[B] = { + val source = this match { case Map(source, g, index) => // Allowed to do fixed number of map operations fused before // resetting the counter in order to avoid stack overflows; @@ -112,6 +113,9 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { Map(this, f, 0) } + IOTracing.check(source) + } + /** * Monadic bind on `IO`, used for sequentially composing two `IO` * actions, where the value produced by the first `IO` is passed as @@ -127,8 +131,10 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * failures would be completely silent and `IO` references would * never terminate on evaluation. */ - final def flatMap[B](f: A => IO[B]): IO[B] = - Bind(this, f) + final def flatMap[B](f: A => IO[B]): IO[B] = { + val source = Bind(this, f) + IOTracing.check(source) + } /** * Materializes any sequenced exceptions into value space, where @@ -740,6 +746,9 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { def redeemWith[B](recover: Throwable => IO[B], bind: A => IO[B]): IO[B] = IO.Bind(this, new IOFrame.RedeemWith(recover, bind)) + def traced: IO[A] = + IOTracing(this) + override def toString: String = this match { case Pure(a) => s"IO($a)" case RaiseError(e) => s"IO(throw $e)" @@ -1554,6 +1563,10 @@ object IO extends IOInstances { def contextShift(ec: ExecutionContext): ContextShift[IO] = IOContextShift(ec) + def introspect: IO[List[TraceElement]] = + Introspect + + /* -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-= */ /* IO's internal encoding: */ @@ -1610,6 +1623,10 @@ object IO extends IOInstances { restore: (A, Throwable, IOConnection, IOConnection) => IOConnection ) extends IO[A] + final private[effect] case class Trace[+A](source: IO[A], stackTrace: List[TraceElement]) extends IO[A] + + final private[effect] case object Introspect extends IO[List[TraceElement]] + /* -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-= */ /** diff --git a/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala b/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala index 59b7d7b940..df6130834a 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala @@ -24,6 +24,8 @@ import scala.concurrent.{ExecutionContext, Promise} import scala.util.control.NonFatal import java.util.concurrent.atomic.AtomicBoolean +import cats.effect.tracing.FiberTracing + private[effect] object IOBracket { /** @@ -59,6 +61,7 @@ private[effect] object IOBracket { // This class switches from being a Callback to a Runnable, but relies on // the internal IO callback protocol to be respected (called at most once) private[this] var result: Either[Throwable, A] = _ + private[this] var tracingStatus: Boolean = _ def apply(ea: Either[Throwable, A]): Unit = { if (result ne null) { @@ -67,6 +70,7 @@ private[effect] object IOBracket { // Introducing a light async boundary, otherwise executing the required // logic directly will yield a StackOverflowException result = ea + ec.execute(this) } diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index a6cfac33bc..67a4a29ceb 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -17,13 +17,17 @@ package cats.effect.internals import cats.effect.IO -import cats.effect.IO.{Async, Bind, ContextSwitch, Delay, Map, Pure, RaiseError, Suspend} +import cats.effect.IO.{Async, Bind, ContextSwitch, Delay, Introspect, Map, Pure, RaiseError, Suspend, Trace} +import cats.effect.tracing.{FiberTracing, TraceElement} + import scala.util.control.NonFatal private[effect] object IORunLoop { private type Current = IO[Any] private type Bind = Any => IO[Any] private type CallStack = ArrayStack[Bind] + // TODO: replace with a mutable ring buffer + private type FiberTrace = List[TraceElement] private type Callback = Either[Throwable, Any] => Unit /** @@ -31,14 +35,14 @@ private[effect] object IORunLoop { * with the result when completed. */ def start[A](source: IO[A], cb: Either[Throwable, A] => Unit): Unit = - loop(source, IOConnection.uncancelable, cb.asInstanceOf[Callback], null, null, null) + loop(source, IOConnection.uncancelable, cb.asInstanceOf[Callback], null, null, null, null) /** * Evaluates the given `IO` reference, calling the given callback * with the result when completed. */ def startCancelable[A](source: IO[A], conn: IOConnection, cb: Either[Throwable, A] => Unit): Unit = - loop(source, conn, cb.asInstanceOf[Callback], null, null, null) + loop(source, conn, cb.asInstanceOf[Callback], null, null, null, null) /** * Loop for evaluating an `IO` value. @@ -53,7 +57,8 @@ private[effect] object IORunLoop { cb: Either[Throwable, Any] => Unit, rcbRef: RestartCallback, bFirstRef: Bind, - bRestRef: CallStack + bRestRef: CallStack, + traceRef: FiberTrace ): Unit = { var currentIO: Current = source // Can change on a context switch @@ -61,6 +66,7 @@ private[effect] object IORunLoop { var bFirst: Bind = bFirstRef var bRest: CallStack = bRestRef var rcb: RestartCallback = rcbRef + var trace: FiberTrace = traceRef // Values from Pure and Delay are unboxed in this var, // for code reuse between Pure and Delay var hasUnboxed: Boolean = false @@ -100,6 +106,7 @@ private[effect] object IORunLoop { case RaiseError(ex) => findErrorHandler(bFirst, bRest) match { case null => + FiberTracing.reset() cb(Left(ex)) return case bind => @@ -121,10 +128,11 @@ private[effect] object IORunLoop { case async @ Async(_, _) => if (conn eq null) conn = IOConnection() if (rcb eq null) rcb = new RestartCallback(conn, cb.asInstanceOf[Callback]) - rcb.start(async, bFirst, bRest) + rcb.start(async, bFirst, bRest, trace) return case ContextSwitch(next, modify, restore) => + // TODO: any tracing implications here? val old = if (conn ne null) conn else IOConnection() conn = modify(old) currentIO = next @@ -133,11 +141,22 @@ private[effect] object IORunLoop { if (restore ne null) currentIO = Bind(next, new RestoreContext(old, restore)) } + + case Trace(source, stackTrace) => + if (trace eq null) trace = Nil + trace = stackTrace ++ trace + currentIO = source + + case Introspect => + val returnTrace: List[TraceElement] = if (trace eq null) Nil else trace + hasUnboxed = true + unboxed = returnTrace } if (hasUnboxed) { popNextBind(bFirst, bRest) match { case null => + // TODO: reset status here? cb(Right(unboxed)) return case bind => @@ -169,6 +188,7 @@ private[effect] object IORunLoop { var currentIO: Current = source var bFirst: Bind = null var bRest: CallStack = null + var trace: FiberTrace = null // Values from Pure and Delay are unboxed in this var, // for code reuse between Pure and Delay var hasUnboxed: Boolean = false @@ -227,10 +247,23 @@ private[effect] object IORunLoop { case Async(_, _) => // Cannot inline the code of this method — as it would // box those vars in scala.runtime.ObjectRef! - return suspendAsync(currentIO.asInstanceOf[IO.Async[A]], bFirst, bRest) + // TODO: Since IO.traced is implemented in terms of IOBracket + // we may not need to concern ourselves with tracing status here? + return suspendAsync(currentIO.asInstanceOf[IO.Async[A]], bFirst, bRest, trace) + + case Trace(source, stackTrace) => + if (trace eq null) trace = Nil + trace = stackTrace ++ trace + currentIO = source + + case Introspect => + val returnTrace: List[TraceElement] = if (trace eq null) Nil else trace + hasUnboxed = true + unboxed = returnTrace + case _ => return Async { (conn, cb) => - loop(currentIO, conn, cb.asInstanceOf[Callback], null, bFirst, bRest) + loop(currentIO, conn, cb.asInstanceOf[Callback], null, bFirst, bRest, trace) } } @@ -255,13 +288,13 @@ private[effect] object IORunLoop { // $COVERAGE-ON$ } - private def suspendAsync[A](currentIO: IO.Async[A], bFirst: Bind, bRest: CallStack): IO[A] = + private def suspendAsync[A](currentIO: IO.Async[A], bFirst: Bind, bRest: CallStack, trace: FiberTrace): IO[A] = // Hitting an async boundary means we have to stop, however // if we had previous `flatMap` operations then we need to resume // the loop with the collected stack if (bFirst != null || (bRest != null && !bRest.isEmpty)) Async { (conn, cb) => - loop(currentIO, conn, cb.asInstanceOf[Callback], null, bFirst, bRest) + loop(currentIO, conn, cb.asInstanceOf[Callback], null, bFirst, bRest, trace) } else currentIO @@ -336,6 +369,8 @@ private[effect] object IORunLoop { private[this] var trampolineAfter = false private[this] var bFirst: Bind = _ private[this] var bRest: CallStack = _ + private[this] var tracingStatus: Boolean = false + private[this] var trace: FiberTrace = _ // Used in combination with trampolineAfter = true private[this] var value: Either[Throwable, Any] = _ @@ -343,11 +378,15 @@ private[effect] object IORunLoop { def contextSwitch(conn: IOConnection): Unit = this.conn = conn - def start(task: IO.Async[Any], bFirst: Bind, bRest: CallStack): Unit = { + def start(task: IO.Async[Any], bFirst: Bind, bRest: CallStack, trace: FiberTrace): Unit = { canCall = true this.bFirst = bFirst this.bRest = bRest + this.trace = trace this.trampolineAfter = task.trampolineAfter + // Save the bound tracing status to recover after the continuation completes + this.tracingStatus = FiberTracing.getAndReset() + // Go, go, go task.k(conn, this) } @@ -356,16 +395,23 @@ private[effect] object IORunLoop { // Allow GC to collect val bFirst = this.bFirst val bRest = this.bRest + val trace = this.trace this.bFirst = null this.bRest = null + this.trace = null + + // TODO: Set only if tracing? + // Recover tracing status + println("recovering status") + FiberTracing.set(tracingStatus) // Auto-cancelable logic: in case the connection was cancelled, // we interrupt the bind continuation if (!conn.isCanceled) either match { case Right(success) => - loop(Pure(success), conn, cb, this, bFirst, bRest) + loop(Pure(success), conn, cb, this, bFirst, bRest, trace) case Left(e) => - loop(RaiseError(e), conn, cb, this, bFirst, bRest) + loop(RaiseError(e), conn, cb, this, bFirst, bRest, trace) } } diff --git a/core/shared/src/main/scala/cats/effect/internals/IOStart.scala b/core/shared/src/main/scala/cats/effect/internals/IOStart.scala index 10f0711183..2cd968e29c 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOStart.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOStart.scala @@ -37,6 +37,8 @@ private[effect] object IOStart { p.success(ea) () } + // TODO: We need to ensure that the thread that begins executing the new IO + // has a tracing status reset because of how IOBracket is implemented. IORunLoop.startCancelable(IOForkedStart(fa, cs), conn2, cb0) cb(Right(fiber(p, conn2))) diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala new file mode 100644 index 0000000000..e57543bad4 --- /dev/null +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -0,0 +1,66 @@ +/* + * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers + * + * Licensed 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 cats.effect.internals + +import cats.effect.{ExitCase, IO} +import cats.effect.IO.Trace +import cats.effect.tracing.{FiberTracing, TraceElement} + +private[effect] object IOTracing { + + def apply[A](source: IO[A]): IO[A] = + IOBracket(enable)(_ => source)(disable) + + private def enable: IO[Boolean] = + IO.delay { + val os = FiberTracing.get() + FiberTracing.set(true) + os + } + + // TODO: This finalizer doesn't actually work in the + // case of cancellation because it is invoked by the thread + // that initiated cancellation. + // This leaves the thread the cancelled fiber was bound to in + // an invalid state, so instead we have to reset the status when we + // begin interpreting a new IO. + private def disable(oldStatus: Boolean, exitCase: ExitCase[Throwable]): IO[Unit] = + IO.delay { + val isSameFiber = exitCase match { + case ExitCase.Completed => true + case ExitCase.Error(_) => true + case _ => false + } + + if (isSameFiber) + FiberTracing.set(oldStatus) + + () + } + + def check[A](source: IO[A]): IO[A] = { + if (FiberTracing.get()) { + // The userspace method invocation is at least two frames away + // TODO: filtering here? + val stackTrace = new Throwable().getStackTrace.toList.map(TraceElement.fromStackTraceElement) + Trace(source, stackTrace) + } else { + source + } + } + +} diff --git a/core/shared/src/main/scala/cats/effect/tracing/FiberTracing.scala b/core/shared/src/main/scala/cats/effect/tracing/FiberTracing.scala new file mode 100644 index 0000000000..75db735a4c --- /dev/null +++ b/core/shared/src/main/scala/cats/effect/tracing/FiberTracing.scala @@ -0,0 +1,46 @@ +/* + * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers + * + * Licensed 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 cats.effect.tracing + +private[effect] object FiberTracing { + + def get(): Boolean = + tracingStatus.get() + + def set(newStatus: Boolean): Unit = + tracingStatus.set(newStatus) + + def reset(): Unit = + tracingStatus.remove() + + def getAndReset(): Boolean = { + val s = get() + reset() + s + } + + // TODO: Create a ached global flag so thread-local doesn't + // have to be read when tracing is disabled + + /** + * Thread-local storage for fiber tracing status. + */ + private val tracingStatus = new ThreadLocal[Boolean] { + override def initialValue(): Boolean = false + } + +} diff --git a/core/shared/src/main/scala/cats/effect/tracing/TraceElement.scala b/core/shared/src/main/scala/cats/effect/tracing/TraceElement.scala new file mode 100644 index 0000000000..138853d3b5 --- /dev/null +++ b/core/shared/src/main/scala/cats/effect/tracing/TraceElement.scala @@ -0,0 +1,24 @@ +/* + * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers + * + * Licensed 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 cats.effect.tracing + +final case class TraceElement(className: String, methodName: String, fileName: String, lineNumber: Int) + +object TraceElement { + def fromStackTraceElement(ste: StackTraceElement): TraceElement = + TraceElement(ste.getClassName, ste.getMethodName, ste.getFileName, ste.getLineNumber) +} diff --git a/core/shared/src/main/scala/cats/effect/tracing/TracingStatus.scala b/core/shared/src/main/scala/cats/effect/tracing/TracingStatus.scala new file mode 100644 index 0000000000..e37bef6963 --- /dev/null +++ b/core/shared/src/main/scala/cats/effect/tracing/TracingStatus.scala @@ -0,0 +1,23 @@ +/* + * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers + * + * Licensed 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 cats.effect.tracing + +sealed abstract class TracingStatus + +object TracingStatus { + case object None extends TracingStatus +} From 5301fe92f1235ef1659f9ee24e6895a90b93f828 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Sun, 26 Apr 2020 22:16:59 -0500 Subject: [PATCH 02/78] Introduce global tracing flag --- .../benchmarks/SingleMapCallBenchmark.scala | 5 ++ .../effect/internals/TracingPlatform.java | 58 +++++++++++++++++++ .../src/main/scala/cats/effect/IO.scala | 10 ++-- .../cats/effect/internals/IOBracket.scala | 3 - .../cats/effect/internals/IORunLoop.scala | 11 +--- .../cats/effect/internals/IOTracing.scala | 52 +++++------------ .../cats/effect/tracing/FiberTracing.scala | 46 --------------- .../cats/effect/tracing/TracingStatus.scala | 6 +- 8 files changed, 87 insertions(+), 104 deletions(-) create mode 100644 benchmarks/shared/src/main/scala/cats/effect/benchmarks/SingleMapCallBenchmark.scala create mode 100644 core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java delete mode 100644 core/shared/src/main/scala/cats/effect/tracing/FiberTracing.scala diff --git a/benchmarks/shared/src/main/scala/cats/effect/benchmarks/SingleMapCallBenchmark.scala b/benchmarks/shared/src/main/scala/cats/effect/benchmarks/SingleMapCallBenchmark.scala new file mode 100644 index 0000000000..055595623b --- /dev/null +++ b/benchmarks/shared/src/main/scala/cats/effect/benchmarks/SingleMapCallBenchmark.scala @@ -0,0 +1,5 @@ +package cats.effect.benchmarks + +class SingleMapCallBenchmark { + +} diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java new file mode 100644 index 0000000000..fc37544dee --- /dev/null +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java @@ -0,0 +1,58 @@ +/* + * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers + * + * Licensed 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 cats.effect.internals; + +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +/** + * + * Motivation: + * In Scala, object-level variable reads cause a volatile read. + * Instead, + */ +class TracingPlatform { + + /** + * A boolean variable that controls tracing globally. For tracing to + * take effect, this flag must be enabled. + */ + public static final boolean tracingEnabled; + + /** + * Global, thread-safe cache for traces. Keys are generally + * lambda references. + * + * TODO: Could this be a thread-local? + * If every thread eventually calculates its own set, + * there should be no issue? + * + * TODO: Bound the cache. + */ + public static final Map traceCache; + + static { + tracingEnabled = Optional.ofNullable(System.getProperty("cats.effect.tracingEnabled")) + .filter(x -> !x.isEmpty()) + .map(x -> Boolean.valueOf(x)) // TODO: this can throw + .orElse(false); + + traceCache = new HashMap<>(); + } + +} diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index 2b7f83d5da..b7902186ed 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -102,7 +102,8 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * never terminate on evaluation. */ final def map[B](f: A => B): IO[B] = { - val source = this match { +// val source = + this match { case Map(source, g, index) => // Allowed to do fixed number of map operations fused before // resetting the counter in order to avoid stack overflows; @@ -113,7 +114,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { Map(this, f, 0) } - IOTracing.check(source) +// IOTracing.check(source, f.asInstanceOf[AnyRef]) } /** @@ -133,7 +134,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { */ final def flatMap[B](f: A => IO[B]): IO[B] = { val source = Bind(this, f) - IOTracing.check(source) + IOTracing.check(source, f.asInstanceOf[AnyRef]) } /** @@ -746,9 +747,6 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { def redeemWith[B](recover: Throwable => IO[B], bind: A => IO[B]): IO[B] = IO.Bind(this, new IOFrame.RedeemWith(recover, bind)) - def traced: IO[A] = - IOTracing(this) - override def toString: String = this match { case Pure(a) => s"IO($a)" case RaiseError(e) => s"IO(throw $e)" diff --git a/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala b/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala index df6130834a..309413dac4 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala @@ -24,8 +24,6 @@ import scala.concurrent.{ExecutionContext, Promise} import scala.util.control.NonFatal import java.util.concurrent.atomic.AtomicBoolean -import cats.effect.tracing.FiberTracing - private[effect] object IOBracket { /** @@ -61,7 +59,6 @@ private[effect] object IOBracket { // This class switches from being a Callback to a Runnable, but relies on // the internal IO callback protocol to be respected (called at most once) private[this] var result: Either[Throwable, A] = _ - private[this] var tracingStatus: Boolean = _ def apply(ea: Either[Throwable, A]): Unit = { if (result ne null) { diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index 67a4a29ceb..c633c9359d 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -18,7 +18,7 @@ package cats.effect.internals import cats.effect.IO import cats.effect.IO.{Async, Bind, ContextSwitch, Delay, Introspect, Map, Pure, RaiseError, Suspend, Trace} -import cats.effect.tracing.{FiberTracing, TraceElement} +import cats.effect.tracing.TraceElement import scala.util.control.NonFatal @@ -106,7 +106,6 @@ private[effect] object IORunLoop { case RaiseError(ex) => findErrorHandler(bFirst, bRest) match { case null => - FiberTracing.reset() cb(Left(ex)) return case bind => @@ -369,7 +368,6 @@ private[effect] object IORunLoop { private[this] var trampolineAfter = false private[this] var bFirst: Bind = _ private[this] var bRest: CallStack = _ - private[this] var tracingStatus: Boolean = false private[this] var trace: FiberTrace = _ // Used in combination with trampolineAfter = true @@ -384,8 +382,6 @@ private[effect] object IORunLoop { this.bRest = bRest this.trace = trace this.trampolineAfter = task.trampolineAfter - // Save the bound tracing status to recover after the continuation completes - this.tracingStatus = FiberTracing.getAndReset() // Go, go, go task.k(conn, this) @@ -400,11 +396,6 @@ private[effect] object IORunLoop { this.bRest = null this.trace = null - // TODO: Set only if tracing? - // Recover tracing status - println("recovering status") - FiberTracing.set(tracingStatus) - // Auto-cancelable logic: in case the connection was cancelled, // we interrupt the bind continuation if (!conn.isCanceled) either match { diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index e57543bad4..ee7615c3b9 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -16,48 +16,26 @@ package cats.effect.internals -import cats.effect.{ExitCase, IO} -import cats.effect.IO.Trace -import cats.effect.tracing.{FiberTracing, TraceElement} +import cats.effect.IO +import cats.effect.tracing.TraceElement +import TracingPlatform.{tracingEnabled, traceCache} private[effect] object IOTracing { - def apply[A](source: IO[A]): IO[A] = - IOBracket(enable)(_ => source)(disable) - - private def enable: IO[Boolean] = - IO.delay { - val os = FiberTracing.get() - FiberTracing.set(true) - os - } - - // TODO: This finalizer doesn't actually work in the - // case of cancellation because it is invoked by the thread - // that initiated cancellation. - // This leaves the thread the cancelled fiber was bound to in - // an invalid state, so instead we have to reset the status when we - // begin interpreting a new IO. - private def disable(oldStatus: Boolean, exitCase: ExitCase[Throwable]): IO[Unit] = - IO.delay { - val isSameFiber = exitCase match { - case ExitCase.Completed => true - case ExitCase.Error(_) => true - case _ => false - } - - if (isSameFiber) - FiberTracing.set(oldStatus) - - () - } - - def check[A](source: IO[A]): IO[A] = { - if (FiberTracing.get()) { + // TODO: Lazily evaluate key? + // calculating this key has a cost. inline the checks + def check[A](source: IO[A], key: AnyRef): IO[A] = { + if (tracingEnabled) { // The userspace method invocation is at least two frames away // TODO: filtering here? - val stackTrace = new Throwable().getStackTrace.toList.map(TraceElement.fromStackTraceElement) - Trace(source, stackTrace) + val cachedRef = traceCache.get(key) + if (cachedRef eq null) { + val stackTrace = new Throwable().getStackTrace.toList.map(TraceElement.fromStackTraceElement) + traceCache.put(key, stackTrace) + IO.Trace(source, stackTrace) + } else { + IO.Trace(source, cachedRef.asInstanceOf[List[TraceElement]]) + } } else { source } diff --git a/core/shared/src/main/scala/cats/effect/tracing/FiberTracing.scala b/core/shared/src/main/scala/cats/effect/tracing/FiberTracing.scala deleted file mode 100644 index 75db735a4c..0000000000 --- a/core/shared/src/main/scala/cats/effect/tracing/FiberTracing.scala +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers - * - * Licensed 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 cats.effect.tracing - -private[effect] object FiberTracing { - - def get(): Boolean = - tracingStatus.get() - - def set(newStatus: Boolean): Unit = - tracingStatus.set(newStatus) - - def reset(): Unit = - tracingStatus.remove() - - def getAndReset(): Boolean = { - val s = get() - reset() - s - } - - // TODO: Create a ached global flag so thread-local doesn't - // have to be read when tracing is disabled - - /** - * Thread-local storage for fiber tracing status. - */ - private val tracingStatus = new ThreadLocal[Boolean] { - override def initialValue(): Boolean = false - } - -} diff --git a/core/shared/src/main/scala/cats/effect/tracing/TracingStatus.scala b/core/shared/src/main/scala/cats/effect/tracing/TracingStatus.scala index e37bef6963..a4103d933c 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/TracingStatus.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/TracingStatus.scala @@ -16,8 +16,10 @@ package cats.effect.tracing -sealed abstract class TracingStatus +private[effect] sealed abstract class TracingStatus -object TracingStatus { +private[effect] object TracingStatus { case object None extends TracingStatus + + case object Slug extends TracingStatus } From 6c404dd054e1c5dffba0f72de7ae8c85eb148322 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Sun, 26 Apr 2020 22:42:47 -0500 Subject: [PATCH 03/78] Capture traces in more functions --- .../effect/internals/TracingPlatform.java | 11 ++++++ .../src/main/scala/cats/effect/IO.scala | 30 ++++++++++------ .../cats/effect/internals/IORunLoop.scala | 35 +++++++++---------- .../cats/effect/internals/IOTracing.scala | 24 +++++++------ .../scala/cats/effect/tracing/IOTrace.scala | 26 ++++++++++++++ .../cats/effect/tracing/TracingStatus.scala | 11 +++++- 6 files changed, 97 insertions(+), 40 deletions(-) create mode 100644 core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java index fc37544dee..4dce1011d6 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java @@ -34,6 +34,12 @@ class TracingPlatform { */ public static final boolean tracingEnabled; + // TODO: The following variables can probably be kept in Scala + // because they'll only be used when tracing and tracing is + // pretty expensive anyway. + + public static final int tracingMode; + /** * Global, thread-safe cache for traces. Keys are generally * lambda references. @@ -52,6 +58,11 @@ class TracingPlatform { .map(x -> Boolean.valueOf(x)) // TODO: this can throw .orElse(false); + tracingMode = Optional.ofNullable(System.getProperty("cats.effect.tracingMode")) + .filter(x -> !x.isEmpty()) + .map(x -> Integer.valueOf(x)) // TODO: this can throw + .orElse(0); + traceCache = new HashMap<>(); } diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index b7902186ed..797a6dc456 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -26,7 +26,7 @@ import scala.concurrent.{ExecutionContext, Future, Promise, TimeoutException} import scala.util.control.NonFatal import scala.util.{Failure, Left, Right, Success, Try} import cats.data.Ior -import cats.effect.tracing.TraceElement +import cats.effect.tracing.IOTrace /** * A pure abstraction representing the intention to perform a @@ -134,7 +134,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { */ final def flatMap[B](f: A => IO[B]): IO[B] = { val source = Bind(this, f) - IOTracing.check(source, f.asInstanceOf[AnyRef]) + IOTracing.apply(source, f.asInstanceOf[AnyRef]) } /** @@ -1207,13 +1207,16 @@ object IO extends IOInstances { * * @see [[asyncF]] and [[cancelable]] */ - def async[A](k: (Either[Throwable, A] => Unit) => Unit): IO[A] = - Async { (_, cb) => + def async[A](k: (Either[Throwable, A] => Unit) => Unit): IO[A] = { + val source = Async[A] { (_, cb) => val cb2 = Callback.asyncIdempotent(null, cb) try k(cb2) catch { case NonFatal(t) => cb2(Left(t)) } } + IOTracing(source, k) + } + /** * Suspends an asynchronous side effect in `IO`, this being a variant * of [[async]] that takes a pure registration function. @@ -1238,8 +1241,8 @@ object IO extends IOInstances { * * @see [[async]] and [[cancelable]] */ - def asyncF[A](k: (Either[Throwable, A] => Unit) => IO[Unit]): IO[A] = - Async { (conn, cb) => + def asyncF[A](k: (Either[Throwable, A] => Unit) => IO[Unit]): IO[A] = { + val source = Async[A] { (conn, cb) => // Must create new connection, otherwise we can have a race // condition b/t the bind continuation and `startCancelable` below val conn2 = IOConnection() @@ -1252,6 +1255,9 @@ object IO extends IOInstances { IORunLoop.startCancelable(fa, conn2, Callback.report) } + IOTracing(source, k) + } + /** * Builds a cancelable `IO`. * @@ -1291,8 +1297,8 @@ object IO extends IOInstances { * @see [[asyncF]] for a more potent version that does hook into * the underlying cancelation model */ - def cancelable[A](k: (Either[Throwable, A] => Unit) => CancelToken[IO]): IO[A] = - Async { (conn, cb) => + def cancelable[A](k: (Either[Throwable, A] => Unit) => CancelToken[IO]): IO[A] = { + val source = Async[A] { (conn, cb) => val cb2 = Callback.asyncIdempotent(conn, cb) val ref = ForwardCancelable() conn.push(ref.cancel) @@ -1311,6 +1317,8 @@ object IO extends IOInstances { else ref.complete(IO.unit) } + IOTracing.apply(source, k) + } /** * Constructs an `IO` which sequences the specified exception. @@ -1561,7 +1569,7 @@ object IO extends IOInstances { def contextShift(ec: ExecutionContext): ContextShift[IO] = IOContextShift(ec) - def introspect: IO[List[TraceElement]] = + def introspect: IO[IOTrace] = Introspect @@ -1621,9 +1629,9 @@ object IO extends IOInstances { restore: (A, Throwable, IOConnection, IOConnection) => IOConnection ) extends IO[A] - final private[effect] case class Trace[+A](source: IO[A], stackTrace: List[TraceElement]) extends IO[A] + final private[effect] case class Trace[+A](source: IO[A], stackTrace: IOTrace) extends IO[A] - final private[effect] case object Introspect extends IO[List[TraceElement]] + final private[effect] case object Introspect extends IO[IOTrace] /* -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-= */ diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index c633c9359d..028988e606 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -18,7 +18,7 @@ package cats.effect.internals import cats.effect.IO import cats.effect.IO.{Async, Bind, ContextSwitch, Delay, Introspect, Map, Pure, RaiseError, Suspend, Trace} -import cats.effect.tracing.TraceElement +import cats.effect.tracing.IOTrace import scala.util.control.NonFatal @@ -27,7 +27,6 @@ private[effect] object IORunLoop { private type Bind = Any => IO[Any] private type CallStack = ArrayStack[Bind] // TODO: replace with a mutable ring buffer - private type FiberTrace = List[TraceElement] private type Callback = Either[Throwable, Any] => Unit /** @@ -58,7 +57,7 @@ private[effect] object IORunLoop { rcbRef: RestartCallback, bFirstRef: Bind, bRestRef: CallStack, - traceRef: FiberTrace + traceRef: IOTrace ): Unit = { var currentIO: Current = source // Can change on a context switch @@ -66,7 +65,7 @@ private[effect] object IORunLoop { var bFirst: Bind = bFirstRef var bRest: CallStack = bRestRef var rcb: RestartCallback = rcbRef - var trace: FiberTrace = traceRef + var trace: IOTrace = traceRef // Values from Pure and Delay are unboxed in this var, // for code reuse between Pure and Delay var hasUnboxed: Boolean = false @@ -141,15 +140,15 @@ private[effect] object IORunLoop { currentIO = Bind(next, new RestoreContext(old, restore)) } - case Trace(source, stackTrace) => - if (trace eq null) trace = Nil - trace = stackTrace ++ trace + case Trace(source, currTrace) => + if (trace eq null) trace = IOTrace.Empty + trace = trace.and(currTrace) currentIO = source case Introspect => - val returnTrace: List[TraceElement] = if (trace eq null) Nil else trace + val retTrace = if (trace eq null) IOTrace.Empty else trace hasUnboxed = true - unboxed = returnTrace + unboxed = retTrace } if (hasUnboxed) { @@ -187,7 +186,7 @@ private[effect] object IORunLoop { var currentIO: Current = source var bFirst: Bind = null var bRest: CallStack = null - var trace: FiberTrace = null + var trace: IOTrace = null // Values from Pure and Delay are unboxed in this var, // for code reuse between Pure and Delay var hasUnboxed: Boolean = false @@ -250,15 +249,15 @@ private[effect] object IORunLoop { // we may not need to concern ourselves with tracing status here? return suspendAsync(currentIO.asInstanceOf[IO.Async[A]], bFirst, bRest, trace) - case Trace(source, stackTrace) => - if (trace eq null) trace = Nil - trace = stackTrace ++ trace + case Trace(source, currTrace) => + if (trace eq null) trace = IOTrace.Empty + trace = trace.and(currTrace) currentIO = source case Introspect => - val returnTrace: List[TraceElement] = if (trace eq null) Nil else trace + val retTrace = if (trace eq null) IOTrace.Empty else trace hasUnboxed = true - unboxed = returnTrace + unboxed = retTrace case _ => return Async { (conn, cb) => @@ -287,7 +286,7 @@ private[effect] object IORunLoop { // $COVERAGE-ON$ } - private def suspendAsync[A](currentIO: IO.Async[A], bFirst: Bind, bRest: CallStack, trace: FiberTrace): IO[A] = + private def suspendAsync[A](currentIO: IO.Async[A], bFirst: Bind, bRest: CallStack, trace: IOTrace): IO[A] = // Hitting an async boundary means we have to stop, however // if we had previous `flatMap` operations then we need to resume // the loop with the collected stack @@ -368,7 +367,7 @@ private[effect] object IORunLoop { private[this] var trampolineAfter = false private[this] var bFirst: Bind = _ private[this] var bRest: CallStack = _ - private[this] var trace: FiberTrace = _ + private[this] var trace: IOTrace = _ // Used in combination with trampolineAfter = true private[this] var value: Either[Throwable, Any] = _ @@ -376,7 +375,7 @@ private[effect] object IORunLoop { def contextSwitch(conn: IOConnection): Unit = this.conn = conn - def start(task: IO.Async[Any], bFirst: Bind, bRest: CallStack, trace: FiberTrace): Unit = { + def start(task: IO.Async[Any], bFirst: Bind, bRest: CallStack, trace: IOTrace): Unit = { canCall = true this.bFirst = bFirst this.bRest = bRest diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index ee7615c3b9..6e3942fc9d 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -17,28 +17,32 @@ package cats.effect.internals import cats.effect.IO -import cats.effect.tracing.TraceElement +import cats.effect.tracing.{IOTrace, TraceElement} import TracingPlatform.{tracingEnabled, traceCache} private[effect] object IOTracing { // TODO: Lazily evaluate key? // calculating this key has a cost. inline the checks - def check[A](source: IO[A], key: AnyRef): IO[A] = { + def apply[A](source: IO[A], lambda: AnyRef): IO[A] = { if (tracingEnabled) { - // The userspace method invocation is at least two frames away - // TODO: filtering here? - val cachedRef = traceCache.get(key) - if (cachedRef eq null) { - val stackTrace = new Throwable().getStackTrace.toList.map(TraceElement.fromStackTraceElement) - traceCache.put(key, stackTrace) - IO.Trace(source, stackTrace) + val traceRef = traceCache.get(lambda) + if (traceRef eq null) { + val fiberTrace = createTrace() + traceCache.put(lambda, fiberTrace) + IO.Trace(source, fiberTrace) } else { - IO.Trace(source, cachedRef.asInstanceOf[List[TraceElement]]) + IO.Trace(source, traceRef.asInstanceOf[IOTrace]) } } else { source } } + def createTrace(): IOTrace = { + // TODO: calculate trace here + val lines = new Throwable().getStackTrace.toList.map(TraceElement.fromStackTraceElement) + IOTrace(lines) + } + } diff --git a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala new file mode 100644 index 0000000000..c6b403886c --- /dev/null +++ b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala @@ -0,0 +1,26 @@ +/* + * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers + * + * Licensed 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 cats.effect.tracing + +final case class IOTrace(lines: List[TraceElement]) { + def and(that: IOTrace): IOTrace = + IOTrace(that.lines ++ lines) +} + +object IOTrace { + val Empty = IOTrace(List()) +} diff --git a/core/shared/src/main/scala/cats/effect/tracing/TracingStatus.scala b/core/shared/src/main/scala/cats/effect/tracing/TracingStatus.scala index a4103d933c..4fb2789c44 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/TracingStatus.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/TracingStatus.scala @@ -19,7 +19,16 @@ package cats.effect.tracing private[effect] sealed abstract class TracingStatus private[effect] object TracingStatus { - case object None extends TracingStatus + + case object Rabbit extends TracingStatus case object Slug extends TracingStatus + + def fromString(value: String): TracingStatus = + value.toLowerCase() match { + case "rabbit" => Rabbit + case "slug" => Slug + case _ => Rabbit + } + } From 887a3d72c4aa2bcaaa50c0894bd34958097f2afa Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Mon, 27 Apr 2020 00:37:17 -0500 Subject: [PATCH 04/78] IOContext for threading state across asynchronous boundaries --- .../effect/internals/TracingPlatform.java | 35 ++--------- .../scala/cats/effect/internals/IOTimer.scala | 2 +- .../src/main/scala/cats/effect/IO.scala | 15 +++-- .../effect/internals/ForwardCancelable.scala | 4 +- .../cats/effect/internals/IOBracket.scala | 9 +-- .../cats/effect/internals/IOContext.scala | 36 +++++++++++ .../cats/effect/internals/IOParMap.scala | 2 +- .../scala/cats/effect/internals/IORace.scala | 4 +- .../cats/effect/internals/IORunLoop.scala | 61 ++++++++++--------- .../scala/cats/effect/internals/IOShift.scala | 2 +- .../scala/cats/effect/internals/IOStart.scala | 2 +- .../cats/effect/internals/IOTracing.scala | 45 ++++++++++---- .../scala/cats/effect/internals/package.scala | 2 +- .../scala/cats/effect/tracing/IOTrace.scala | 10 ++- .../{TraceElement.scala => TraceLine.scala} | 9 +-- 15 files changed, 138 insertions(+), 100 deletions(-) create mode 100644 core/shared/src/main/scala/cats/effect/internals/IOContext.scala rename core/shared/src/main/scala/cats/effect/tracing/{TraceElement.scala => TraceLine.scala} (66%) diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java index 4dce1011d6..b80d0cc584 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java @@ -16,8 +16,6 @@ package cats.effect.internals; -import java.util.HashMap; -import java.util.Map; import java.util.Optional; /** @@ -34,36 +32,11 @@ class TracingPlatform { */ public static final boolean tracingEnabled; - // TODO: The following variables can probably be kept in Scala - // because they'll only be used when tracing and tracing is - // pretty expensive anyway. - - public static final int tracingMode; - - /** - * Global, thread-safe cache for traces. Keys are generally - * lambda references. - * - * TODO: Could this be a thread-local? - * If every thread eventually calculates its own set, - * there should be no issue? - * - * TODO: Bound the cache. - */ - public static final Map traceCache; - static { - tracingEnabled = Optional.ofNullable(System.getProperty("cats.effect.tracingEnabled")) - .filter(x -> !x.isEmpty()) - .map(x -> Boolean.valueOf(x)) // TODO: this can throw - .orElse(false); - - tracingMode = Optional.ofNullable(System.getProperty("cats.effect.tracingMode")) - .filter(x -> !x.isEmpty()) - .map(x -> Integer.valueOf(x)) // TODO: this can throw - .orElse(0); - - traceCache = new HashMap<>(); + tracingEnabled = Optional.ofNullable(System.getProperty("cats.effect.tracing.enabled")) + .filter(x -> !x.isEmpty()) + .map(x -> Boolean.valueOf(x)) + .orElse(true); } } diff --git a/core/jvm/src/main/scala/cats/effect/internals/IOTimer.scala b/core/jvm/src/main/scala/cats/effect/internals/IOTimer.scala index cf5e897712..0049769200 100644 --- a/core/jvm/src/main/scala/cats/effect/internals/IOTimer.scala +++ b/core/jvm/src/main/scala/cats/effect/internals/IOTimer.scala @@ -38,7 +38,7 @@ final private[internals] class IOTimer private (ec: ExecutionContext, sc: Schedu override def sleep(timespan: FiniteDuration): IO[Unit] = IO.Async(new IOForkedStart[Unit] { - def apply(conn: IOConnection, cb: T[Unit]): Unit = { + def apply(conn: IOConnection, ctx: IOContext, cb: T[Unit]): Unit = { // Doing what IO.cancelable does val ref = ForwardCancelable() conn.push(ref.cancel) diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index 797a6dc456..3aaed6f13b 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -102,8 +102,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * never terminate on evaluation. */ final def map[B](f: A => B): IO[B] = { -// val source = - this match { + val source = this match { case Map(source, g, index) => // Allowed to do fixed number of map operations fused before // resetting the counter in order to avoid stack overflows; @@ -114,7 +113,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { Map(this, f, 0) } -// IOTracing.check(source, f.asInstanceOf[AnyRef]) + IOTracing.apply(source, f.asInstanceOf[AnyRef]) } /** @@ -1208,7 +1207,7 @@ object IO extends IOInstances { * @see [[asyncF]] and [[cancelable]] */ def async[A](k: (Either[Throwable, A] => Unit) => Unit): IO[A] = { - val source = Async[A] { (_, cb) => + val source = Async[A] { (_, _, cb) => val cb2 = Callback.asyncIdempotent(null, cb) try k(cb2) catch { case NonFatal(t) => cb2(Left(t)) } @@ -1242,7 +1241,7 @@ object IO extends IOInstances { * @see [[async]] and [[cancelable]] */ def asyncF[A](k: (Either[Throwable, A] => Unit) => IO[Unit]): IO[A] = { - val source = Async[A] { (conn, cb) => + val source = Async[A] { (conn, _, cb) => // Must create new connection, otherwise we can have a race // condition b/t the bind continuation and `startCancelable` below val conn2 = IOConnection() @@ -1298,7 +1297,7 @@ object IO extends IOInstances { * the underlying cancelation model */ def cancelable[A](k: (Either[Throwable, A] => Unit) => CancelToken[IO]): IO[A] = { - val source = Async[A] { (conn, cb) => + val source = Async[A] { (conn, _, cb) => val cb2 = Callback.asyncIdempotent(conn, cb) val ref = ForwardCancelable() conn.push(ref.cancel) @@ -1488,7 +1487,7 @@ object IO extends IOInstances { * }}} */ val cancelBoundary: IO[Unit] = { - val start: Start[Unit] = (_, cb) => cb(Callback.rightUnit) + val start: Start[Unit] = (_, _, cb) => cb(Callback.rightUnit) Async(start, trampolineAfter = true) } @@ -1612,7 +1611,7 @@ object IO extends IOInstances { * signal downstream */ final private[effect] case class Async[+A]( - k: (IOConnection, Either[Throwable, A] => Unit) => Unit, + k: (IOConnection, IOContext, Either[Throwable, A] => Unit) => Unit, trampolineAfter: Boolean = false ) extends IO[A] diff --git a/core/shared/src/main/scala/cats/effect/internals/ForwardCancelable.scala b/core/shared/src/main/scala/cats/effect/internals/ForwardCancelable.scala index f1ac626cfa..ea30c9eb2b 100644 --- a/core/shared/src/main/scala/cats/effect/internals/ForwardCancelable.scala +++ b/core/shared/src/main/scala/cats/effect/internals/ForwardCancelable.scala @@ -35,11 +35,11 @@ final private[effect] class ForwardCancelable private () { private[this] val state = new AtomicReference[State](init) val cancel: CancelToken[IO] = { - @tailrec def loop(conn: IOConnection, cb: Callback.T[Unit]): Unit = + @tailrec def loop(conn: IOConnection, ctx: IOContext, cb: Callback.T[Unit]): Unit = state.get() match { case current @ Empty(list) => if (!state.compareAndSet(current, Empty(cb :: list))) - loop(conn, cb) + loop(conn, ctx, cb) case Active(token) => state.lazySet(finished) // GC purposes diff --git a/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala b/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala index 309413dac4..2e002597a3 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala @@ -30,7 +30,7 @@ private[effect] object IOBracket { * Implementation for `IO.bracketCase`. */ def apply[A, B](acquire: IO[A])(use: A => IO[B])(release: (A, ExitCase[Throwable]) => IO[Unit]): IO[B] = - IO.Async { (conn, cb) => + IO.Async { (conn, ctx, cb) => // Placeholder for the future finalizer val deferredRelease = ForwardCancelable() conn.push(deferredRelease.cancel) @@ -40,7 +40,7 @@ private[effect] object IOBracket { if (!conn.isCanceled) { // Note `acquire` is uncancelable due to usage of `IORunLoop.start` // (in other words it is disconnected from our IOConnection) - IORunLoop.start[A](acquire, new BracketStart(use, release, conn, deferredRelease, cb)) + IORunLoop.restart[A](acquire, ctx, new BracketStart(use, release, conn, ctx, deferredRelease, cb)) } else { deferredRelease.complete(IO.unit) } @@ -51,6 +51,7 @@ private[effect] object IOBracket { use: A => IO[B], release: (A, ExitCase[Throwable]) => IO[Unit], conn: IOConnection, + ctx: IOContext, deferredRelease: ForwardCancelable, cb: Callback.T[B] ) extends (Either[Throwable, A] => Unit) @@ -88,7 +89,7 @@ private[effect] object IOBracket { fb.flatMap(frame) } // Actual execution - IORunLoop.startCancelable(onNext, conn, cb) + IORunLoop.restartCancelable(onNext, conn, ctx, cb) } case error @ Left(_) => @@ -101,7 +102,7 @@ private[effect] object IOBracket { * Implementation for `IO.guaranteeCase`. */ def guaranteeCase[A](source: IO[A], release: ExitCase[Throwable] => IO[Unit]): IO[A] = - IO.Async { (conn, cb) => + IO.Async { (conn, _, cb) => // Light async boundary, otherwise this will trigger a StackOverflowException ec.execute(new Runnable { def run(): Unit = { diff --git a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala new file mode 100644 index 0000000000..b9b60c6d49 --- /dev/null +++ b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala @@ -0,0 +1,36 @@ +/* + * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers + * + * Licensed 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 cats.effect.internals + +import cats.effect.tracing.IOTrace + +final private[effect] class IOContext private () { + + // This is declared volatile but it is accessed + // from at most one thread at a time. + @volatile var trace: IOTrace = IOTrace.Empty + + def pushTrace(that: IOTrace): Unit = { + trace = trace.push(that) + } + +} + +object IOContext { + def newContext: IOContext = + new IOContext +} diff --git a/core/shared/src/main/scala/cats/effect/internals/IOParMap.scala b/core/shared/src/main/scala/cats/effect/internals/IOParMap.scala index 9a84a81bda..d867777903 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOParMap.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOParMap.scala @@ -29,7 +29,7 @@ private[effect] object IOParMap { def apply[A, B, C](cs: ContextShift[IO], fa: IO[A], fb: IO[B])(f: (A, B) => C): IO[C] = IO.Async( new IOForkedStart[C] { - def apply(conn: IOConnection, cb: Callback.T[C]) = + def apply(conn: IOConnection, ctx: IOContext, cb: Callback.T[C]) = // For preventing stack-overflow errors; using a // trampolined execution context, so no thread forks TrampolineEC.immediate.execute(new ParMapRunnable(cs, fa, fb, f, conn, cb)) diff --git a/core/shared/src/main/scala/cats/effect/internals/IORace.scala b/core/shared/src/main/scala/cats/effect/internals/IORace.scala index ad61b93571..a2aa57d4b4 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORace.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORace.scala @@ -58,7 +58,7 @@ private[effect] object IORace { Logger.reportFailure(err) } - val start: Start[Either[A, B]] = (conn, cb) => { + val start: Start[Either[A, B]] = (conn, _, cb) => { val active = new AtomicBoolean(true) // Cancelable connection for the left value val connL = IOConnection() @@ -94,7 +94,7 @@ private[effect] object IORace { * Implementation for `IO.racePair` */ def pair[A, B](cs: ContextShift[IO], lh: IO[A], rh: IO[B]): IO[Pair[A, B]] = { - val start: Start[Pair[A, B]] = (conn, cb) => { + val start: Start[Pair[A, B]] = (conn, _, cb) => { val active = new AtomicBoolean(true) // Cancelable connection for the left value val connL = IOConnection() diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index 028988e606..6c26b7e24a 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -18,7 +18,6 @@ package cats.effect.internals import cats.effect.IO import cats.effect.IO.{Async, Bind, ContextSwitch, Delay, Introspect, Map, Pure, RaiseError, Suspend, Trace} -import cats.effect.tracing.IOTrace import scala.util.control.NonFatal @@ -36,6 +35,9 @@ private[effect] object IORunLoop { def start[A](source: IO[A], cb: Either[Throwable, A] => Unit): Unit = loop(source, IOConnection.uncancelable, cb.asInstanceOf[Callback], null, null, null, null) + def restart[A](source: IO[A], ctx: IOContext, cb: Either[Throwable, A] => Unit): Unit = + loop(source, IOConnection.uncancelable, cb.asInstanceOf[Callback], ctx, null, null, null) + /** * Evaluates the given `IO` reference, calling the given callback * with the result when completed. @@ -43,6 +45,9 @@ private[effect] object IORunLoop { def startCancelable[A](source: IO[A], conn: IOConnection, cb: Either[Throwable, A] => Unit): Unit = loop(source, conn, cb.asInstanceOf[Callback], null, null, null, null) + def restartCancelable[A](source: IO[A], conn: IOConnection, ctx: IOContext, cb: Either[Throwable, A] => Unit): Unit = + loop(source, conn, cb.asInstanceOf[Callback], ctx, null, null, null) + /** * Loop for evaluating an `IO` value. * @@ -54,18 +59,18 @@ private[effect] object IORunLoop { source: Current, cancelable: IOConnection, cb: Either[Throwable, Any] => Unit, + ctxRef: IOContext, rcbRef: RestartCallback, bFirstRef: Bind, - bRestRef: CallStack, - traceRef: IOTrace + bRestRef: CallStack ): Unit = { var currentIO: Current = source // Can change on a context switch var conn: IOConnection = cancelable + var ctx: IOContext = ctxRef var bFirst: Bind = bFirstRef var bRest: CallStack = bRestRef var rcb: RestartCallback = rcbRef - var trace: IOTrace = traceRef // Values from Pure and Delay are unboxed in this var, // for code reuse between Pure and Delay var hasUnboxed: Boolean = false @@ -125,8 +130,8 @@ private[effect] object IORunLoop { case async @ Async(_, _) => if (conn eq null) conn = IOConnection() - if (rcb eq null) rcb = new RestartCallback(conn, cb.asInstanceOf[Callback]) - rcb.start(async, bFirst, bRest, trace) + if (rcb eq null) rcb = new RestartCallback(conn, ctx, cb.asInstanceOf[Callback]) + rcb.start(async, bFirst, bRest) return case ContextSwitch(next, modify, restore) => @@ -141,14 +146,14 @@ private[effect] object IORunLoop { } case Trace(source, currTrace) => - if (trace eq null) trace = IOTrace.Empty - trace = trace.and(currTrace) + if (ctx eq null) ctx = IOContext.newContext + ctx.pushTrace(currTrace) currentIO = source case Introspect => - val retTrace = if (trace eq null) IOTrace.Empty else trace + if (ctx eq null) ctx = IOContext.newContext hasUnboxed = true - unboxed = retTrace + unboxed = ctx.trace } if (hasUnboxed) { @@ -186,7 +191,7 @@ private[effect] object IORunLoop { var currentIO: Current = source var bFirst: Bind = null var bRest: CallStack = null - var trace: IOTrace = null + var ctx: IOContext = null // Values from Pure and Delay are unboxed in this var, // for code reuse between Pure and Delay var hasUnboxed: Boolean = false @@ -247,21 +252,21 @@ private[effect] object IORunLoop { // box those vars in scala.runtime.ObjectRef! // TODO: Since IO.traced is implemented in terms of IOBracket // we may not need to concern ourselves with tracing status here? - return suspendAsync(currentIO.asInstanceOf[IO.Async[A]], bFirst, bRest, trace) + return suspendAsync(currentIO.asInstanceOf[IO.Async[A]], bFirst, bRest) case Trace(source, currTrace) => - if (trace eq null) trace = IOTrace.Empty - trace = trace.and(currTrace) + if (ctx eq null) ctx = IOContext.newContext + ctx.pushTrace(currTrace) currentIO = source case Introspect => - val retTrace = if (trace eq null) IOTrace.Empty else trace + if (ctx eq null) ctx = IOContext.newContext hasUnboxed = true - unboxed = retTrace + unboxed = ctx.trace case _ => - return Async { (conn, cb) => - loop(currentIO, conn, cb.asInstanceOf[Callback], null, bFirst, bRest, trace) + return Async { (conn, ctx, cb) => + loop(currentIO, conn, cb.asInstanceOf[Callback], ctx, null, bFirst, bRest) } } @@ -286,13 +291,13 @@ private[effect] object IORunLoop { // $COVERAGE-ON$ } - private def suspendAsync[A](currentIO: IO.Async[A], bFirst: Bind, bRest: CallStack, trace: IOTrace): IO[A] = + private def suspendAsync[A](currentIO: IO.Async[A], bFirst: Bind, bRest: CallStack): IO[A] = // Hitting an async boundary means we have to stop, however // if we had previous `flatMap` operations then we need to resume // the loop with the collected stack if (bFirst != null || (bRest != null && !bRest.isEmpty)) - Async { (conn, cb) => - loop(currentIO, conn, cb.asInstanceOf[Callback], null, bFirst, bRest, trace) + Async { (conn, ctx, cb) => + loop(currentIO, conn, cb.asInstanceOf[Callback], ctx, null, bFirst, bRest) } else currentIO @@ -358,7 +363,7 @@ private[effect] object IORunLoop { * It's an ugly, mutable implementation. * For internal use only, here be dragons! */ - final private class RestartCallback(connInit: IOConnection, cb: Callback) extends Callback with Runnable { + final private class RestartCallback(connInit: IOConnection, ctx: IOContext, cb: Callback) extends Callback with Runnable { import TrampolineEC.{immediate => ec} // can change on a ContextSwitch @@ -367,7 +372,6 @@ private[effect] object IORunLoop { private[this] var trampolineAfter = false private[this] var bFirst: Bind = _ private[this] var bRest: CallStack = _ - private[this] var trace: IOTrace = _ // Used in combination with trampolineAfter = true private[this] var value: Either[Throwable, Any] = _ @@ -375,33 +379,30 @@ private[effect] object IORunLoop { def contextSwitch(conn: IOConnection): Unit = this.conn = conn - def start(task: IO.Async[Any], bFirst: Bind, bRest: CallStack, trace: IOTrace): Unit = { + def start(task: IO.Async[Any], bFirst: Bind, bRest: CallStack): Unit = { canCall = true this.bFirst = bFirst this.bRest = bRest - this.trace = trace this.trampolineAfter = task.trampolineAfter // Go, go, go - task.k(conn, this) + task.k(conn, ctx, this) } private[this] def signal(either: Either[Throwable, Any]): Unit = { // Allow GC to collect val bFirst = this.bFirst val bRest = this.bRest - val trace = this.trace this.bFirst = null this.bRest = null - this.trace = null // Auto-cancelable logic: in case the connection was cancelled, // we interrupt the bind continuation if (!conn.isCanceled) either match { case Right(success) => - loop(Pure(success), conn, cb, this, bFirst, bRest, trace) + loop(Pure(success), conn, cb, ctx, this, bFirst, bRest) case Left(e) => - loop(RaiseError(e), conn, cb, this, bFirst, bRest, trace) + loop(RaiseError(e), conn, cb, ctx, this, bFirst, bRest) } } diff --git a/core/shared/src/main/scala/cats/effect/internals/IOShift.scala b/core/shared/src/main/scala/cats/effect/internals/IOShift.scala index 3eccb05286..fa01e57a8b 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOShift.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOShift.scala @@ -25,7 +25,7 @@ private[effect] object IOShift { /** Implementation for `IO.shift`. */ def apply(ec: ExecutionContext): IO[Unit] = IO.Async(new IOForkedStart[Unit] { - def apply(conn: IOConnection, cb: Callback.T[Unit]): Unit = + def apply(conn: IOConnection, ctx: IOContext, cb: Callback.T[Unit]): Unit = ec.execute(new Tick(cb)) }) diff --git a/core/shared/src/main/scala/cats/effect/internals/IOStart.scala b/core/shared/src/main/scala/cats/effect/internals/IOStart.scala index 2cd968e29c..8474e209c5 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOStart.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOStart.scala @@ -26,7 +26,7 @@ private[effect] object IOStart { * Implementation for `IO.start`. */ def apply[A](cs: ContextShift[IO], fa: IO[A]): IO[Fiber[IO, A]] = { - val start: Start[Fiber[IO, A]] = (_, cb) => { + val start: Start[Fiber[IO, A]] = (_, _, cb) => { // Memoization val p = Promise[Either[Throwable, A]]() diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index 6e3942fc9d..ee08e45e3d 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -17,8 +17,10 @@ package cats.effect.internals import cats.effect.IO -import cats.effect.tracing.{IOTrace, TraceElement} -import TracingPlatform.{tracingEnabled, traceCache} +import cats.effect.tracing.{IOTrace, TraceLine} +import cats.effect.internals.TracingPlatform.tracingEnabled + +import scala.collection.mutable private[effect] object IOTracing { @@ -26,23 +28,42 @@ private[effect] object IOTracing { // calculating this key has a cost. inline the checks def apply[A](source: IO[A], lambda: AnyRef): IO[A] = { if (tracingEnabled) { - val traceRef = traceCache.get(lambda) - if (traceRef eq null) { - val fiberTrace = createTrace() - traceCache.put(lambda, fiberTrace) - IO.Trace(source, fiberTrace) - } else { - IO.Trace(source, traceRef.asInstanceOf[IOTrace]) + val cachedTrace = traceCache.get(lambda) + cachedTrace match { + case Some(trace) => + IO.Trace(source, trace) + case None => { + val fiberTrace = buildTrace() + traceCache.put(lambda, fiberTrace) + IO.Trace(source, fiberTrace) + } } } else { source } } - def createTrace(): IOTrace = { - // TODO: calculate trace here - val lines = new Throwable().getStackTrace.toList.map(TraceElement.fromStackTraceElement) + private def buildTrace(): IOTrace = { + // TODO: proper trace calculation + val lines = new Throwable() + .getStackTrace + .toList + .map(TraceLine.fromStackTraceElement) + .filter(_.className.startsWith("cats.effect.internals.Main")) + IOTrace(lines) } + /** + * Global, thread-safe cache for traces. Keys are generally + * lambda references. + * + * TODO: Could this be a thread-local? + * If every thread eventually calculates its own set, + * there should be no issue? + * + * TODO: Bound the cache. + */ + private val traceCache: mutable.Map[AnyRef, IOTrace] = new mutable.HashMap() + } diff --git a/core/shared/src/main/scala/cats/effect/internals/package.scala b/core/shared/src/main/scala/cats/effect/internals/package.scala index 5590d0a4c3..a24e14033c 100644 --- a/core/shared/src/main/scala/cats/effect/internals/package.scala +++ b/core/shared/src/main/scala/cats/effect/internals/package.scala @@ -22,5 +22,5 @@ package object internals { * Handy alias for the registration functions of [[IO.Async]]. */ private[effect] type Start[+A] = - (IOConnection, Callback.T[A]) => Unit + (IOConnection, IOContext, Callback.T[A]) => Unit } diff --git a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala index c6b403886c..2313f2379d 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala @@ -16,9 +16,15 @@ package cats.effect.tracing -final case class IOTrace(lines: List[TraceElement]) { - def and(that: IOTrace): IOTrace = +final case class IOTrace(lines: List[TraceLine]) { + def push(that: IOTrace): IOTrace = IOTrace(that.lines ++ lines) + + def printTrace(): Unit = { + lines.foreach { line => + println(s"\t${line.className}.${line.methodName} (${line.fileName}:${line.lineNumber})") + } + } } object IOTrace { diff --git a/core/shared/src/main/scala/cats/effect/tracing/TraceElement.scala b/core/shared/src/main/scala/cats/effect/tracing/TraceLine.scala similarity index 66% rename from core/shared/src/main/scala/cats/effect/tracing/TraceElement.scala rename to core/shared/src/main/scala/cats/effect/tracing/TraceLine.scala index 138853d3b5..fc10ea631d 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/TraceElement.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/TraceLine.scala @@ -16,9 +16,10 @@ package cats.effect.tracing -final case class TraceElement(className: String, methodName: String, fileName: String, lineNumber: Int) +// TODO: Track information about what combinator was used etc. +final case class TraceLine(className: String, methodName: String, fileName: String, lineNumber: Int) -object TraceElement { - def fromStackTraceElement(ste: StackTraceElement): TraceElement = - TraceElement(ste.getClassName, ste.getMethodName, ste.getFileName, ste.getLineNumber) +object TraceLine { + def fromStackTraceElement(ste: StackTraceElement): TraceLine = + TraceLine(ste.getClassName, ste.getMethodName, ste.getFileName, ste.getLineNumber) } From 3159e9e3d3702e098ea30cc60741bb043f9fe00e Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Mon, 27 Apr 2020 00:58:34 -0500 Subject: [PATCH 05/78] Inspect tracing mode --- .../internals/TracingFlagsPlatform.scala | 29 +++++++++++++ .../src/main/scala/cats/effect/IO.scala | 4 +- .../cats/effect/internals/IOContext.scala | 11 +++-- .../cats/effect/internals/IORunLoop.scala | 4 +- .../cats/effect/internals/IOTracing.scala | 41 ++++++++++--------- .../scala/cats/effect/tracing/IOTrace.scala | 13 +++--- .../cats/effect/tracing/TraceFrame.scala | 19 +++++++++ ...{TracingStatus.scala => TracingMode.scala} | 16 ++++---- 8 files changed, 97 insertions(+), 40 deletions(-) create mode 100644 core/jvm/src/main/scala/cats/effect/internals/TracingFlagsPlatform.scala create mode 100644 core/shared/src/main/scala/cats/effect/tracing/TraceFrame.scala rename core/shared/src/main/scala/cats/effect/tracing/{TracingStatus.scala => TracingMode.scala} (69%) diff --git a/core/jvm/src/main/scala/cats/effect/internals/TracingFlagsPlatform.scala b/core/jvm/src/main/scala/cats/effect/internals/TracingFlagsPlatform.scala new file mode 100644 index 0000000000..19d3741cc8 --- /dev/null +++ b/core/jvm/src/main/scala/cats/effect/internals/TracingFlagsPlatform.scala @@ -0,0 +1,29 @@ +/* + * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers + * + * Licensed 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 cats.effect.internals + +import cats.effect.tracing.TracingMode + +// TODO: Extend `TracingPlatform` and inspect bytecode for static final field access +private[effect] object TracingFlagsPlatform { + + val tracingMode: TracingMode = + Option(System.getProperty("cats.effect.tracing.mode")) + .flatMap(TracingMode.fromString) + .getOrElse(TracingMode.Rabbit) + +} diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index 3aaed6f13b..a2a1321769 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -26,7 +26,7 @@ import scala.concurrent.{ExecutionContext, Future, Promise, TimeoutException} import scala.util.control.NonFatal import scala.util.{Failure, Left, Right, Success, Try} import cats.data.Ior -import cats.effect.tracing.IOTrace +import cats.effect.tracing.{IOTrace, TraceFrame} /** * A pure abstraction representing the intention to perform a @@ -1628,7 +1628,7 @@ object IO extends IOInstances { restore: (A, Throwable, IOConnection, IOConnection) => IOConnection ) extends IO[A] - final private[effect] case class Trace[+A](source: IO[A], stackTrace: IOTrace) extends IO[A] + final private[effect] case class Trace[+A](source: IO[A], frame: TraceFrame) extends IO[A] final private[effect] case object Introspect extends IO[IOTrace] diff --git a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala index b9b60c6d49..f818c8a570 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala @@ -16,16 +16,21 @@ package cats.effect.internals -import cats.effect.tracing.IOTrace +import cats.effect.tracing.{IOTrace, TraceFrame} +/** + * IOContext holds state related to the execution of an IO and + * should be threaded across multiple invocations of the run-loop + * for the same fiber. + */ final private[effect] class IOContext private () { // This is declared volatile but it is accessed // from at most one thread at a time. @volatile var trace: IOTrace = IOTrace.Empty - def pushTrace(that: IOTrace): Unit = { - trace = trace.push(that) + def pushFrame(that: TraceFrame): Unit = { + trace = trace.pushFrame(that) } } diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index 6c26b7e24a..fda21a7d13 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -147,7 +147,7 @@ private[effect] object IORunLoop { case Trace(source, currTrace) => if (ctx eq null) ctx = IOContext.newContext - ctx.pushTrace(currTrace) + ctx.pushFrame(currTrace) currentIO = source case Introspect => @@ -256,7 +256,7 @@ private[effect] object IORunLoop { case Trace(source, currTrace) => if (ctx eq null) ctx = IOContext.newContext - ctx.pushTrace(currTrace) + ctx.pushFrame(currTrace) currentIO = source case Introspect => diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index ee08e45e3d..351c486407 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -17,8 +17,9 @@ package cats.effect.internals import cats.effect.IO -import cats.effect.tracing.{IOTrace, TraceLine} +import cats.effect.tracing.{TraceFrame, TraceLine, TracingMode} import cats.effect.internals.TracingPlatform.tracingEnabled +import cats.effect.internals.TracingFlagsPlatform.tracingMode import scala.collection.mutable @@ -28,42 +29,42 @@ private[effect] object IOTracing { // calculating this key has a cost. inline the checks def apply[A](source: IO[A], lambda: AnyRef): IO[A] = { if (tracingEnabled) { - val cachedTrace = traceCache.get(lambda) - cachedTrace match { - case Some(trace) => - IO.Trace(source, trace) - case None => { - val fiberTrace = buildTrace() - traceCache.put(lambda, fiberTrace) - IO.Trace(source, fiberTrace) - } + val frame = tracingMode match { + case TracingMode.Rabbit => + frameCache.get(lambda) match { + case Some(fr) => fr + case None => { + val fr = buildFrame() + frameCache.put(lambda, fr) + fr + } + } + case TracingMode.Slug => buildFrame() } + + IO.Trace(source, frame) } else { source } } - private def buildTrace(): IOTrace = { + private def buildFrame(): TraceFrame = { // TODO: proper trace calculation val lines = new Throwable() .getStackTrace .toList .map(TraceLine.fromStackTraceElement) - .filter(_.className.startsWith("cats.effect.internals.Main")) +// .filter(_.className.startsWith("cats.effect.internals.Main")) - IOTrace(lines) + TraceFrame(lines) } /** - * Global, thread-safe cache for traces. Keys are generally - * lambda references. - * - * TODO: Could this be a thread-local? - * If every thread eventually calculates its own set, - * there should be no issue? + * Cache for trace frames. Keys are object references for lambdas. * + * TODO: Switch to thread-local or j.u.chm? * TODO: Bound the cache. */ - private val traceCache: mutable.Map[AnyRef, IOTrace] = new mutable.HashMap() + private val frameCache: mutable.Map[AnyRef, TraceFrame] = new mutable.HashMap() } diff --git a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala index 2313f2379d..e475a94961 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala @@ -16,13 +16,16 @@ package cats.effect.tracing -final case class IOTrace(lines: List[TraceLine]) { - def push(that: IOTrace): IOTrace = - IOTrace(that.lines ++ lines) +final case class IOTrace(frames: List[TraceFrame]) { + def pushFrame(frame: TraceFrame): IOTrace = + IOTrace(frame :: frames) def printTrace(): Unit = { - lines.foreach { line => - println(s"\t${line.className}.${line.methodName} (${line.fileName}:${line.lineNumber})") + frames.foreach { f => + println("New frame") + f.lines.foreach { l => + println(s"\t${l.className}.${l.methodName} (${l.fileName}:${l.lineNumber})") + } } } } diff --git a/core/shared/src/main/scala/cats/effect/tracing/TraceFrame.scala b/core/shared/src/main/scala/cats/effect/tracing/TraceFrame.scala new file mode 100644 index 0000000000..78c94b4070 --- /dev/null +++ b/core/shared/src/main/scala/cats/effect/tracing/TraceFrame.scala @@ -0,0 +1,19 @@ +/* + * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers + * + * Licensed 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 cats.effect.tracing + +final case class TraceFrame(lines: List[TraceLine]) diff --git a/core/shared/src/main/scala/cats/effect/tracing/TracingStatus.scala b/core/shared/src/main/scala/cats/effect/tracing/TracingMode.scala similarity index 69% rename from core/shared/src/main/scala/cats/effect/tracing/TracingStatus.scala rename to core/shared/src/main/scala/cats/effect/tracing/TracingMode.scala index 4fb2789c44..710cc4f1bf 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/TracingStatus.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/TracingMode.scala @@ -16,19 +16,19 @@ package cats.effect.tracing -private[effect] sealed abstract class TracingStatus +private[effect] sealed abstract class TracingMode -private[effect] object TracingStatus { +private[effect] object TracingMode { - case object Rabbit extends TracingStatus + case object Rabbit extends TracingMode - case object Slug extends TracingStatus + case object Slug extends TracingMode - def fromString(value: String): TracingStatus = + def fromString(value: String): Option[TracingMode] = value.toLowerCase() match { - case "rabbit" => Rabbit - case "slug" => Slug - case _ => Rabbit + case "rabbit" => Some(Rabbit) + case "slug" => Some(Slug) + case _ => None } } From 2b5f569e9cc948488ce45f7157ee3dd785944e27 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Mon, 27 Apr 2020 01:06:25 -0500 Subject: [PATCH 06/78] wip --- .../cats/effect/internals/IOTracing.scala | 25 +++++++++---------- .../scala/cats/effect/tracing/IOTrace.scala | 2 ++ 2 files changed, 14 insertions(+), 13 deletions(-) diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index 351c486407..c7ce77d75e 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -25,29 +25,28 @@ import scala.collection.mutable private[effect] object IOTracing { - // TODO: Lazily evaluate key? - // calculating this key has a cost. inline the checks - def apply[A](source: IO[A], lambda: AnyRef): IO[A] = { + def apply[A](source: IO[A], lambdaRef: AnyRef): IO[A] = { + // TODO: consider inlining this conditional at call-sites if (tracingEnabled) { val frame = tracingMode match { - case TracingMode.Rabbit => - frameCache.get(lambda) match { - case Some(fr) => fr - case None => { - val fr = buildFrame() - frameCache.put(lambda, fr) - fr - } - } + case TracingMode.Rabbit => buildCachedFrame(lambdaRef) case TracingMode.Slug => buildFrame() } - IO.Trace(source, frame) } else { source } } + private def buildCachedFrame(lambdaRef: AnyRef): TraceFrame = + frameCache.get(lambdaRef) match { + case Some(fr) => fr + case None => + val fr = buildFrame() + frameCache.put(lambdaRef, fr) + fr + } + private def buildFrame(): TraceFrame = { // TODO: proper trace calculation val lines = new Throwable() diff --git a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala index e475a94961..fa77452477 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala @@ -17,6 +17,8 @@ package cats.effect.tracing final case class IOTrace(frames: List[TraceFrame]) { + // TODO: for infinite loops, frames represents an unbounded memory leak + // we should implement a ring buffer with a configurable frame buffer size def pushFrame(frame: TraceFrame): IOTrace = IOTrace(frame :: frames) From 41f06b30884bbce851dec5ec3288b0c563a73824 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Mon, 27 Apr 2020 01:08:00 -0500 Subject: [PATCH 07/78] Add Example --- .../scala/cats/effect/internals/Example.scala | 40 +++++++++++++++++++ .../internals/TracingFlagsPlatform.scala | 1 + 2 files changed, 41 insertions(+) create mode 100644 core/jvm/src/main/scala/cats/effect/internals/Example.scala diff --git a/core/jvm/src/main/scala/cats/effect/internals/Example.scala b/core/jvm/src/main/scala/cats/effect/internals/Example.scala new file mode 100644 index 0000000000..630e97814d --- /dev/null +++ b/core/jvm/src/main/scala/cats/effect/internals/Example.scala @@ -0,0 +1,40 @@ +/* + * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers + * + * Licensed 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 cats.effect.internals + +import cats.effect.{ExitCode, IO, IOApp} + +object Example extends IOApp { + + def program: IO[Unit] = for { + _ <- IO.delay(println("1")) + _ <- IO.delay(println("2")) + _ <- IO.shift + _ <- IO.unit.bracket(_ => IO.delay(println("3")) + .flatMap(_ => IO.unit))(_ => IO.unit) + _ <- IO.delay(println("4")) + _ <- IO.delay(println("5")) + } yield () + + override def run(args: List[String]): IO[ExitCode] = + for { + _ <- IO.suspend(program) + trace <- IO.introspect + _ <- IO.delay(trace.printTrace()) + } yield ExitCode.Success + +} diff --git a/core/jvm/src/main/scala/cats/effect/internals/TracingFlagsPlatform.scala b/core/jvm/src/main/scala/cats/effect/internals/TracingFlagsPlatform.scala index 19d3741cc8..e905517aab 100644 --- a/core/jvm/src/main/scala/cats/effect/internals/TracingFlagsPlatform.scala +++ b/core/jvm/src/main/scala/cats/effect/internals/TracingFlagsPlatform.scala @@ -21,6 +21,7 @@ import cats.effect.tracing.TracingMode // TODO: Extend `TracingPlatform` and inspect bytecode for static final field access private[effect] object TracingFlagsPlatform { + // TODO: configure this lexically somehow val tracingMode: TracingMode = Option(System.getProperty("cats.effect.tracing.mode")) .flatMap(TracingMode.fromString) From f0a715f0aafc6034e5254a7d877862f35be83044 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Mon, 27 Apr 2020 01:17:34 -0500 Subject: [PATCH 08/78] Hold frames in IOContext --- .../scala/cats/effect/internals/IOContext.scala | 14 ++++++++++---- .../scala/cats/effect/internals/IORunLoop.scala | 5 +++-- .../main/scala/cats/effect/tracing/IOTrace.scala | 10 +--------- 3 files changed, 14 insertions(+), 15 deletions(-) diff --git a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala index f818c8a570..a3b731c10f 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala @@ -25,14 +25,20 @@ import cats.effect.tracing.{IOTrace, TraceFrame} */ final private[effect] class IOContext private () { - // This is declared volatile but it is accessed - // from at most one thread at a time. - @volatile var trace: IOTrace = IOTrace.Empty + // We had to do this because of IOBracket implementation + // and how it invokes a new run-loop. + // TODO: for infinite loops, `frames` represents an unbounded memory leak + // we should implement a ring buffer with a configurable frame buffer size + @volatile var frames: List[TraceFrame] = Nil def pushFrame(that: TraceFrame): Unit = { - trace = trace.pushFrame(that) + // Accessed from at most one thread at a time + frames = that :: frames } + def getTrace: IOTrace = + IOTrace(frames) + } object IOContext { diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index fda21a7d13..f417d74741 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -153,7 +153,7 @@ private[effect] object IORunLoop { case Introspect => if (ctx eq null) ctx = IOContext.newContext hasUnboxed = true - unboxed = ctx.trace + unboxed = ctx.getTrace } if (hasUnboxed) { @@ -260,9 +260,10 @@ private[effect] object IORunLoop { currentIO = source case Introspect => + // This can be implemented in terms of Async now if (ctx eq null) ctx = IOContext.newContext hasUnboxed = true - unboxed = ctx.trace + unboxed = ctx.getTrace case _ => return Async { (conn, ctx, cb) => diff --git a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala index fa77452477..b2fa37d681 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala @@ -17,21 +17,13 @@ package cats.effect.tracing final case class IOTrace(frames: List[TraceFrame]) { - // TODO: for infinite loops, frames represents an unbounded memory leak - // we should implement a ring buffer with a configurable frame buffer size - def pushFrame(frame: TraceFrame): IOTrace = - IOTrace(frame :: frames) - def printTrace(): Unit = { + def printTrace(): Unit = frames.foreach { f => println("New frame") f.lines.foreach { l => println(s"\t${l.className}.${l.methodName} (${l.fileName}:${l.lineNumber})") } } - } -} -object IOTrace { - val Empty = IOTrace(List()) } From 995ac9fceba0c64177de2989e9c9f66a45cd4ec6 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Mon, 27 Apr 2020 01:20:10 -0500 Subject: [PATCH 09/78] scalafmt --- .../scala/cats/effect/internals/Example.scala | 25 +++++++++++-------- .../src/main/scala/cats/effect/IO.scala | 1 - .../cats/effect/internals/IOContext.scala | 3 +-- .../cats/effect/internals/IORunLoop.scala | 4 ++- .../cats/effect/internals/IOTracing.scala | 9 +++---- .../cats/effect/tracing/TracingMode.scala | 6 ++--- 6 files changed, 24 insertions(+), 24 deletions(-) diff --git a/core/jvm/src/main/scala/cats/effect/internals/Example.scala b/core/jvm/src/main/scala/cats/effect/internals/Example.scala index 630e97814d..21721c8d63 100644 --- a/core/jvm/src/main/scala/cats/effect/internals/Example.scala +++ b/core/jvm/src/main/scala/cats/effect/internals/Example.scala @@ -20,21 +20,24 @@ import cats.effect.{ExitCode, IO, IOApp} object Example extends IOApp { - def program: IO[Unit] = for { - _ <- IO.delay(println("1")) - _ <- IO.delay(println("2")) - _ <- IO.shift - _ <- IO.unit.bracket(_ => IO.delay(println("3")) - .flatMap(_ => IO.unit))(_ => IO.unit) - _ <- IO.delay(println("4")) - _ <- IO.delay(println("5")) - } yield () + def program: IO[Unit] = + for { + _ <- IO.delay(println("1")) + _ <- IO.delay(println("2")) + _ <- IO.shift + _ <- IO.unit.bracket(_ => + IO.delay(println("3")) + .flatMap(_ => IO.unit) + )(_ => IO.unit) + _ <- IO.delay(println("4")) + _ <- IO.delay(println("5")) + } yield () override def run(args: List[String]): IO[ExitCode] = for { - _ <- IO.suspend(program) + _ <- IO.suspend(program) trace <- IO.introspect - _ <- IO.delay(trace.printTrace()) + _ <- IO.delay(trace.printTrace()) } yield ExitCode.Success } diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index a2a1321769..29831c1913 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -1571,7 +1571,6 @@ object IO extends IOInstances { def introspect: IO[IOTrace] = Introspect - /* -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-= */ /* IO's internal encoding: */ diff --git a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala index a3b731c10f..9ed90ceb48 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala @@ -31,10 +31,9 @@ final private[effect] class IOContext private () { // we should implement a ring buffer with a configurable frame buffer size @volatile var frames: List[TraceFrame] = Nil - def pushFrame(that: TraceFrame): Unit = { + def pushFrame(that: TraceFrame): Unit = // Accessed from at most one thread at a time frames = that :: frames - } def getTrace: IOTrace = IOTrace(frames) diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index f417d74741..1a528f18e7 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -364,7 +364,9 @@ private[effect] object IORunLoop { * It's an ugly, mutable implementation. * For internal use only, here be dragons! */ - final private class RestartCallback(connInit: IOConnection, ctx: IOContext, cb: Callback) extends Callback with Runnable { + final private class RestartCallback(connInit: IOConnection, ctx: IOContext, cb: Callback) + extends Callback + with Runnable { import TrampolineEC.{immediate => ec} // can change on a ContextSwitch diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index c7ce77d75e..04129f3b4a 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -25,18 +25,17 @@ import scala.collection.mutable private[effect] object IOTracing { - def apply[A](source: IO[A], lambdaRef: AnyRef): IO[A] = { + def apply[A](source: IO[A], lambdaRef: AnyRef): IO[A] = // TODO: consider inlining this conditional at call-sites if (tracingEnabled) { val frame = tracingMode match { case TracingMode.Rabbit => buildCachedFrame(lambdaRef) - case TracingMode.Slug => buildFrame() + case TracingMode.Slug => buildFrame() } IO.Trace(source, frame) } else { source } - } private def buildCachedFrame(lambdaRef: AnyRef): TraceFrame = frameCache.get(lambdaRef) match { @@ -49,9 +48,7 @@ private[effect] object IOTracing { private def buildFrame(): TraceFrame = { // TODO: proper trace calculation - val lines = new Throwable() - .getStackTrace - .toList + val lines = new Throwable().getStackTrace.toList .map(TraceLine.fromStackTraceElement) // .filter(_.className.startsWith("cats.effect.internals.Main")) diff --git a/core/shared/src/main/scala/cats/effect/tracing/TracingMode.scala b/core/shared/src/main/scala/cats/effect/tracing/TracingMode.scala index 710cc4f1bf..5b740c0f24 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/TracingMode.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/TracingMode.scala @@ -16,7 +16,7 @@ package cats.effect.tracing -private[effect] sealed abstract class TracingMode +sealed abstract private[effect] class TracingMode private[effect] object TracingMode { @@ -27,8 +27,8 @@ private[effect] object TracingMode { def fromString(value: String): Option[TracingMode] = value.toLowerCase() match { case "rabbit" => Some(Rabbit) - case "slug" => Some(Slug) - case _ => None + case "slug" => Some(Slug) + case _ => None } } From c8890f1083941916d728f5c628a06f265b2ce602 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Mon, 27 Apr 2020 01:26:35 -0500 Subject: [PATCH 10/78] Clean up --- .../effect/internals/TracingPlatform.java | 24 ++++++++----------- .../internals/TracingFlagsPlatform.scala | 5 ++-- 2 files changed, 13 insertions(+), 16 deletions(-) diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java index b80d0cc584..285f32a6db 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java @@ -19,24 +19,20 @@ import java.util.Optional; /** - * - * Motivation: - * In Scala, object-level variable reads cause a volatile read. - * Instead, + * Scala companion object field accesses cost a volatile read. + * Since this flag is read at the construction of IO nodes, + * we are opting to source this flag from a Java class to + * bypass the volatile read and squeeze out as much performance + * as possible. */ class TracingPlatform { /** - * A boolean variable that controls tracing globally. For tracing to - * take effect, this flag must be enabled. + * A boolean flag that controls tracing for a JVM process. */ - public static final boolean tracingEnabled; - - static { - tracingEnabled = Optional.ofNullable(System.getProperty("cats.effect.tracing.enabled")) - .filter(x -> !x.isEmpty()) - .map(x -> Boolean.valueOf(x)) - .orElse(true); - } + public static final boolean tracingEnabled = Optional.ofNullable(System.getProperty("cats.effect.tracing.enabled")) + .filter(x -> !x.isEmpty()) + .map(x -> Boolean.valueOf(x)) + .orElse(true); } diff --git a/core/jvm/src/main/scala/cats/effect/internals/TracingFlagsPlatform.scala b/core/jvm/src/main/scala/cats/effect/internals/TracingFlagsPlatform.scala index e905517aab..cfc6ae5ba9 100644 --- a/core/jvm/src/main/scala/cats/effect/internals/TracingFlagsPlatform.scala +++ b/core/jvm/src/main/scala/cats/effect/internals/TracingFlagsPlatform.scala @@ -18,10 +18,11 @@ package cats.effect.internals import cats.effect.tracing.TracingMode -// TODO: Extend `TracingPlatform` and inspect bytecode for static final field access private[effect] object TracingFlagsPlatform { - // TODO: configure this lexically somehow + // TODO: Extend `TracingPlatform` and inspect bytecode for static final field access + + // TODO: Configure this lexically and introduce a Disabled mode. val tracingMode: TracingMode = Option(System.getProperty("cats.effect.tracing.mode")) .flatMap(TracingMode.fromString) From b6bb1e5a9bdf5511e7a1e3831ac59c5e976eecea Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Mon, 27 Apr 2020 01:41:45 -0500 Subject: [PATCH 11/78] disabled mode --- .../src/main/scala/cats/effect/internals/IOTracing.scala | 8 ++++---- .../src/main/scala/cats/effect/tracing/TracingMode.scala | 9 ++++++--- 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index 04129f3b4a..84be97197b 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -28,11 +28,11 @@ private[effect] object IOTracing { def apply[A](source: IO[A], lambdaRef: AnyRef): IO[A] = // TODO: consider inlining this conditional at call-sites if (tracingEnabled) { - val frame = tracingMode match { - case TracingMode.Rabbit => buildCachedFrame(lambdaRef) - case TracingMode.Slug => buildFrame() + tracingMode match { + case TracingMode.Disabled => source + case TracingMode.Rabbit => IO.Trace(source, buildCachedFrame(lambdaRef)) + case TracingMode.Slug => IO.Trace(source, buildFrame()) } - IO.Trace(source, frame) } else { source } diff --git a/core/shared/src/main/scala/cats/effect/tracing/TracingMode.scala b/core/shared/src/main/scala/cats/effect/tracing/TracingMode.scala index 5b740c0f24..cd7aa015e1 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/TracingMode.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/TracingMode.scala @@ -20,15 +20,18 @@ sealed abstract private[effect] class TracingMode private[effect] object TracingMode { + case object Disabled extends TracingMode + case object Rabbit extends TracingMode case object Slug extends TracingMode def fromString(value: String): Option[TracingMode] = value.toLowerCase() match { - case "rabbit" => Some(Rabbit) - case "slug" => Some(Slug) - case _ => None + case "disabled" => Some(Disabled) + case "rabbit" => Some(Rabbit) + case "slug" => Some(Slug) + case _ => None } } From ff2ec7e9fc752a86e528057e6412f794331a2337 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Mon, 27 Apr 2020 19:03:04 -0500 Subject: [PATCH 12/78] wip --- .../shared/src/main/scala/cats/effect/internals/IOContext.scala | 2 +- .../shared/src/main/scala/cats/effect/internals/IOTracing.scala | 2 +- core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala index 9ed90ceb48..69c496a6b4 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala @@ -33,7 +33,7 @@ final private[effect] class IOContext private () { def pushFrame(that: TraceFrame): Unit = // Accessed from at most one thread at a time - frames = that :: frames + frames = (that :: frames) def getTrace: IOTrace = IOTrace(frames) diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index 84be97197b..e0d4ef9ed9 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -50,7 +50,7 @@ private[effect] object IOTracing { // TODO: proper trace calculation val lines = new Throwable().getStackTrace.toList .map(TraceLine.fromStackTraceElement) -// .filter(_.className.startsWith("cats.effect.internals.Main")) + .filter(_.className.startsWith("cats.effect.internals.Example")) TraceFrame(lines) } diff --git a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala index b2fa37d681..5bfb3c445e 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala @@ -20,7 +20,7 @@ final case class IOTrace(frames: List[TraceFrame]) { def printTrace(): Unit = frames.foreach { f => - println("New frame") +// println("New frame") f.lines.foreach { l => println(s"\t${l.className}.${l.methodName} (${l.fileName}:${l.lineNumber})") } From 72d972385b59c44c77b29c6d0d173c6281b754c1 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Tue, 28 Apr 2020 02:47:08 -0500 Subject: [PATCH 13/78] Basic trace calculation --- ...Platform.java => TracingPlatformFast.java} | 4 +- .../scala/cats/effect/internals/Example.scala | 4 +- ...gsPlatform.scala => TracingPlatform.scala} | 8 ++-- .../src/main/scala/cats/effect/IO.scala | 2 +- .../cats/effect/internals/IOContext.scala | 24 +++++++----- .../cats/effect/internals/IORunLoop.scala | 6 +-- .../cats/effect/internals/IOTracing.scala | 39 ++++++++++++------- .../scala/cats/effect/tracing/IOTrace.scala | 2 +- 8 files changed, 50 insertions(+), 39 deletions(-) rename core/jvm/src/main/java/cats/effect/internals/{TracingPlatform.java => TracingPlatformFast.java} (95%) rename core/jvm/src/main/scala/cats/effect/internals/{TracingFlagsPlatform.scala => TracingPlatform.scala} (78%) diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java similarity index 95% rename from core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java rename to core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java index 285f32a6db..9b04bcd470 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java @@ -25,7 +25,7 @@ * bypass the volatile read and squeeze out as much performance * as possible. */ -class TracingPlatform { +public class TracingPlatformFast { /** * A boolean flag that controls tracing for a JVM process. @@ -33,6 +33,6 @@ class TracingPlatform { public static final boolean tracingEnabled = Optional.ofNullable(System.getProperty("cats.effect.tracing.enabled")) .filter(x -> !x.isEmpty()) .map(x -> Boolean.valueOf(x)) - .orElse(true); + .orElse(false); } diff --git a/core/jvm/src/main/scala/cats/effect/internals/Example.scala b/core/jvm/src/main/scala/cats/effect/internals/Example.scala index 21721c8d63..d056c7dff2 100644 --- a/core/jvm/src/main/scala/cats/effect/internals/Example.scala +++ b/core/jvm/src/main/scala/cats/effect/internals/Example.scala @@ -14,7 +14,7 @@ * limitations under the License. */ -package cats.effect.internals +package org.simpleapp.example import cats.effect.{ExitCode, IO, IOApp} @@ -36,7 +36,7 @@ object Example extends IOApp { override def run(args: List[String]): IO[ExitCode] = for { _ <- IO.suspend(program) - trace <- IO.introspect + trace <- IO.backtrace _ <- IO.delay(trace.printTrace()) } yield ExitCode.Success diff --git a/core/jvm/src/main/scala/cats/effect/internals/TracingFlagsPlatform.scala b/core/jvm/src/main/scala/cats/effect/internals/TracingPlatform.scala similarity index 78% rename from core/jvm/src/main/scala/cats/effect/internals/TracingFlagsPlatform.scala rename to core/jvm/src/main/scala/cats/effect/internals/TracingPlatform.scala index cfc6ae5ba9..78e1fc018e 100644 --- a/core/jvm/src/main/scala/cats/effect/internals/TracingFlagsPlatform.scala +++ b/core/jvm/src/main/scala/cats/effect/internals/TracingPlatform.scala @@ -18,14 +18,12 @@ package cats.effect.internals import cats.effect.tracing.TracingMode -private[effect] object TracingFlagsPlatform { +private[effect] object TracingPlatform { - // TODO: Extend `TracingPlatform` and inspect bytecode for static final field access - - // TODO: Configure this lexically and introduce a Disabled mode. + // TODO: Configure this lexically val tracingMode: TracingMode = Option(System.getProperty("cats.effect.tracing.mode")) .flatMap(TracingMode.fromString) - .getOrElse(TracingMode.Rabbit) + .getOrElse(TracingMode.Disabled) } diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index 29831c1913..11b6251ac0 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -1568,7 +1568,7 @@ object IO extends IOInstances { def contextShift(ec: ExecutionContext): ContextShift[IO] = IOContextShift(ec) - def introspect: IO[IOTrace] = + def backtrace: IO[IOTrace] = Introspect /* -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-= */ diff --git a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala index 69c496a6b4..4fa8119b07 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala @@ -25,15 +25,21 @@ import cats.effect.tracing.{IOTrace, TraceFrame} */ final private[effect] class IOContext private () { - // We had to do this because of IOBracket implementation - // and how it invokes a new run-loop. - // TODO: for infinite loops, `frames` represents an unbounded memory leak - // we should implement a ring buffer with a configurable frame buffer size - @volatile var frames: List[TraceFrame] = Nil - - def pushFrame(that: TraceFrame): Unit = - // Accessed from at most one thread at a time - frames = (that :: frames) + // We have to use a volatile here because of IOBracket implementation + // and how it invokes a new run-loop "asynchronously." + // Ideally we could use a mutable, ring buffer here. + @volatile var frames: Vector[TraceFrame] = Vector.empty + + def pushFrame(fr: TraceFrame): Unit = { + // Accessed from at most one thread at a time, + // so no race condition will occur + val currFrames = frames + if (currFrames.length >= 1000) { + frames = fr +: currFrames.dropRight(1) + } else { + frames = fr +: currFrames + } + } def getTrace: IOTrace = IOTrace(frames) diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index 1a528f18e7..b4e1d7da5b 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -135,7 +135,6 @@ private[effect] object IORunLoop { return case ContextSwitch(next, modify, restore) => - // TODO: any tracing implications here? val old = if (conn ne null) conn else IOConnection() conn = modify(old) currentIO = next @@ -159,7 +158,6 @@ private[effect] object IORunLoop { if (hasUnboxed) { popNextBind(bFirst, bRest) match { case null => - // TODO: reset status here? cb(Right(unboxed)) return case bind => @@ -250,8 +248,6 @@ private[effect] object IORunLoop { case Async(_, _) => // Cannot inline the code of this method — as it would // box those vars in scala.runtime.ObjectRef! - // TODO: Since IO.traced is implemented in terms of IOBracket - // we may not need to concern ourselves with tracing status here? return suspendAsync(currentIO.asInstanceOf[IO.Async[A]], bFirst, bRest) case Trace(source, currTrace) => @@ -260,7 +256,7 @@ private[effect] object IORunLoop { currentIO = source case Introspect => - // This can be implemented in terms of Async now + // TODO: This can be implemented in terms of Async now if (ctx eq null) ctx = IOContext.newContext hasUnboxed = true unboxed = ctx.getTrace diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index e0d4ef9ed9..bae1e57577 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -16,12 +16,12 @@ package cats.effect.internals +import java.util.concurrent.ConcurrentHashMap + import cats.effect.IO import cats.effect.tracing.{TraceFrame, TraceLine, TracingMode} -import cats.effect.internals.TracingPlatform.tracingEnabled -import cats.effect.internals.TracingFlagsPlatform.tracingMode - -import scala.collection.mutable +import cats.effect.internals.TracingPlatform.tracingMode +import cats.effect.internals.TracingPlatformFast.tracingEnabled private[effect] object IOTracing { @@ -37,20 +37,23 @@ private[effect] object IOTracing { source } - private def buildCachedFrame(lambdaRef: AnyRef): TraceFrame = - frameCache.get(lambdaRef) match { - case Some(fr) => fr - case None => - val fr = buildFrame() - frameCache.put(lambdaRef, fr) - fr + private def buildCachedFrame(lambdaRef: AnyRef): TraceFrame = { + val cachedFr = frameCache.get(lambdaRef) + if (cachedFr eq null) { + val fr = buildFrame() + frameCache.put(lambdaRef, fr) + fr + } else { + cachedFr } + } private def buildFrame(): TraceFrame = { // TODO: proper trace calculation val lines = new Throwable().getStackTrace.toList .map(TraceLine.fromStackTraceElement) - .filter(_.className.startsWith("cats.effect.internals.Example")) + .find(l => !classBlacklist.exists(b => l.className.startsWith(b))) + .toList TraceFrame(lines) } @@ -58,9 +61,17 @@ private[effect] object IOTracing { /** * Cache for trace frames. Keys are object references for lambdas. * - * TODO: Switch to thread-local or j.u.chm? + * TODO: Consider thread-local or a regular, mutable map.h * TODO: Bound the cache. */ - private val frameCache: mutable.Map[AnyRef, TraceFrame] = new mutable.HashMap() + private val frameCache: ConcurrentHashMap[AnyRef, TraceFrame] = new ConcurrentHashMap[AnyRef, TraceFrame]() + + private val classBlacklist = List( + "cats.effect.", + "sbt.", + "java.", + "sun.", + "scala." + ) } diff --git a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala index 5bfb3c445e..0bafb43b7d 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala @@ -16,7 +16,7 @@ package cats.effect.tracing -final case class IOTrace(frames: List[TraceFrame]) { +final case class IOTrace(frames: Vector[TraceFrame]) { def printTrace(): Unit = frames.foreach { f => From bf659ad07dab2df4de4318e8ee8dfe2f8352a47e Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Tue, 28 Apr 2020 03:11:40 -0500 Subject: [PATCH 14/78] WIP --- .../cats/effect/internals/TracingPlatformFast.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java index 9b04bcd470..b8fa3224f8 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java @@ -19,16 +19,16 @@ import java.util.Optional; /** - * Scala companion object field accesses cost a volatile read. - * Since this flag is read at the construction of IO nodes, - * we are opting to source this flag from a Java class to - * bypass the volatile read and squeeze out as much performance - * as possible. + * Scala object field accesses cost a volatile read across modules. + * Since this flag is read during construction of IO nodes, we are opting to + * hold this flag in a Java class to bypass the volatile read. */ public class TracingPlatformFast { /** - * A boolean flag that controls tracing for a JVM process. + * A boolean flag that enables or disables tracing for a JVM process. + * Since it is declared static and final, the JIT compiler has the liberty + * to completely eliminate code paths consequent to the conditional. */ public static final boolean tracingEnabled = Optional.ofNullable(System.getProperty("cats.effect.tracing.enabled")) .filter(x -> !x.isEmpty()) From c23b4dfb5427873dfa024ca1a2123b28ff331266 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Tue, 28 Apr 2020 21:35:28 -0500 Subject: [PATCH 15/78] comment --- .../src/main/scala/cats/effect/internals/IOTracing.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index bae1e57577..6c60c0a58d 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -61,8 +61,8 @@ private[effect] object IOTracing { /** * Cache for trace frames. Keys are object references for lambdas. * - * TODO: Consider thread-local or a regular, mutable map.h - * TODO: Bound the cache. + * TODO: Consider thread-local or a regular, mutable map. + * TODO: LRU max-bounded cache. */ private val frameCache: ConcurrentHashMap[AnyRef, TraceFrame] = new ConcurrentHashMap[AnyRef, TraceFrame]() From 4666088c9c7555240ebbfdf4ec21ac4a92e813b3 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Tue, 28 Apr 2020 22:34:41 -0500 Subject: [PATCH 16/78] Lexically scoped tracing via ThreadLocal --- .../effect/internals/TracingPlatformFast.java | 2 +- .../scala/cats/effect/internals/Example.scala | 7 +++- .../effect/internals/TracingPlatform.scala | 29 --------------- .../src/main/scala/cats/effect/IO.scala | 8 ++++- .../cats/effect/internals/IOBracket.scala | 8 +++-- .../cats/effect/internals/IOContext.scala | 2 +- .../cats/effect/internals/IORunLoop.scala | 34 +++++++++++++----- .../scala/cats/effect/internals/IOStart.scala | 2 -- .../cats/effect/internals/IOTracing.scala | 36 ++++++++++++++++--- 9 files changed, 79 insertions(+), 49 deletions(-) delete mode 100644 core/jvm/src/main/scala/cats/effect/internals/TracingPlatform.scala diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java index b8fa3224f8..c6d0c43661 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java @@ -33,6 +33,6 @@ public class TracingPlatformFast { public static final boolean tracingEnabled = Optional.ofNullable(System.getProperty("cats.effect.tracing.enabled")) .filter(x -> !x.isEmpty()) .map(x -> Boolean.valueOf(x)) - .orElse(false); + .orElse(true); } diff --git a/core/jvm/src/main/scala/cats/effect/internals/Example.scala b/core/jvm/src/main/scala/cats/effect/internals/Example.scala index d056c7dff2..bbfa4bc34f 100644 --- a/core/jvm/src/main/scala/cats/effect/internals/Example.scala +++ b/core/jvm/src/main/scala/cats/effect/internals/Example.scala @@ -35,7 +35,12 @@ object Example extends IOApp { override def run(args: List[String]): IO[ExitCode] = for { - _ <- IO.suspend(program) + _ <- IO.suspend(program).rabbitTrace + _ <- IO.delay("10") + _ <- IO.delay("11") + _ <- IO.delay("12") + _ <- IO.delay("13") + _ <- IO.delay("14") trace <- IO.backtrace _ <- IO.delay(trace.printTrace()) } yield ExitCode.Success diff --git a/core/jvm/src/main/scala/cats/effect/internals/TracingPlatform.scala b/core/jvm/src/main/scala/cats/effect/internals/TracingPlatform.scala deleted file mode 100644 index 78e1fc018e..0000000000 --- a/core/jvm/src/main/scala/cats/effect/internals/TracingPlatform.scala +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers - * - * Licensed 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 cats.effect.internals - -import cats.effect.tracing.TracingMode - -private[effect] object TracingPlatform { - - // TODO: Configure this lexically - val tracingMode: TracingMode = - Option(System.getProperty("cats.effect.tracing.mode")) - .flatMap(TracingMode.fromString) - .getOrElse(TracingMode.Disabled) - -} diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index 11b6251ac0..e17e7a9e57 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -26,7 +26,7 @@ import scala.concurrent.{ExecutionContext, Future, Promise, TimeoutException} import scala.util.control.NonFatal import scala.util.{Failure, Left, Right, Success, Try} import cats.data.Ior -import cats.effect.tracing.{IOTrace, TraceFrame} +import cats.effect.tracing.{IOTrace, TraceFrame, TracingMode} /** * A pure abstraction representing the intention to perform a @@ -785,6 +785,12 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * */ def <&[B](another: IO[B])(implicit p: NonEmptyParallel[IO]): IO[A] = p.parProductL(this)(another) + + def slugTrace: IO[A] = + IOTracing.tracedLocally(this, TracingMode.Slug) + + def rabbitTrace: IO[A] = + IOTracing.tracedLocally(this, TracingMode.Rabbit) } abstract private[effect] class IOParallelNewtype extends internals.IOTimerRef with internals.IOCompanionBinaryCompat { diff --git a/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala b/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala index 2e002597a3..245ba02894 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala @@ -24,6 +24,8 @@ import scala.concurrent.{ExecutionContext, Promise} import scala.util.control.NonFatal import java.util.concurrent.atomic.AtomicBoolean +import cats.effect.tracing.TracingMode + private[effect] object IOBracket { /** @@ -40,7 +42,8 @@ private[effect] object IOBracket { if (!conn.isCanceled) { // Note `acquire` is uncancelable due to usage of `IORunLoop.start` // (in other words it is disconnected from our IOConnection) - IORunLoop.restart[A](acquire, ctx, new BracketStart(use, release, conn, ctx, deferredRelease, cb)) + val tMode = IOTracing.getLocalTracingMode() + IORunLoop.restart[A](acquire, ctx, tMode, new BracketStart(use, release, conn, ctx, tMode, deferredRelease, cb)) } else { deferredRelease.complete(IO.unit) } @@ -52,6 +55,7 @@ private[effect] object IOBracket { release: (A, ExitCase[Throwable]) => IO[Unit], conn: IOConnection, ctx: IOContext, + mode: TracingMode, deferredRelease: ForwardCancelable, cb: Callback.T[B] ) extends (Either[Throwable, A] => Unit) @@ -89,7 +93,7 @@ private[effect] object IOBracket { fb.flatMap(frame) } // Actual execution - IORunLoop.restartCancelable(onNext, conn, ctx, cb) + IORunLoop.restartCancelable(onNext, conn, ctx, mode, cb) } case error @ Left(_) => diff --git a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala index 4fa8119b07..f631d0b2b5 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala @@ -47,6 +47,6 @@ final private[effect] class IOContext private () { } object IOContext { - def newContext: IOContext = + def apply(): IOContext = new IOContext } diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index b4e1d7da5b..ebe854669c 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -18,6 +18,7 @@ package cats.effect.internals import cats.effect.IO import cats.effect.IO.{Async, Bind, ContextSwitch, Delay, Introspect, Map, Pure, RaiseError, Suspend, Trace} +import cats.effect.tracing.TracingMode import scala.util.control.NonFatal @@ -32,21 +33,29 @@ private[effect] object IORunLoop { * Evaluates the given `IO` reference, calling the given callback * with the result when completed. */ - def start[A](source: IO[A], cb: Either[Throwable, A] => Unit): Unit = + def start[A](source: IO[A], cb: Either[Throwable, A] => Unit): Unit = { + IOTracing.setLocalTracingMode(TracingMode.Disabled) loop(source, IOConnection.uncancelable, cb.asInstanceOf[Callback], null, null, null, null) + } - def restart[A](source: IO[A], ctx: IOContext, cb: Either[Throwable, A] => Unit): Unit = + def restart[A](source: IO[A], ctx: IOContext, mode: TracingMode, cb: Either[Throwable, A] => Unit): Unit = { + IOTracing.setLocalTracingMode(mode) loop(source, IOConnection.uncancelable, cb.asInstanceOf[Callback], ctx, null, null, null) + } /** * Evaluates the given `IO` reference, calling the given callback * with the result when completed. */ - def startCancelable[A](source: IO[A], conn: IOConnection, cb: Either[Throwable, A] => Unit): Unit = + def startCancelable[A](source: IO[A], conn: IOConnection, cb: Either[Throwable, A] => Unit): Unit = { + IOTracing.setLocalTracingMode(TracingMode.Disabled) loop(source, conn, cb.asInstanceOf[Callback], null, null, null, null) + } - def restartCancelable[A](source: IO[A], conn: IOConnection, ctx: IOContext, cb: Either[Throwable, A] => Unit): Unit = + def restartCancelable[A](source: IO[A], conn: IOConnection, ctx: IOContext, mode: TracingMode, cb: Either[Throwable, A] => Unit): Unit = { + IOTracing.setLocalTracingMode(mode) loop(source, conn, cb.asInstanceOf[Callback], ctx, null, null, null) + } /** * Loop for evaluating an `IO` value. @@ -130,6 +139,7 @@ private[effect] object IORunLoop { case async @ Async(_, _) => if (conn eq null) conn = IOConnection() + if (ctx eq null) ctx = IOContext() if (rcb eq null) rcb = new RestartCallback(conn, ctx, cb.asInstanceOf[Callback]) rcb.start(async, bFirst, bRest) return @@ -145,14 +155,15 @@ private[effect] object IORunLoop { } case Trace(source, currTrace) => - if (ctx eq null) ctx = IOContext.newContext + if (ctx eq null) ctx = IOContext() ctx.pushFrame(currTrace) currentIO = source case Introspect => - if (ctx eq null) ctx = IOContext.newContext + if (ctx eq null) ctx = IOContext() hasUnboxed = true unboxed = ctx.getTrace + } if (hasUnboxed) { @@ -251,13 +262,13 @@ private[effect] object IORunLoop { return suspendAsync(currentIO.asInstanceOf[IO.Async[A]], bFirst, bRest) case Trace(source, currTrace) => - if (ctx eq null) ctx = IOContext.newContext + if (ctx eq null) ctx = IOContext() ctx.pushFrame(currTrace) currentIO = source case Introspect => // TODO: This can be implemented in terms of Async now - if (ctx eq null) ctx = IOContext.newContext + if (ctx eq null) ctx = IOContext() hasUnboxed = true unboxed = ctx.getTrace @@ -371,6 +382,7 @@ private[effect] object IORunLoop { private[this] var trampolineAfter = false private[this] var bFirst: Bind = _ private[this] var bRest: CallStack = _ + private[this] var tMode: TracingMode = _ // Used in combination with trampolineAfter = true private[this] var value: Either[Throwable, Any] = _ @@ -383,6 +395,7 @@ private[effect] object IORunLoop { this.bFirst = bFirst this.bRest = bRest this.trampolineAfter = task.trampolineAfter + this.tMode = IOTracing.getLocalTracingMode() // Go, go, go task.k(conn, ctx, this) @@ -395,6 +408,11 @@ private[effect] object IORunLoop { this.bFirst = null this.bRest = null + // The continuation may have been invoked on a new execution context, + // so let's recover the tracing mode here. + IOTracing.setLocalTracingMode(this.tMode) + this.tMode = null + // Auto-cancelable logic: in case the connection was cancelled, // we interrupt the bind continuation if (!conn.isCanceled) either match { diff --git a/core/shared/src/main/scala/cats/effect/internals/IOStart.scala b/core/shared/src/main/scala/cats/effect/internals/IOStart.scala index 8474e209c5..db00a7b96a 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOStart.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOStart.scala @@ -37,8 +37,6 @@ private[effect] object IOStart { p.success(ea) () } - // TODO: We need to ensure that the thread that begins executing the new IO - // has a tracing status reset because of how IOBracket is implemented. IORunLoop.startCancelable(IOForkedStart(fa, cs), conn2, cb0) cb(Right(fiber(p, conn2))) diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index 6c60c0a58d..a98fbc0fdb 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -20,7 +20,6 @@ import java.util.concurrent.ConcurrentHashMap import cats.effect.IO import cats.effect.tracing.{TraceFrame, TraceLine, TracingMode} -import cats.effect.internals.TracingPlatform.tracingMode import cats.effect.internals.TracingPlatformFast.tracingEnabled private[effect] object IOTracing { @@ -28,7 +27,7 @@ private[effect] object IOTracing { def apply[A](source: IO[A], lambdaRef: AnyRef): IO[A] = // TODO: consider inlining this conditional at call-sites if (tracingEnabled) { - tracingMode match { + localTracingMode.get() match { case TracingMode.Disabled => source case TracingMode.Rabbit => IO.Trace(source, buildCachedFrame(lambdaRef)) case TracingMode.Slug => IO.Trace(source, buildFrame()) @@ -37,6 +36,30 @@ private[effect] object IOTracing { source } + def tracedLocally[A](source: IO[A], mode: TracingMode): IO[A] = + if (tracingEnabled) { + IO.suspend { + val old = localTracingMode.get() + localTracingMode.set(mode) + + // We don't need to reset the status here in the event of cancellation. + source.attempt.flatMap { e => + localTracingMode.set(old) + IO.fromEither(e) + } + } + } else { + source + } + + def getLocalTracingMode(): TracingMode = + localTracingMode.get() + + def setLocalTracingMode(mode: TracingMode): Unit = + if (tracingEnabled) { + localTracingMode.set(mode) + } + private def buildCachedFrame(lambdaRef: AnyRef): TraceFrame = { val cachedFr = frameCache.get(lambdaRef) if (cachedFr eq null) { @@ -59,13 +82,18 @@ private[effect] object IOTracing { } /** - * Cache for trace frames. Keys are object references for lambdas. + * Cache for trace frames. Keys are references to: + * - lambdas * * TODO: Consider thread-local or a regular, mutable map. - * TODO: LRU max-bounded cache. + * TODO: LRU entry-bounded cache. */ private val frameCache: ConcurrentHashMap[AnyRef, TraceFrame] = new ConcurrentHashMap[AnyRef, TraceFrame]() + private val localTracingMode: ThreadLocal[TracingMode] = new ThreadLocal[TracingMode] { + override def initialValue(): TracingMode = TracingMode.Disabled + } + private val classBlacklist = List( "cats.effect.", "sbt.", From fe61325aebc85c514c5853fe057f41f323eb0e72 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Fri, 1 May 2020 01:51:21 -0500 Subject: [PATCH 17/78] Check flag --- .../effect/internals/TracingPlatformFast.java | 2 +- .../scala/cats/effect/internals/Example.scala | 12 +-- .../src/main/scala/cats/effect/IO.scala | 76 +++++++++++++------ .../cats/effect/internals/IORunLoop.scala | 49 ++++++++---- .../cats/effect/internals/IOTracing.scala | 42 ++++------ .../scala/cats/effect/tracing/IOTrace.scala | 7 +- 6 files changed, 117 insertions(+), 71 deletions(-) diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java index c6d0c43661..b8fa3224f8 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java @@ -33,6 +33,6 @@ public class TracingPlatformFast { public static final boolean tracingEnabled = Optional.ofNullable(System.getProperty("cats.effect.tracing.enabled")) .filter(x -> !x.isEmpty()) .map(x -> Boolean.valueOf(x)) - .orElse(true); + .orElse(false); } diff --git a/core/jvm/src/main/scala/cats/effect/internals/Example.scala b/core/jvm/src/main/scala/cats/effect/internals/Example.scala index bbfa4bc34f..033265be63 100644 --- a/core/jvm/src/main/scala/cats/effect/internals/Example.scala +++ b/core/jvm/src/main/scala/cats/effect/internals/Example.scala @@ -20,6 +20,12 @@ import cats.effect.{ExitCode, IO, IOApp} object Example extends IOApp { + def program2: IO[Unit] = + for { + _ <- IO.delay(println("7")) + _ <- IO.delay(println("8")) + } yield () + def program: IO[Unit] = for { _ <- IO.delay(println("1")) @@ -27,7 +33,7 @@ object Example extends IOApp { _ <- IO.shift _ <- IO.unit.bracket(_ => IO.delay(println("3")) - .flatMap(_ => IO.unit) + .flatMap(_ => program2) )(_ => IO.unit) _ <- IO.delay(println("4")) _ <- IO.delay(println("5")) @@ -37,10 +43,6 @@ object Example extends IOApp { for { _ <- IO.suspend(program).rabbitTrace _ <- IO.delay("10") - _ <- IO.delay("11") - _ <- IO.delay("12") - _ <- IO.delay("13") - _ <- IO.delay("14") trace <- IO.backtrace _ <- IO.delay(trace.printTrace()) } yield ExitCode.Success diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index e17e7a9e57..95b55a85db 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -27,6 +27,7 @@ import scala.util.control.NonFatal import scala.util.{Failure, Left, Right, Success, Try} import cats.data.Ior import cats.effect.tracing.{IOTrace, TraceFrame, TracingMode} +import cats.effect.internals.TracingPlatformFast.tracingEnabled /** * A pure abstraction representing the intention to perform a @@ -102,18 +103,22 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * never terminate on evaluation. */ final def map[B](f: A => B): IO[B] = { - val source = this match { - case Map(source, g, index) => - // Allowed to do fixed number of map operations fused before - // resetting the counter in order to avoid stack overflows; - // See `IOPlatform` for details on this maximum. - if (index != fusionMaxStackDepth) Map(source, g.andThen(f), index + 1) - else Map(this, f, 0) - case _ => - Map(this, f, 0) + if (tracingEnabled) { + // Don't perform map fusion when tracing is enabled. + // We may not actually have to do this + IOTracing(Map(this, f, 0), f) + } else { + this match { + case Map(source, g, index) => + // Allowed to do fixed number of map operations fused before + // resetting the counter in order to avoid stack overflows; + // See `IOPlatform` for details on this maximum. + if (index != fusionMaxStackDepth) Map(source, g.andThen(f), index + 1) + else Map(this, f, 0) + case _ => + Map(this, f, 0) + } } - - IOTracing.apply(source, f.asInstanceOf[AnyRef]) } /** @@ -132,8 +137,12 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * never terminate on evaluation. */ final def flatMap[B](f: A => IO[B]): IO[B] = { - val source = Bind(this, f) - IOTracing.apply(source, f.asInstanceOf[AnyRef]) + val nextIo = Bind(this, f) + if (tracingEnabled) { + IOTracing(nextIo, f) + } else { + nextIo + } } /** @@ -787,10 +796,18 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { p.parProductL(this)(another) def slugTrace: IO[A] = - IOTracing.tracedLocally(this, TracingMode.Slug) + if (tracingEnabled) { + IOTracing.tracedLocally(this, TracingMode.Slug) + } else { + this + } def rabbitTrace: IO[A] = - IOTracing.tracedLocally(this, TracingMode.Rabbit) + if (tracingEnabled) { + IOTracing.tracedLocally(this, TracingMode.Rabbit) + } else { + this + } } abstract private[effect] class IOParallelNewtype extends internals.IOTimerRef with internals.IOCompanionBinaryCompat { @@ -1213,13 +1230,17 @@ object IO extends IOInstances { * @see [[asyncF]] and [[cancelable]] */ def async[A](k: (Either[Throwable, A] => Unit) => Unit): IO[A] = { - val source = Async[A] { (_, _, cb) => + val nextIo = Async[A] { (_, _, cb) => val cb2 = Callback.asyncIdempotent(null, cb) try k(cb2) catch { case NonFatal(t) => cb2(Left(t)) } } - IOTracing(source, k) + if (tracingEnabled) { + IOTracing(nextIo, k) + } else { + nextIo + } } /** @@ -1247,7 +1268,7 @@ object IO extends IOInstances { * @see [[async]] and [[cancelable]] */ def asyncF[A](k: (Either[Throwable, A] => Unit) => IO[Unit]): IO[A] = { - val source = Async[A] { (conn, _, cb) => + val nextIo = Async[A] { (conn, _, cb) => // Must create new connection, otherwise we can have a race // condition b/t the bind continuation and `startCancelable` below val conn2 = IOConnection() @@ -1260,7 +1281,11 @@ object IO extends IOInstances { IORunLoop.startCancelable(fa, conn2, Callback.report) } - IOTracing(source, k) + if (tracingEnabled) { + IOTracing(nextIo, k) + } else { + nextIo + } } /** @@ -1303,7 +1328,7 @@ object IO extends IOInstances { * the underlying cancelation model */ def cancelable[A](k: (Either[Throwable, A] => Unit) => CancelToken[IO]): IO[A] = { - val source = Async[A] { (conn, _, cb) => + val nextIo = Async[A] { (conn, _, cb) => val cb2 = Callback.asyncIdempotent(conn, cb) val ref = ForwardCancelable() conn.push(ref.cancel) @@ -1322,7 +1347,12 @@ object IO extends IOInstances { else ref.complete(IO.unit) } - IOTracing.apply(source, k) + + if (tracingEnabled) { + IOTracing(nextIo, k) + } else { + nextIo + } } /** @@ -1598,7 +1628,7 @@ object IO extends IOInstances { /** Corresponds to [[IO.map]]. */ final private[effect] case class Map[E, +A](source: IO[E], f: E => A, index: Int) extends IO[A] with (E => IO[A]) { override def apply(value: E): IO[A] = - new Pure(f(value)) + Pure(f(value)) } /** @@ -1633,7 +1663,7 @@ object IO extends IOInstances { restore: (A, Throwable, IOConnection, IOConnection) => IOConnection ) extends IO[A] - final private[effect] case class Trace[+A](source: IO[A], frame: TraceFrame) extends IO[A] + final private[effect] case class Trace[A](source: IO[A], frame: TraceFrame) extends IO[A] final private[effect] case object Introspect extends IO[IOTrace] diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index ebe854669c..3c010a9a2d 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -19,6 +19,7 @@ package cats.effect.internals import cats.effect.IO import cats.effect.IO.{Async, Bind, ContextSwitch, Delay, Introspect, Map, Pure, RaiseError, Suspend, Trace} import cats.effect.tracing.TracingMode +import cats.effect.internals.TracingPlatformFast.tracingEnabled import scala.util.control.NonFatal @@ -34,12 +35,16 @@ private[effect] object IORunLoop { * with the result when completed. */ def start[A](source: IO[A], cb: Either[Throwable, A] => Unit): Unit = { - IOTracing.setLocalTracingMode(TracingMode.Disabled) + if (tracingEnabled) { + IOTracing.setLocalTracingMode(TracingMode.Disabled) + } loop(source, IOConnection.uncancelable, cb.asInstanceOf[Callback], null, null, null, null) } def restart[A](source: IO[A], ctx: IOContext, mode: TracingMode, cb: Either[Throwable, A] => Unit): Unit = { - IOTracing.setLocalTracingMode(mode) + if (tracingEnabled) { + IOTracing.setLocalTracingMode(mode) + } loop(source, IOConnection.uncancelable, cb.asInstanceOf[Callback], ctx, null, null, null) } @@ -48,12 +53,16 @@ private[effect] object IORunLoop { * with the result when completed. */ def startCancelable[A](source: IO[A], conn: IOConnection, cb: Either[Throwable, A] => Unit): Unit = { - IOTracing.setLocalTracingMode(TracingMode.Disabled) + if (tracingEnabled) { + IOTracing.setLocalTracingMode(TracingMode.Disabled) + } loop(source, conn, cb.asInstanceOf[Callback], null, null, null, null) } def restartCancelable[A](source: IO[A], conn: IOConnection, ctx: IOContext, mode: TracingMode, cb: Either[Throwable, A] => Unit): Unit = { - IOTracing.setLocalTracingMode(mode) + if (tracingEnabled) { + IOTracing.setLocalTracingMode(mode) + } loop(source, conn, cb.asInstanceOf[Callback], ctx, null, null, null) } @@ -94,6 +103,10 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } +// if (tracingEnabled) { +// if (ctx eq null) ctx = IOContext() +// ctx.pushFrame(bind.trace) +// } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa @@ -134,11 +147,17 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } +// if (tracingEnabled) { +// if (ctx eq null) ctx = IOContext() +// ctx.pushFrame(bindNext.trace) +// } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa case async @ Async(_, _) => if (conn eq null) conn = IOConnection() + // We need to initialize an IOContext because the continuation + // may produce trace frames. if (ctx eq null) ctx = IOContext() if (rcb eq null) rcb = new RestartCallback(conn, ctx, cb.asInstanceOf[Callback]) rcb.start(async, bFirst, bRest) @@ -154,9 +173,9 @@ private[effect] object IORunLoop { currentIO = Bind(next, new RestoreContext(old, restore)) } - case Trace(source, currTrace) => + case Trace(source, frame) => if (ctx eq null) ctx = IOContext() - ctx.pushFrame(currTrace) + ctx.pushFrame(frame) currentIO = source case Introspect => @@ -261,9 +280,9 @@ private[effect] object IORunLoop { // box those vars in scala.runtime.ObjectRef! return suspendAsync(currentIO.asInstanceOf[IO.Async[A]], bFirst, bRest) - case Trace(source, currTrace) => + case Trace(source, frame) => if (ctx eq null) ctx = IOContext() - ctx.pushFrame(currTrace) + ctx.pushFrame(frame) currentIO = source case Introspect => @@ -395,7 +414,9 @@ private[effect] object IORunLoop { this.bFirst = bFirst this.bRest = bRest this.trampolineAfter = task.trampolineAfter - this.tMode = IOTracing.getLocalTracingMode() + if (tracingEnabled) { + this.tMode = IOTracing.getLocalTracingMode() + } // Go, go, go task.k(conn, ctx, this) @@ -408,10 +429,12 @@ private[effect] object IORunLoop { this.bFirst = null this.bRest = null - // The continuation may have been invoked on a new execution context, - // so let's recover the tracing mode here. - IOTracing.setLocalTracingMode(this.tMode) - this.tMode = null + if (tracingEnabled) { + // The continuation may have been invoked on a new execution context, + // so let's recover the tracing mode here. + IOTracing.setLocalTracingMode(this.tMode) + this.tMode = null + } // Auto-cancelable logic: in case the connection was cancelled, // we interrupt the bind continuation diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index a98fbc0fdb..88f6b782ac 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -20,45 +20,35 @@ import java.util.concurrent.ConcurrentHashMap import cats.effect.IO import cats.effect.tracing.{TraceFrame, TraceLine, TracingMode} -import cats.effect.internals.TracingPlatformFast.tracingEnabled private[effect] object IOTracing { - def apply[A](source: IO[A], lambdaRef: AnyRef): IO[A] = - // TODO: consider inlining this conditional at call-sites - if (tracingEnabled) { - localTracingMode.get() match { - case TracingMode.Disabled => source - case TracingMode.Rabbit => IO.Trace(source, buildCachedFrame(lambdaRef)) - case TracingMode.Slug => IO.Trace(source, buildFrame()) - } - } else { - source + def apply[A](source: IO[A], ref: AnyRef): IO[A] = + localTracingMode.get() match { + case TracingMode.Disabled => source + case TracingMode.Rabbit => IO.Trace(source, buildCachedFrame(ref)) + case TracingMode.Slug => IO.Trace(source, buildFrame()) } def tracedLocally[A](source: IO[A], mode: TracingMode): IO[A] = - if (tracingEnabled) { - IO.suspend { - val old = localTracingMode.get() - localTracingMode.set(mode) - - // We don't need to reset the status here in the event of cancellation. - source.attempt.flatMap { e => - localTracingMode.set(old) - IO.fromEither(e) - } + IO.suspend { + val old = localTracingMode.get() + localTracingMode.set(mode) + + // Rethrow any exceptions that `source` produces after resettubg nide, + // In the event of cancellation, the mode will be reset when the + // thread grabs a new task to run (via Async). + source.attempt.flatMap { e => + localTracingMode.set(old) + IO.fromEither(e) } - } else { - source } def getLocalTracingMode(): TracingMode = localTracingMode.get() def setLocalTracingMode(mode: TracingMode): Unit = - if (tracingEnabled) { - localTracingMode.set(mode) - } + localTracingMode.set(mode) private def buildCachedFrame(lambdaRef: AnyRef): TraceFrame = { val cachedFr = frameCache.get(lambdaRef) diff --git a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala index 0bafb43b7d..9bd81fc665 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala @@ -18,12 +18,13 @@ package cats.effect.tracing final case class IOTrace(frames: Vector[TraceFrame]) { - def printTrace(): Unit = + def printTrace(): Unit = { + System.err.println("IOTrace") frames.foreach { f => -// println("New frame") f.lines.foreach { l => - println(s"\t${l.className}.${l.methodName} (${l.fileName}:${l.lineNumber})") + System.err.println(s"\tat ${l.className}.${l.methodName} (${l.fileName}:${l.lineNumber})") } } + } } From 99eb9cabd06a914d4356e96b3336ad921e0141a4 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Fri, 1 May 2020 20:55:33 -0500 Subject: [PATCH 18/78] WIP --- .../effect/internals/TracingPlatformFast.java | 2 +- .../scala/cats/effect/internals/Example.scala | 17 ++++++++------ .../cats/effect/internals/IOBracket.scala | 20 ++++++++++++---- .../cats/effect/internals/IOTracing.scala | 23 ++++++++++++------- .../scala/cats/effect/tracing/IOTrace.scala | 4 ++-- .../cats/effect/tracing/TraceFrame.scala | 2 +- 6 files changed, 45 insertions(+), 23 deletions(-) diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java index b8fa3224f8..c6d0c43661 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java @@ -33,6 +33,6 @@ public class TracingPlatformFast { public static final boolean tracingEnabled = Optional.ofNullable(System.getProperty("cats.effect.tracing.enabled")) .filter(x -> !x.isEmpty()) .map(x -> Boolean.valueOf(x)) - .orElse(false); + .orElse(true); } diff --git a/core/jvm/src/main/scala/cats/effect/internals/Example.scala b/core/jvm/src/main/scala/cats/effect/internals/Example.scala index 033265be63..a2efa4f99f 100644 --- a/core/jvm/src/main/scala/cats/effect/internals/Example.scala +++ b/core/jvm/src/main/scala/cats/effect/internals/Example.scala @@ -20,23 +20,26 @@ import cats.effect.{ExitCode, IO, IOApp} object Example extends IOApp { + def print(msg: String): IO[Unit] = + IO.delay(println(msg)) + def program2: IO[Unit] = for { - _ <- IO.delay(println("7")) - _ <- IO.delay(println("8")) + _ <- print("7") + _ <- print("8") } yield () def program: IO[Unit] = for { - _ <- IO.delay(println("1")) - _ <- IO.delay(println("2")) + _ <- print("1") + _ <- print("2") _ <- IO.shift _ <- IO.unit.bracket(_ => - IO.delay(println("3")) + print("3") .flatMap(_ => program2) )(_ => IO.unit) - _ <- IO.delay(println("4")) - _ <- IO.delay(println("5")) + _ <- print("4") + _ <- print("5") } yield () override def run(args: List[String]): IO[ExitCode] = diff --git a/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala b/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala index 245ba02894..aa9a69a5a0 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala @@ -25,14 +25,15 @@ import scala.util.control.NonFatal import java.util.concurrent.atomic.AtomicBoolean import cats.effect.tracing.TracingMode +import cats.effect.internals.TracingPlatformFast.tracingEnabled private[effect] object IOBracket { /** * Implementation for `IO.bracketCase`. */ - def apply[A, B](acquire: IO[A])(use: A => IO[B])(release: (A, ExitCase[Throwable]) => IO[Unit]): IO[B] = - IO.Async { (conn, ctx, cb) => + def apply[A, B](acquire: IO[A])(use: A => IO[B])(release: (A, ExitCase[Throwable]) => IO[Unit]): IO[B] = { + val nextIo = IO.Async[B] { (conn, ctx, cb) => // Placeholder for the future finalizer val deferredRelease = ForwardCancelable() conn.push(deferredRelease.cancel) @@ -42,13 +43,24 @@ private[effect] object IOBracket { if (!conn.isCanceled) { // Note `acquire` is uncancelable due to usage of `IORunLoop.start` // (in other words it is disconnected from our IOConnection) - val tMode = IOTracing.getLocalTracingMode() - IORunLoop.restart[A](acquire, ctx, tMode, new BracketStart(use, release, conn, ctx, tMode, deferredRelease, cb)) + val mode = if (tracingEnabled) { + IOTracing.getLocalTracingMode() + } else { + TracingMode.Disabled + } + IORunLoop.restart[A](acquire, ctx, mode, new BracketStart(use, release, conn, ctx, mode, deferredRelease, cb)) } else { deferredRelease.complete(IO.unit) } } + if (tracingEnabled) { + IOTracing(nextIo, use) + } else { + nextIo + } + } + // Internals of `IO.bracketCase`. final private class BracketStart[A, B]( use: A => IO[B], diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index 88f6b782ac..79caadacf4 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -26,8 +26,8 @@ private[effect] object IOTracing { def apply[A](source: IO[A], ref: AnyRef): IO[A] = localTracingMode.get() match { case TracingMode.Disabled => source - case TracingMode.Rabbit => IO.Trace(source, buildCachedFrame(ref)) - case TracingMode.Slug => IO.Trace(source, buildFrame()) + case TracingMode.Rabbit => IO.Trace(source, buildCachedFrame(source, ref)) + case TracingMode.Slug => IO.Trace(source, buildFrame(source)) } def tracedLocally[A](source: IO[A], mode: TracingMode): IO[A] = @@ -50,10 +50,10 @@ private[effect] object IOTracing { def setLocalTracingMode(mode: TracingMode): Unit = localTracingMode.set(mode) - private def buildCachedFrame(lambdaRef: AnyRef): TraceFrame = { + private def buildCachedFrame(source: IO[Any], lambdaRef: AnyRef): TraceFrame = { val cachedFr = frameCache.get(lambdaRef) if (cachedFr eq null) { - val fr = buildFrame() + val fr = buildFrame(source) frameCache.put(lambdaRef, fr) fr } else { @@ -61,14 +61,21 @@ private[effect] object IOTracing { } } - private def buildFrame(): TraceFrame = { + private def buildFrame(source: IO[Any]): TraceFrame = { // TODO: proper trace calculation - val lines = new Throwable().getStackTrace.toList + val line = new Throwable().getStackTrace.toList .map(TraceLine.fromStackTraceElement) .find(l => !classBlacklist.exists(b => l.className.startsWith(b))) - .toList + .headOption - TraceFrame(lines) + val op = source match { + case _: IO.Map[_, _] => "map" + case _: IO.Bind[_, _] => "bind" + case _: IO.Async[_] => "async" + case _ => "unknown" + } + + TraceFrame(op, line) } /** diff --git a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala index 9bd81fc665..b9e74ab4d0 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala @@ -21,8 +21,8 @@ final case class IOTrace(frames: Vector[TraceFrame]) { def printTrace(): Unit = { System.err.println("IOTrace") frames.foreach { f => - f.lines.foreach { l => - System.err.println(s"\tat ${l.className}.${l.methodName} (${l.fileName}:${l.lineNumber})") + f.line.foreach { l => + System.err.println(s"\t${f.op} at ${l.className}.${l.methodName} (${l.fileName}:${l.lineNumber})") } } } diff --git a/core/shared/src/main/scala/cats/effect/tracing/TraceFrame.scala b/core/shared/src/main/scala/cats/effect/tracing/TraceFrame.scala index 78c94b4070..50e7fa4eb4 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/TraceFrame.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/TraceFrame.scala @@ -16,4 +16,4 @@ package cats.effect.tracing -final case class TraceFrame(lines: List[TraceLine]) +final case class TraceFrame(op: String, line: Option[TraceLine]) From 09da37fdab464f4c8f6576abf19f558974b59a35 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Fri, 1 May 2020 20:56:24 -0500 Subject: [PATCH 19/78] default to false --- .../main/java/cats/effect/internals/TracingPlatformFast.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java index c6d0c43661..b8fa3224f8 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java @@ -33,6 +33,6 @@ public class TracingPlatformFast { public static final boolean tracingEnabled = Optional.ofNullable(System.getProperty("cats.effect.tracing.enabled")) .filter(x -> !x.isEmpty()) .map(x -> Boolean.valueOf(x)) - .orElse(true); + .orElse(false); } From 1cbe1f561313ed4587fc0d4d2b188ca49f21ed0d Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Sun, 10 May 2020 04:10:11 -0500 Subject: [PATCH 20/78] address pr feedback wip Delete SingleMapCallBenchmark.scala Delete SingleMapCallBenchmark.scala --- .../benchmarks/SingleMapCallBenchmark.scala | 5 -- .../effect/internals/TracingPlatformFast.java | 2 +- .../scala/cats/effect/internals/Example.scala | 23 ++++++++ .../src/main/scala/cats/effect/IO.scala | 19 ++++--- .../cats/effect/internals/IOBracket.scala | 6 ++- .../cats/effect/internals/IORunLoop.scala | 13 +++-- .../cats/effect/internals/IOTracing.scala | 54 ++++++++++--------- 7 files changed, 75 insertions(+), 47 deletions(-) delete mode 100644 benchmarks/shared/src/main/scala/cats/effect/benchmarks/SingleMapCallBenchmark.scala diff --git a/benchmarks/shared/src/main/scala/cats/effect/benchmarks/SingleMapCallBenchmark.scala b/benchmarks/shared/src/main/scala/cats/effect/benchmarks/SingleMapCallBenchmark.scala deleted file mode 100644 index 055595623b..0000000000 --- a/benchmarks/shared/src/main/scala/cats/effect/benchmarks/SingleMapCallBenchmark.scala +++ /dev/null @@ -1,5 +0,0 @@ -package cats.effect.benchmarks - -class SingleMapCallBenchmark { - -} diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java index b8fa3224f8..c6d0c43661 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java @@ -33,6 +33,6 @@ public class TracingPlatformFast { public static final boolean tracingEnabled = Optional.ofNullable(System.getProperty("cats.effect.tracing.enabled")) .filter(x -> !x.isEmpty()) .map(x -> Boolean.valueOf(x)) - .orElse(false); + .orElse(true); } diff --git a/core/jvm/src/main/scala/cats/effect/internals/Example.scala b/core/jvm/src/main/scala/cats/effect/internals/Example.scala index a2efa4f99f..91124c4bcc 100644 --- a/core/jvm/src/main/scala/cats/effect/internals/Example.scala +++ b/core/jvm/src/main/scala/cats/effect/internals/Example.scala @@ -20,6 +20,29 @@ import cats.effect.{ExitCode, IO, IOApp} object Example extends IOApp { + /* + The output of this program should be: + 1 + 2 + 3 + 7 + 8 + 4 + 5 + IOTrace + map at org.simpleapp.example.Example$.$anonfun$program$8 (Example.scala:42) + bind at org.simpleapp.example.Example$.$anonfun$program$7 (Example.scala:41) + map at org.simpleapp.example.Example$.$anonfun$program2$1 (Example.scala:29) + bind at org.simpleapp.example.Example$.program2 (Example.scala:28) + bind at org.simpleapp.example.Example$.$anonfun$program$4 (Example.scala:39) + async at org.simpleapp.example.Example$.$anonfun$program$3 (Example.scala:40) + bind at org.simpleapp.example.Example$.$anonfun$program$3 (Example.scala:37) + bind at org.simpleapp.example.Example$.$anonfun$program$2 (Example.scala:36) + bind at org.simpleapp.example.Example$.$anonfun$program$1 (Example.scala:35) + bind at org.simpleapp.example.Example$.program (Example.scala:34) + bind at org.simpleapp.example.Example$.run (Example.scala:47) + */ + def print(msg: String): IO[Unit] = IO.delay(println(msg)) diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index 95b55a85db..bdd620a741 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -102,11 +102,11 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * failures would be completely silent and `IO` references would * never terminate on evaluation. */ - final def map[B](f: A => B): IO[B] = { + final def map[B](f: A => B): IO[B] = if (tracingEnabled) { // Don't perform map fusion when tracing is enabled. // We may not actually have to do this - IOTracing(Map(this, f, 0), f) + IOTracing(Map(this, f, 0), f.getClass) } else { this match { case Map(source, g, index) => @@ -119,7 +119,6 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { Map(this, f, 0) } } - } /** * Monadic bind on `IO`, used for sequentially composing two `IO` @@ -139,7 +138,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { final def flatMap[B](f: A => IO[B]): IO[B] = { val nextIo = Bind(this, f) if (tracingEnabled) { - IOTracing(nextIo, f) + IOTracing(nextIo, f.getClass) } else { nextIo } @@ -797,14 +796,14 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { def slugTrace: IO[A] = if (tracingEnabled) { - IOTracing.tracedLocally(this, TracingMode.Slug) + IOTracing.locallyTraced(this, TracingMode.Slug) } else { this } def rabbitTrace: IO[A] = if (tracingEnabled) { - IOTracing.tracedLocally(this, TracingMode.Rabbit) + IOTracing.locallyTraced(this, TracingMode.Rabbit) } else { this } @@ -1237,7 +1236,7 @@ object IO extends IOInstances { } if (tracingEnabled) { - IOTracing(nextIo, k) + IOTracing(nextIo, k.getClass) } else { nextIo } @@ -1282,7 +1281,7 @@ object IO extends IOInstances { } if (tracingEnabled) { - IOTracing(nextIo, k) + IOTracing(nextIo, k.getClass) } else { nextIo } @@ -1349,7 +1348,7 @@ object IO extends IOInstances { } if (tracingEnabled) { - IOTracing(nextIo, k) + IOTracing(nextIo, k.getClass) } else { nextIo } @@ -1604,7 +1603,7 @@ object IO extends IOInstances { def contextShift(ec: ExecutionContext): ContextShift[IO] = IOContextShift(ec) - def backtrace: IO[IOTrace] = + val backtrace: IO[IOTrace] = Introspect /* -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-= */ diff --git a/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala b/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala index aa9a69a5a0..a9f153b723 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala @@ -46,7 +46,7 @@ private[effect] object IOBracket { val mode = if (tracingEnabled) { IOTracing.getLocalTracingMode() } else { - TracingMode.Disabled + TracingDisabled } IORunLoop.restart[A](acquire, ctx, mode, new BracketStart(use, release, conn, ctx, mode, deferredRelease, cb)) } else { @@ -55,7 +55,7 @@ private[effect] object IOBracket { } if (tracingEnabled) { - IOTracing(nextIo, use) + IOTracing(nextIo, use.getClass) } else { nextIo } @@ -209,4 +209,6 @@ private[effect] object IOBracket { old.pop() old } + + private[this] val TracingDisabled: TracingMode = TracingMode.Disabled } diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index 3c010a9a2d..0f9389acce 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -27,7 +27,6 @@ private[effect] object IORunLoop { private type Current = IO[Any] private type Bind = Any => IO[Any] private type CallStack = ArrayStack[Bind] - // TODO: replace with a mutable ring buffer private type Callback = Either[Throwable, Any] => Unit /** @@ -36,7 +35,7 @@ private[effect] object IORunLoop { */ def start[A](source: IO[A], cb: Either[Throwable, A] => Unit): Unit = { if (tracingEnabled) { - IOTracing.setLocalTracingMode(TracingMode.Disabled) + IOTracing.setLocalTracingMode(TracingDisabled) } loop(source, IOConnection.uncancelable, cb.asInstanceOf[Callback], null, null, null, null) } @@ -54,12 +53,16 @@ private[effect] object IORunLoop { */ def startCancelable[A](source: IO[A], conn: IOConnection, cb: Either[Throwable, A] => Unit): Unit = { if (tracingEnabled) { - IOTracing.setLocalTracingMode(TracingMode.Disabled) + IOTracing.setLocalTracingMode(TracingDisabled) } loop(source, conn, cb.asInstanceOf[Callback], null, null, null, null) } - def restartCancelable[A](source: IO[A], conn: IOConnection, ctx: IOContext, mode: TracingMode, cb: Either[Throwable, A] => Unit): Unit = { + def restartCancelable[A](source: IO[A], + conn: IOConnection, + ctx: IOContext, + mode: TracingMode, + cb: Either[Throwable, A] => Unit): Unit = { if (tracingEnabled) { IOTracing.setLocalTracingMode(mode) } @@ -481,4 +484,6 @@ private[effect] object IORunLoop { * cancelled status, to interrupt synchronous flatMap loops. */ private[this] val maxAutoCancelableBatchSize = 512 + + private[this] val TracingDisabled = TracingMode.Disabled } diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index 79caadacf4..19d84b5b6b 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -23,25 +23,32 @@ import cats.effect.tracing.{TraceFrame, TraceLine, TracingMode} private[effect] object IOTracing { - def apply[A](source: IO[A], ref: AnyRef): IO[A] = + def apply[A](source: IO[A], clazz: Class[_]): IO[A] = localTracingMode.get() match { case TracingMode.Disabled => source - case TracingMode.Rabbit => IO.Trace(source, buildCachedFrame(source, ref)) + case TracingMode.Rabbit => IO.Trace(source, buildCachedFrame(source, clazz)) case TracingMode.Slug => IO.Trace(source, buildFrame(source)) } - def tracedLocally[A](source: IO[A], mode: TracingMode): IO[A] = + def locallyTraced[A](source: IO[A], newMode: TracingMode): IO[A] = IO.suspend { - val old = localTracingMode.get() - localTracingMode.set(mode) - - // Rethrow any exceptions that `source` produces after resettubg nide, - // In the event of cancellation, the mode will be reset when the - // thread grabs a new task to run (via Async). - source.attempt.flatMap { e => - localTracingMode.set(old) - IO.fromEither(e) - } + val oldMode = localTracingMode.get() + localTracingMode.set(newMode) + + // In the event of cancellation, the tracing mode will be reset + // when the thread grabs a new task to run (via Async). + source.redeemWith( + e => + IO.suspend { + localTracingMode.set(oldMode) + IO.raiseError(e) + }, + a => + IO.suspend { + localTracingMode.set(oldMode) + IO.pure(a) + } + ) } def getLocalTracingMode(): TracingMode = @@ -50,11 +57,11 @@ private[effect] object IOTracing { def setLocalTracingMode(mode: TracingMode): Unit = localTracingMode.set(mode) - private def buildCachedFrame(source: IO[Any], lambdaRef: AnyRef): TraceFrame = { - val cachedFr = frameCache.get(lambdaRef) + private def buildCachedFrame(source: IO[Any], clazz: Class[_]): TraceFrame = { + val cachedFr = frameCache.get(clazz) if (cachedFr eq null) { val fr = buildFrame(source) - frameCache.put(lambdaRef, fr) + frameCache.put(clazz, fr) fr } else { cachedFr @@ -69,10 +76,10 @@ private[effect] object IOTracing { .headOption val op = source match { - case _: IO.Map[_, _] => "map" + case _: IO.Map[_, _] => "map" case _: IO.Bind[_, _] => "bind" - case _: IO.Async[_] => "async" - case _ => "unknown" + case _: IO.Async[_] => "async" + case _ => "unknown" } TraceFrame(op, line) @@ -80,15 +87,12 @@ private[effect] object IOTracing { /** * Cache for trace frames. Keys are references to: - * - lambdas - * - * TODO: Consider thread-local or a regular, mutable map. - * TODO: LRU entry-bounded cache. + * - lambda classes */ - private val frameCache: ConcurrentHashMap[AnyRef, TraceFrame] = new ConcurrentHashMap[AnyRef, TraceFrame]() + private val frameCache: ConcurrentHashMap[Class[_], TraceFrame] = new ConcurrentHashMap[Class[_], TraceFrame]() private val localTracingMode: ThreadLocal[TracingMode] = new ThreadLocal[TracingMode] { - override def initialValue(): TracingMode = TracingMode.Disabled + override def initialValue(): TracingMode = TracingMode.Rabbit } private val classBlacklist = List( From 3a2c39eb55d69e4eadecf7763b0079596f2d07d1 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Sun, 10 May 2020 04:14:49 -0500 Subject: [PATCH 21/78] final class --- .../main/java/cats/effect/internals/TracingPlatformFast.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java index c6d0c43661..549266e383 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java @@ -23,7 +23,7 @@ * Since this flag is read during construction of IO nodes, we are opting to * hold this flag in a Java class to bypass the volatile read. */ -public class TracingPlatformFast { +public final class TracingPlatformFast { /** * A boolean flag that enables or disables tracing for a JVM process. From 0772c7ea6a4d90750d65aaecb46ba0bfa61567dc Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Mon, 11 May 2020 21:28:28 -0500 Subject: [PATCH 22/78] isTracingEnabled --- .../effect/internals/TracingPlatformFast.java | 2 +- .../shared/src/main/scala/cats/effect/IO.scala | 16 ++++++++-------- .../cats/effect/internals/IOBracket.scala | 6 +++--- .../cats/effect/internals/IORunLoop.scala | 18 +++++++++--------- 4 files changed, 21 insertions(+), 21 deletions(-) diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java index 549266e383..ca03914cb0 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java @@ -30,7 +30,7 @@ public final class TracingPlatformFast { * Since it is declared static and final, the JIT compiler has the liberty * to completely eliminate code paths consequent to the conditional. */ - public static final boolean tracingEnabled = Optional.ofNullable(System.getProperty("cats.effect.tracing.enabled")) + public static final boolean isTracingEnabled = Optional.ofNullable(System.getProperty("cats.effect.tracing.enabled")) .filter(x -> !x.isEmpty()) .map(x -> Boolean.valueOf(x)) .orElse(true); diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index bdd620a741..8b37c0ac31 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -27,7 +27,7 @@ import scala.util.control.NonFatal import scala.util.{Failure, Left, Right, Success, Try} import cats.data.Ior import cats.effect.tracing.{IOTrace, TraceFrame, TracingMode} -import cats.effect.internals.TracingPlatformFast.tracingEnabled +import cats.effect.internals.TracingPlatformFast.isTracingEnabled /** * A pure abstraction representing the intention to perform a @@ -103,7 +103,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * never terminate on evaluation. */ final def map[B](f: A => B): IO[B] = - if (tracingEnabled) { + if (isTracingEnabled) { // Don't perform map fusion when tracing is enabled. // We may not actually have to do this IOTracing(Map(this, f, 0), f.getClass) @@ -137,7 +137,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { */ final def flatMap[B](f: A => IO[B]): IO[B] = { val nextIo = Bind(this, f) - if (tracingEnabled) { + if (isTracingEnabled) { IOTracing(nextIo, f.getClass) } else { nextIo @@ -795,14 +795,14 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { p.parProductL(this)(another) def slugTrace: IO[A] = - if (tracingEnabled) { + if (isTracingEnabled) { IOTracing.locallyTraced(this, TracingMode.Slug) } else { this } def rabbitTrace: IO[A] = - if (tracingEnabled) { + if (isTracingEnabled) { IOTracing.locallyTraced(this, TracingMode.Rabbit) } else { this @@ -1235,7 +1235,7 @@ object IO extends IOInstances { catch { case NonFatal(t) => cb2(Left(t)) } } - if (tracingEnabled) { + if (isTracingEnabled) { IOTracing(nextIo, k.getClass) } else { nextIo @@ -1280,7 +1280,7 @@ object IO extends IOInstances { IORunLoop.startCancelable(fa, conn2, Callback.report) } - if (tracingEnabled) { + if (isTracingEnabled) { IOTracing(nextIo, k.getClass) } else { nextIo @@ -1347,7 +1347,7 @@ object IO extends IOInstances { ref.complete(IO.unit) } - if (tracingEnabled) { + if (isTracingEnabled) { IOTracing(nextIo, k.getClass) } else { nextIo diff --git a/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala b/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala index a9f153b723..20b036495b 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala @@ -25,7 +25,7 @@ import scala.util.control.NonFatal import java.util.concurrent.atomic.AtomicBoolean import cats.effect.tracing.TracingMode -import cats.effect.internals.TracingPlatformFast.tracingEnabled +import cats.effect.internals.TracingPlatformFast.isTracingEnabled private[effect] object IOBracket { @@ -43,7 +43,7 @@ private[effect] object IOBracket { if (!conn.isCanceled) { // Note `acquire` is uncancelable due to usage of `IORunLoop.start` // (in other words it is disconnected from our IOConnection) - val mode = if (tracingEnabled) { + val mode = if (isTracingEnabled) { IOTracing.getLocalTracingMode() } else { TracingDisabled @@ -54,7 +54,7 @@ private[effect] object IOBracket { } } - if (tracingEnabled) { + if (isTracingEnabled) { IOTracing(nextIo, use.getClass) } else { nextIo diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index 0f9389acce..3da207d779 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -19,7 +19,7 @@ package cats.effect.internals import cats.effect.IO import cats.effect.IO.{Async, Bind, ContextSwitch, Delay, Introspect, Map, Pure, RaiseError, Suspend, Trace} import cats.effect.tracing.TracingMode -import cats.effect.internals.TracingPlatformFast.tracingEnabled +import cats.effect.internals.TracingPlatformFast.isTracingEnabled import scala.util.control.NonFatal @@ -34,14 +34,14 @@ private[effect] object IORunLoop { * with the result when completed. */ def start[A](source: IO[A], cb: Either[Throwable, A] => Unit): Unit = { - if (tracingEnabled) { + if (isTracingEnabled) { IOTracing.setLocalTracingMode(TracingDisabled) } loop(source, IOConnection.uncancelable, cb.asInstanceOf[Callback], null, null, null, null) } def restart[A](source: IO[A], ctx: IOContext, mode: TracingMode, cb: Either[Throwable, A] => Unit): Unit = { - if (tracingEnabled) { + if (isTracingEnabled) { IOTracing.setLocalTracingMode(mode) } loop(source, IOConnection.uncancelable, cb.asInstanceOf[Callback], ctx, null, null, null) @@ -52,7 +52,7 @@ private[effect] object IORunLoop { * with the result when completed. */ def startCancelable[A](source: IO[A], conn: IOConnection, cb: Either[Throwable, A] => Unit): Unit = { - if (tracingEnabled) { + if (isTracingEnabled) { IOTracing.setLocalTracingMode(TracingDisabled) } loop(source, conn, cb.asInstanceOf[Callback], null, null, null, null) @@ -63,7 +63,7 @@ private[effect] object IORunLoop { ctx: IOContext, mode: TracingMode, cb: Either[Throwable, A] => Unit): Unit = { - if (tracingEnabled) { + if (isTracingEnabled) { IOTracing.setLocalTracingMode(mode) } loop(source, conn, cb.asInstanceOf[Callback], ctx, null, null, null) @@ -106,7 +106,7 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } -// if (tracingEnabled) { +// if (isTracingEnabled) { // if (ctx eq null) ctx = IOContext() // ctx.pushFrame(bind.trace) // } @@ -150,7 +150,7 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } -// if (tracingEnabled) { +// if (isTracingEnabled) { // if (ctx eq null) ctx = IOContext() // ctx.pushFrame(bindNext.trace) // } @@ -417,7 +417,7 @@ private[effect] object IORunLoop { this.bFirst = bFirst this.bRest = bRest this.trampolineAfter = task.trampolineAfter - if (tracingEnabled) { + if (isTracingEnabled) { this.tMode = IOTracing.getLocalTracingMode() } @@ -432,7 +432,7 @@ private[effect] object IORunLoop { this.bFirst = null this.bRest = null - if (tracingEnabled) { + if (isTracingEnabled) { // The continuation may have been invoked on a new execution context, // so let's recover the tracing mode here. IOTracing.setLocalTracingMode(this.tMode) From abce87fb9d8c3f7209fb3c536dd82bfd9ebc96af Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Sat, 16 May 2020 16:42:12 -0500 Subject: [PATCH 23/78] Mutable, thread unsafe IOContext --- .../cats/effect/internals/IOBracket.scala | 52 +++++++++++++------ .../cats/effect/internals/IOContext.scala | 14 ++--- .../cats/effect/internals/IOTracing.scala | 1 + 3 files changed, 41 insertions(+), 26 deletions(-) diff --git a/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala b/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala index 20b036495b..56fbab1faa 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala @@ -16,6 +16,7 @@ package cats.effect.internals +import cats.implicits._ import cats.effect.IO.ContextSwitch import cats.effect.{CancelToken, ExitCase, IO} import cats.effect.internals.TrampolineEC.immediate @@ -29,6 +30,8 @@ import cats.effect.internals.TracingPlatformFast.isTracingEnabled private[effect] object IOBracket { + private[this] type Acquire[A] = (A, IOContext) + /** * Implementation for `IO.bracketCase`. */ @@ -41,14 +44,17 @@ private[effect] object IOBracket { // was cancelled already, to ensure that `cancel` really blocks if we // start `acquire` — n.b. `isCanceled` is visible here due to `push` if (!conn.isCanceled) { - // Note `acquire` is uncancelable due to usage of `IORunLoop.start` + // Note `acquireWithContext` is uncancelable due to usage of `IORunLoop.restart` // (in other words it is disconnected from our IOConnection) - val mode = if (isTracingEnabled) { - IOTracing.getLocalTracingMode() - } else { - TracingDisabled - } - IORunLoop.restart[A](acquire, ctx, mode, new BracketStart(use, release, conn, ctx, mode, deferredRelease, cb)) + val acquireWithContext = acquire.product(ioContext) + val tracingMode = activeTracingMode + + IORunLoop.restart[Acquire[A]]( + acquireWithContext, + ctx, + tracingMode, + new BracketStart(use, release, conn, tracingMode, deferredRelease, cb) + ) } else { deferredRelease.complete(IO.unit) } @@ -66,18 +72,17 @@ private[effect] object IOBracket { use: A => IO[B], release: (A, ExitCase[Throwable]) => IO[Unit], conn: IOConnection, - ctx: IOContext, - mode: TracingMode, + tracingMode: TracingMode, deferredRelease: ForwardCancelable, cb: Callback.T[B] - ) extends (Either[Throwable, A] => Unit) + ) extends (Either[Throwable, Acquire[A]] => Unit) with Runnable { // This runnable is a dirty optimization to avoid some memory allocations; // This class switches from being a Callback to a Runnable, but relies on // the internal IO callback protocol to be respected (called at most once) - private[this] var result: Either[Throwable, A] = _ + private[this] var result: Either[Throwable, Acquire[A]] = _ - def apply(ea: Either[Throwable, A]): Unit = { + def apply(ea: Either[Throwable, Acquire[A]]): Unit = { if (result ne null) { throw new IllegalStateException("callback called multiple times!") } @@ -90,7 +95,7 @@ private[effect] object IOBracket { def run(): Unit = result match { case Right(a) => - val frame = new BracketReleaseFrame[A, B](a, release) + val frame = new BracketReleaseFrame[A, B](a._1, release) // Registering our cancelable token ensures that in case // cancellation is detected, `release` gets called @@ -100,12 +105,12 @@ private[effect] object IOBracket { if (!conn.isCanceled) { val onNext = { val fb = - try use(a) + try use(a._1) catch { case NonFatal(e) => IO.raiseError(e) } fb.flatMap(frame) } // Actual execution - IORunLoop.restartCancelable(onNext, conn, ctx, mode, cb) + IORunLoop.restartCancelable(onNext, conn, a._2, tracingMode, cb) } case error @ Left(_) => @@ -118,7 +123,7 @@ private[effect] object IOBracket { * Implementation for `IO.guaranteeCase`. */ def guaranteeCase[A](source: IO[A], release: ExitCase[Throwable] => IO[Unit]): IO[A] = - IO.Async { (conn, _, cb) => + IO.Async { (conn, ctx, cb) => // Light async boundary, otherwise this will trigger a StackOverflowException ec.execute(new Runnable { def run(): Unit = { @@ -131,10 +136,11 @@ private[effect] object IOBracket { // the connection was already cancelled — n.b. we don't need // to trigger `release` otherwise, because it already happened if (!conn.isCanceled) { - IORunLoop.startCancelable(onNext, conn, cb) + IORunLoop.restartCancelable(onNext, conn, ctx, activeTracingMode, cb) } } }) + // TODO: Trace here? } final private class BracketReleaseFrame[A, B](a: A, releaseFn: (A, ExitCase[Throwable]) => IO[Unit]) @@ -210,5 +216,17 @@ private[effect] object IOBracket { old } + private[this] val ioContext: IO[IOContext] = + IO.Async { (_, ctx, cb) => + cb(Right(ctx)) + } + private[this] val TracingDisabled: TracingMode = TracingMode.Disabled + + private def activeTracingMode: TracingMode = + if (isTracingEnabled) { + IOTracing.getLocalTracingMode() + } else { + TracingDisabled + } } diff --git a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala index f631d0b2b5..197e6504a2 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala @@ -19,20 +19,16 @@ package cats.effect.internals import cats.effect.tracing.{IOTrace, TraceFrame} /** - * IOContext holds state related to the execution of an IO and - * should be threaded across multiple invocations of the run-loop - * for the same fiber. + * INTERNAL API — Holds state related to the execution of + * an IO and should be threaded across multiple invocations + * of the run-loop associated with the same fiber. */ final private[effect] class IOContext private () { - // We have to use a volatile here because of IOBracket implementation - // and how it invokes a new run-loop "asynchronously." - // Ideally we could use a mutable, ring buffer here. - @volatile var frames: Vector[TraceFrame] = Vector.empty + // TODO: Replace this with a performant mutable ring buffer + private var frames: Vector[TraceFrame] = Vector.empty def pushFrame(fr: TraceFrame): Unit = { - // Accessed from at most one thread at a time, - // so no race condition will occur val currFrames = frames if (currFrames.length >= 1000) { frames = fr +: currFrames.dropRight(1) diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index 19d84b5b6b..d4ddef3c3c 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -97,6 +97,7 @@ private[effect] object IOTracing { private val classBlacklist = List( "cats.effect.", + "cats.", "sbt.", "java.", "sun.", From 7556b2ce50fc6c8c561a79303833350ffdd85a0c Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Sun, 17 May 2020 16:26:21 -0500 Subject: [PATCH 24/78] Ring buffer to back trace frames --- .../effect/internals/TracingPlatformFast.java | 16 +++++ .../scala/cats/effect/internals/Example.scala | 24 +++---- .../cats/effect/internals/IOContext.scala | 15 ++-- .../cats/effect/internals/RingBuffer.scala | 71 +++++++++++++++++++ .../scala/cats/effect/tracing/IOTrace.scala | 4 +- 5 files changed, 108 insertions(+), 22 deletions(-) create mode 100644 core/shared/src/main/scala/cats/effect/internals/RingBuffer.scala diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java index ca03914cb0..1ac6c26781 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java @@ -35,4 +35,20 @@ public final class TracingPlatformFast { .map(x -> Boolean.valueOf(x)) .orElse(true); + /** + * The number of trace lines to retain during tracing. If more trace + * lines are produced, then the oldest trace lines will be discarded. + * Automatically rounded up to the nearest power of 2. + */ + public static final int maxTraceFrameSize = Optional.ofNullable(System.getProperty("cats.effect.tracing.maxTraceFrameSize")) + .filter(x -> !x.isEmpty()) + .flatMap(x -> { + try { + return Optional.of(Integer.valueOf(x)); + } catch (Exception e) { + return Optional.empty(); + } + }) + .orElse(512); + } diff --git a/core/jvm/src/main/scala/cats/effect/internals/Example.scala b/core/jvm/src/main/scala/cats/effect/internals/Example.scala index 91124c4bcc..0c8f287791 100644 --- a/core/jvm/src/main/scala/cats/effect/internals/Example.scala +++ b/core/jvm/src/main/scala/cats/effect/internals/Example.scala @@ -29,18 +29,18 @@ object Example extends IOApp { 8 4 5 - IOTrace - map at org.simpleapp.example.Example$.$anonfun$program$8 (Example.scala:42) - bind at org.simpleapp.example.Example$.$anonfun$program$7 (Example.scala:41) - map at org.simpleapp.example.Example$.$anonfun$program2$1 (Example.scala:29) - bind at org.simpleapp.example.Example$.program2 (Example.scala:28) - bind at org.simpleapp.example.Example$.$anonfun$program$4 (Example.scala:39) - async at org.simpleapp.example.Example$.$anonfun$program$3 (Example.scala:40) - bind at org.simpleapp.example.Example$.$anonfun$program$3 (Example.scala:37) - bind at org.simpleapp.example.Example$.$anonfun$program$2 (Example.scala:36) - bind at org.simpleapp.example.Example$.$anonfun$program$1 (Example.scala:35) - bind at org.simpleapp.example.Example$.program (Example.scala:34) - bind at org.simpleapp.example.Example$.run (Example.scala:47) + IOTrace: 0 omitted frames + bind at org.simpleapp.example.Example$.run (Example.scala:70) + bind at org.simpleapp.example.Example$.program (Example.scala:57) + bind at org.simpleapp.example.Example$.$anonfun$program$1 (Example.scala:58) + bind at org.simpleapp.example.Example$.$anonfun$program$2 (Example.scala:59) + bind at org.simpleapp.example.Example$.$anonfun$program$3 (Example.scala:60) + async at org.simpleapp.example.Example$.$anonfun$program$3 (Example.scala:63) + bind at org.simpleapp.example.Example$.$anonfun$program$4 (Example.scala:62) + bind at org.simpleapp.example.Example$.program2 (Example.scala:51) + map at org.simpleapp.example.Example$.$anonfun$program2$1 (Example.scala:52) + bind at org.simpleapp.example.Example$.$anonfun$program$7 (Example.scala:64) + map at org.simpleapp.example.Example$.$anonfun$program$8 (Example.scala:65) */ def print(msg: String): IO[Unit] = diff --git a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala index 197e6504a2..78766c8f1e 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala @@ -17,6 +17,7 @@ package cats.effect.internals import cats.effect.tracing.{IOTrace, TraceFrame} +import cats.effect.internals.TracingPlatformFast.maxTraceFrameSize /** * INTERNAL API — Holds state related to the execution of @@ -25,20 +26,18 @@ import cats.effect.tracing.{IOTrace, TraceFrame} */ final private[effect] class IOContext private () { - // TODO: Replace this with a performant mutable ring buffer - private var frames: Vector[TraceFrame] = Vector.empty + private val frames: RingBuffer[TraceFrame] = new RingBuffer(maxTraceFrameSize) + private var omitted: Int = 0 def pushFrame(fr: TraceFrame): Unit = { - val currFrames = frames - if (currFrames.length >= 1000) { - frames = fr +: currFrames.dropRight(1) - } else { - frames = fr +: currFrames + val a = frames.push(fr) + if (a != null) { + omitted += 1 } } def getTrace: IOTrace = - IOTrace(frames) + IOTrace(frames.toList.toVector, omitted) } diff --git a/core/shared/src/main/scala/cats/effect/internals/RingBuffer.scala b/core/shared/src/main/scala/cats/effect/internals/RingBuffer.scala new file mode 100644 index 0000000000..8673820b19 --- /dev/null +++ b/core/shared/src/main/scala/cats/effect/internals/RingBuffer.scala @@ -0,0 +1,71 @@ +/* + * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers + * + * Licensed 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 cats.effect.internals + +/** + * Provides a fast, mutable ring buffer. + * + * INTERNAL API. + */ +final private[internals] class RingBuffer[A <: AnyRef](size: Int) { + + import RingBuffer._ + + private[this] val capacity = nextPowerOfTwo(size) + private[this] val mask = capacity - 1 + + // TODO: this can be an expensive allocation + // either construct it lazily or expand it on-demand + private[this] val array: Array[AnyRef] = new Array(capacity) + private[this] var writeIndex: Int = 0 + private[this] var readIndex: Int = 0 + + def push(a: A): A = { + val wi = writeIndex & mask + if (writeIndex == readIndex + capacity) { + val old = array(wi) + array(wi) = a + // TODO: overflow at int.maxvalue? + writeIndex = writeIndex + 1 + readIndex = readIndex + 1 + old.asInstanceOf[A] + } else { + array(wi) = a + writeIndex = writeIndex + 1 + null.asInstanceOf[A] + } + } + + // TODO: expose this as an iterator instead? + def toList: List[A] = + (readIndex until writeIndex).toList + .map(i => array(i & mask).asInstanceOf[A]) + +} + +object RingBuffer { + + // TODO: bounds check at int.maxvalue ? + private def nextPowerOfTwo(i: Int): Int = { + var n = 1 + while (n < i) { + n = n * 2 + } + n + } + +} diff --git a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala index b9e74ab4d0..83071d68a7 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala @@ -16,10 +16,10 @@ package cats.effect.tracing -final case class IOTrace(frames: Vector[TraceFrame]) { +final case class IOTrace(frames: Vector[TraceFrame], omitted: Int) { def printTrace(): Unit = { - System.err.println("IOTrace") + System.err.println(s"IOTrace: $omitted omitted frames") frames.foreach { f => f.line.foreach { l => System.err.println(s"\t${f.op} at ${l.className}.${l.methodName} (${l.fileName}:${l.lineNumber})") From 4f4dbd56d16eb884abe94efcd8a06361aedd2a8d Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Sat, 23 May 2020 18:17:43 -0500 Subject: [PATCH 25/78] Hold trace in constructor argument --- .../effect/benchmarks/DeepBindBenchmark.scala | 22 ++--- .../effect/benchmarks/MapCallsBenchmark.scala | 12 +-- .../effect/internals/TracingPlatformFast.java | 9 ++ .../src/main/scala/cats/effect/Async.scala | 8 +- .../main/scala/cats/effect/Concurrent.scala | 8 +- .../src/main/scala/cats/effect/IO.scala | 97 +++++++++++-------- .../src/main/scala/cats/effect/SyncIO.scala | 2 +- .../cats/effect/internals/IOBracket.scala | 11 ++- .../cats/effect/internals/IOForkedStart.scala | 4 +- .../cats/effect/internals/IORunLoop.scala | 36 +++---- .../cats/effect/internals/IOTracing.scala | 51 ++++------ .../cats/effect/tracing/TracingMode.scala | 19 ++-- 12 files changed, 139 insertions(+), 140 deletions(-) diff --git a/benchmarks/shared/src/main/scala/cats/effect/benchmarks/DeepBindBenchmark.scala b/benchmarks/shared/src/main/scala/cats/effect/benchmarks/DeepBindBenchmark.scala index 62efa04967..c7ce40d2be 100644 --- a/benchmarks/shared/src/main/scala/cats/effect/benchmarks/DeepBindBenchmark.scala +++ b/benchmarks/shared/src/main/scala/cats/effect/benchmarks/DeepBindBenchmark.scala @@ -65,15 +65,15 @@ class DeepBindBenchmark { loop(0).unsafeRunSync() } - @Benchmark - def async(): Int = { - def loop(i: Int): IO[Int] = - for { - j <- IO(i) - _ <- IO.shift - _ <- if (j > size) IO(j) else loop(j + 1) - } yield j - - loop(0).unsafeRunSync() - } +// @Benchmark +// def async(): Int = { +// def loop(i: Int): IO[Int] = +// for { +// j <- IO(i) +// _ <- IO.shift +// _ <- if (j > size) IO(j) else loop(j + 1) +// } yield j +// +// loop(0).unsafeRunSync() +// } } diff --git a/benchmarks/shared/src/main/scala/cats/effect/benchmarks/MapCallsBenchmark.scala b/benchmarks/shared/src/main/scala/cats/effect/benchmarks/MapCallsBenchmark.scala index 6105d4818e..2474be2933 100644 --- a/benchmarks/shared/src/main/scala/cats/effect/benchmarks/MapCallsBenchmark.scala +++ b/benchmarks/shared/src/main/scala/cats/effect/benchmarks/MapCallsBenchmark.scala @@ -40,13 +40,13 @@ class MapCallsBenchmark { import MapCallsBenchmark.test @Benchmark - def one(): Long = test(12000, 1) + def one(): Long = test(1, 1) - @Benchmark - def batch30(): Long = test(12000 / 30, 30) - - @Benchmark - def batch120(): Long = test(12000 / 120, 120) +// @Benchmark +// def batch30(): Long = test(12000 / 30, 30) +// +// @Benchmark +// def batch120(): Long = test(12000 / 120, 120) } object MapCallsBenchmark { diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java index 1ac6c26781..4fd12043b2 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java @@ -17,6 +17,7 @@ package cats.effect.internals; import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; /** * Scala object field accesses cost a volatile read across modules. @@ -51,4 +52,12 @@ public final class TracingPlatformFast { }) .orElse(512); + /** + * Cache for trace frames. Keys are references to: + * - lambda classes + */ + public static final ConcurrentHashMap, Object> frameCache = new ConcurrentHashMap<>(); + + public static final ThreadLocal localTracingMode = ThreadLocal.withInitial(() -> 1); + } diff --git a/core/shared/src/main/scala/cats/effect/Async.scala b/core/shared/src/main/scala/cats/effect/Async.scala index 8007f65382..d8085c40f1 100644 --- a/core/shared/src/main/scala/cats/effect/Async.scala +++ b/core/shared/src/main/scala/cats/effect/Async.scala @@ -308,13 +308,13 @@ object Async { */ def liftIO[F[_], A](io: IO[A])(implicit F: Async[F]): F[A] = io match { - case Pure(a) => F.pure(a) - case RaiseError(e) => F.raiseError(e) - case Delay(thunk) => F.delay(thunk()) + case Pure(a, _) => F.pure(a) + case RaiseError(e) => F.raiseError(e) + case Delay(thunk, _) => F.delay(thunk()) case _ => F.suspend { IORunLoop.step(io) match { - case Pure(a) => F.pure(a) + case Pure(a, _) => F.pure(a) case RaiseError(e) => F.raiseError(e) case async => F.async(async.unsafeRunAsync) } diff --git a/core/shared/src/main/scala/cats/effect/Concurrent.scala b/core/shared/src/main/scala/cats/effect/Concurrent.scala index 1dcf6f6915..b2d774d3b7 100644 --- a/core/shared/src/main/scala/cats/effect/Concurrent.scala +++ b/core/shared/src/main/scala/cats/effect/Concurrent.scala @@ -406,13 +406,13 @@ object Concurrent { */ def liftIO[F[_], A](ioa: IO[A])(implicit F: Concurrent[F]): F[A] = ioa match { - case Pure(a) => F.pure(a) - case RaiseError(e) => F.raiseError(e) - case Delay(thunk) => F.delay(thunk()) + case Pure(a, _) => F.pure(a) + case RaiseError(e) => F.raiseError(e) + case Delay(thunk, _) => F.delay(thunk()) case _ => F.suspend { IORunLoop.step(ioa) match { - case Pure(a) => F.pure(a) + case Pure(a, _) => F.pure(a) case RaiseError(e) => F.raiseError(e) case async => F.cancelable(cb => liftIO(async.unsafeRunCancelable(cb))(F)) diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index 8b37c0ac31..4719f0b4c5 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -19,6 +19,7 @@ package effect import cats.effect.internals._ import cats.effect.internals.IOPlatform.fusionMaxStackDepth +import cats.effect.internals.TracingPlatformFast.isTracingEnabled import scala.annotation.unchecked.uncheckedVariance import scala.concurrent.duration._ @@ -27,7 +28,6 @@ import scala.util.control.NonFatal import scala.util.{Failure, Left, Right, Success, Try} import cats.data.Ior import cats.effect.tracing.{IOTrace, TraceFrame, TracingMode} -import cats.effect.internals.TracingPlatformFast.isTracingEnabled /** * A pure abstraction representing the intention to perform a @@ -106,17 +106,22 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { if (isTracingEnabled) { // Don't perform map fusion when tracing is enabled. // We may not actually have to do this - IOTracing(Map(this, f, 0), f.getClass) + val trace = if (isTracingEnabled) { + IOTracing(Map.getClass, f.getClass) + } else { + null + } + Map(this, f, 0, trace) } else { this match { - case Map(source, g, index) => + case Map(source, g, index, _) => // Allowed to do fixed number of map operations fused before // resetting the counter in order to avoid stack overflows; // See `IOPlatform` for details on this maximum. - if (index != fusionMaxStackDepth) Map(source, g.andThen(f), index + 1) - else Map(this, f, 0) + if (index != fusionMaxStackDepth) Map(source, g.andThen(f), index + 1, null) + else Map(this, f, 0, null) case _ => - Map(this, f, 0) + Map(this, f, 0, null) } } @@ -136,12 +141,12 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * never terminate on evaluation. */ final def flatMap[B](f: A => IO[B]): IO[B] = { - val nextIo = Bind(this, f) - if (isTracingEnabled) { - IOTracing(nextIo, f.getClass) + val trace = if (isTracingEnabled) { + IOTracing(Bind.getClass, f.getClass) } else { - nextIo + null } + Bind(this, f, trace) } /** @@ -158,7 +163,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * @see [[IO.raiseError]] */ def attempt: IO[Either[Throwable, A]] = - Bind(this, AttemptIO.asInstanceOf[A => IO[Either[Throwable, A]]]) + Bind(this, AttemptIO.asInstanceOf[A => IO[Either[Throwable, A]]], null) /** * Produces an `IO` reference that should execute the source on @@ -339,7 +344,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { */ final def unsafeRunTimed(limit: Duration): Option[A] = IORunLoop.step(this) match { - case Pure(a) => Some(a) + case Pure(a, _) => Some(a) case RaiseError(e) => throw e case self @ Async(_, _) => IOPlatform.unsafeResync(self, limit) @@ -686,7 +691,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * Implements `ApplicativeError.handleErrorWith`. */ def handleErrorWith[AA >: A](f: Throwable => IO[AA]): IO[AA] = - IO.Bind(this, new IOFrame.ErrorHandler(f)) + IO.Bind(this, new IOFrame.ErrorHandler(f), null) /** * Zips both this action and the parameter in parallel. @@ -720,7 +725,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * in case it ends in success */ def redeem[B](recover: Throwable => B, map: A => B): IO[B] = - IO.Bind(this, new IOFrame.Redeem(recover, map)) + IO.Bind(this, new IOFrame.Redeem(recover, map), null) /** * Returns a new value that transforms the result of the source, @@ -752,10 +757,10 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * in case of success */ def redeemWith[B](recover: Throwable => IO[B], bind: A => IO[B]): IO[B] = - IO.Bind(this, new IOFrame.RedeemWith(recover, bind)) + IO.Bind(this, new IOFrame.RedeemWith(recover, bind), null) override def toString: String = this match { - case Pure(a) => s"IO($a)" + case Pure(a, _) => s"IO($a)" case RaiseError(e) => s"IO(throw $e)" case _ => "IO$" + System.identityHashCode(this) } @@ -1140,7 +1145,7 @@ object IO extends IOInstances { * into the `IO`. */ def delay[A](body: => A): IO[A] = - Delay(() => body) + Delay(() => body, null) /** * Suspends a synchronous side effect which produces an `IO` in `IO`. @@ -1163,7 +1168,7 @@ object IO extends IOInstances { * (when evaluated) than `IO(42)`, due to avoiding the allocation of * extra thunks. */ - def pure[A](a: A): IO[A] = Pure(a) + def pure[A](a: A): IO[A] = Pure(a, null) /** Alias for `IO.pure(())`. */ val unit: IO[Unit] = pure(()) @@ -1235,11 +1240,12 @@ object IO extends IOInstances { catch { case NonFatal(t) => cb2(Left(t)) } } - if (isTracingEnabled) { - IOTracing(nextIo, k.getClass) - } else { - nextIo - } +// if (isTracingEnabled) { +// IOTracing(nextIo, k.getClass) +// } else { +// nextIo +// } + nextIo } /** @@ -1280,11 +1286,12 @@ object IO extends IOInstances { IORunLoop.startCancelable(fa, conn2, Callback.report) } - if (isTracingEnabled) { - IOTracing(nextIo, k.getClass) - } else { - nextIo - } +// if (isTracingEnabled) { +// IOTracing(nextIo, k.getClass) +// } else { +// nextIo +// } + nextIo } /** @@ -1347,11 +1354,12 @@ object IO extends IOInstances { ref.complete(IO.unit) } - if (isTracingEnabled) { - IOTracing(nextIo, k.getClass) - } else { - nextIo - } +// if (isTracingEnabled) { +// IOTracing(nextIo, k.getClass) +// } else { +// nextIo +// } + nextIo } /** @@ -1610,10 +1618,10 @@ object IO extends IOInstances { /* IO's internal encoding: */ /** Corresponds to [[IO.pure]]. */ - final private[effect] case class Pure[+A](a: A) extends IO[A] + final private[effect] case class Pure[+A](a: A, trace: TraceFrame) extends IO[A] /** Corresponds to [[IO.apply]]. */ - final private[effect] case class Delay[+A](thunk: () => A) extends IO[A] + final private[effect] case class Delay[+A](thunk: () => A, trace: TraceFrame) extends IO[A] /** Corresponds to [[IO.raiseError]]. */ final private[effect] case class RaiseError(e: Throwable) extends IO[Nothing] @@ -1622,12 +1630,14 @@ object IO extends IOInstances { final private[effect] case class Suspend[+A](thunk: () => IO[A]) extends IO[A] /** Corresponds to [[IO.flatMap]]. */ - final private[effect] case class Bind[E, +A](source: IO[E], f: E => IO[A]) extends IO[A] + final private[effect] case class Bind[E, +A](source: IO[E], f: E => IO[A], trace: TraceFrame) extends IO[A] /** Corresponds to [[IO.map]]. */ - final private[effect] case class Map[E, +A](source: IO[E], f: E => A, index: Int) extends IO[A] with (E => IO[A]) { + final private[effect] case class Map[E, +A](source: IO[E], f: E => A, index: Int, trace: TraceFrame) + extends IO[A] + with (E => IO[A]) { override def apply(value: E): IO[A] = - Pure(f(value)) + Pure(f(value), null) } /** @@ -1648,6 +1658,7 @@ object IO extends IOInstances { k: (IOConnection, IOContext, Either[Throwable, A] => Unit) => Unit, trampolineAfter: Boolean = false ) extends IO[A] + with TracedIO[A] /** * An internal state for that optimizes changes to @@ -1662,10 +1673,12 @@ object IO extends IOInstances { restore: (A, Throwable, IOConnection, IOConnection) => IOConnection ) extends IO[A] - final private[effect] case class Trace[A](source: IO[A], frame: TraceFrame) extends IO[A] - final private[effect] case object Introspect extends IO[IOTrace] + private[effect] trait TracedIO[+A] { self: IO[A] => + private[effect] var trace: TraceFrame = null + } + /* -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-= */ /** @@ -1674,8 +1687,8 @@ object IO extends IOInstances { */ private object AttemptIO extends IOFrame[Any, IO[Either[Throwable, Any]]] { override def apply(a: Any) = - Pure(Right(a)) + Pure(Right(a), null) override def recover(e: Throwable) = - Pure(Left(e)) + Pure(Left(e), null) } } diff --git a/core/shared/src/main/scala/cats/effect/SyncIO.scala b/core/shared/src/main/scala/cats/effect/SyncIO.scala index 63906d42be..d9e8218940 100644 --- a/core/shared/src/main/scala/cats/effect/SyncIO.scala +++ b/core/shared/src/main/scala/cats/effect/SyncIO.scala @@ -313,7 +313,7 @@ final class SyncIO[+A] private (private val io: IO[A]) { new SyncIO(io.redeemWith(t => recover(t).io, a => bind(a).io)) override def toString: String = io match { - case IO.Pure(a) => s"SyncIO($a)" + case IO.Pure(a, _) => s"SyncIO($a)" case IO.RaiseError(e) => s"SyncIO(throw $e)" case _ => "SyncIO$" + System.identityHashCode(this) } diff --git a/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala b/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala index 56fbab1faa..d247f1c31f 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala @@ -60,11 +60,12 @@ private[effect] object IOBracket { } } - if (isTracingEnabled) { - IOTracing(nextIo, use.getClass) - } else { - nextIo - } +// if (isTracingEnabled) { +// IOTracing(nextIo, use.getClass) +// } else { +// nextIo +// } + nextIo } // Internals of `IO.bracketCase`. diff --git a/core/shared/src/main/scala/cats/effect/internals/IOForkedStart.scala b/core/shared/src/main/scala/cats/effect/internals/IOForkedStart.scala index 743565d13d..161eb62770 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOForkedStart.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOForkedStart.scala @@ -54,8 +54,8 @@ private[effect] object IOForkedStart { if (limit > 0) { task match { case IO.Async(k, _) => k.isInstanceOf[IOForkedStart[_]] - case IO.Bind(other, _) => detect(other, limit - 1) - case IO.Map(other, _, _) => detect(other, limit - 1) + case IO.Bind(other, _, _) => detect(other, limit - 1) + case IO.Map(other, _, _, _) => detect(other, limit - 1) case IO.ContextSwitch(other, _, _) => detect(other, limit - 1) case _ => false } diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index 3da207d779..1597308ea8 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -17,7 +17,7 @@ package cats.effect.internals import cats.effect.IO -import cats.effect.IO.{Async, Bind, ContextSwitch, Delay, Introspect, Map, Pure, RaiseError, Suspend, Trace} +import cats.effect.IO.{Async, Bind, ContextSwitch, Delay, Introspect, Map, Pure, RaiseError, Suspend} import cats.effect.tracing.TracingMode import cats.effect.internals.TracingPlatformFast.isTracingEnabled @@ -101,7 +101,7 @@ private[effect] object IORunLoop { while ({ currentIO match { - case Bind(fa, bindNext) => + case Bind(fa, bindNext, _) => if (bFirst ne null) { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) @@ -113,11 +113,11 @@ private[effect] object IORunLoop { bFirst = bindNext.asInstanceOf[Bind] currentIO = fa - case Pure(value) => + case Pure(value, _) => unboxed = value.asInstanceOf[AnyRef] hasUnboxed = true - case Delay(thunk) => + case Delay(thunk, _) => try { unboxed = thunk().asInstanceOf[AnyRef] hasUnboxed = true @@ -145,7 +145,7 @@ private[effect] object IORunLoop { currentIO = fa } - case bindNext @ Map(fa, _, _) => + case bindNext @ Map(fa, _, _, _) => if (bFirst ne null) { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) @@ -173,14 +173,9 @@ private[effect] object IORunLoop { if (conn ne old) { if (rcb ne null) rcb.contextSwitch(conn) if (restore ne null) - currentIO = Bind(next, new RestoreContext(old, restore)) + currentIO = Bind(next, new RestoreContext(old, restore), null) } - case Trace(source, frame) => - if (ctx eq null) ctx = IOContext() - ctx.pushFrame(frame) - currentIO = source - case Introspect => if (ctx eq null) ctx = IOContext() hasUnboxed = true @@ -230,7 +225,7 @@ private[effect] object IORunLoop { while ({ currentIO match { - case Bind(fa, bindNext) => + case Bind(fa, bindNext, _) => if (bFirst ne null) { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) @@ -238,11 +233,11 @@ private[effect] object IORunLoop { bFirst = bindNext.asInstanceOf[Bind] currentIO = fa - case Pure(value) => + case Pure(value, _) => unboxed = value.asInstanceOf[AnyRef] hasUnboxed = true - case Delay(thunk) => + case Delay(thunk, _) => try { unboxed = thunk().asInstanceOf[AnyRef] hasUnboxed = true @@ -270,7 +265,7 @@ private[effect] object IORunLoop { currentIO = fa } - case bindNext @ Map(fa, _, _) => + case bindNext @ Map(fa, _, _, _) => if (bFirst ne null) { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) @@ -283,11 +278,6 @@ private[effect] object IORunLoop { // box those vars in scala.runtime.ObjectRef! return suspendAsync(currentIO.asInstanceOf[IO.Async[A]], bFirst, bRest) - case Trace(source, frame) => - if (ctx eq null) ctx = IOContext() - ctx.pushFrame(frame) - currentIO = source - case Introspect => // TODO: This can be implemented in terms of Async now if (ctx eq null) ctx = IOContext() @@ -303,7 +293,7 @@ private[effect] object IORunLoop { if (hasUnboxed) { popNextBind(bFirst, bRest) match { case null => - return (if (currentIO ne null) currentIO else Pure(unboxed)) + return (if (currentIO ne null) currentIO else Pure(unboxed, null)) .asInstanceOf[IO[A]] case bind => currentIO = @@ -443,7 +433,7 @@ private[effect] object IORunLoop { // we interrupt the bind continuation if (!conn.isCanceled) either match { case Right(success) => - loop(Pure(success), conn, cb, ctx, this, bFirst, bRest) + loop(Pure(success, null), conn, cb, ctx, this, bFirst, bRest) case Left(e) => loop(RaiseError(e), conn, cb, ctx, this, bFirst, bRest) } @@ -474,7 +464,7 @@ private[effect] object IORunLoop { restore: (Any, Throwable, IOConnection, IOConnection) => IOConnection ) extends IOFrame[Any, IO[Any]] { def apply(a: Any): IO[Any] = - ContextSwitch(Pure(a), current => restore(a, null, old, current), null) + ContextSwitch(Pure(a, null), current => restore(a, null, old, current), null) def recover(e: Throwable): IO[Any] = ContextSwitch(RaiseError(e), current => restore(null, e, old, current), null) } diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index d4ddef3c3c..26154f89be 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -16,24 +16,28 @@ package cats.effect.internals -import java.util.concurrent.ConcurrentHashMap +import cats.effect.internals.TracingPlatformFast.{frameCache, localTracingMode} import cats.effect.IO import cats.effect.tracing.{TraceFrame, TraceLine, TracingMode} private[effect] object IOTracing { - def apply[A](source: IO[A], clazz: Class[_]): IO[A] = - localTracingMode.get() match { - case TracingMode.Disabled => source - case TracingMode.Rabbit => IO.Trace(source, buildCachedFrame(source, clazz)) - case TracingMode.Slug => IO.Trace(source, buildFrame(source)) + def apply[A](source: Class[_], clazz: Class[_]): TraceFrame = { + val mode = localTracingMode.get() + if (mode == 1) { + buildCachedFrame(source, clazz) + } else if (mode == 2) { + buildFrame(source) + } else { + null } + } def locallyTraced[A](source: IO[A], newMode: TracingMode): IO[A] = IO.suspend { val oldMode = localTracingMode.get() - localTracingMode.set(newMode) + localTracingMode.set(newMode.tag) // In the event of cancellation, the tracing mode will be reset // when the thread grabs a new task to run (via Async). @@ -52,47 +56,30 @@ private[effect] object IOTracing { } def getLocalTracingMode(): TracingMode = - localTracingMode.get() + TracingMode.fromInt(localTracingMode.get()) def setLocalTracingMode(mode: TracingMode): Unit = - localTracingMode.set(mode) + localTracingMode.set(mode.tag) - private def buildCachedFrame(source: IO[Any], clazz: Class[_]): TraceFrame = { - val cachedFr = frameCache.get(clazz) + private def buildCachedFrame(sourceClass: Class[_], keyClass: Class[_]): TraceFrame = { + val cachedFr = frameCache.get(keyClass).asInstanceOf[TraceFrame] if (cachedFr eq null) { - val fr = buildFrame(source) - frameCache.put(clazz, fr) + val fr = buildFrame(sourceClass) + frameCache.put(keyClass, fr) fr } else { cachedFr } } - private def buildFrame(source: IO[Any]): TraceFrame = { + private def buildFrame(sourceClass: Class[_]): TraceFrame = { // TODO: proper trace calculation val line = new Throwable().getStackTrace.toList .map(TraceLine.fromStackTraceElement) .find(l => !classBlacklist.exists(b => l.className.startsWith(b))) .headOption - val op = source match { - case _: IO.Map[_, _] => "map" - case _: IO.Bind[_, _] => "bind" - case _: IO.Async[_] => "async" - case _ => "unknown" - } - - TraceFrame(op, line) - } - - /** - * Cache for trace frames. Keys are references to: - * - lambda classes - */ - private val frameCache: ConcurrentHashMap[Class[_], TraceFrame] = new ConcurrentHashMap[Class[_], TraceFrame]() - - private val localTracingMode: ThreadLocal[TracingMode] = new ThreadLocal[TracingMode] { - override def initialValue(): TracingMode = TracingMode.Rabbit + TraceFrame(sourceClass.getSimpleName, line) } private val classBlacklist = List( diff --git a/core/shared/src/main/scala/cats/effect/tracing/TracingMode.scala b/core/shared/src/main/scala/cats/effect/tracing/TracingMode.scala index cd7aa015e1..00c31a834c 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/TracingMode.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/TracingMode.scala @@ -16,22 +16,21 @@ package cats.effect.tracing -sealed abstract private[effect] class TracingMode +sealed abstract private[effect] class TracingMode(val tag: Int) private[effect] object TracingMode { - case object Disabled extends TracingMode + case object Disabled extends TracingMode(0) - case object Rabbit extends TracingMode + case object Rabbit extends TracingMode(1) - case object Slug extends TracingMode + case object Slug extends TracingMode(2) - def fromString(value: String): Option[TracingMode] = - value.toLowerCase() match { - case "disabled" => Some(Disabled) - case "rabbit" => Some(Rabbit) - case "slug" => Some(Slug) - case _ => None + def fromInt(value: Int): TracingMode = + value match { + case 1 => Rabbit + case 2 => Slug + case _ => Disabled } } From 6e9ceab0cd55a1462c0a2dc1bf6df62a2c003198 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Sat, 23 May 2020 18:53:54 -0500 Subject: [PATCH 26/78] Add trace to IOContext --- .../effect/internals/TracingPlatformFast.java | 2 +- .../cats/effect/internals/IORunLoop.scala | 20 +++++++++---------- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java index 4fd12043b2..854d9f0ecb 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java @@ -34,7 +34,7 @@ public final class TracingPlatformFast { public static final boolean isTracingEnabled = Optional.ofNullable(System.getProperty("cats.effect.tracing.enabled")) .filter(x -> !x.isEmpty()) .map(x -> Boolean.valueOf(x)) - .orElse(true); + .orElse(false); /** * The number of trace lines to retain during tracing. If more trace diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index 1597308ea8..17e57e5456 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -101,15 +101,15 @@ private[effect] object IORunLoop { while ({ currentIO match { - case Bind(fa, bindNext, _) => + case Bind(fa, bindNext, trace) => if (bFirst ne null) { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } -// if (isTracingEnabled) { -// if (ctx eq null) ctx = IOContext() -// ctx.pushFrame(bind.trace) -// } + if (isTracingEnabled) { + if (ctx eq null) ctx = IOContext() + if (trace ne null) ctx.pushFrame(trace) + } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa @@ -145,15 +145,15 @@ private[effect] object IORunLoop { currentIO = fa } - case bindNext @ Map(fa, _, _, _) => + case bindNext @ Map(fa, _, _, trace) => if (bFirst ne null) { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } -// if (isTracingEnabled) { -// if (ctx eq null) ctx = IOContext() -// ctx.pushFrame(bindNext.trace) -// } + if (isTracingEnabled) { + if (ctx eq null) ctx = IOContext() + if (trace ne null) ctx.pushFrame(trace) + } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa From 91ab5b78d40c8f2cf30710a88103ef426c952afa Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Sat, 23 May 2020 19:02:07 -0500 Subject: [PATCH 27/78] Don't trace Pure --- core/shared/src/main/scala/cats/effect/Async.scala | 4 ++-- .../src/main/scala/cats/effect/Concurrent.scala | 4 ++-- core/shared/src/main/scala/cats/effect/IO.scala | 14 +++++++------- .../shared/src/main/scala/cats/effect/SyncIO.scala | 2 +- .../scala/cats/effect/internals/IORunLoop.scala | 10 +++++----- 5 files changed, 17 insertions(+), 17 deletions(-) diff --git a/core/shared/src/main/scala/cats/effect/Async.scala b/core/shared/src/main/scala/cats/effect/Async.scala index d8085c40f1..4822a2238c 100644 --- a/core/shared/src/main/scala/cats/effect/Async.scala +++ b/core/shared/src/main/scala/cats/effect/Async.scala @@ -308,13 +308,13 @@ object Async { */ def liftIO[F[_], A](io: IO[A])(implicit F: Async[F]): F[A] = io match { - case Pure(a, _) => F.pure(a) + case Pure(a) => F.pure(a) case RaiseError(e) => F.raiseError(e) case Delay(thunk, _) => F.delay(thunk()) case _ => F.suspend { IORunLoop.step(io) match { - case Pure(a, _) => F.pure(a) + case Pure(a) => F.pure(a) case RaiseError(e) => F.raiseError(e) case async => F.async(async.unsafeRunAsync) } diff --git a/core/shared/src/main/scala/cats/effect/Concurrent.scala b/core/shared/src/main/scala/cats/effect/Concurrent.scala index b2d774d3b7..5ccfd35e7f 100644 --- a/core/shared/src/main/scala/cats/effect/Concurrent.scala +++ b/core/shared/src/main/scala/cats/effect/Concurrent.scala @@ -406,13 +406,13 @@ object Concurrent { */ def liftIO[F[_], A](ioa: IO[A])(implicit F: Concurrent[F]): F[A] = ioa match { - case Pure(a, _) => F.pure(a) + case Pure(a) => F.pure(a) case RaiseError(e) => F.raiseError(e) case Delay(thunk, _) => F.delay(thunk()) case _ => F.suspend { IORunLoop.step(ioa) match { - case Pure(a, _) => F.pure(a) + case Pure(a) => F.pure(a) case RaiseError(e) => F.raiseError(e) case async => F.cancelable(cb => liftIO(async.unsafeRunCancelable(cb))(F)) diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index 4719f0b4c5..f6f4b8dbf8 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -344,7 +344,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { */ final def unsafeRunTimed(limit: Duration): Option[A] = IORunLoop.step(this) match { - case Pure(a, _) => Some(a) + case Pure(a) => Some(a) case RaiseError(e) => throw e case self @ Async(_, _) => IOPlatform.unsafeResync(self, limit) @@ -760,7 +760,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { IO.Bind(this, new IOFrame.RedeemWith(recover, bind), null) override def toString: String = this match { - case Pure(a, _) => s"IO($a)" + case Pure(a) => s"IO($a)" case RaiseError(e) => s"IO(throw $e)" case _ => "IO$" + System.identityHashCode(this) } @@ -1168,7 +1168,7 @@ object IO extends IOInstances { * (when evaluated) than `IO(42)`, due to avoiding the allocation of * extra thunks. */ - def pure[A](a: A): IO[A] = Pure(a, null) + def pure[A](a: A): IO[A] = Pure(a) /** Alias for `IO.pure(())`. */ val unit: IO[Unit] = pure(()) @@ -1618,7 +1618,7 @@ object IO extends IOInstances { /* IO's internal encoding: */ /** Corresponds to [[IO.pure]]. */ - final private[effect] case class Pure[+A](a: A, trace: TraceFrame) extends IO[A] + final private[effect] case class Pure[+A](a: A) extends IO[A] /** Corresponds to [[IO.apply]]. */ final private[effect] case class Delay[+A](thunk: () => A, trace: TraceFrame) extends IO[A] @@ -1637,7 +1637,7 @@ object IO extends IOInstances { extends IO[A] with (E => IO[A]) { override def apply(value: E): IO[A] = - Pure(f(value), null) + Pure(f(value)) } /** @@ -1687,8 +1687,8 @@ object IO extends IOInstances { */ private object AttemptIO extends IOFrame[Any, IO[Either[Throwable, Any]]] { override def apply(a: Any) = - Pure(Right(a), null) + Pure(Right(a)) override def recover(e: Throwable) = - Pure(Left(e), null) + Pure(Left(e)) } } diff --git a/core/shared/src/main/scala/cats/effect/SyncIO.scala b/core/shared/src/main/scala/cats/effect/SyncIO.scala index d9e8218940..71e779102d 100644 --- a/core/shared/src/main/scala/cats/effect/SyncIO.scala +++ b/core/shared/src/main/scala/cats/effect/SyncIO.scala @@ -313,7 +313,7 @@ final class SyncIO[+A] private (private val io: IO[A]) { new SyncIO(io.redeemWith(t => recover(t).io, a => bind(a).io)) override def toString: String = io match { - case IO.Pure(a, _) => s"SyncIO($a)" + case IO.Pure(a) => s"SyncIO($a)" case IO.RaiseError(e) => s"SyncIO(throw $e)" case _ => "SyncIO$" + System.identityHashCode(this) } diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index 17e57e5456..cf3712d655 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -113,7 +113,7 @@ private[effect] object IORunLoop { bFirst = bindNext.asInstanceOf[Bind] currentIO = fa - case Pure(value, _) => + case Pure(value) => unboxed = value.asInstanceOf[AnyRef] hasUnboxed = true @@ -233,7 +233,7 @@ private[effect] object IORunLoop { bFirst = bindNext.asInstanceOf[Bind] currentIO = fa - case Pure(value, _) => + case Pure(value) => unboxed = value.asInstanceOf[AnyRef] hasUnboxed = true @@ -293,7 +293,7 @@ private[effect] object IORunLoop { if (hasUnboxed) { popNextBind(bFirst, bRest) match { case null => - return (if (currentIO ne null) currentIO else Pure(unboxed, null)) + return (if (currentIO ne null) currentIO else Pure(unboxed)) .asInstanceOf[IO[A]] case bind => currentIO = @@ -433,7 +433,7 @@ private[effect] object IORunLoop { // we interrupt the bind continuation if (!conn.isCanceled) either match { case Right(success) => - loop(Pure(success, null), conn, cb, ctx, this, bFirst, bRest) + loop(Pure(success), conn, cb, ctx, this, bFirst, bRest) case Left(e) => loop(RaiseError(e), conn, cb, ctx, this, bFirst, bRest) } @@ -464,7 +464,7 @@ private[effect] object IORunLoop { restore: (Any, Throwable, IOConnection, IOConnection) => IOConnection ) extends IOFrame[Any, IO[Any]] { def apply(a: Any): IO[Any] = - ContextSwitch(Pure(a, null), current => restore(a, null, old, current), null) + ContextSwitch(Pure(a), current => restore(a, null, old, current), null) def recover(e: Throwable): IO[Any] = ContextSwitch(RaiseError(e), current => restore(null, e, old, current), null) } From 3c792cf3cba012205835e6072ba8fe323b1658a7 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Sat, 23 May 2020 21:51:38 -0500 Subject: [PATCH 28/78] Trace nodes --- .../effect/benchmarks/DeepBindBenchmark.scala | 20 +++---- .../benchmarks/ShallowBindBenchmark.scala | 34 +++++------ .../effect/internals/TracingPlatformFast.java | 2 +- .../src/main/scala/cats/effect/Async.scala | 8 +-- .../main/scala/cats/effect/Concurrent.scala | 8 +-- .../src/main/scala/cats/effect/IO.scala | 59 ++++++++----------- .../src/main/scala/cats/effect/SyncIO.scala | 2 +- .../cats/effect/internals/IOForkedStart.scala | 4 +- .../cats/effect/internals/IORunLoop.scala | 30 ++++------ .../cats/effect/internals/IOTracing.scala | 22 +++---- 10 files changed, 89 insertions(+), 100 deletions(-) diff --git a/benchmarks/shared/src/main/scala/cats/effect/benchmarks/DeepBindBenchmark.scala b/benchmarks/shared/src/main/scala/cats/effect/benchmarks/DeepBindBenchmark.scala index c7ce40d2be..31a0705264 100644 --- a/benchmarks/shared/src/main/scala/cats/effect/benchmarks/DeepBindBenchmark.scala +++ b/benchmarks/shared/src/main/scala/cats/effect/benchmarks/DeepBindBenchmark.scala @@ -43,16 +43,16 @@ class DeepBindBenchmark { @Param(Array("3000")) var size: Int = _ - @Benchmark - def pure(): Int = { - def loop(i: Int): IO[Int] = - for { - j <- IO.pure(i) - _ <- if (j > size) IO.pure(j) else loop(j + 1) - } yield j - - loop(0).unsafeRunSync() - } +// @Benchmark +// def pure(): Int = { +// def loop(i: Int): IO[Int] = +// for { +// j <- IO.pure(i) +// _ <- if (j > size) IO.pure(j) else loop(j + 1) +// } yield j +// +// loop(0).unsafeRunSync() +// } @Benchmark def delay(): Int = { diff --git a/benchmarks/shared/src/main/scala/cats/effect/benchmarks/ShallowBindBenchmark.scala b/benchmarks/shared/src/main/scala/cats/effect/benchmarks/ShallowBindBenchmark.scala index 78cddc1f32..36ac6b29ee 100644 --- a/benchmarks/shared/src/main/scala/cats/effect/benchmarks/ShallowBindBenchmark.scala +++ b/benchmarks/shared/src/main/scala/cats/effect/benchmarks/ShallowBindBenchmark.scala @@ -54,21 +54,21 @@ class ShallowBindBenchmark { .unsafeRunSync() } - @Benchmark - def delay(): Int = { - def loop(i: Int): IO[Int] = - if (i < size) IO(i + 1).flatMap(loop) - else IO(i) - - IO(0).flatMap(loop).unsafeRunSync() - } - - @Benchmark - def async(): Int = { - def loop(i: Int): IO[Int] = - if (i < size) IO.shift.flatMap(_ => IO.pure(i + 1)).flatMap(loop) - else IO.shift.flatMap(_ => IO.pure(i)) - - IO(0).flatMap(loop).unsafeRunSync() - } +// @Benchmark +// def delay(): Int = { +// def loop(i: Int): IO[Int] = +// if (i < size) IO(i + 1).flatMap(loop) +// else IO(i) +// +// IO(0).flatMap(loop).unsafeRunSync() +// } +// +// @Benchmark +// def async(): Int = { +// def loop(i: Int): IO[Int] = +// if (i < size) IO.shift.flatMap(_ => IO.pure(i + 1)).flatMap(loop) +// else IO.shift.flatMap(_ => IO.pure(i)) +// +// IO(0).flatMap(loop).unsafeRunSync() +// } } diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java index 854d9f0ecb..4fd12043b2 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java @@ -34,7 +34,7 @@ public final class TracingPlatformFast { public static final boolean isTracingEnabled = Optional.ofNullable(System.getProperty("cats.effect.tracing.enabled")) .filter(x -> !x.isEmpty()) .map(x -> Boolean.valueOf(x)) - .orElse(false); + .orElse(true); /** * The number of trace lines to retain during tracing. If more trace diff --git a/core/shared/src/main/scala/cats/effect/Async.scala b/core/shared/src/main/scala/cats/effect/Async.scala index 4822a2238c..8007f65382 100644 --- a/core/shared/src/main/scala/cats/effect/Async.scala +++ b/core/shared/src/main/scala/cats/effect/Async.scala @@ -308,13 +308,13 @@ object Async { */ def liftIO[F[_], A](io: IO[A])(implicit F: Async[F]): F[A] = io match { - case Pure(a) => F.pure(a) - case RaiseError(e) => F.raiseError(e) - case Delay(thunk, _) => F.delay(thunk()) + case Pure(a) => F.pure(a) + case RaiseError(e) => F.raiseError(e) + case Delay(thunk) => F.delay(thunk()) case _ => F.suspend { IORunLoop.step(io) match { - case Pure(a) => F.pure(a) + case Pure(a) => F.pure(a) case RaiseError(e) => F.raiseError(e) case async => F.async(async.unsafeRunAsync) } diff --git a/core/shared/src/main/scala/cats/effect/Concurrent.scala b/core/shared/src/main/scala/cats/effect/Concurrent.scala index 5ccfd35e7f..1dcf6f6915 100644 --- a/core/shared/src/main/scala/cats/effect/Concurrent.scala +++ b/core/shared/src/main/scala/cats/effect/Concurrent.scala @@ -406,13 +406,13 @@ object Concurrent { */ def liftIO[F[_], A](ioa: IO[A])(implicit F: Concurrent[F]): F[A] = ioa match { - case Pure(a) => F.pure(a) - case RaiseError(e) => F.raiseError(e) - case Delay(thunk, _) => F.delay(thunk()) + case Pure(a) => F.pure(a) + case RaiseError(e) => F.raiseError(e) + case Delay(thunk) => F.delay(thunk()) case _ => F.suspend { IORunLoop.step(ioa) match { - case Pure(a) => F.pure(a) + case Pure(a) => F.pure(a) case RaiseError(e) => F.raiseError(e) case async => F.cancelable(cb => liftIO(async.unsafeRunCancelable(cb))(F)) diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index f6f4b8dbf8..bba8ff1c45 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -106,22 +106,22 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { if (isTracingEnabled) { // Don't perform map fusion when tracing is enabled. // We may not actually have to do this - val trace = if (isTracingEnabled) { - IOTracing(Map.getClass, f.getClass) - } else { - null - } - Map(this, f, 0, trace) +// val trace = if (isTracingEnabled) { +// IOTracing(Map.getClass, f.getClass) +// } else { +// null +// } + Map(this, f, 0) } else { this match { - case Map(source, g, index, _) => + case Map(source, g, index) => // Allowed to do fixed number of map operations fused before // resetting the counter in order to avoid stack overflows; // See `IOPlatform` for details on this maximum. - if (index != fusionMaxStackDepth) Map(source, g.andThen(f), index + 1, null) - else Map(this, f, 0, null) + if (index != fusionMaxStackDepth) Map(source, g.andThen(f), index + 1) + else Map(this, f, 0) case _ => - Map(this, f, 0, null) + Map(this, f, 0) } } @@ -140,14 +140,12 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * failures would be completely silent and `IO` references would * never terminate on evaluation. */ - final def flatMap[B](f: A => IO[B]): IO[B] = { - val trace = if (isTracingEnabled) { - IOTracing(Bind.getClass, f.getClass) + final def flatMap[B](f: A => IO[B]): IO[B] = + if (isTracingEnabled) { + IOTracing(Bind(this, f), f.getClass) } else { - null + Bind(this, f) } - Bind(this, f, trace) - } /** * Materializes any sequenced exceptions into value space, where @@ -163,7 +161,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * @see [[IO.raiseError]] */ def attempt: IO[Either[Throwable, A]] = - Bind(this, AttemptIO.asInstanceOf[A => IO[Either[Throwable, A]]], null) + Bind(this, AttemptIO.asInstanceOf[A => IO[Either[Throwable, A]]]) /** * Produces an `IO` reference that should execute the source on @@ -344,7 +342,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { */ final def unsafeRunTimed(limit: Duration): Option[A] = IORunLoop.step(this) match { - case Pure(a) => Some(a) + case Pure(a) => Some(a) case RaiseError(e) => throw e case self @ Async(_, _) => IOPlatform.unsafeResync(self, limit) @@ -691,7 +689,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * Implements `ApplicativeError.handleErrorWith`. */ def handleErrorWith[AA >: A](f: Throwable => IO[AA]): IO[AA] = - IO.Bind(this, new IOFrame.ErrorHandler(f), null) + IO.Bind(this, new IOFrame.ErrorHandler(f)) /** * Zips both this action and the parameter in parallel. @@ -725,7 +723,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * in case it ends in success */ def redeem[B](recover: Throwable => B, map: A => B): IO[B] = - IO.Bind(this, new IOFrame.Redeem(recover, map), null) + IO.Bind(this, new IOFrame.Redeem(recover, map)) /** * Returns a new value that transforms the result of the source, @@ -757,10 +755,10 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * in case of success */ def redeemWith[B](recover: Throwable => IO[B], bind: A => IO[B]): IO[B] = - IO.Bind(this, new IOFrame.RedeemWith(recover, bind), null) + IO.Bind(this, new IOFrame.RedeemWith(recover, bind)) override def toString: String = this match { - case Pure(a) => s"IO($a)" + case Pure(a) => s"IO($a)" case RaiseError(e) => s"IO(throw $e)" case _ => "IO$" + System.identityHashCode(this) } @@ -1145,7 +1143,7 @@ object IO extends IOInstances { * into the `IO`. */ def delay[A](body: => A): IO[A] = - Delay(() => body, null) + Delay(() => body) /** * Suspends a synchronous side effect which produces an `IO` in `IO`. @@ -1621,7 +1619,7 @@ object IO extends IOInstances { final private[effect] case class Pure[+A](a: A) extends IO[A] /** Corresponds to [[IO.apply]]. */ - final private[effect] case class Delay[+A](thunk: () => A, trace: TraceFrame) extends IO[A] + final private[effect] case class Delay[+A](thunk: () => A) extends IO[A] /** Corresponds to [[IO.raiseError]]. */ final private[effect] case class RaiseError(e: Throwable) extends IO[Nothing] @@ -1630,12 +1628,10 @@ object IO extends IOInstances { final private[effect] case class Suspend[+A](thunk: () => IO[A]) extends IO[A] /** Corresponds to [[IO.flatMap]]. */ - final private[effect] case class Bind[E, +A](source: IO[E], f: E => IO[A], trace: TraceFrame) extends IO[A] + final private[effect] case class Bind[E, +A](source: IO[E], f: E => IO[A]) extends IO[A] /** Corresponds to [[IO.map]]. */ - final private[effect] case class Map[E, +A](source: IO[E], f: E => A, index: Int, trace: TraceFrame) - extends IO[A] - with (E => IO[A]) { + final private[effect] case class Map[E, +A](source: IO[E], f: E => A, index: Int) extends IO[A] with (E => IO[A]) { override def apply(value: E): IO[A] = Pure(f(value)) } @@ -1658,7 +1654,8 @@ object IO extends IOInstances { k: (IOConnection, IOContext, Either[Throwable, A] => Unit) => Unit, trampolineAfter: Boolean = false ) extends IO[A] - with TracedIO[A] + + final private[effect] case class Trace[A](source: IO[A], frame: TraceFrame) extends IO[A] /** * An internal state for that optimizes changes to @@ -1675,10 +1672,6 @@ object IO extends IOInstances { final private[effect] case object Introspect extends IO[IOTrace] - private[effect] trait TracedIO[+A] { self: IO[A] => - private[effect] var trace: TraceFrame = null - } - /* -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-= */ /** diff --git a/core/shared/src/main/scala/cats/effect/SyncIO.scala b/core/shared/src/main/scala/cats/effect/SyncIO.scala index 71e779102d..63906d42be 100644 --- a/core/shared/src/main/scala/cats/effect/SyncIO.scala +++ b/core/shared/src/main/scala/cats/effect/SyncIO.scala @@ -313,7 +313,7 @@ final class SyncIO[+A] private (private val io: IO[A]) { new SyncIO(io.redeemWith(t => recover(t).io, a => bind(a).io)) override def toString: String = io match { - case IO.Pure(a) => s"SyncIO($a)" + case IO.Pure(a) => s"SyncIO($a)" case IO.RaiseError(e) => s"SyncIO(throw $e)" case _ => "SyncIO$" + System.identityHashCode(this) } diff --git a/core/shared/src/main/scala/cats/effect/internals/IOForkedStart.scala b/core/shared/src/main/scala/cats/effect/internals/IOForkedStart.scala index 161eb62770..743565d13d 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOForkedStart.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOForkedStart.scala @@ -54,8 +54,8 @@ private[effect] object IOForkedStart { if (limit > 0) { task match { case IO.Async(k, _) => k.isInstanceOf[IOForkedStart[_]] - case IO.Bind(other, _, _) => detect(other, limit - 1) - case IO.Map(other, _, _, _) => detect(other, limit - 1) + case IO.Bind(other, _) => detect(other, limit - 1) + case IO.Map(other, _, _) => detect(other, limit - 1) case IO.ContextSwitch(other, _, _) => detect(other, limit - 1) case _ => false } diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index cf3712d655..4334d47f84 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -17,7 +17,7 @@ package cats.effect.internals import cats.effect.IO -import cats.effect.IO.{Async, Bind, ContextSwitch, Delay, Introspect, Map, Pure, RaiseError, Suspend} +import cats.effect.IO.{Async, Bind, ContextSwitch, Delay, Introspect, Map, Pure, RaiseError, Suspend, Trace} import cats.effect.tracing.TracingMode import cats.effect.internals.TracingPlatformFast.isTracingEnabled @@ -101,15 +101,11 @@ private[effect] object IORunLoop { while ({ currentIO match { - case Bind(fa, bindNext, trace) => + case Bind(fa, bindNext) => if (bFirst ne null) { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if (isTracingEnabled) { - if (ctx eq null) ctx = IOContext() - if (trace ne null) ctx.pushFrame(trace) - } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa @@ -117,7 +113,7 @@ private[effect] object IORunLoop { unboxed = value.asInstanceOf[AnyRef] hasUnboxed = true - case Delay(thunk, _) => + case Delay(thunk) => try { unboxed = thunk().asInstanceOf[AnyRef] hasUnboxed = true @@ -145,15 +141,11 @@ private[effect] object IORunLoop { currentIO = fa } - case bindNext @ Map(fa, _, _, trace) => + case bindNext @ Map(fa, _, _) => if (bFirst ne null) { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if (isTracingEnabled) { - if (ctx eq null) ctx = IOContext() - if (trace ne null) ctx.pushFrame(trace) - } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa @@ -173,14 +165,18 @@ private[effect] object IORunLoop { if (conn ne old) { if (rcb ne null) rcb.contextSwitch(conn) if (restore ne null) - currentIO = Bind(next, new RestoreContext(old, restore), null) + currentIO = Bind(next, new RestoreContext(old, restore)) } + case Trace(source, frame) => + if (ctx eq null) ctx = IOContext() + ctx.pushFrame(frame) + currentIO = source + case Introspect => if (ctx eq null) ctx = IOContext() hasUnboxed = true unboxed = ctx.getTrace - } if (hasUnboxed) { @@ -225,7 +221,7 @@ private[effect] object IORunLoop { while ({ currentIO match { - case Bind(fa, bindNext, _) => + case Bind(fa, bindNext) => if (bFirst ne null) { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) @@ -237,7 +233,7 @@ private[effect] object IORunLoop { unboxed = value.asInstanceOf[AnyRef] hasUnboxed = true - case Delay(thunk, _) => + case Delay(thunk) => try { unboxed = thunk().asInstanceOf[AnyRef] hasUnboxed = true @@ -265,7 +261,7 @@ private[effect] object IORunLoop { currentIO = fa } - case bindNext @ Map(fa, _, _, _) => + case bindNext @ Map(fa, _, _) => if (bFirst ne null) { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index 26154f89be..79a499aaed 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -17,22 +17,22 @@ package cats.effect.internals import cats.effect.internals.TracingPlatformFast.{frameCache, localTracingMode} - import cats.effect.IO +import cats.effect.IO.Trace import cats.effect.tracing.{TraceFrame, TraceLine, TracingMode} private[effect] object IOTracing { - def apply[A](source: Class[_], clazz: Class[_]): TraceFrame = { - val mode = localTracingMode.get() - if (mode == 1) { - buildCachedFrame(source, clazz) - } else if (mode == 2) { - buildFrame(source) - } else { - null - } - } + def apply[A](source: IO[A], clazz: Class[_]): IO[A] = +// val mode = localTracingMode.get() +// if (mode == 1) { +// Trace(source, buildCachedFrame(source.getClass, clazz)) +// } else if (mode == 2) { +// Trace(source, buildFrame(source.getClass)) +// } else { +// source +// } + Trace(source, buildCachedFrame(source.getClass, clazz)) def locallyTraced[A](source: IO[A], newMode: TracingMode): IO[A] = IO.suspend { From 21313088b1e4b18061bdf7014832126a7c6af8e4 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Sat, 23 May 2020 22:38:51 -0500 Subject: [PATCH 29/78] Pretty printing and demangling --- .../scala/cats/effect/internals/Example.scala | 2 +- .../cats/effect/internals/IOTracing.scala | 4 +-- .../scala/cats/effect/tracing/IOTrace.scala | 32 ++++++++++++++++--- .../{TraceLine.scala => StackTraceLine.scala} | 20 +++++++++--- .../cats/effect/tracing/TraceFrame.scala | 2 +- 5 files changed, 48 insertions(+), 12 deletions(-) rename core/shared/src/main/scala/cats/effect/tracing/{TraceLine.scala => StackTraceLine.scala} (51%) diff --git a/core/jvm/src/main/scala/cats/effect/internals/Example.scala b/core/jvm/src/main/scala/cats/effect/internals/Example.scala index 0c8f287791..ec9227ac95 100644 --- a/core/jvm/src/main/scala/cats/effect/internals/Example.scala +++ b/core/jvm/src/main/scala/cats/effect/internals/Example.scala @@ -70,7 +70,7 @@ object Example extends IOApp { _ <- IO.suspend(program).rabbitTrace _ <- IO.delay("10") trace <- IO.backtrace - _ <- IO.delay(trace.printTrace()) + _ <- IO.delay(trace.prettyPrint()) } yield ExitCode.Success } diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index 79a499aaed..f9554a1e4b 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -19,7 +19,7 @@ package cats.effect.internals import cats.effect.internals.TracingPlatformFast.{frameCache, localTracingMode} import cats.effect.IO import cats.effect.IO.Trace -import cats.effect.tracing.{TraceFrame, TraceLine, TracingMode} +import cats.effect.tracing.{TraceFrame, StackTraceLine, TracingMode} private[effect] object IOTracing { @@ -75,7 +75,7 @@ private[effect] object IOTracing { private def buildFrame(sourceClass: Class[_]): TraceFrame = { // TODO: proper trace calculation val line = new Throwable().getStackTrace.toList - .map(TraceLine.fromStackTraceElement) + .map(StackTraceLine.fromStackTraceElement) .find(l => !classBlacklist.exists(b => l.className.startsWith(b))) .headOption diff --git a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala index 83071d68a7..bec0e8c3e7 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala @@ -18,11 +18,35 @@ package cats.effect.tracing final case class IOTrace(frames: Vector[TraceFrame], omitted: Int) { - def printTrace(): Unit = { + def prettyPrint(): Unit = { System.err.println(s"IOTrace: $omitted omitted frames") - frames.foreach { f => - f.line.foreach { l => - System.err.println(s"\t${f.op} at ${l.className}.${l.methodName} (${l.fileName}:${l.lineNumber})") + val render = loop("", 0, true, frames.toList) + System.err.println(render) + } + + private def loop(acc: String, indent: Int, init: Boolean, rest: List[TraceFrame]): String = { + val TurnRight = "╰" + val InverseTurnRight = "╭" + val TurnDown = "╮" + val Junction = "├" + val Line = "│" + + rest match { + case k :: ks => { + val acc2 = if (init) { + InverseTurnRight + s" ${k.op}\n" + } else { + Junction + s" ${k.op}\n" + } + + val inner = Line + " " + TurnRight + TurnDown + "\n" + val demangled = k.line.map(_.demangled) + val traceLine = Line + " " + TurnRight + " " + demangled.map(l => s"${l.className}.${l.methodName} (${l.fileName}:${l.lineNumber})").mkString + "\n" + + loop(acc + acc2 + inner + traceLine + Line + "\n", indent, false, ks) + } + case Nil => { + acc + TurnRight + " Done" } } } diff --git a/core/shared/src/main/scala/cats/effect/tracing/TraceLine.scala b/core/shared/src/main/scala/cats/effect/tracing/StackTraceLine.scala similarity index 51% rename from core/shared/src/main/scala/cats/effect/tracing/TraceLine.scala rename to core/shared/src/main/scala/cats/effect/tracing/StackTraceLine.scala index fc10ea631d..cd3432ebf1 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/TraceLine.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/StackTraceLine.scala @@ -17,9 +17,21 @@ package cats.effect.tracing // TODO: Track information about what combinator was used etc. -final case class TraceLine(className: String, methodName: String, fileName: String, lineNumber: Int) +final case class StackTraceLine(className: String, methodName: String, fileName: String, lineNumber: Int) { + import StackTraceLine._ + def demangled: StackTraceLine = { + val newClassName = className.replaceAll("\\$", "") + val newMethodName = anonfunRegex.findFirstMatchIn(methodName) match { + case Some(mat) => mat.group(1) + case None => methodName + } + StackTraceLine(newClassName, newMethodName, fileName, lineNumber) + } +} + +object StackTraceLine { + val anonfunRegex = "^\\$+anonfun\\$+(.+)\\$+\\d+$".r -object TraceLine { - def fromStackTraceElement(ste: StackTraceElement): TraceLine = - TraceLine(ste.getClassName, ste.getMethodName, ste.getFileName, ste.getLineNumber) + def fromStackTraceElement(ste: StackTraceElement): StackTraceLine = + StackTraceLine(ste.getClassName, ste.getMethodName, ste.getFileName, ste.getLineNumber) } diff --git a/core/shared/src/main/scala/cats/effect/tracing/TraceFrame.scala b/core/shared/src/main/scala/cats/effect/tracing/TraceFrame.scala index 50e7fa4eb4..5e0dbe90a2 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/TraceFrame.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/TraceFrame.scala @@ -16,4 +16,4 @@ package cats.effect.tracing -final case class TraceFrame(op: String, line: Option[TraceLine]) +final case class TraceFrame(op: String, line: Option[StackTraceLine]) From e17a79538589a94cae684dc532c02748eb9d399b Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Sun, 24 May 2020 01:42:27 -0500 Subject: [PATCH 30/78] Trace tags --- .../effect/internals/TracingPlatformFast.java | 2 +- .../scala/cats/effect/internals/Example.scala | 1 + .../src/main/scala/cats/effect/IO.scala | 4 +- .../cats/effect/internals/IOContext.scala | 4 +- .../cats/effect/internals/IOTracing.scala | 20 ++++---- .../scala/cats/effect/tracing/IOTrace.scala | 51 ++++++++++++++++--- .../cats/effect/tracing/StackTraceLine.scala | 37 -------------- .../cats/effect/tracing/TraceFrame.scala | 2 +- .../scala/cats/effect/tracing/TraceTag.scala | 24 +++++++++ 9 files changed, 85 insertions(+), 60 deletions(-) delete mode 100644 core/shared/src/main/scala/cats/effect/tracing/StackTraceLine.scala create mode 100644 core/shared/src/main/scala/cats/effect/tracing/TraceTag.scala diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java index 4fd12043b2..7ba0a26b02 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java @@ -41,7 +41,7 @@ public final class TracingPlatformFast { * lines are produced, then the oldest trace lines will be discarded. * Automatically rounded up to the nearest power of 2. */ - public static final int maxTraceFrameSize = Optional.ofNullable(System.getProperty("cats.effect.tracing.maxTraceFrameSize")) + public static final int maxTraceDepth = Optional.ofNullable(System.getProperty("cats.effect.tracing.maxTraceDepth")) .filter(x -> !x.isEmpty()) .flatMap(x -> { try { diff --git a/core/jvm/src/main/scala/cats/effect/internals/Example.scala b/core/jvm/src/main/scala/cats/effect/internals/Example.scala index ec9227ac95..b111725a3a 100644 --- a/core/jvm/src/main/scala/cats/effect/internals/Example.scala +++ b/core/jvm/src/main/scala/cats/effect/internals/Example.scala @@ -71,6 +71,7 @@ object Example extends IOApp { _ <- IO.delay("10") trace <- IO.backtrace _ <- IO.delay(trace.prettyPrint()) + _ <- IO.delay(trace.rawPrint()) } yield ExitCode.Success } diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index bba8ff1c45..612cc33596 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -27,7 +27,7 @@ import scala.concurrent.{ExecutionContext, Future, Promise, TimeoutException} import scala.util.control.NonFatal import scala.util.{Failure, Left, Right, Success, Try} import cats.data.Ior -import cats.effect.tracing.{IOTrace, TraceFrame, TracingMode} +import cats.effect.tracing.{IOTrace, TraceFrame, TraceTag, TracingMode} /** * A pure abstraction representing the intention to perform a @@ -142,7 +142,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { */ final def flatMap[B](f: A => IO[B]): IO[B] = if (isTracingEnabled) { - IOTracing(Bind(this, f), f.getClass) + IOTracing(Bind(this, f), TraceTag.Bind, f.getClass) } else { Bind(this, f) } diff --git a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala index 78766c8f1e..2fcca7a50c 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala @@ -17,7 +17,7 @@ package cats.effect.internals import cats.effect.tracing.{IOTrace, TraceFrame} -import cats.effect.internals.TracingPlatformFast.maxTraceFrameSize +import cats.effect.internals.TracingPlatformFast.maxTraceDepth /** * INTERNAL API — Holds state related to the execution of @@ -26,7 +26,7 @@ import cats.effect.internals.TracingPlatformFast.maxTraceFrameSize */ final private[effect] class IOContext private () { - private val frames: RingBuffer[TraceFrame] = new RingBuffer(maxTraceFrameSize) + private val frames: RingBuffer[TraceFrame] = new RingBuffer(maxTraceDepth) private var omitted: Int = 0 def pushFrame(fr: TraceFrame): Unit = { diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index f9554a1e4b..29f7e0e8c6 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -19,11 +19,11 @@ package cats.effect.internals import cats.effect.internals.TracingPlatformFast.{frameCache, localTracingMode} import cats.effect.IO import cats.effect.IO.Trace -import cats.effect.tracing.{TraceFrame, StackTraceLine, TracingMode} +import cats.effect.tracing.{TraceFrame, TraceTag, TracingMode} private[effect] object IOTracing { - def apply[A](source: IO[A], clazz: Class[_]): IO[A] = + def apply[A](source: IO[A], traceTag: TraceTag, clazz: Class[_]): IO[A] = // val mode = localTracingMode.get() // if (mode == 1) { // Trace(source, buildCachedFrame(source.getClass, clazz)) @@ -32,7 +32,7 @@ private[effect] object IOTracing { // } else { // source // } - Trace(source, buildCachedFrame(source.getClass, clazz)) + Trace(source, buildCachedFrame(traceTag, clazz)) def locallyTraced[A](source: IO[A], newMode: TracingMode): IO[A] = IO.suspend { @@ -61,10 +61,10 @@ private[effect] object IOTracing { def setLocalTracingMode(mode: TracingMode): Unit = localTracingMode.set(mode.tag) - private def buildCachedFrame(sourceClass: Class[_], keyClass: Class[_]): TraceFrame = { + private def buildCachedFrame(traceTag: TraceTag, keyClass: Class[_]): TraceFrame = { val cachedFr = frameCache.get(keyClass).asInstanceOf[TraceFrame] if (cachedFr eq null) { - val fr = buildFrame(sourceClass) + val fr = buildFrame(traceTag) frameCache.put(keyClass, fr) fr } else { @@ -72,14 +72,12 @@ private[effect] object IOTracing { } } - private def buildFrame(sourceClass: Class[_]): TraceFrame = { + private def buildFrame(traceTag: TraceTag): TraceFrame = { // TODO: proper trace calculation - val line = new Throwable().getStackTrace.toList - .map(StackTraceLine.fromStackTraceElement) - .find(l => !classBlacklist.exists(b => l.className.startsWith(b))) - .headOption + val stackTrace = new Throwable().getStackTrace.toList + .dropWhile(l => classBlacklist.exists(b => l.getClassName.startsWith(b))) - TraceFrame(sourceClass.getSimpleName, line) + TraceFrame(traceTag, stackTrace) } private val classBlacklist = List( diff --git a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala index bec0e8c3e7..8b39a4fa22 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala @@ -18,10 +18,32 @@ package cats.effect.tracing final case class IOTrace(frames: Vector[TraceFrame], omitted: Int) { - def prettyPrint(): Unit = { + import IOTrace._ + + def rawPrint(): Unit = { + def renderStackTraceElement(ste: StackTraceElement): String = { + val className = ste.getClassName.replaceAll("\\$", "") + val methodName = anonfunRegex.findFirstMatchIn(ste.getMethodName) match { + case Some(mat) => mat.group(1) + case None => ste.getMethodName + } + + s"$className.$methodName (${ste.getFileName}:${ste.getLineNumber})" + } + System.err.println(s"IOTrace: $omitted omitted frames") + frames.foreach { f => + val desc = s"\t${f.tag.name} at " + f.stackTrace.headOption.map(renderStackTraceElement).getOrElse("(...)") + System.err.println(desc) + } + System.err.println() + } + + def prettyPrint(): Unit = { val render = loop("", 0, true, frames.toList) + System.err.println(s"IOTrace: $omitted omitted frames") System.err.println(render) + System.err.println() } private def loop(acc: String, indent: Int, init: Boolean, rest: List[TraceFrame]): String = { @@ -31,19 +53,32 @@ final case class IOTrace(frames: Vector[TraceFrame], omitted: Int) { val Junction = "├" val Line = "│" + def renderStackTraceElement(ste: StackTraceElement, last: Boolean): String = { + val className = ste.getClassName.replaceAll("\\$", "") + val methodName = anonfunRegex.findFirstMatchIn(ste.getMethodName) match { + case Some(mat) => mat.group(1) + case None => ste.getMethodName + } + + val junc = if (last) TurnRight else Junction + + Line + " " + junc + s" $className.$methodName (${ste.getFileName}:${ste.getLineNumber})\n" + } + rest match { case k :: ks => { val acc2 = if (init) { - InverseTurnRight + s" ${k.op}\n" + InverseTurnRight + s" ${k.tag.name}\n" } else { - Junction + s" ${k.op}\n" + Junction + s" ${k.tag.name}\n" } val inner = Line + " " + TurnRight + TurnDown + "\n" - val demangled = k.line.map(_.demangled) - val traceLine = Line + " " + TurnRight + " " + demangled.map(l => s"${l.className}.${l.methodName} (${l.fileName}:${l.lineNumber})").mkString + "\n" + val innerLines = k.stackTrace.zipWithIndex.map { + case (ste, i) => renderStackTraceElement(ste, i == k.stackTrace.length - 1) + }.mkString - loop(acc + acc2 + inner + traceLine + Line + "\n", indent, false, ks) + loop(acc + acc2 + inner + innerLines + Line + "\n", indent, false, ks) } case Nil => { acc + TurnRight + " Done" @@ -52,3 +87,7 @@ final case class IOTrace(frames: Vector[TraceFrame], omitted: Int) { } } + +object IOTrace { + private[effect] val anonfunRegex = "^\\$+anonfun\\$+(.+)\\$+\\d+$".r +} diff --git a/core/shared/src/main/scala/cats/effect/tracing/StackTraceLine.scala b/core/shared/src/main/scala/cats/effect/tracing/StackTraceLine.scala deleted file mode 100644 index cd3432ebf1..0000000000 --- a/core/shared/src/main/scala/cats/effect/tracing/StackTraceLine.scala +++ /dev/null @@ -1,37 +0,0 @@ -/* - * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers - * - * Licensed 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 cats.effect.tracing - -// TODO: Track information about what combinator was used etc. -final case class StackTraceLine(className: String, methodName: String, fileName: String, lineNumber: Int) { - import StackTraceLine._ - def demangled: StackTraceLine = { - val newClassName = className.replaceAll("\\$", "") - val newMethodName = anonfunRegex.findFirstMatchIn(methodName) match { - case Some(mat) => mat.group(1) - case None => methodName - } - StackTraceLine(newClassName, newMethodName, fileName, lineNumber) - } -} - -object StackTraceLine { - val anonfunRegex = "^\\$+anonfun\\$+(.+)\\$+\\d+$".r - - def fromStackTraceElement(ste: StackTraceElement): StackTraceLine = - StackTraceLine(ste.getClassName, ste.getMethodName, ste.getFileName, ste.getLineNumber) -} diff --git a/core/shared/src/main/scala/cats/effect/tracing/TraceFrame.scala b/core/shared/src/main/scala/cats/effect/tracing/TraceFrame.scala index 5e0dbe90a2..e651e8015e 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/TraceFrame.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/TraceFrame.scala @@ -16,4 +16,4 @@ package cats.effect.tracing -final case class TraceFrame(op: String, line: Option[StackTraceLine]) +final case class TraceFrame(tag: TraceTag, stackTrace: List[StackTraceElement]) diff --git a/core/shared/src/main/scala/cats/effect/tracing/TraceTag.scala b/core/shared/src/main/scala/cats/effect/tracing/TraceTag.scala new file mode 100644 index 0000000000..9cd5f963d1 --- /dev/null +++ b/core/shared/src/main/scala/cats/effect/tracing/TraceTag.scala @@ -0,0 +1,24 @@ +/* + * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers + * + * Licensed 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 cats.effect.tracing + +sealed abstract class TraceTag(val name: String) + +object TraceTag { + case object Bind extends TraceTag("FlatMap") + case object Map extends TraceTag("Map") +} From e36263d66811bd9f8f4e09f53284ed42bd6e330b Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Sun, 24 May 2020 02:07:46 -0500 Subject: [PATCH 31/78] Trace more instructions --- .../src/main/scala/cats/effect/IO.scala | 65 +++++++++++-------- .../scala/cats/effect/tracing/TraceTag.scala | 14 +++- 2 files changed, 48 insertions(+), 31 deletions(-) diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index 612cc33596..2de4d3aef9 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -106,12 +106,12 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { if (isTracingEnabled) { // Don't perform map fusion when tracing is enabled. // We may not actually have to do this -// val trace = if (isTracingEnabled) { -// IOTracing(Map.getClass, f.getClass) -// } else { -// null -// } - Map(this, f, 0) + val nextIo = Map(this, f, 0) + if (isTracingEnabled) { + IOTracing(nextIo, TraceTag.Map, f.getClass) + } else { + nextIo + } } else { this match { case Map(source, g, index) => @@ -580,8 +580,14 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * canceled, receiving as input the resource that needs to * be released */ - final def bracket[B](use: A => IO[B])(release: A => IO[Unit]): IO[B] = - bracketCase(use)((a, _) => release(a)) + final def bracket[B](use: A => IO[B])(release: A => IO[Unit]): IO[B] = { + val nextIo = IOBracket(this)(use)((a, _) => release(a)) + if (isTracingEnabled) { + IOTracing(nextIo, TraceTag.Bracket, use.getClass) + } else { + nextIo + } + } /** * Returns a new `IO` task that treats the source task as the @@ -615,8 +621,14 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * release, along with the result of `use` * (cancellation, error or successful result) */ - def bracketCase[B](use: A => IO[B])(release: (A, ExitCase[Throwable]) => IO[Unit]): IO[B] = - IOBracket(this)(use)(release) + def bracketCase[B](use: A => IO[B])(release: (A, ExitCase[Throwable]) => IO[Unit]): IO[B] = { + val nextIo = IOBracket(this)(use)(release) + if (isTracingEnabled) { + IOTracing(nextIo, TraceTag.BracketCase, use.getClass) + } else { + nextIo + } + } /** * Executes the given `finalizer` when the source is finished, @@ -1238,12 +1250,11 @@ object IO extends IOInstances { catch { case NonFatal(t) => cb2(Left(t)) } } -// if (isTracingEnabled) { -// IOTracing(nextIo, k.getClass) -// } else { -// nextIo -// } - nextIo + if (isTracingEnabled) { + IOTracing(nextIo, TraceTag.Async, k.getClass) + } else { + nextIo + } } /** @@ -1284,12 +1295,11 @@ object IO extends IOInstances { IORunLoop.startCancelable(fa, conn2, Callback.report) } -// if (isTracingEnabled) { -// IOTracing(nextIo, k.getClass) -// } else { -// nextIo -// } - nextIo + if (isTracingEnabled) { + IOTracing(nextIo, TraceTag.AsyncF, k.getClass) + } else { + nextIo + } } /** @@ -1352,12 +1362,11 @@ object IO extends IOInstances { ref.complete(IO.unit) } -// if (isTracingEnabled) { -// IOTracing(nextIo, k.getClass) -// } else { -// nextIo -// } - nextIo + if (isTracingEnabled) { + IOTracing(nextIo, TraceTag.Cancelable, k.getClass) + } else { + nextIo + } } /** diff --git a/core/shared/src/main/scala/cats/effect/tracing/TraceTag.scala b/core/shared/src/main/scala/cats/effect/tracing/TraceTag.scala index 9cd5f963d1..4402f72e12 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/TraceTag.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/TraceTag.scala @@ -16,9 +16,17 @@ package cats.effect.tracing -sealed abstract class TraceTag(val name: String) +final case class TraceTag(name: String) object TraceTag { - case object Bind extends TraceTag("FlatMap") - case object Map extends TraceTag("Map") + val Attempt = TraceTag("attempt") + val Bind = TraceTag("flatMap") + val Map = TraceTag("map") + + val Async = TraceTag("async") + val AsyncF = TraceTag("asyncF") + val Cancelable = TraceTag("cancelable") + + val Bracket = TraceTag("bracket") + val BracketCase = TraceTag("bracketCase") } From bfa4e1b2e3a11913f236651f7a94ecc4b1775c4c Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Tue, 26 May 2020 01:50:28 -0500 Subject: [PATCH 32/78] Hybrid tracing --- .../effect/benchmarks/DeepBindBenchmark.scala | 28 +++---- .../effect/internals/TracingPlatformFast.java | 18 +++-- .../src/main/scala/cats/effect/IO.scala | 78 +++++++++++-------- .../cats/effect/internals/IOForkedStart.scala | 4 +- .../cats/effect/internals/IORunLoop.scala | 26 +++++-- .../cats/effect/internals/IOTracing.scala | 8 +- .../scala/cats/effect/tracing/TraceTag.scala | 4 + 7 files changed, 108 insertions(+), 58 deletions(-) diff --git a/benchmarks/shared/src/main/scala/cats/effect/benchmarks/DeepBindBenchmark.scala b/benchmarks/shared/src/main/scala/cats/effect/benchmarks/DeepBindBenchmark.scala index 31a0705264..95811c2a37 100644 --- a/benchmarks/shared/src/main/scala/cats/effect/benchmarks/DeepBindBenchmark.scala +++ b/benchmarks/shared/src/main/scala/cats/effect/benchmarks/DeepBindBenchmark.scala @@ -43,28 +43,28 @@ class DeepBindBenchmark { @Param(Array("3000")) var size: Int = _ -// @Benchmark -// def pure(): Int = { -// def loop(i: Int): IO[Int] = -// for { -// j <- IO.pure(i) -// _ <- if (j > size) IO.pure(j) else loop(j + 1) -// } yield j -// -// loop(0).unsafeRunSync() -// } - @Benchmark - def delay(): Int = { + def pure(): Int = { def loop(i: Int): IO[Int] = for { - j <- IO(i) - _ <- if (j > size) IO(j) else loop(j + 1) + j <- IO.pure(i) + _ <- if (j > size) IO.pure(j) else loop(j + 1) } yield j loop(0).unsafeRunSync() } +// @Benchmark +// def delay(): Int = { +// def loop(i: Int): IO[Int] = +// for { +// j <- IO(i) +// _ <- if (j > size) IO(j) else loop(j + 1) +// } yield j +// +// loop(0).unsafeRunSync() +// } + // @Benchmark // def async(): Int = { // def loop(i: Int): IO[Int] = diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java index 7ba0a26b02..49c8bb6f63 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java @@ -20,21 +20,25 @@ import java.util.concurrent.ConcurrentHashMap; /** - * Scala object field accesses cost a volatile read across modules. - * Since this flag is read during construction of IO nodes, we are opting to - * hold this flag in a Java class to bypass the volatile read. + * Holds platform-specific flags that control tracing behavior. + * + * The Scala compiler inserts a volatile bitmap access for module field accesses. + * Since several of these flags are read during IO node construction, we are opting + * to hold this flag in a Java class to bypass that and squeeze out more performance. + * + * INTERNAL API. */ public final class TracingPlatformFast { /** * A boolean flag that enables or disables tracing for a JVM process. - * Since it is declared static and final, the JIT compiler has the liberty + * Since it is declared static and final, The JIT compiler has the liberty * to completely eliminate code paths consequent to the conditional. */ public static final boolean isTracingEnabled = Optional.ofNullable(System.getProperty("cats.effect.tracing.enabled")) .filter(x -> !x.isEmpty()) .map(x -> Boolean.valueOf(x)) - .orElse(true); + .orElse(false); /** * The number of trace lines to retain during tracing. If more trace @@ -58,6 +62,10 @@ public final class TracingPlatformFast { */ public static final ConcurrentHashMap, Object> frameCache = new ConcurrentHashMap<>(); + /** + * Thread-local state that stores the lexical tracing + * mode for the fiber bound to the current thread. + */ public static final ThreadLocal localTracingMode = ThreadLocal.withInitial(() -> 1); } diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index 2de4d3aef9..2b26900987 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -105,23 +105,18 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { final def map[B](f: A => B): IO[B] = if (isTracingEnabled) { // Don't perform map fusion when tracing is enabled. - // We may not actually have to do this - val nextIo = Map(this, f, 0) - if (isTracingEnabled) { - IOTracing(nextIo, TraceTag.Map, f.getClass) - } else { - nextIo - } + // We may end up removing map fusion altogether. + Map(this, f, 0, IOTracing.trace(TraceTag.Map, f.getClass)) } else { this match { - case Map(source, g, index) => + case Map(source, g, index, null) => // Allowed to do fixed number of map operations fused before // resetting the counter in order to avoid stack overflows; // See `IOPlatform` for details on this maximum. - if (index != fusionMaxStackDepth) Map(source, g.andThen(f), index + 1) - else Map(this, f, 0) + if (index != fusionMaxStackDepth) Map(source, g.andThen(f), index + 1, null) + else Map(this, f, 0, null) case _ => - Map(this, f, 0) + Map(this, f, 0, null) } } @@ -140,12 +135,14 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * failures would be completely silent and `IO` references would * never terminate on evaluation. */ - final def flatMap[B](f: A => IO[B]): IO[B] = - if (isTracingEnabled) { - IOTracing(Bind(this, f), TraceTag.Bind, f.getClass) + final def flatMap[B](f: A => IO[B]): IO[B] = { + val trace = if (isTracingEnabled) { + IOTracing.trace(TraceTag.Bind, f.getClass) } else { - Bind(this, f) + null } + Bind(this, f, trace) + } /** * Materializes any sequenced exceptions into value space, where @@ -161,7 +158,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * @see [[IO.raiseError]] */ def attempt: IO[Either[Throwable, A]] = - Bind(this, AttemptIO.asInstanceOf[A => IO[Either[Throwable, A]]]) + Bind(this, AttemptIO.asInstanceOf[A => IO[Either[Throwable, A]]], null) /** * Produces an `IO` reference that should execute the source on @@ -583,7 +580,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { final def bracket[B](use: A => IO[B])(release: A => IO[Unit]): IO[B] = { val nextIo = IOBracket(this)(use)((a, _) => release(a)) if (isTracingEnabled) { - IOTracing(nextIo, TraceTag.Bracket, use.getClass) + IOTracing.cached(nextIo, TraceTag.Bracket, use.getClass) } else { nextIo } @@ -624,7 +621,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { def bracketCase[B](use: A => IO[B])(release: (A, ExitCase[Throwable]) => IO[Unit]): IO[B] = { val nextIo = IOBracket(this)(use)(release) if (isTracingEnabled) { - IOTracing(nextIo, TraceTag.BracketCase, use.getClass) + IOTracing.cached(nextIo, TraceTag.BracketCase, use.getClass) } else { nextIo } @@ -701,7 +698,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * Implements `ApplicativeError.handleErrorWith`. */ def handleErrorWith[AA >: A](f: Throwable => IO[AA]): IO[AA] = - IO.Bind(this, new IOFrame.ErrorHandler(f)) + IO.Bind(this, new IOFrame.ErrorHandler(f), null) /** * Zips both this action and the parameter in parallel. @@ -735,7 +732,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * in case it ends in success */ def redeem[B](recover: Throwable => B, map: A => B): IO[B] = - IO.Bind(this, new IOFrame.Redeem(recover, map)) + IO.Bind(this, new IOFrame.Redeem(recover, map), null) /** * Returns a new value that transforms the result of the source, @@ -767,7 +764,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * in case of success */ def redeemWith[B](recover: Throwable => IO[B], bind: A => IO[B]): IO[B] = - IO.Bind(this, new IOFrame.RedeemWith(recover, bind)) + IO.Bind(this, new IOFrame.RedeemWith(recover, bind), null) override def toString: String = this match { case Pure(a) => s"IO($a)" @@ -1154,8 +1151,14 @@ object IO extends IOInstances { * Any exceptions thrown by the effect will be caught and sequenced * into the `IO`. */ - def delay[A](body: => A): IO[A] = - Delay(() => body) + def delay[A](body: => A): IO[A] = { + val nextIo = Delay(() => body) + if (isTracingEnabled) { + IOTracing.uncached(nextIo, TraceTag.Delay) + } else { + nextIo + } + } /** * Suspends a synchronous side effect which produces an `IO` in `IO`. @@ -1165,8 +1168,14 @@ object IO extends IOInstances { * thrown by the side effect will be caught and sequenced into the * `IO`. */ - def suspend[A](thunk: => IO[A]): IO[A] = - Suspend(() => thunk) + def suspend[A](thunk: => IO[A]): IO[A] = { + val nextIo = Suspend(() => thunk) + if (isTracingEnabled) { + IOTracing.uncached(nextIo, TraceTag.Suspend) + } else { + nextIo + } + } /** * Suspends a pure value in `IO`. @@ -1178,7 +1187,12 @@ object IO extends IOInstances { * (when evaluated) than `IO(42)`, due to avoiding the allocation of * extra thunks. */ - def pure[A](a: A): IO[A] = Pure(a) + def pure[A](a: A): IO[A] = + if (isTracingEnabled) { + IOTracing.uncached(Pure(a), TraceTag.Pure) + } else { + Pure(a) + } /** Alias for `IO.pure(())`. */ val unit: IO[Unit] = pure(()) @@ -1251,7 +1265,7 @@ object IO extends IOInstances { } if (isTracingEnabled) { - IOTracing(nextIo, TraceTag.Async, k.getClass) + IOTracing.cached(nextIo, TraceTag.Async, k.getClass) } else { nextIo } @@ -1296,7 +1310,7 @@ object IO extends IOInstances { } if (isTracingEnabled) { - IOTracing(nextIo, TraceTag.AsyncF, k.getClass) + IOTracing.cached(nextIo, TraceTag.AsyncF, k.getClass) } else { nextIo } @@ -1363,7 +1377,7 @@ object IO extends IOInstances { } if (isTracingEnabled) { - IOTracing(nextIo, TraceTag.Cancelable, k.getClass) + IOTracing.cached(nextIo, TraceTag.Cancelable, k.getClass) } else { nextIo } @@ -1637,10 +1651,12 @@ object IO extends IOInstances { final private[effect] case class Suspend[+A](thunk: () => IO[A]) extends IO[A] /** Corresponds to [[IO.flatMap]]. */ - final private[effect] case class Bind[E, +A](source: IO[E], f: E => IO[A]) extends IO[A] + final private[effect] case class Bind[E, +A](source: IO[E], f: E => IO[A], trace: TraceFrame) extends IO[A] /** Corresponds to [[IO.map]]. */ - final private[effect] case class Map[E, +A](source: IO[E], f: E => A, index: Int) extends IO[A] with (E => IO[A]) { + final private[effect] case class Map[E, +A](source: IO[E], f: E => A, index: Int, trace: TraceFrame) + extends IO[A] + with (E => IO[A]) { override def apply(value: E): IO[A] = Pure(f(value)) } diff --git a/core/shared/src/main/scala/cats/effect/internals/IOForkedStart.scala b/core/shared/src/main/scala/cats/effect/internals/IOForkedStart.scala index 743565d13d..161eb62770 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOForkedStart.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOForkedStart.scala @@ -54,8 +54,8 @@ private[effect] object IOForkedStart { if (limit > 0) { task match { case IO.Async(k, _) => k.isInstanceOf[IOForkedStart[_]] - case IO.Bind(other, _) => detect(other, limit - 1) - case IO.Map(other, _, _) => detect(other, limit - 1) + case IO.Bind(other, _, _) => detect(other, limit - 1) + case IO.Map(other, _, _, _) => detect(other, limit - 1) case IO.ContextSwitch(other, _, _) => detect(other, limit - 1) case _ => false } diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index 4334d47f84..0e04ad2776 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -101,11 +101,15 @@ private[effect] object IORunLoop { while ({ currentIO match { - case Bind(fa, bindNext) => + case Bind(fa, bindNext, trace) => if (bFirst ne null) { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } + if (isTracingEnabled) { + if (ctx eq null) ctx = IOContext() + if (trace ne null) ctx.pushFrame(trace) + } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa @@ -141,11 +145,15 @@ private[effect] object IORunLoop { currentIO = fa } - case bindNext @ Map(fa, _, _) => + case bindNext @ Map(fa, _, _, trace) => if (bFirst ne null) { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } + if (isTracingEnabled) { + if (ctx eq null) ctx = IOContext() + if (trace ne null) ctx.pushFrame(trace) + } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa @@ -165,7 +173,7 @@ private[effect] object IORunLoop { if (conn ne old) { if (rcb ne null) rcb.contextSwitch(conn) if (restore ne null) - currentIO = Bind(next, new RestoreContext(old, restore)) + currentIO = Bind(next, new RestoreContext(old, restore), null) } case Trace(source, frame) => @@ -221,11 +229,15 @@ private[effect] object IORunLoop { while ({ currentIO match { - case Bind(fa, bindNext) => + case Bind(fa, bindNext, trace) => if (bFirst ne null) { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } + if (isTracingEnabled) { + if (ctx eq null) ctx = IOContext() + if (trace ne null) ctx.pushFrame(trace) + } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa @@ -261,11 +273,15 @@ private[effect] object IORunLoop { currentIO = fa } - case bindNext @ Map(fa, _, _) => + case bindNext @ Map(fa, _, _, trace) => if (bFirst ne null) { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } + if (isTracingEnabled) { + if (ctx eq null) ctx = IOContext() + if (trace ne null) ctx.pushFrame(trace) + } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index 29f7e0e8c6..fd8c7f052d 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -23,7 +23,10 @@ import cats.effect.tracing.{TraceFrame, TraceTag, TracingMode} private[effect] object IOTracing { - def apply[A](source: IO[A], traceTag: TraceTag, clazz: Class[_]): IO[A] = + def uncached[A](source: IO[A], traceTag: TraceTag): IO[A] = + Trace(source, buildFrame(traceTag)) + + def cached[A](source: IO[A], traceTag: TraceTag, clazz: Class[_]): IO[A] = // val mode = localTracingMode.get() // if (mode == 1) { // Trace(source, buildCachedFrame(source.getClass, clazz)) @@ -34,6 +37,9 @@ private[effect] object IOTracing { // } Trace(source, buildCachedFrame(traceTag, clazz)) + def trace(traceTag: TraceTag, clazz: Class[_]): TraceFrame = + buildCachedFrame(traceTag, clazz) + def locallyTraced[A](source: IO[A], newMode: TracingMode): IO[A] = IO.suspend { val oldMode = localTracingMode.get() diff --git a/core/shared/src/main/scala/cats/effect/tracing/TraceTag.scala b/core/shared/src/main/scala/cats/effect/tracing/TraceTag.scala index 4402f72e12..ff39b04036 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/TraceTag.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/TraceTag.scala @@ -19,6 +19,10 @@ package cats.effect.tracing final case class TraceTag(name: String) object TraceTag { + val Pure = TraceTag("pure") + val Delay = TraceTag("delay") + val Suspend = TraceTag("suspend") + val Attempt = TraceTag("attempt") val Bind = TraceTag("flatMap") val Map = TraceTag("map") From 308789f43ea1f4bcf4189953c96b7ce56f6b6a2f Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Wed, 27 May 2020 20:45:59 -0500 Subject: [PATCH 33/78] Remove Introspect instruction --- .../effect/internals/TracingPlatformFast.java | 2 +- .../scala/cats/effect/internals/Example.scala | 4 +- .../src/main/scala/cats/effect/IO.scala | 4 +- .../cats/effect/internals/IOContext.scala | 7 ++- .../cats/effect/internals/IORunLoop.scala | 13 +--- .../cats/effect/internals/IOTracing.scala | 54 ++++++++++------- .../scala/cats/effect/tracing/IOTrace.scala | 59 +++++++++++-------- 7 files changed, 78 insertions(+), 65 deletions(-) diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java index 49c8bb6f63..9c50af3f6c 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java @@ -38,7 +38,7 @@ public final class TracingPlatformFast { public static final boolean isTracingEnabled = Optional.ofNullable(System.getProperty("cats.effect.tracing.enabled")) .filter(x -> !x.isEmpty()) .map(x -> Boolean.valueOf(x)) - .orElse(false); + .orElse(true); /** * The number of trace lines to retain during tracing. If more trace diff --git a/core/jvm/src/main/scala/cats/effect/internals/Example.scala b/core/jvm/src/main/scala/cats/effect/internals/Example.scala index b111725a3a..d398bb1e4c 100644 --- a/core/jvm/src/main/scala/cats/effect/internals/Example.scala +++ b/core/jvm/src/main/scala/cats/effect/internals/Example.scala @@ -70,8 +70,8 @@ object Example extends IOApp { _ <- IO.suspend(program).rabbitTrace _ <- IO.delay("10") trace <- IO.backtrace - _ <- IO.delay(trace.prettyPrint()) - _ <- IO.delay(trace.rawPrint()) + _ <- trace.prettyPrint + _ <- trace.compactPrint } yield ExitCode.Success } diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index 2b26900987..468e16e440 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -1633,7 +1633,7 @@ object IO extends IOInstances { IOContextShift(ec) val backtrace: IO[IOTrace] = - Introspect + IOTracing.backtrace /* -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-= */ /* IO's internal encoding: */ @@ -1695,8 +1695,6 @@ object IO extends IOInstances { restore: (A, Throwable, IOConnection, IOConnection) => IOConnection ) extends IO[A] - final private[effect] case object Introspect extends IO[IOTrace] - /* -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-= */ /** diff --git a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala index 2fcca7a50c..767305a4dc 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala @@ -26,7 +26,7 @@ import cats.effect.internals.TracingPlatformFast.maxTraceDepth */ final private[effect] class IOContext private () { - private val frames: RingBuffer[TraceFrame] = new RingBuffer(maxTraceDepth) + private var frames: RingBuffer[TraceFrame] = new RingBuffer(maxTraceDepth) private var omitted: Int = 0 def pushFrame(fr: TraceFrame): Unit = { @@ -36,6 +36,11 @@ final private[effect] class IOContext private () { } } + def resetTrace(): Unit = { + frames = new RingBuffer(maxTraceDepth) + omitted = 0 + } + def getTrace: IOTrace = IOTrace(frames.toList.toVector, omitted) diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index 0e04ad2776..ab7f1e71ba 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -17,7 +17,7 @@ package cats.effect.internals import cats.effect.IO -import cats.effect.IO.{Async, Bind, ContextSwitch, Delay, Introspect, Map, Pure, RaiseError, Suspend, Trace} +import cats.effect.IO.{Async, Bind, ContextSwitch, Delay, Map, Pure, RaiseError, Suspend, Trace} import cats.effect.tracing.TracingMode import cats.effect.internals.TracingPlatformFast.isTracingEnabled @@ -180,11 +180,6 @@ private[effect] object IORunLoop { if (ctx eq null) ctx = IOContext() ctx.pushFrame(frame) currentIO = source - - case Introspect => - if (ctx eq null) ctx = IOContext() - hasUnboxed = true - unboxed = ctx.getTrace } if (hasUnboxed) { @@ -290,12 +285,6 @@ private[effect] object IORunLoop { // box those vars in scala.runtime.ObjectRef! return suspendAsync(currentIO.asInstanceOf[IO.Async[A]], bFirst, bRest) - case Introspect => - // TODO: This can be implemented in terms of Async now - if (ctx eq null) ctx = IOContext() - hasUnboxed = true - unboxed = ctx.getTrace - case _ => return Async { (conn, ctx, cb) => loop(currentIO, conn, cb.asInstanceOf[Callback], ctx, null, bFirst, bRest) diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index fd8c7f052d..001795cc80 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -19,7 +19,7 @@ package cats.effect.internals import cats.effect.internals.TracingPlatformFast.{frameCache, localTracingMode} import cats.effect.IO import cats.effect.IO.Trace -import cats.effect.tracing.{TraceFrame, TraceTag, TracingMode} +import cats.effect.tracing.{IOTrace, TraceFrame, TraceTag, TracingMode} private[effect] object IOTracing { @@ -41,25 +41,28 @@ private[effect] object IOTracing { buildCachedFrame(traceTag, clazz) def locallyTraced[A](source: IO[A], newMode: TracingMode): IO[A] = - IO.suspend { - val oldMode = localTracingMode.get() - localTracingMode.set(newMode.tag) - - // In the event of cancellation, the tracing mode will be reset - // when the thread grabs a new task to run (via Async). - source.redeemWith( - e => - IO.suspend { - localTracingMode.set(oldMode) - IO.raiseError(e) - }, - a => - IO.suspend { - localTracingMode.set(oldMode) - IO.pure(a) - } - ) - } + for { + _ <- resetTrace + a <- IO.suspend { + val oldMode = localTracingMode.get() + localTracingMode.set(newMode.tag) + + // In the event of cancellation, the tracing mode will be reset + // when the thread grabs a new task to run (via Async). + source.redeemWith( + e => + IO.suspend { + localTracingMode.set(oldMode) + IO.raiseError(e) + }, + a => + IO.suspend { + localTracingMode.set(oldMode) + IO.pure(a) + } + ) + } + } yield a def getLocalTracingMode(): TracingMode = TracingMode.fromInt(localTracingMode.get()) @@ -67,6 +70,17 @@ private[effect] object IOTracing { def setLocalTracingMode(mode: TracingMode): Unit = localTracingMode.set(mode.tag) + val backtrace: IO[IOTrace] = + IO.Async { (_, ctx, cb) => + cb(Right(ctx.getTrace)) + } + + private val resetTrace: IO[Unit] = + IO.Async { (_, ctx, cb) => + ctx.resetTrace() + cb(Right(())) + } + private def buildCachedFrame(traceTag: TraceTag, keyClass: Class[_]): TraceFrame = { val cachedFr = frameCache.get(keyClass).asInstanceOf[TraceFrame] if (cachedFr eq null) { diff --git a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala index 8b39a4fa22..6206de4143 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala @@ -16,36 +16,39 @@ package cats.effect.tracing +import cats.effect.IO + final case class IOTrace(frames: Vector[TraceFrame], omitted: Int) { import IOTrace._ - def rawPrint(): Unit = { + def compact: String = { def renderStackTraceElement(ste: StackTraceElement): String = { - val className = ste.getClassName.replaceAll("\\$", "") - val methodName = anonfunRegex.findFirstMatchIn(ste.getMethodName) match { - case Some(mat) => mat.group(1) - case None => ste.getMethodName - } - + val className = demangleClassName(ste.getClassName) + val methodName = demangleMethodName(ste.getMethodName) s"$className.$methodName (${ste.getFileName}:${ste.getLineNumber})" } - System.err.println(s"IOTrace: $omitted omitted frames") - frames.foreach { f => - val desc = s"\t${f.tag.name} at " + f.stackTrace.headOption.map(renderStackTraceElement).getOrElse("(...)") - System.err.println(desc) - } - System.err.println() + val acc0 = s"IOTrace: $omitted omitted frames\n" + val acc1 = frames.foldLeft(acc0)((acc, f) => + acc + s"\t${f.tag.name} at " + f.stackTrace.headOption.map(renderStackTraceElement).getOrElse("(...)") + "\n" + ) + "\n" + + acc1 } - def prettyPrint(): Unit = { - val render = loop("", 0, true, frames.toList) - System.err.println(s"IOTrace: $omitted omitted frames") - System.err.println(render) - System.err.println() + def compactPrint: IO[Unit] = + IO(System.err.println(compact)) + + def pretty: String = { + val acc0 = s"IOTrace: $omitted omitted frames" + val acc1 = acc0 + loop("", 0, true, frames.toList) + acc1 } + def prettyPrint: IO[Unit] = + IO(System.err.println(pretty)) + private def loop(acc: String, indent: Int, init: Boolean, rest: List[TraceFrame]): String = { val TurnRight = "╰" val InverseTurnRight = "╭" @@ -54,11 +57,8 @@ final case class IOTrace(frames: Vector[TraceFrame], omitted: Int) { val Line = "│" def renderStackTraceElement(ste: StackTraceElement, last: Boolean): String = { - val className = ste.getClassName.replaceAll("\\$", "") - val methodName = anonfunRegex.findFirstMatchIn(ste.getMethodName) match { - case Some(mat) => mat.group(1) - case None => ste.getMethodName - } + val className = demangleClassName(ste.getClassName) + val methodName = demangleMethodName(ste.getMethodName) val junc = if (last) TurnRight else Junction @@ -80,12 +80,19 @@ final case class IOTrace(frames: Vector[TraceFrame], omitted: Int) { loop(acc + acc2 + inner + innerLines + Line + "\n", indent, false, ks) } - case Nil => { - acc + TurnRight + " Done" - } + case Nil => acc } } + private def demangleClassName(className: String): String = + className.replaceAll("\\$", "") + + private def demangleMethodName(methodName: String): String = + anonfunRegex.findFirstMatchIn(methodName) match { + case Some(mat) => mat.group(1) + case None => methodName + } + } object IOTrace { From 77ccab3416803cc4af2f306011e2cbf59a9ec684 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Thu, 28 May 2020 00:21:01 -0500 Subject: [PATCH 34/78] Ring buffer tests --- .../cats/effect/internals/RingBuffer.scala | 14 +++-- .../effect/internals/RingBufferTests.scala | 54 +++++++++++++++++++ 2 files changed, 64 insertions(+), 4 deletions(-) create mode 100644 core/shared/src/test/scala/cats/effect/internals/RingBufferTests.scala diff --git a/core/shared/src/main/scala/cats/effect/internals/RingBuffer.scala b/core/shared/src/main/scala/cats/effect/internals/RingBuffer.scala index 8673820b19..2ffc7237f6 100644 --- a/core/shared/src/main/scala/cats/effect/internals/RingBuffer.scala +++ b/core/shared/src/main/scala/cats/effect/internals/RingBuffer.scala @@ -25,18 +25,18 @@ final private[internals] class RingBuffer[A <: AnyRef](size: Int) { import RingBuffer._ - private[this] val capacity = nextPowerOfTwo(size) - private[this] val mask = capacity - 1 + private[this] val length = nextPowerOfTwo(size) + private[this] val mask = length - 1 // TODO: this can be an expensive allocation // either construct it lazily or expand it on-demand - private[this] val array: Array[AnyRef] = new Array(capacity) + private[this] val array: Array[AnyRef] = new Array(length) private[this] var writeIndex: Int = 0 private[this] var readIndex: Int = 0 def push(a: A): A = { val wi = writeIndex & mask - if (writeIndex == readIndex + capacity) { + if (writeIndex == readIndex + length) { val old = array(wi) array(wi) = a // TODO: overflow at int.maxvalue? @@ -50,6 +50,12 @@ final private[internals] class RingBuffer[A <: AnyRef](size: Int) { } } + def capacity: Int = + length + + def isEmpty: Boolean = + readIndex == writeIndex + // TODO: expose this as an iterator instead? def toList: List[A] = (readIndex until writeIndex).toList diff --git a/core/shared/src/test/scala/cats/effect/internals/RingBufferTests.scala b/core/shared/src/test/scala/cats/effect/internals/RingBufferTests.scala new file mode 100644 index 0000000000..b9e7ec01d4 --- /dev/null +++ b/core/shared/src/test/scala/cats/effect/internals/RingBufferTests.scala @@ -0,0 +1,54 @@ +/* + * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers + * + * Licensed 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 cats.effect.internals + +import org.scalatest.funsuite.AnyFunSuite +import org.scalatest.matchers.should.Matchers + +class RingBufferTests extends AnyFunSuite with Matchers with TestUtils { + test("empty ring buffer") { + val buffer = new RingBuffer[Integer](2) + buffer.isEmpty shouldBe true + } + + test("non-empty ring buffer") { + val buffer = new RingBuffer[Integer](2) + buffer.push(0) + buffer.isEmpty shouldBe false + } + + test("size is a power of two") { + new RingBuffer[Integer](0).capacity shouldBe 1 + new RingBuffer[Integer](1).capacity shouldBe 1 + new RingBuffer[Integer](2).capacity shouldBe 2 + new RingBuffer[Integer](3).capacity shouldBe 4 + new RingBuffer[Integer](127).capacity shouldBe 128 + new RingBuffer[Integer](13000).capacity shouldBe 16384 + } + + test("reading and writing elements") { + val buffer = new RingBuffer[Integer](4) + for (i <- 0 to 3) buffer.push(i) + buffer.toList shouldBe List(0, 1, 2, 3) + } + + test("overwriting elements") { + val buffer = new RingBuffer[Integer](4) + for (i <- 0 to 100) buffer.push(i) + buffer.toList shouldBe List(97, 98, 99, 100) + } +} From d234eec201a1a16209483cd9be7dffb833ec72d1 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Thu, 28 May 2020 21:40:09 -0500 Subject: [PATCH 35/78] Use private[this] in IOTracing --- .../effect/benchmarks/DeepBindBenchmark.scala | 20 +++++----- .../effect/internals/TracingPlatformFast.java | 14 +------ .../cats/effect/internals/IOTracing.scala | 37 +++++++++++++------ 3 files changed, 36 insertions(+), 35 deletions(-) diff --git a/benchmarks/shared/src/main/scala/cats/effect/benchmarks/DeepBindBenchmark.scala b/benchmarks/shared/src/main/scala/cats/effect/benchmarks/DeepBindBenchmark.scala index 95811c2a37..c7ce40d2be 100644 --- a/benchmarks/shared/src/main/scala/cats/effect/benchmarks/DeepBindBenchmark.scala +++ b/benchmarks/shared/src/main/scala/cats/effect/benchmarks/DeepBindBenchmark.scala @@ -54,16 +54,16 @@ class DeepBindBenchmark { loop(0).unsafeRunSync() } -// @Benchmark -// def delay(): Int = { -// def loop(i: Int): IO[Int] = -// for { -// j <- IO(i) -// _ <- if (j > size) IO(j) else loop(j + 1) -// } yield j -// -// loop(0).unsafeRunSync() -// } + @Benchmark + def delay(): Int = { + def loop(i: Int): IO[Int] = + for { + j <- IO(i) + _ <- if (j > size) IO(j) else loop(j + 1) + } yield j + + loop(0).unsafeRunSync() + } // @Benchmark // def async(): Int = { diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java index 9c50af3f6c..9b0335d69a 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java @@ -38,7 +38,7 @@ public final class TracingPlatformFast { public static final boolean isTracingEnabled = Optional.ofNullable(System.getProperty("cats.effect.tracing.enabled")) .filter(x -> !x.isEmpty()) .map(x -> Boolean.valueOf(x)) - .orElse(true); + .orElse(false); /** * The number of trace lines to retain during tracing. If more trace @@ -56,16 +56,4 @@ public final class TracingPlatformFast { }) .orElse(512); - /** - * Cache for trace frames. Keys are references to: - * - lambda classes - */ - public static final ConcurrentHashMap, Object> frameCache = new ConcurrentHashMap<>(); - - /** - * Thread-local state that stores the lexical tracing - * mode for the fiber bound to the current thread. - */ - public static final ThreadLocal localTracingMode = ThreadLocal.withInitial(() -> 1); - } diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index 001795cc80..bf961493b2 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -16,7 +16,8 @@ package cats.effect.internals -import cats.effect.internals.TracingPlatformFast.{frameCache, localTracingMode} +import java.util.concurrent.ConcurrentHashMap + import cats.effect.IO import cats.effect.IO.Trace import cats.effect.tracing.{IOTrace, TraceFrame, TraceTag, TracingMode} @@ -45,7 +46,7 @@ private[effect] object IOTracing { _ <- resetTrace a <- IO.suspend { val oldMode = localTracingMode.get() - localTracingMode.set(newMode.tag) + localTracingMode.set(newMode) // In the event of cancellation, the tracing mode will be reset // when the thread grabs a new task to run (via Async). @@ -65,24 +66,18 @@ private[effect] object IOTracing { } yield a def getLocalTracingMode(): TracingMode = - TracingMode.fromInt(localTracingMode.get()) + localTracingMode.get() def setLocalTracingMode(mode: TracingMode): Unit = - localTracingMode.set(mode.tag) + localTracingMode.set(mode) val backtrace: IO[IOTrace] = IO.Async { (_, ctx, cb) => cb(Right(ctx.getTrace)) } - private val resetTrace: IO[Unit] = - IO.Async { (_, ctx, cb) => - ctx.resetTrace() - cb(Right(())) - } - private def buildCachedFrame(traceTag: TraceTag, keyClass: Class[_]): TraceFrame = { - val cachedFr = frameCache.get(keyClass).asInstanceOf[TraceFrame] + val cachedFr = frameCache.get(keyClass) if (cachedFr eq null) { val fr = buildFrame(traceTag) frameCache.put(keyClass, fr) @@ -100,7 +95,25 @@ private[effect] object IOTracing { TraceFrame(traceTag, stackTrace) } - private val classBlacklist = List( + private[this] val resetTrace: IO[Unit] = + IO.Async { (_, ctx, cb) => + ctx.resetTrace() + cb(Right(())) + } + + /** + * Cache for trace frames. Keys are references to: + * - lambda classes + */ + private[this] val frameCache: ConcurrentHashMap[Class[_], TraceFrame] = new ConcurrentHashMap() + + /** + * Thread-local state that stores the lexical tracing + * mode for the fiber bound to the current thread. + */ + private[this] val localTracingMode: ThreadLocal[TracingMode] = ThreadLocal.withInitial(() => TracingMode.Rabbit) + + private[this] val classBlacklist = List( "cats.effect.", "cats.", "sbt.", From e52aee478baaa8ddda9c3406ccb13034efef5d1c Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Thu, 28 May 2020 21:43:47 -0500 Subject: [PATCH 36/78] wip --- core/shared/src/main/scala/cats/effect/internals/IOTracing.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index bf961493b2..2c226b698c 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -71,6 +71,7 @@ private[effect] object IOTracing { def setLocalTracingMode(mode: TracingMode): Unit = localTracingMode.set(mode) + // TODO: def might be faster than value val backtrace: IO[IOTrace] = IO.Async { (_, ctx, cb) => cb(Right(ctx.getTrace)) From 74f7e3d324e3ddba3bbc297213af4d1c63086633 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Fri, 29 May 2020 00:00:54 -0500 Subject: [PATCH 37/78] optimizations --- .../effect/benchmarks/DeepBindBenchmark.scala | 20 ++++++------- .../effect/internals/TracingPlatformFast.java | 1 - .../cats/effect/internals/IORunLoop.scala | 12 +++++--- .../cats/effect/internals/IOTracing.scala | 29 ++++++++++++------- 4 files changed, 36 insertions(+), 26 deletions(-) diff --git a/benchmarks/shared/src/main/scala/cats/effect/benchmarks/DeepBindBenchmark.scala b/benchmarks/shared/src/main/scala/cats/effect/benchmarks/DeepBindBenchmark.scala index c7ce40d2be..95811c2a37 100644 --- a/benchmarks/shared/src/main/scala/cats/effect/benchmarks/DeepBindBenchmark.scala +++ b/benchmarks/shared/src/main/scala/cats/effect/benchmarks/DeepBindBenchmark.scala @@ -54,16 +54,16 @@ class DeepBindBenchmark { loop(0).unsafeRunSync() } - @Benchmark - def delay(): Int = { - def loop(i: Int): IO[Int] = - for { - j <- IO(i) - _ <- if (j > size) IO(j) else loop(j + 1) - } yield j - - loop(0).unsafeRunSync() - } +// @Benchmark +// def delay(): Int = { +// def loop(i: Int): IO[Int] = +// for { +// j <- IO(i) +// _ <- if (j > size) IO(j) else loop(j + 1) +// } yield j +// +// loop(0).unsafeRunSync() +// } // @Benchmark // def async(): Int = { diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java index 9b0335d69a..e1b48eb8f8 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java @@ -17,7 +17,6 @@ package cats.effect.internals; import java.util.Optional; -import java.util.concurrent.ConcurrentHashMap; /** * Holds platform-specific flags that control tracing behavior. diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index ab7f1e71ba..5dba213359 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -101,12 +101,13 @@ private[effect] object IORunLoop { while ({ currentIO match { - case Bind(fa, bindNext, trace) => + case bind @ Bind(fa, bindNext, _) => if (bFirst ne null) { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } if (isTracingEnabled) { + val trace = bind.trace if (ctx eq null) ctx = IOContext() if (trace ne null) ctx.pushFrame(trace) } @@ -145,12 +146,13 @@ private[effect] object IORunLoop { currentIO = fa } - case bindNext @ Map(fa, _, _, trace) => + case bindNext @ Map(fa, _, _, _) => if (bFirst ne null) { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } if (isTracingEnabled) { + val trace = bindNext.trace if (ctx eq null) ctx = IOContext() if (trace ne null) ctx.pushFrame(trace) } @@ -224,12 +226,13 @@ private[effect] object IORunLoop { while ({ currentIO match { - case Bind(fa, bindNext, trace) => + case bind @ Bind(fa, bindNext, _) => if (bFirst ne null) { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } if (isTracingEnabled) { + val trace = bind.trace if (ctx eq null) ctx = IOContext() if (trace ne null) ctx.pushFrame(trace) } @@ -268,12 +271,13 @@ private[effect] object IORunLoop { currentIO = fa } - case bindNext @ Map(fa, _, _, trace) => + case bindNext @ Map(fa, _, _, _) => if (bFirst ne null) { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } if (isTracingEnabled) { + val trace = bindNext.trace if (ctx eq null) ctx = IOContext() if (trace ne null) ctx.pushFrame(trace) } diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index 2c226b698c..de70df6fb9 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -24,19 +24,26 @@ import cats.effect.tracing.{IOTrace, TraceFrame, TraceTag, TracingMode} private[effect] object IOTracing { - def uncached[A](source: IO[A], traceTag: TraceTag): IO[A] = - Trace(source, buildFrame(traceTag)) - - def cached[A](source: IO[A], traceTag: TraceTag, clazz: Class[_]): IO[A] = -// val mode = localTracingMode.get() -// if (mode == 1) { -// Trace(source, buildCachedFrame(source.getClass, clazz)) -// } else if (mode == 2) { -// Trace(source, buildFrame(source.getClass)) -// } else { -// source + // TODO: It may be worth tracking mode in the global flag + // marking the ones we want. This avoids a thread-local on uncachable + // nodes which incurs the bulk of the performance hit. + def uncached[A](source: IO[A], traceTag: TraceTag): IO[A] = { + if (false) { + println(traceTag) + } +// Trace(source, buildFrame(traceTag)) + source + } + + // TODO: Avoid trace tag for primitive ops and rely on class + def cached[A](source: IO[A], traceTag: TraceTag, clazz: Class[_]): IO[A] = { +// localTracingMode.get() match { +// case TracingMode.Rabbit => Trace(source, buildCachedFrame(traceTag, clazz)) +// case TracingMode.Slug => Trace(source, buildFrame(traceTag)) +// case _ => source // } Trace(source, buildCachedFrame(traceTag, clazz)) + } def trace(traceTag: TraceTag, clazz: Class[_]): TraceFrame = buildCachedFrame(traceTag, clazz) From 2eb49f47dc5c5b116b241b4cb32852448db24c75 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Mon, 1 Jun 2020 19:34:41 -0500 Subject: [PATCH 38/78] Use AnyRef to capture trace frame --- core/shared/src/main/scala/cats/effect/IO.scala | 4 ++-- .../main/scala/cats/effect/internals/IORunLoop.scala | 10 +++++----- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index 468e16e440..c2e4e9ee3b 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -1651,10 +1651,10 @@ object IO extends IOInstances { final private[effect] case class Suspend[+A](thunk: () => IO[A]) extends IO[A] /** Corresponds to [[IO.flatMap]]. */ - final private[effect] case class Bind[E, +A](source: IO[E], f: E => IO[A], trace: TraceFrame) extends IO[A] + final private[effect] case class Bind[E, +A](source: IO[E], f: E => IO[A], trace: AnyRef) extends IO[A] /** Corresponds to [[IO.map]]. */ - final private[effect] case class Map[E, +A](source: IO[E], f: E => A, index: Int, trace: TraceFrame) + final private[effect] case class Map[E, +A](source: IO[E], f: E => A, index: Int, trace: AnyRef) extends IO[A] with (E => IO[A]) { override def apply(value: E): IO[A] = diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index 5dba213359..b4a7aaeb59 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -18,7 +18,7 @@ package cats.effect.internals import cats.effect.IO import cats.effect.IO.{Async, Bind, ContextSwitch, Delay, Map, Pure, RaiseError, Suspend, Trace} -import cats.effect.tracing.TracingMode +import cats.effect.tracing.{TraceFrame, TracingMode} import cats.effect.internals.TracingPlatformFast.isTracingEnabled import scala.util.control.NonFatal @@ -107,7 +107,7 @@ private[effect] object IORunLoop { bRest.push(bFirst) } if (isTracingEnabled) { - val trace = bind.trace + val trace = bind.trace.asInstanceOf[TraceFrame] if (ctx eq null) ctx = IOContext() if (trace ne null) ctx.pushFrame(trace) } @@ -152,7 +152,7 @@ private[effect] object IORunLoop { bRest.push(bFirst) } if (isTracingEnabled) { - val trace = bindNext.trace + val trace = bindNext.trace.asInstanceOf[TraceFrame] if (ctx eq null) ctx = IOContext() if (trace ne null) ctx.pushFrame(trace) } @@ -232,7 +232,7 @@ private[effect] object IORunLoop { bRest.push(bFirst) } if (isTracingEnabled) { - val trace = bind.trace + val trace = bind.trace.asInstanceOf[TraceFrame] if (ctx eq null) ctx = IOContext() if (trace ne null) ctx.pushFrame(trace) } @@ -277,7 +277,7 @@ private[effect] object IORunLoop { bRest.push(bFirst) } if (isTracingEnabled) { - val trace = bindNext.trace + val trace = bindNext.trace.asInstanceOf[TraceFrame] if (ctx eq null) ctx = IOContext() if (trace ne null) ctx.pushFrame(trace) } From 7b980d805b85997090c4b1434adc0e69293d9935 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Wed, 3 Jun 2020 21:56:18 -0500 Subject: [PATCH 39/78] Don't push to ring buffer when not in a tracing region --- .../effect/internals/TracingPlatformFast.java | 2 +- .../src/main/scala/cats/effect/IO.scala | 12 ++--- .../cats/effect/internals/IORunLoop.scala | 27 ++++++++--- .../cats/effect/internals/IOTracing.scala | 48 ++++++++----------- 4 files changed, 49 insertions(+), 40 deletions(-) diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java index e1b48eb8f8..17e0d7a68e 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java @@ -37,7 +37,7 @@ public final class TracingPlatformFast { public static final boolean isTracingEnabled = Optional.ofNullable(System.getProperty("cats.effect.tracing.enabled")) .filter(x -> !x.isEmpty()) .map(x -> Boolean.valueOf(x)) - .orElse(false); + .orElse(true); /** * The number of trace lines to retain during tracing. If more trace diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index c2e4e9ee3b..2980774ddb 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -135,14 +135,12 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * failures would be completely silent and `IO` references would * never terminate on evaluation. */ - final def flatMap[B](f: A => IO[B]): IO[B] = { - val trace = if (isTracingEnabled) { - IOTracing.trace(TraceTag.Bind, f.getClass) + final def flatMap[B](f: A => IO[B]): IO[B] = + if (isTracingEnabled) { + Bind(this, f, IOTracing.trace(TraceTag.Bind, f.getClass)) } else { - null + Bind(this, f, null) } - Bind(this, f, trace) - } /** * Materializes any sequenced exceptions into value space, where @@ -1682,6 +1680,8 @@ object IO extends IOInstances { final private[effect] case class Trace[A](source: IO[A], frame: TraceFrame) extends IO[A] +// final private[effect] case class SetTracingMode(mode: TracingMode) extends IO[Nothing] + /** * An internal state for that optimizes changes to * [[internals.IOConnection]]. diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index b4a7aaeb59..3244aaaecd 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -19,6 +19,8 @@ package cats.effect.internals import cats.effect.IO import cats.effect.IO.{Async, Bind, ContextSwitch, Delay, Map, Pure, RaiseError, Suspend, Trace} import cats.effect.tracing.{TraceFrame, TracingMode} + +//TODO: define in a private variable? import cats.effect.internals.TracingPlatformFast.isTracingEnabled import scala.util.control.NonFatal @@ -223,6 +225,12 @@ private[effect] object IORunLoop { // for code reuse between Pure and Delay var hasUnboxed: Boolean = false var unboxed: AnyRef = null + val inTracingRegion: Boolean = if (isTracingEnabled) { + IOTracing.getLocalTracingMode() match { + case TracingMode.Disabled => false + case _ => true + } + } else false while ({ currentIO match { @@ -231,10 +239,9 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if (isTracingEnabled) { - val trace = bind.trace.asInstanceOf[TraceFrame] + if (isTracingEnabled && inTracingRegion) { if (ctx eq null) ctx = IOContext() - if (trace ne null) ctx.pushFrame(trace) + if (bind.trace ne null) ctx.pushFrame(bind.trace.asInstanceOf[TraceFrame]) } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa @@ -276,14 +283,22 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if (isTracingEnabled) { - val trace = bindNext.trace.asInstanceOf[TraceFrame] + if (isTracingEnabled && inTracingRegion) { if (ctx eq null) ctx = IOContext() - if (trace ne null) ctx.pushFrame(trace) + if (bindNext.trace ne null) ctx.pushFrame(bindNext.trace.asInstanceOf[TraceFrame]) } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa + case Trace(source, frame) => + // We should never see a Trace node where the following + // conditional evaluates to false. + if (isTracingEnabled && inTracingRegion) { + if (ctx eq null) ctx = IOContext() + ctx.pushFrame(frame) + } + currentIO = source + case Async(_, _) => // Cannot inline the code of this method — as it would // box those vars in scala.runtime.ObjectRef! diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index de70df6fb9..7a62923cdc 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -27,23 +27,19 @@ private[effect] object IOTracing { // TODO: It may be worth tracking mode in the global flag // marking the ones we want. This avoids a thread-local on uncachable // nodes which incurs the bulk of the performance hit. - def uncached[A](source: IO[A], traceTag: TraceTag): IO[A] = { - if (false) { - println(traceTag) + def uncached[A](source: IO[A], traceTag: TraceTag): IO[A] = + localTracingMode.get() match { + case TracingMode.Slug => Trace(source, buildFrame(traceTag)) + case _ => source } -// Trace(source, buildFrame(traceTag)) - source - } // TODO: Avoid trace tag for primitive ops and rely on class - def cached[A](source: IO[A], traceTag: TraceTag, clazz: Class[_]): IO[A] = { -// localTracingMode.get() match { -// case TracingMode.Rabbit => Trace(source, buildCachedFrame(traceTag, clazz)) -// case TracingMode.Slug => Trace(source, buildFrame(traceTag)) -// case _ => source -// } - Trace(source, buildCachedFrame(traceTag, clazz)) - } + def cached[A](source: IO[A], traceTag: TraceTag, clazz: Class[_]): IO[A] = + localTracingMode.get() match { + case TracingMode.Rabbit => Trace(source, buildCachedFrame(traceTag, clazz)) + case TracingMode.Slug => Trace(source, buildFrame(traceTag)) + case TracingMode.Disabled => source + } def trace(traceTag: TraceTag, clazz: Class[_]): TraceFrame = buildCachedFrame(traceTag, clazz) @@ -56,18 +52,16 @@ private[effect] object IOTracing { localTracingMode.set(newMode) // In the event of cancellation, the tracing mode will be reset - // when the thread grabs a new task to run (via Async). + // when the thread grabs a new task to run (via Async or IORunLoop.start). source.redeemWith( - e => - IO.suspend { - localTracingMode.set(oldMode) - IO.raiseError(e) - }, - a => - IO.suspend { - localTracingMode.set(oldMode) - IO.pure(a) - } + e => IO.suspend { + localTracingMode.set(oldMode) + IO.raiseError(e) + }, + a => IO.suspend { + localTracingMode.set(oldMode) + IO.pure(a) + } ) } } yield a @@ -95,7 +89,7 @@ private[effect] object IOTracing { } } - private def buildFrame(traceTag: TraceTag): TraceFrame = { + def buildFrame(traceTag: TraceTag): TraceFrame = { // TODO: proper trace calculation val stackTrace = new Throwable().getStackTrace.toList .dropWhile(l => classBlacklist.exists(b => l.getClassName.startsWith(b))) @@ -119,7 +113,7 @@ private[effect] object IOTracing { * Thread-local state that stores the lexical tracing * mode for the fiber bound to the current thread. */ - private[this] val localTracingMode: ThreadLocal[TracingMode] = ThreadLocal.withInitial(() => TracingMode.Rabbit) + private[this] val localTracingMode: ThreadLocal[TracingMode] = ThreadLocal.withInitial(() => TracingMode.Disabled) private[this] val classBlacklist = List( "cats.effect.", From bf95848d8077508d06f2ecacbd262e2907782a1e Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Wed, 3 Jun 2020 23:28:34 -0500 Subject: [PATCH 40/78] wip --- .../src/main/scala/cats/effect/internals/IORunLoop.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index 3244aaaecd..7f38acd1c3 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -241,7 +241,8 @@ private[effect] object IORunLoop { } if (isTracingEnabled && inTracingRegion) { if (ctx eq null) ctx = IOContext() - if (bind.trace ne null) ctx.pushFrame(bind.trace.asInstanceOf[TraceFrame]) + val trace = bind.trace + if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa @@ -285,7 +286,8 @@ private[effect] object IORunLoop { } if (isTracingEnabled && inTracingRegion) { if (ctx eq null) ctx = IOContext() - if (bindNext.trace ne null) ctx.pushFrame(bindNext.trace.asInstanceOf[TraceFrame]) + val trace = bindNext.trace + if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa From 273a0e4b466536bc25bbcb764b1848f55c088346 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Sun, 7 Jun 2020 22:20:37 -0500 Subject: [PATCH 41/78] Replace global tracing boolean with global tracing mode --- .../effect/internals/TracingPlatformFast.java | 22 ++-- .../scala/cats/effect/internals/Example.scala | 2 +- .../src/main/scala/cats/effect/IO.scala | 96 +++++++--------- .../cats/effect/internals/IOBracket.scala | 21 +--- .../cats/effect/internals/IOContext.scala | 8 ++ .../cats/effect/internals/IORunLoop.scala | 108 +++++++----------- .../cats/effect/internals/IOTracing.scala | 87 ++++++-------- 7 files changed, 144 insertions(+), 200 deletions(-) diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java index 17e0d7a68e..b820b4ce5f 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java @@ -30,14 +30,22 @@ public final class TracingPlatformFast { /** - * A boolean flag that enables or disables tracing for a JVM process. - * Since it is declared static and final, The JIT compiler has the liberty - * to completely eliminate code paths consequent to the conditional. + * An integer flag that sets a global tracing mode for a JVM process. + * 0 - DISABLED + * 1 - RABBIT + * 2 - SLUG + * TODO: move to enum */ - public static final boolean isTracingEnabled = Optional.ofNullable(System.getProperty("cats.effect.tracing.enabled")) - .filter(x -> !x.isEmpty()) - .map(x -> Boolean.valueOf(x)) - .orElse(true); + public static final int tracingMode = Optional.ofNullable(System.getProperty("cats.effect.tracing.mode")) + .filter(x -> !x.isEmpty()) + .flatMap(x -> { + try { + return Optional.of(Integer.valueOf(x)); + } catch (Exception e) { + return Optional.empty(); + } + }) + .orElse(0); /** * The number of trace lines to retain during tracing. If more trace diff --git a/core/jvm/src/main/scala/cats/effect/internals/Example.scala b/core/jvm/src/main/scala/cats/effect/internals/Example.scala index d398bb1e4c..84b8766dd2 100644 --- a/core/jvm/src/main/scala/cats/effect/internals/Example.scala +++ b/core/jvm/src/main/scala/cats/effect/internals/Example.scala @@ -67,7 +67,7 @@ object Example extends IOApp { override def run(args: List[String]): IO[ExitCode] = for { - _ <- IO.suspend(program).rabbitTrace + _ <- IO.suspend(program).traced _ <- IO.delay("10") trace <- IO.backtrace _ <- trace.prettyPrint diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index 2980774ddb..01b212933f 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -19,7 +19,7 @@ package effect import cats.effect.internals._ import cats.effect.internals.IOPlatform.fusionMaxStackDepth -import cats.effect.internals.TracingPlatformFast.isTracingEnabled +import cats.effect.internals.TracingPlatformFast.tracingMode import scala.annotation.unchecked.uncheckedVariance import scala.concurrent.duration._ @@ -27,7 +27,7 @@ import scala.concurrent.{ExecutionContext, Future, Promise, TimeoutException} import scala.util.control.NonFatal import scala.util.{Failure, Left, Right, Success, Try} import cats.data.Ior -import cats.effect.tracing.{IOTrace, TraceFrame, TraceTag, TracingMode} +import cats.effect.tracing.{IOTrace, TraceFrame, TraceTag} /** * A pure abstraction representing the intention to perform a @@ -103,10 +103,12 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * never terminate on evaluation. */ final def map[B](f: A => B): IO[B] = - if (isTracingEnabled) { - // Don't perform map fusion when tracing is enabled. - // We may end up removing map fusion altogether. + // Don't perform map fusion when tracing is enabled. + // We may end up removing map fusion altogether. + if (tracingMode == 1) { Map(this, f, 0, IOTracing.trace(TraceTag.Map, f.getClass)) + } else if (tracingMode == 2) { + Map(this, f, 0, null) } else { this match { case Map(source, g, index, null) => @@ -135,13 +137,18 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * failures would be completely silent and `IO` references would * never terminate on evaluation. */ - final def flatMap[B](f: A => IO[B]): IO[B] = - if (isTracingEnabled) { - Bind(this, f, IOTracing.trace(TraceTag.Bind, f.getClass)) + final def flatMap[B](f: A => IO[B]): IO[B] = { + val trace = if (tracingMode == 1) { + IOTracing.trace(TraceTag.Bind, f.getClass) + } else if (tracingMode == 2) { + null } else { - Bind(this, f, null) + null } + Bind(this, f, trace) + } + /** * Materializes any sequenced exceptions into value space, where * they may be handled. @@ -577,11 +584,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { */ final def bracket[B](use: A => IO[B])(release: A => IO[Unit]): IO[B] = { val nextIo = IOBracket(this)(use)((a, _) => release(a)) - if (isTracingEnabled) { - IOTracing.cached(nextIo, TraceTag.Bracket, use.getClass) - } else { - nextIo - } + nextIo } /** @@ -618,11 +621,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { */ def bracketCase[B](use: A => IO[B])(release: (A, ExitCase[Throwable]) => IO[Unit]): IO[B] = { val nextIo = IOBracket(this)(use)(release) - if (isTracingEnabled) { - IOTracing.cached(nextIo, TraceTag.BracketCase, use.getClass) - } else { - nextIo - } + nextIo } /** @@ -804,19 +803,8 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { def <&[B](another: IO[B])(implicit p: NonEmptyParallel[IO]): IO[A] = p.parProductL(this)(another) - def slugTrace: IO[A] = - if (isTracingEnabled) { - IOTracing.locallyTraced(this, TracingMode.Slug) - } else { - this - } - - def rabbitTrace: IO[A] = - if (isTracingEnabled) { - IOTracing.locallyTraced(this, TracingMode.Rabbit) - } else { - this - } + def traced: IO[A] = + IOTracing.traced(this) } abstract private[effect] class IOParallelNewtype extends internals.IOTimerRef with internals.IOCompanionBinaryCompat { @@ -1151,8 +1139,10 @@ object IO extends IOInstances { */ def delay[A](body: => A): IO[A] = { val nextIo = Delay(() => body) - if (isTracingEnabled) { + if (tracingMode == 1) { IOTracing.uncached(nextIo, TraceTag.Delay) + } else if (tracingMode == 2) { + nextIo } else { nextIo } @@ -1168,8 +1158,10 @@ object IO extends IOInstances { */ def suspend[A](thunk: => IO[A]): IO[A] = { val nextIo = Suspend(() => thunk) - if (isTracingEnabled) { + if (tracingMode == 1) { IOTracing.uncached(nextIo, TraceTag.Suspend) + } else if (tracingMode == 2) { + nextIo } else { nextIo } @@ -1185,12 +1177,16 @@ object IO extends IOInstances { * (when evaluated) than `IO(42)`, due to avoiding the allocation of * extra thunks. */ - def pure[A](a: A): IO[A] = - if (isTracingEnabled) { - IOTracing.uncached(Pure(a), TraceTag.Pure) + def pure[A](a: A): IO[A] = { + val nextIo = Pure(a) + if (tracingMode == 1) { + IOTracing.uncached(nextIo, TraceTag.Pure) + } else if (tracingMode == 2) { + nextIo } else { - Pure(a) + nextIo } + } /** Alias for `IO.pure(())`. */ val unit: IO[Unit] = pure(()) @@ -1262,11 +1258,7 @@ object IO extends IOInstances { catch { case NonFatal(t) => cb2(Left(t)) } } - if (isTracingEnabled) { - IOTracing.cached(nextIo, TraceTag.Async, k.getClass) - } else { - nextIo - } + nextIo } /** @@ -1307,11 +1299,7 @@ object IO extends IOInstances { IORunLoop.startCancelable(fa, conn2, Callback.report) } - if (isTracingEnabled) { - IOTracing.cached(nextIo, TraceTag.AsyncF, k.getClass) - } else { - nextIo - } + nextIo } /** @@ -1374,11 +1362,7 @@ object IO extends IOInstances { ref.complete(IO.unit) } - if (isTracingEnabled) { - IOTracing.cached(nextIo, TraceTag.Cancelable, k.getClass) - } else { - nextIo - } + nextIo } /** @@ -1631,7 +1615,9 @@ object IO extends IOInstances { IOContextShift(ec) val backtrace: IO[IOTrace] = - IOTracing.backtrace + IO.Async { (_, ctx, cb) => + cb(Right(ctx.getTrace)) + } /* -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-= */ /* IO's internal encoding: */ @@ -1680,7 +1666,7 @@ object IO extends IOInstances { final private[effect] case class Trace[A](source: IO[A], frame: TraceFrame) extends IO[A] -// final private[effect] case class SetTracingMode(mode: TracingMode) extends IO[Nothing] + final private[effect] case class CollectTraces(collect: Boolean) extends IO[Unit] /** * An internal state for that optimizes changes to diff --git a/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala b/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala index d247f1c31f..b8c902ee80 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala @@ -25,9 +25,6 @@ import scala.concurrent.{ExecutionContext, Promise} import scala.util.control.NonFatal import java.util.concurrent.atomic.AtomicBoolean -import cats.effect.tracing.TracingMode -import cats.effect.internals.TracingPlatformFast.isTracingEnabled - private[effect] object IOBracket { private[this] type Acquire[A] = (A, IOContext) @@ -47,13 +44,11 @@ private[effect] object IOBracket { // Note `acquireWithContext` is uncancelable due to usage of `IORunLoop.restart` // (in other words it is disconnected from our IOConnection) val acquireWithContext = acquire.product(ioContext) - val tracingMode = activeTracingMode IORunLoop.restart[Acquire[A]]( acquireWithContext, ctx, - tracingMode, - new BracketStart(use, release, conn, tracingMode, deferredRelease, cb) + new BracketStart(use, release, conn, deferredRelease, cb) ) } else { deferredRelease.complete(IO.unit) @@ -73,7 +68,6 @@ private[effect] object IOBracket { use: A => IO[B], release: (A, ExitCase[Throwable]) => IO[Unit], conn: IOConnection, - tracingMode: TracingMode, deferredRelease: ForwardCancelable, cb: Callback.T[B] ) extends (Either[Throwable, Acquire[A]] => Unit) @@ -111,7 +105,7 @@ private[effect] object IOBracket { fb.flatMap(frame) } // Actual execution - IORunLoop.restartCancelable(onNext, conn, a._2, tracingMode, cb) + IORunLoop.restartCancelable(onNext, conn, a._2, cb) } case error @ Left(_) => @@ -137,7 +131,7 @@ private[effect] object IOBracket { // the connection was already cancelled — n.b. we don't need // to trigger `release` otherwise, because it already happened if (!conn.isCanceled) { - IORunLoop.restartCancelable(onNext, conn, ctx, activeTracingMode, cb) + IORunLoop.restartCancelable(onNext, conn, ctx, cb) } } }) @@ -221,13 +215,4 @@ private[effect] object IOBracket { IO.Async { (_, ctx, cb) => cb(Right(ctx)) } - - private[this] val TracingDisabled: TracingMode = TracingMode.Disabled - - private def activeTracingMode: TracingMode = - if (isTracingEnabled) { - IOTracing.getLocalTracingMode() - } else { - TracingDisabled - } } diff --git a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala index 767305a4dc..d03d569563 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala @@ -28,6 +28,7 @@ final private[effect] class IOContext private () { private var frames: RingBuffer[TraceFrame] = new RingBuffer(maxTraceDepth) private var omitted: Int = 0 + private var collectTraces: Boolean = false def pushFrame(fr: TraceFrame): Unit = { val a = frames.push(fr) @@ -44,6 +45,13 @@ final private[effect] class IOContext private () { def getTrace: IOTrace = IOTrace(frames.toList.toVector, omitted) + def isCollectingTraces: Boolean = + collectTraces + + def setCollectTraces(b: Boolean): Unit = { + collectTraces = b + } + } object IOContext { diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index 7f38acd1c3..da5700769d 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -17,11 +17,10 @@ package cats.effect.internals import cats.effect.IO -import cats.effect.IO.{Async, Bind, ContextSwitch, Delay, Map, Pure, RaiseError, Suspend, Trace} -import cats.effect.tracing.{TraceFrame, TracingMode} +import cats.effect.IO.{Async, Bind, CollectTraces, ContextSwitch, Delay, Map, Pure, RaiseError, Suspend, Trace} +import cats.effect.tracing.TraceFrame -//TODO: define in a private variable? -import cats.effect.internals.TracingPlatformFast.isTracingEnabled +import cats.effect.internals.TracingPlatformFast.tracingMode import scala.util.control.NonFatal @@ -36,16 +35,10 @@ private[effect] object IORunLoop { * with the result when completed. */ def start[A](source: IO[A], cb: Either[Throwable, A] => Unit): Unit = { - if (isTracingEnabled) { - IOTracing.setLocalTracingMode(TracingDisabled) - } loop(source, IOConnection.uncancelable, cb.asInstanceOf[Callback], null, null, null, null) } - def restart[A](source: IO[A], ctx: IOContext, mode: TracingMode, cb: Either[Throwable, A] => Unit): Unit = { - if (isTracingEnabled) { - IOTracing.setLocalTracingMode(mode) - } + def restart[A](source: IO[A], ctx: IOContext, cb: Either[Throwable, A] => Unit): Unit = { loop(source, IOConnection.uncancelable, cb.asInstanceOf[Callback], ctx, null, null, null) } @@ -54,20 +47,10 @@ private[effect] object IORunLoop { * with the result when completed. */ def startCancelable[A](source: IO[A], conn: IOConnection, cb: Either[Throwable, A] => Unit): Unit = { - if (isTracingEnabled) { - IOTracing.setLocalTracingMode(TracingDisabled) - } loop(source, conn, cb.asInstanceOf[Callback], null, null, null, null) } - def restartCancelable[A](source: IO[A], - conn: IOConnection, - ctx: IOContext, - mode: TracingMode, - cb: Either[Throwable, A] => Unit): Unit = { - if (isTracingEnabled) { - IOTracing.setLocalTracingMode(mode) - } + def restartCancelable[A](source: IO[A], conn: IOConnection, ctx: IOContext, cb: Either[Throwable, A] => Unit): Unit = { loop(source, conn, cb.asInstanceOf[Callback], ctx, null, null, null) } @@ -91,6 +74,7 @@ private[effect] object IORunLoop { // Can change on a context switch var conn: IOConnection = cancelable var ctx: IOContext = ctxRef + var collectTraces: Boolean = if (ctx ne null) ctx.isCollectingTraces else false var bFirst: Bind = bFirstRef var bRest: CallStack = bRestRef var rcb: RestartCallback = rcbRef @@ -108,10 +92,10 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if (isTracingEnabled) { - val trace = bind.trace.asInstanceOf[TraceFrame] + if ((tracingMode == 1 || tracingMode == 2) && collectTraces) { + val trace = bind.trace if (ctx eq null) ctx = IOContext() - if (trace ne null) ctx.pushFrame(trace) + if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa @@ -153,10 +137,10 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if (isTracingEnabled) { - val trace = bindNext.trace.asInstanceOf[TraceFrame] + if ((tracingMode == 1 || tracingMode == 2) && collectTraces) { + val trace = bindNext.trace if (ctx eq null) ctx = IOContext() - if (trace ne null) ctx.pushFrame(trace) + if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa @@ -166,8 +150,8 @@ private[effect] object IORunLoop { // We need to initialize an IOContext because the continuation // may produce trace frames. if (ctx eq null) ctx = IOContext() - if (rcb eq null) rcb = new RestartCallback(conn, ctx, cb.asInstanceOf[Callback]) - rcb.start(async, bFirst, bRest) + if (rcb eq null) rcb = new RestartCallback(conn, cb.asInstanceOf[Callback]) + rcb.start(async, ctx, bFirst, bRest) return case ContextSwitch(next, modify, restore) => @@ -184,6 +168,13 @@ private[effect] object IORunLoop { if (ctx eq null) ctx = IOContext() ctx.pushFrame(frame) currentIO = source + + case CollectTraces(collect) => + if (ctx eq null) ctx = IOContext() + ctx.setCollectTraces(collect) + collectTraces = collect + unboxed = ().asInstanceOf[AnyRef] + hasUnboxed = true } if (hasUnboxed) { @@ -221,16 +212,11 @@ private[effect] object IORunLoop { var bFirst: Bind = null var bRest: CallStack = null var ctx: IOContext = null + var collectTraces: Boolean = false // Values from Pure and Delay are unboxed in this var, // for code reuse between Pure and Delay var hasUnboxed: Boolean = false var unboxed: AnyRef = null - val inTracingRegion: Boolean = if (isTracingEnabled) { - IOTracing.getLocalTracingMode() match { - case TracingMode.Disabled => false - case _ => true - } - } else false while ({ currentIO match { @@ -239,9 +225,9 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if (isTracingEnabled && inTracingRegion) { - if (ctx eq null) ctx = IOContext() + if ((tracingMode == 1 || tracingMode == 2) && collectTraces) { val trace = bind.trace + if (ctx eq null) ctx = IOContext() if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) } bFirst = bindNext.asInstanceOf[Bind] @@ -284,27 +270,30 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if (isTracingEnabled && inTracingRegion) { - if (ctx eq null) ctx = IOContext() + if ((tracingMode == 1 || tracingMode == 2) && collectTraces) { val trace = bindNext.trace + if (ctx eq null) ctx = IOContext() if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa case Trace(source, frame) => - // We should never see a Trace node where the following - // conditional evaluates to false. - if (isTracingEnabled && inTracingRegion) { - if (ctx eq null) ctx = IOContext() - ctx.pushFrame(frame) - } + if (ctx eq null) ctx = IOContext() + ctx.pushFrame(frame) currentIO = source + case CollectTraces(collect) => + if (ctx eq null) ctx = IOContext() + ctx.setCollectTraces(collect) + collectTraces = collect + unboxed = ().asInstanceOf[AnyRef] + hasUnboxed = true + case Async(_, _) => // Cannot inline the code of this method — as it would // box those vars in scala.runtime.ObjectRef! - return suspendAsync(currentIO.asInstanceOf[IO.Async[A]], bFirst, bRest) + return suspendAsync(currentIO.asInstanceOf[IO.Async[A]], ctx, bFirst, bRest) case _ => return Async { (conn, ctx, cb) => @@ -333,12 +322,13 @@ private[effect] object IORunLoop { // $COVERAGE-ON$ } - private def suspendAsync[A](currentIO: IO.Async[A], bFirst: Bind, bRest: CallStack): IO[A] = + private def suspendAsync[A](currentIO: IO.Async[A], ctx: IOContext, bFirst: Bind, bRest: CallStack): IO[A] = // Hitting an async boundary means we have to stop, however // if we had previous `flatMap` operations then we need to resume // the loop with the collected stack if (bFirst != null || (bRest != null && !bRest.isEmpty)) - Async { (conn, ctx, cb) => + Async { (conn, _, cb) => + // TODO: Pass in old IOContext loop(currentIO, conn, cb.asInstanceOf[Callback], ctx, null, bFirst, bRest) } else @@ -405,7 +395,7 @@ private[effect] object IORunLoop { * It's an ugly, mutable implementation. * For internal use only, here be dragons! */ - final private class RestartCallback(connInit: IOConnection, ctx: IOContext, cb: Callback) + final private class RestartCallback(connInit: IOConnection, cb: Callback) extends Callback with Runnable { import TrampolineEC.{immediate => ec} @@ -416,7 +406,8 @@ private[effect] object IORunLoop { private[this] var trampolineAfter = false private[this] var bFirst: Bind = _ private[this] var bRest: CallStack = _ - private[this] var tMode: TracingMode = _ + // TODO: can this have an initial implementation ? + private[this] var ctx: IOContext = _ // Used in combination with trampolineAfter = true private[this] var value: Either[Throwable, Any] = _ @@ -424,14 +415,12 @@ private[effect] object IORunLoop { def contextSwitch(conn: IOConnection): Unit = this.conn = conn - def start(task: IO.Async[Any], bFirst: Bind, bRest: CallStack): Unit = { + def start(task: IO.Async[Any], ctx: IOContext, bFirst: Bind, bRest: CallStack): Unit = { canCall = true this.bFirst = bFirst this.bRest = bRest this.trampolineAfter = task.trampolineAfter - if (isTracingEnabled) { - this.tMode = IOTracing.getLocalTracingMode() - } + this.ctx = ctx // Go, go, go task.k(conn, ctx, this) @@ -444,13 +433,6 @@ private[effect] object IORunLoop { this.bFirst = null this.bRest = null - if (isTracingEnabled) { - // The continuation may have been invoked on a new execution context, - // so let's recover the tracing mode here. - IOTracing.setLocalTracingMode(this.tMode) - this.tMode = null - } - // Auto-cancelable logic: in case the connection was cancelled, // we interrupt the bind continuation if (!conn.isCanceled) either match { @@ -496,6 +478,4 @@ private[effect] object IORunLoop { * cancelled status, to interrupt synchronous flatMap loops. */ private[this] val maxAutoCancelableBatchSize = 512 - - private[this] val TracingDisabled = TracingMode.Disabled } diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index 7a62923cdc..d55a35b64f 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -18,65 +18,37 @@ package cats.effect.internals import java.util.concurrent.ConcurrentHashMap +import cats.implicits._ import cats.effect.IO -import cats.effect.IO.Trace -import cats.effect.tracing.{IOTrace, TraceFrame, TraceTag, TracingMode} +import cats.effect.IO.{CollectTraces, Pure, RaiseError, Trace} +import cats.effect.tracing.{TraceFrame, TraceTag} private[effect] object IOTracing { - // TODO: It may be worth tracking mode in the global flag - // marking the ones we want. This avoids a thread-local on uncachable - // nodes which incurs the bulk of the performance hit. - def uncached[A](source: IO[A], traceTag: TraceTag): IO[A] = - localTracingMode.get() match { - case TracingMode.Slug => Trace(source, buildFrame(traceTag)) - case _ => source - } + def uncached[A](source: IO[A], traceTag: TraceTag): IO[A] = { +// localTracingMode.get() match { +// case TracingMode.Slug => Trace(source, buildFrame(traceTag)) +// case _ => source +// } + Trace(source, buildFrame(traceTag)) + } // TODO: Avoid trace tag for primitive ops and rely on class - def cached[A](source: IO[A], traceTag: TraceTag, clazz: Class[_]): IO[A] = - localTracingMode.get() match { - case TracingMode.Rabbit => Trace(source, buildCachedFrame(traceTag, clazz)) - case TracingMode.Slug => Trace(source, buildFrame(traceTag)) - case TracingMode.Disabled => source - } + def cached[A](source: IO[A], traceTag: TraceTag, clazz: Class[_]): IO[A] = { +// localTracingMode.get() match { +// case TracingMode.Rabbit => Trace(source, buildCachedFrame(traceTag, clazz)) +// case TracingMode.Slug => Trace(source, buildFrame(traceTag)) +// case TracingMode.Disabled => source +// } + println(clazz) + Trace(source, buildFrame(traceTag)) + } def trace(traceTag: TraceTag, clazz: Class[_]): TraceFrame = buildCachedFrame(traceTag, clazz) - def locallyTraced[A](source: IO[A], newMode: TracingMode): IO[A] = - for { - _ <- resetTrace - a <- IO.suspend { - val oldMode = localTracingMode.get() - localTracingMode.set(newMode) - - // In the event of cancellation, the tracing mode will be reset - // when the thread grabs a new task to run (via Async or IORunLoop.start). - source.redeemWith( - e => IO.suspend { - localTracingMode.set(oldMode) - IO.raiseError(e) - }, - a => IO.suspend { - localTracingMode.set(oldMode) - IO.pure(a) - } - ) - } - } yield a - - def getLocalTracingMode(): TracingMode = - localTracingMode.get() - - def setLocalTracingMode(mode: TracingMode): Unit = - localTracingMode.set(mode) - - // TODO: def might be faster than value - val backtrace: IO[IOTrace] = - IO.Async { (_, ctx, cb) => - cb(Right(ctx.getTrace)) - } + def traced[A](source: IO[A]): IO[A] = + resetTrace *> enableCollection *> source.flatMap(DisableCollection.asInstanceOf[A => IO[A]]) private def buildCachedFrame(traceTag: TraceTag, keyClass: Class[_]): TraceFrame = { val cachedFr = frameCache.get(keyClass) @@ -97,6 +69,17 @@ private[effect] object IOTracing { TraceFrame(traceTag, stackTrace) } + private[this] val enableCollection: IO[Unit] = CollectTraces(true) + + private[this] val disableCollection: IO[Unit] = CollectTraces(false) + + private object DisableCollection extends IOFrame[Any, IO[Any]] { + override def apply(a: Any) = + disableCollection *> Pure(a) + override def recover(e: Throwable) = + disableCollection *> RaiseError(e) + } + private[this] val resetTrace: IO[Unit] = IO.Async { (_, ctx, cb) => ctx.resetTrace() @@ -109,12 +92,6 @@ private[effect] object IOTracing { */ private[this] val frameCache: ConcurrentHashMap[Class[_], TraceFrame] = new ConcurrentHashMap() - /** - * Thread-local state that stores the lexical tracing - * mode for the fiber bound to the current thread. - */ - private[this] val localTracingMode: ThreadLocal[TracingMode] = ThreadLocal.withInitial(() => TracingMode.Disabled) - private[this] val classBlacklist = List( "cats.effect.", "cats.", From e6b629b9795124281940954d510f31b60a7e6356 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Mon, 8 Jun 2020 21:56:09 -0500 Subject: [PATCH 42/78] RunLoop context passing --- .../effect/internals/TracingPlatformFast.java | 4 +- .../scala/cats/effect/internals/Example.scala | 2 +- .../src/main/scala/cats/effect/IO.scala | 20 ++++---- .../cats/effect/internals/IOBracket.scala | 47 ++++++----------- .../cats/effect/internals/IOContext.scala | 25 ++++----- .../cats/effect/internals/IORunLoop.scala | 51 ++++++++++--------- .../cats/effect/internals/IOTracing.scala | 17 +++---- .../scala/cats/effect/tracing/IOTrace.scala | 6 +-- 8 files changed, 74 insertions(+), 98 deletions(-) diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java index b820b4ce5f..c09390ce08 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java @@ -45,7 +45,7 @@ public final class TracingPlatformFast { return Optional.empty(); } }) - .orElse(0); + .orElse(1); /** * The number of trace lines to retain during tracing. If more trace @@ -61,6 +61,6 @@ public final class TracingPlatformFast { return Optional.empty(); } }) - .orElse(512); + .orElse(64); } diff --git a/core/jvm/src/main/scala/cats/effect/internals/Example.scala b/core/jvm/src/main/scala/cats/effect/internals/Example.scala index 84b8766dd2..ab5f81a17c 100644 --- a/core/jvm/src/main/scala/cats/effect/internals/Example.scala +++ b/core/jvm/src/main/scala/cats/effect/internals/Example.scala @@ -57,7 +57,7 @@ object Example extends IOApp { _ <- print("1") _ <- print("2") _ <- IO.shift - _ <- IO.unit.bracket(_ => + _ <- IO.shift.bracket(_ => print("3") .flatMap(_ => program2) )(_ => IO.unit) diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index 01b212933f..65081830a4 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -582,10 +582,8 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * canceled, receiving as input the resource that needs to * be released */ - final def bracket[B](use: A => IO[B])(release: A => IO[Unit]): IO[B] = { - val nextIo = IOBracket(this)(use)((a, _) => release(a)) - nextIo - } + final def bracket[B](use: A => IO[B])(release: A => IO[Unit]): IO[B] = + IOBracket(this)(use)((a, _) => release(a)) /** * Returns a new `IO` task that treats the source task as the @@ -1140,9 +1138,9 @@ object IO extends IOInstances { def delay[A](body: => A): IO[A] = { val nextIo = Delay(() => body) if (tracingMode == 1) { - IOTracing.uncached(nextIo, TraceTag.Delay) - } else if (tracingMode == 2) { nextIo + } else if (tracingMode == 2) { + IOTracing.uncached(nextIo, TraceTag.Delay) } else { nextIo } @@ -1159,9 +1157,9 @@ object IO extends IOInstances { def suspend[A](thunk: => IO[A]): IO[A] = { val nextIo = Suspend(() => thunk) if (tracingMode == 1) { - IOTracing.uncached(nextIo, TraceTag.Suspend) - } else if (tracingMode == 2) { nextIo + } else if (tracingMode == 2) { + IOTracing.uncached(nextIo, TraceTag.Suspend) } else { nextIo } @@ -1180,9 +1178,9 @@ object IO extends IOInstances { def pure[A](a: A): IO[A] = { val nextIo = Pure(a) if (tracingMode == 1) { - IOTracing.uncached(nextIo, TraceTag.Pure) - } else if (tracingMode == 2) { nextIo + } else if (tracingMode == 2) { + IOTracing.uncached(nextIo, TraceTag.Pure) } else { nextIo } @@ -1616,7 +1614,7 @@ object IO extends IOInstances { val backtrace: IO[IOTrace] = IO.Async { (_, ctx, cb) => - cb(Right(ctx.getTrace)) + cb(Right(ctx.trace)) } /* -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-= */ diff --git a/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala b/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala index b8c902ee80..180478e354 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOBracket.scala @@ -16,7 +16,6 @@ package cats.effect.internals -import cats.implicits._ import cats.effect.IO.ContextSwitch import cats.effect.{CancelToken, ExitCase, IO} import cats.effect.internals.TrampolineEC.immediate @@ -27,13 +26,11 @@ import java.util.concurrent.atomic.AtomicBoolean private[effect] object IOBracket { - private[this] type Acquire[A] = (A, IOContext) - /** * Implementation for `IO.bracketCase`. */ - def apply[A, B](acquire: IO[A])(use: A => IO[B])(release: (A, ExitCase[Throwable]) => IO[Unit]): IO[B] = { - val nextIo = IO.Async[B] { (conn, ctx, cb) => + def apply[A, B](acquire: IO[A])(use: A => IO[B])(release: (A, ExitCase[Throwable]) => IO[Unit]): IO[B] = + IO.Async[B] { (conn, ctx, cb) => // Placeholder for the future finalizer val deferredRelease = ForwardCancelable() conn.push(deferredRelease.cancel) @@ -41,43 +38,34 @@ private[effect] object IOBracket { // was cancelled already, to ensure that `cancel` really blocks if we // start `acquire` — n.b. `isCanceled` is visible here due to `push` if (!conn.isCanceled) { - // Note `acquireWithContext` is uncancelable due to usage of `IORunLoop.restart` + // Note `acquire` is uncancelable due to usage of `IORunLoop.restart` // (in other words it is disconnected from our IOConnection) - val acquireWithContext = acquire.product(ioContext) - - IORunLoop.restart[Acquire[A]]( - acquireWithContext, - ctx, - new BracketStart(use, release, conn, deferredRelease, cb) - ) + // We don't need to explicitly pass back a reference to `ctx` because + // it is held in `RestartCallback` and `BracketStart`. + // Updates to it in the run-loop will be visible when the callback is + // invoked, even across asynchronous boundaries. + IORunLoop.restart(acquire, ctx, new BracketStart(use, release, ctx, conn, deferredRelease, cb)) } else { deferredRelease.complete(IO.unit) } } -// if (isTracingEnabled) { -// IOTracing(nextIo, use.getClass) -// } else { -// nextIo -// } - nextIo - } - // Internals of `IO.bracketCase`. final private class BracketStart[A, B]( use: A => IO[B], release: (A, ExitCase[Throwable]) => IO[Unit], + ctx: IOContext, conn: IOConnection, deferredRelease: ForwardCancelable, cb: Callback.T[B] - ) extends (Either[Throwable, Acquire[A]] => Unit) + ) extends (Either[Throwable, A] => Unit) with Runnable { // This runnable is a dirty optimization to avoid some memory allocations; // This class switches from being a Callback to a Runnable, but relies on // the internal IO callback protocol to be respected (called at most once) - private[this] var result: Either[Throwable, Acquire[A]] = _ + private[this] var result: Either[Throwable, A] = _ - def apply(ea: Either[Throwable, Acquire[A]]): Unit = { + def apply(ea: Either[Throwable, A]): Unit = { if (result ne null) { throw new IllegalStateException("callback called multiple times!") } @@ -90,7 +78,7 @@ private[effect] object IOBracket { def run(): Unit = result match { case Right(a) => - val frame = new BracketReleaseFrame[A, B](a._1, release) + val frame = new BracketReleaseFrame[A, B](a, release) // Registering our cancelable token ensures that in case // cancellation is detected, `release` gets called @@ -100,12 +88,12 @@ private[effect] object IOBracket { if (!conn.isCanceled) { val onNext = { val fb = - try use(a._1) + try use(a) catch { case NonFatal(e) => IO.raiseError(e) } fb.flatMap(frame) } // Actual execution - IORunLoop.restartCancelable(onNext, conn, a._2, cb) + IORunLoop.restartCancelable(onNext, conn, ctx, cb) } case error @ Left(_) => @@ -210,9 +198,4 @@ private[effect] object IOBracket { old.pop() old } - - private[this] val ioContext: IO[IOContext] = - IO.Async { (_, ctx, cb) => - cb(Right(ctx)) - } } diff --git a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala index d03d569563..8c0c81fb4c 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala @@ -26,15 +26,15 @@ import cats.effect.internals.TracingPlatformFast.maxTraceDepth */ final private[effect] class IOContext private () { - private var frames: RingBuffer[TraceFrame] = new RingBuffer(maxTraceDepth) - private var omitted: Int = 0 - private var collectTraces: Boolean = false + var frames: RingBuffer[TraceFrame] = new RingBuffer(maxTraceDepth) + var captured: Int = 0 + var omitted: Int = 0 + + var activeCollects: Int = 0 def pushFrame(fr: TraceFrame): Unit = { - val a = frames.push(fr) - if (a != null) { - omitted += 1 - } + captured += 1 + if (frames.push(fr) != null) omitted += 1 } def resetTrace(): Unit = { @@ -42,15 +42,8 @@ final private[effect] class IOContext private () { omitted = 0 } - def getTrace: IOTrace = - IOTrace(frames.toList.toVector, omitted) - - def isCollectingTraces: Boolean = - collectTraces - - def setCollectTraces(b: Boolean): Unit = { - collectTraces = b - } + def trace: IOTrace = + IOTrace(frames.toList.toVector, captured, omitted) } diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index da5700769d..2947a2b6a0 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -34,25 +34,21 @@ private[effect] object IORunLoop { * Evaluates the given `IO` reference, calling the given callback * with the result when completed. */ - def start[A](source: IO[A], cb: Either[Throwable, A] => Unit): Unit = { + def start[A](source: IO[A], cb: Either[Throwable, A] => Unit): Unit = loop(source, IOConnection.uncancelable, cb.asInstanceOf[Callback], null, null, null, null) - } - def restart[A](source: IO[A], ctx: IOContext, cb: Either[Throwable, A] => Unit): Unit = { + def restart[A](source: IO[A], ctx: IOContext, cb: Either[Throwable, A] => Unit): Unit = loop(source, IOConnection.uncancelable, cb.asInstanceOf[Callback], ctx, null, null, null) - } /** * Evaluates the given `IO` reference, calling the given callback * with the result when completed. */ - def startCancelable[A](source: IO[A], conn: IOConnection, cb: Either[Throwable, A] => Unit): Unit = { + def startCancelable[A](source: IO[A], conn: IOConnection, cb: Either[Throwable, A] => Unit): Unit = loop(source, conn, cb.asInstanceOf[Callback], null, null, null, null) - } - def restartCancelable[A](source: IO[A], conn: IOConnection, ctx: IOContext, cb: Either[Throwable, A] => Unit): Unit = { + def restartCancelable[A](source: IO[A], conn: IOConnection, ctx: IOContext, cb: Either[Throwable, A] => Unit): Unit = loop(source, conn, cb.asInstanceOf[Callback], ctx, null, null, null) - } /** * Loop for evaluating an `IO` value. @@ -74,7 +70,7 @@ private[effect] object IORunLoop { // Can change on a context switch var conn: IOConnection = cancelable var ctx: IOContext = ctxRef - var collectTraces: Boolean = if (ctx ne null) ctx.isCollectingTraces else false + var activeCollects: Int = if (ctx ne null) ctx.activeCollects else 0 var bFirst: Bind = bFirstRef var bRest: CallStack = bRestRef var rcb: RestartCallback = rcbRef @@ -92,7 +88,7 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if ((tracingMode == 1 || tracingMode == 2) && collectTraces) { + if ((tracingMode == 1 || tracingMode == 2) && activeCollects > 0) { val trace = bind.trace if (ctx eq null) ctx = IOContext() if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) @@ -137,7 +133,7 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if ((tracingMode == 1 || tracingMode == 2) && collectTraces) { + if ((tracingMode == 1 || tracingMode == 2) && activeCollects > 0) { val trace = bindNext.trace if (ctx eq null) ctx = IOContext() if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) @@ -148,7 +144,7 @@ private[effect] object IORunLoop { case async @ Async(_, _) => if (conn eq null) conn = IOConnection() // We need to initialize an IOContext because the continuation - // may produce trace frames. + // may produce trace frames e.g. IOBracket. if (ctx eq null) ctx = IOContext() if (rcb eq null) rcb = new RestartCallback(conn, cb.asInstanceOf[Callback]) rcb.start(async, ctx, bFirst, bRest) @@ -171,8 +167,13 @@ private[effect] object IORunLoop { case CollectTraces(collect) => if (ctx eq null) ctx = IOContext() - ctx.setCollectTraces(collect) - collectTraces = collect + if (collect) { + activeCollects += 1 + ctx.activeCollects += 1 + } else { + activeCollects -= 1 + ctx.activeCollects -= 1 + } unboxed = ().asInstanceOf[AnyRef] hasUnboxed = true } @@ -212,7 +213,7 @@ private[effect] object IORunLoop { var bFirst: Bind = null var bRest: CallStack = null var ctx: IOContext = null - var collectTraces: Boolean = false + var activeCollects: Int = 0 // Values from Pure and Delay are unboxed in this var, // for code reuse between Pure and Delay var hasUnboxed: Boolean = false @@ -225,7 +226,7 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if ((tracingMode == 1 || tracingMode == 2) && collectTraces) { + if ((tracingMode == 1 || tracingMode == 2) && activeCollects > 0) { val trace = bind.trace if (ctx eq null) ctx = IOContext() if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) @@ -270,7 +271,7 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if ((tracingMode == 1 || tracingMode == 2) && collectTraces) { + if ((tracingMode == 1 || tracingMode == 2) && activeCollects > 0) { val trace = bindNext.trace if (ctx eq null) ctx = IOContext() if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) @@ -285,8 +286,13 @@ private[effect] object IORunLoop { case CollectTraces(collect) => if (ctx eq null) ctx = IOContext() - ctx.setCollectTraces(collect) - collectTraces = collect + if (collect) { + activeCollects += 1 + ctx.activeCollects += 1 + } else { + activeCollects -= 1 + ctx.activeCollects -= 1 + } unboxed = ().asInstanceOf[AnyRef] hasUnboxed = true @@ -395,9 +401,7 @@ private[effect] object IORunLoop { * It's an ugly, mutable implementation. * For internal use only, here be dragons! */ - final private class RestartCallback(connInit: IOConnection, cb: Callback) - extends Callback - with Runnable { + final private class RestartCallback(connInit: IOConnection, cb: Callback) extends Callback with Runnable { import TrampolineEC.{immediate => ec} // can change on a ContextSwitch @@ -406,7 +410,6 @@ private[effect] object IORunLoop { private[this] var trampolineAfter = false private[this] var bFirst: Bind = _ private[this] var bRest: CallStack = _ - // TODO: can this have an initial implementation ? private[this] var ctx: IOContext = _ // Used in combination with trampolineAfter = true @@ -430,8 +433,10 @@ private[effect] object IORunLoop { // Allow GC to collect val bFirst = this.bFirst val bRest = this.bRest + val ctx = this.ctx this.bFirst = null this.bRest = null + this.ctx = null // Auto-cancelable logic: in case the connection was cancelled, // we interrupt the bind continuation diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index d55a35b64f..9fde66124f 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -18,20 +18,18 @@ package cats.effect.internals import java.util.concurrent.ConcurrentHashMap -import cats.implicits._ import cats.effect.IO import cats.effect.IO.{CollectTraces, Pure, RaiseError, Trace} import cats.effect.tracing.{TraceFrame, TraceTag} private[effect] object IOTracing { - def uncached[A](source: IO[A], traceTag: TraceTag): IO[A] = { + def uncached[A](source: IO[A], traceTag: TraceTag): IO[A] = // localTracingMode.get() match { // case TracingMode.Slug => Trace(source, buildFrame(traceTag)) // case _ => source // } Trace(source, buildFrame(traceTag)) - } // TODO: Avoid trace tag for primitive ops and rely on class def cached[A](source: IO[A], traceTag: TraceTag, clazz: Class[_]): IO[A] = { @@ -48,7 +46,7 @@ private[effect] object IOTracing { buildCachedFrame(traceTag, clazz) def traced[A](source: IO[A]): IO[A] = - resetTrace *> enableCollection *> source.flatMap(DisableCollection.asInstanceOf[A => IO[A]]) + resetTrace *> incrementCollection *> source.flatMap(DecrementTraceCollection.asInstanceOf[A => IO[A]]) private def buildCachedFrame(traceTag: TraceTag, keyClass: Class[_]): TraceFrame = { val cachedFr = frameCache.get(keyClass) @@ -62,22 +60,21 @@ private[effect] object IOTracing { } def buildFrame(traceTag: TraceTag): TraceFrame = { - // TODO: proper trace calculation val stackTrace = new Throwable().getStackTrace.toList .dropWhile(l => classBlacklist.exists(b => l.getClassName.startsWith(b))) TraceFrame(traceTag, stackTrace) } - private[this] val enableCollection: IO[Unit] = CollectTraces(true) + private[this] val incrementCollection: IO[Unit] = CollectTraces(true) - private[this] val disableCollection: IO[Unit] = CollectTraces(false) + private[this] val decrementCollection: IO[Unit] = CollectTraces(false) - private object DisableCollection extends IOFrame[Any, IO[Any]] { + private object DecrementTraceCollection extends IOFrame[Any, IO[Any]] { override def apply(a: Any) = - disableCollection *> Pure(a) + decrementCollection *> Pure(a) override def recover(e: Throwable) = - disableCollection *> RaiseError(e) + decrementCollection *> RaiseError(e) } private[this] val resetTrace: IO[Unit] = diff --git a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala index 6206de4143..bd008b1cfd 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala @@ -18,7 +18,7 @@ package cats.effect.tracing import cats.effect.IO -final case class IOTrace(frames: Vector[TraceFrame], omitted: Int) { +final case class IOTrace(frames: Vector[TraceFrame], captured: Int, omitted: Int) { import IOTrace._ @@ -29,7 +29,7 @@ final case class IOTrace(frames: Vector[TraceFrame], omitted: Int) { s"$className.$methodName (${ste.getFileName}:${ste.getLineNumber})" } - val acc0 = s"IOTrace: $omitted omitted frames\n" + val acc0 = s"IOTrace: $captured frames captured, $omitted omitted\n" val acc1 = frames.foldLeft(acc0)((acc, f) => acc + s"\t${f.tag.name} at " + f.stackTrace.headOption.map(renderStackTraceElement).getOrElse("(...)") + "\n" ) + "\n" @@ -41,7 +41,7 @@ final case class IOTrace(frames: Vector[TraceFrame], omitted: Int) { IO(System.err.println(compact)) def pretty: String = { - val acc0 = s"IOTrace: $omitted omitted frames" + val acc0 = s"IOTrace: $captured frames captured, $omitted omitted\n" val acc1 = acc0 + loop("", 0, true, frames.toList) acc1 } From a1855095e4a671e074afd546971c61e089ce0d11 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Mon, 8 Jun 2020 22:17:35 -0500 Subject: [PATCH 43/78] Trace async, asyncF, cancelable --- .../scala/cats/effect/internals/Example.scala | 1 + .../src/main/scala/cats/effect/IO.scala | 125 ++++++++++-------- .../cats/effect/internals/IORunLoop.scala | 19 +-- .../cats/effect/internals/IOTracing.scala | 25 +--- 4 files changed, 91 insertions(+), 79 deletions(-) diff --git a/core/jvm/src/main/scala/cats/effect/internals/Example.scala b/core/jvm/src/main/scala/cats/effect/internals/Example.scala index ab5f81a17c..0cb4f6c627 100644 --- a/core/jvm/src/main/scala/cats/effect/internals/Example.scala +++ b/core/jvm/src/main/scala/cats/effect/internals/Example.scala @@ -57,6 +57,7 @@ object Example extends IOApp { _ <- print("1") _ <- print("2") _ <- IO.shift + _ <- IO.async[Int](cb => cb(Right(32))) _ <- IO.shift.bracket(_ => print("3") .flatMap(_ => program2) diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index 65081830a4..46c398cd90 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -106,9 +106,9 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { // Don't perform map fusion when tracing is enabled. // We may end up removing map fusion altogether. if (tracingMode == 1) { - Map(this, f, 0, IOTracing.trace(TraceTag.Map, f.getClass)) + Map(this, f, 0, IOTracing.cached(TraceTag.Map, f.getClass)) } else if (tracingMode == 2) { - Map(this, f, 0, null) + Map(this, f, 0, IOTracing.uncached(TraceTag.Map, f.getClass)) } else { this match { case Map(source, g, index, null) => @@ -139,9 +139,9 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { */ final def flatMap[B](f: A => IO[B]): IO[B] = { val trace = if (tracingMode == 1) { - IOTracing.trace(TraceTag.Bind, f.getClass) + IOTracing.cached(TraceTag.Bind, f.getClass) } else if (tracingMode == 2) { - null + IOTracing.uncached(TraceTag.Bind, f.getClass) } else { null } @@ -346,7 +346,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { IORunLoop.step(this) match { case Pure(a) => Some(a) case RaiseError(e) => throw e - case self @ Async(_, _) => + case self @ Async(_, _, _) => IOPlatform.unsafeResync(self, limit) case _ => // $COVERAGE-OFF$ @@ -1137,10 +1137,8 @@ object IO extends IOInstances { */ def delay[A](body: => A): IO[A] = { val nextIo = Delay(() => body) - if (tracingMode == 1) { - nextIo - } else if (tracingMode == 2) { - IOTracing.uncached(nextIo, TraceTag.Delay) + if (tracingMode == 2) { + IOTracing.decorated(nextIo, TraceTag.Delay) } else { nextIo } @@ -1156,10 +1154,8 @@ object IO extends IOInstances { */ def suspend[A](thunk: => IO[A]): IO[A] = { val nextIo = Suspend(() => thunk) - if (tracingMode == 1) { - nextIo - } else if (tracingMode == 2) { - IOTracing.uncached(nextIo, TraceTag.Suspend) + if (tracingMode == 2) { + IOTracing.decorated(nextIo, TraceTag.Suspend) } else { nextIo } @@ -1177,10 +1173,8 @@ object IO extends IOInstances { */ def pure[A](a: A): IO[A] = { val nextIo = Pure(a) - if (tracingMode == 1) { - nextIo - } else if (tracingMode == 2) { - IOTracing.uncached(nextIo, TraceTag.Pure) + if (tracingMode == 2) { + IOTracing.decorated(nextIo, TraceTag.Pure) } else { nextIo } @@ -1250,13 +1244,19 @@ object IO extends IOInstances { * @see [[asyncF]] and [[cancelable]] */ def async[A](k: (Either[Throwable, A] => Unit) => Unit): IO[A] = { - val nextIo = Async[A] { (_, _, cb) => + val trace = if (tracingMode == 1) { + IOTracing.cached(TraceTag.Async, k.getClass) + } else if (tracingMode == 2) { + IOTracing.uncached(TraceTag.Async) + } else { + null + } + + Async[A]((_, _, cb) => { val cb2 = Callback.asyncIdempotent(null, cb) try k(cb2) catch { case NonFatal(t) => cb2(Left(t)) } - } - - nextIo + }, trace = trace) } /** @@ -1284,20 +1284,29 @@ object IO extends IOInstances { * @see [[async]] and [[cancelable]] */ def asyncF[A](k: (Either[Throwable, A] => Unit) => IO[Unit]): IO[A] = { - val nextIo = Async[A] { (conn, _, cb) => - // Must create new connection, otherwise we can have a race - // condition b/t the bind continuation and `startCancelable` below - val conn2 = IOConnection() - conn.push(conn2.cancel) - // The callback handles "conn.pop()" - val cb2 = Callback.asyncIdempotent(conn, cb) - val fa = - try k(cb2) - catch { case NonFatal(t) => IO(cb2(Left(t))) } - IORunLoop.startCancelable(fa, conn2, Callback.report) + val trace = if (tracingMode == 1) { + IOTracing.cached(TraceTag.AsyncF, k.getClass) + } else if (tracingMode == 2) { + IOTracing.uncached(TraceTag.AsyncF) + } else { + null } - nextIo + Async[A]( + (conn, _, cb) => { + // Must create new connection, otherwise we can have a race + // condition b/t the bind continuation and `startCancelable` below + val conn2 = IOConnection() + conn.push(conn2.cancel) + // The callback handles "conn.pop()" + val cb2 = Callback.asyncIdempotent(conn, cb) + val fa = + try k(cb2) + catch { case NonFatal(t) => IO(cb2(Left(t))) } + IORunLoop.startCancelable(fa, conn2, Callback.report) + }, + trace = trace + ) } /** @@ -1340,27 +1349,36 @@ object IO extends IOInstances { * the underlying cancelation model */ def cancelable[A](k: (Either[Throwable, A] => Unit) => CancelToken[IO]): IO[A] = { - val nextIo = Async[A] { (conn, _, cb) => - val cb2 = Callback.asyncIdempotent(conn, cb) - val ref = ForwardCancelable() - conn.push(ref.cancel) - // Race condition test — no need to execute `k` if it was already cancelled, - // ensures that fiber.cancel will always wait for the finalizer if `k` - // is executed — note that `isCanceled` is visible here due to `push` - if (!conn.isCanceled) - ref.complete( - try k(cb2) - catch { - case NonFatal(t) => - cb2(Left(t)) - IO.unit - } - ) - else - ref.complete(IO.unit) + val trace = if (tracingMode == 1) { + IOTracing.cached(TraceTag.Cancelable, k.getClass) + } else if (tracingMode == 2) { + IOTracing.uncached(TraceTag.Cancelable) + } else { + null } - nextIo + Async[A]( + (conn, _, cb) => { + val cb2 = Callback.asyncIdempotent(conn, cb) + val ref = ForwardCancelable() + conn.push(ref.cancel) + // Race condition test — no need to execute `k` if it was already cancelled, + // ensures that fiber.cancel will always wait for the finalizer if `k` + // is executed — note that `isCanceled` is visible here due to `push` + if (!conn.isCanceled) + ref.complete( + try k(cb2) + catch { + case NonFatal(t) => + cb2(Left(t)) + IO.unit + } + ) + else + ref.complete(IO.unit) + }, + trace = trace + ) } /** @@ -1659,7 +1677,8 @@ object IO extends IOInstances { */ final private[effect] case class Async[+A]( k: (IOConnection, IOContext, Either[Throwable, A] => Unit) => Unit, - trampolineAfter: Boolean = false + trampolineAfter: Boolean = false, + trace: AnyRef = null ) extends IO[A] final private[effect] case class Trace[A](source: IO[A], frame: TraceFrame) extends IO[A] diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index 2947a2b6a0..5064b4958a 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -89,8 +89,8 @@ private[effect] object IORunLoop { bRest.push(bFirst) } if ((tracingMode == 1 || tracingMode == 2) && activeCollects > 0) { - val trace = bind.trace if (ctx eq null) ctx = IOContext() + val trace = bind.trace if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) } bFirst = bindNext.asInstanceOf[Bind] @@ -134,19 +134,23 @@ private[effect] object IORunLoop { bRest.push(bFirst) } if ((tracingMode == 1 || tracingMode == 2) && activeCollects > 0) { - val trace = bindNext.trace if (ctx eq null) ctx = IOContext() + val trace = bindNext.trace if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa - case async @ Async(_, _) => + case async @ Async(_, _, _) => if (conn eq null) conn = IOConnection() // We need to initialize an IOContext because the continuation // may produce trace frames e.g. IOBracket. if (ctx eq null) ctx = IOContext() if (rcb eq null) rcb = new RestartCallback(conn, cb.asInstanceOf[Callback]) + if ((tracingMode == 1 || tracingMode == 2) && activeCollects > 0) { + val trace = async.trace + if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) + } rcb.start(async, ctx, bFirst, bRest) return @@ -227,8 +231,8 @@ private[effect] object IORunLoop { bRest.push(bFirst) } if ((tracingMode == 1 || tracingMode == 2) && activeCollects > 0) { - val trace = bind.trace if (ctx eq null) ctx = IOContext() + val trace = bind.trace if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) } bFirst = bindNext.asInstanceOf[Bind] @@ -272,8 +276,8 @@ private[effect] object IORunLoop { bRest.push(bFirst) } if ((tracingMode == 1 || tracingMode == 2) && activeCollects > 0) { - val trace = bindNext.trace if (ctx eq null) ctx = IOContext() + val trace = bindNext.trace if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) } bFirst = bindNext.asInstanceOf[Bind] @@ -296,13 +300,13 @@ private[effect] object IORunLoop { unboxed = ().asInstanceOf[AnyRef] hasUnboxed = true - case Async(_, _) => + case Async(_, _, _) => // Cannot inline the code of this method — as it would // box those vars in scala.runtime.ObjectRef! return suspendAsync(currentIO.asInstanceOf[IO.Async[A]], ctx, bFirst, bRest) case _ => - return Async { (conn, ctx, cb) => + return Async { (conn, _, cb) => loop(currentIO, conn, cb.asInstanceOf[Callback], ctx, null, bFirst, bRest) } } @@ -334,7 +338,6 @@ private[effect] object IORunLoop { // the loop with the collected stack if (bFirst != null || (bRest != null && !bRest.isEmpty)) Async { (conn, _, cb) => - // TODO: Pass in old IOContext loop(currentIO, conn, cb.asInstanceOf[Callback], ctx, null, bFirst, bRest) } else diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index 9fde66124f..3d47092caf 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -24,35 +24,24 @@ import cats.effect.tracing.{TraceFrame, TraceTag} private[effect] object IOTracing { - def uncached[A](source: IO[A], traceTag: TraceTag): IO[A] = -// localTracingMode.get() match { -// case TracingMode.Slug => Trace(source, buildFrame(traceTag)) -// case _ => source -// } + def decorated[A](source: IO[A], traceTag: TraceTag): IO[A] = Trace(source, buildFrame(traceTag)) // TODO: Avoid trace tag for primitive ops and rely on class - def cached[A](source: IO[A], traceTag: TraceTag, clazz: Class[_]): IO[A] = { -// localTracingMode.get() match { -// case TracingMode.Rabbit => Trace(source, buildCachedFrame(traceTag, clazz)) -// case TracingMode.Slug => Trace(source, buildFrame(traceTag)) -// case TracingMode.Disabled => source -// } - println(clazz) - Trace(source, buildFrame(traceTag)) - } + def uncached(traceTag: TraceTag): TraceFrame = + buildFrame(traceTag) - def trace(traceTag: TraceTag, clazz: Class[_]): TraceFrame = + def cached(traceTag: TraceTag, clazz: Class[_]): TraceFrame = buildCachedFrame(traceTag, clazz) def traced[A](source: IO[A]): IO[A] = resetTrace *> incrementCollection *> source.flatMap(DecrementTraceCollection.asInstanceOf[A => IO[A]]) - private def buildCachedFrame(traceTag: TraceTag, keyClass: Class[_]): TraceFrame = { - val cachedFr = frameCache.get(keyClass) + private def buildCachedFrame(traceTag: TraceTag, clazz: Class[_]): TraceFrame = { + val cachedFr = frameCache.get(clazz) if (cachedFr eq null) { val fr = buildFrame(traceTag) - frameCache.put(keyClass, fr) + frameCache.put(clazz, fr) fr } else { cachedFr From 7c22a40cf239ddef0853a1890cecb8201c055ed8 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Mon, 8 Jun 2020 22:20:04 -0500 Subject: [PATCH 44/78] fix --- core/shared/src/main/scala/cats/effect/IO.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index 46c398cd90..f6964e7550 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -108,7 +108,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { if (tracingMode == 1) { Map(this, f, 0, IOTracing.cached(TraceTag.Map, f.getClass)) } else if (tracingMode == 2) { - Map(this, f, 0, IOTracing.uncached(TraceTag.Map, f.getClass)) + Map(this, f, 0, IOTracing.uncached(TraceTag.Map)) } else { this match { case Map(source, g, index, null) => @@ -141,7 +141,7 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { val trace = if (tracingMode == 1) { IOTracing.cached(TraceTag.Bind, f.getClass) } else if (tracingMode == 2) { - IOTracing.uncached(TraceTag.Bind, f.getClass) + IOTracing.uncached(TraceTag.Bind) } else { null } From c4630cb61d0d6d70eb3d6cf54b382d8c5e564103 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Mon, 8 Jun 2020 23:00:41 -0500 Subject: [PATCH 45/78] TracingMode in java --- build.sbt | 5 +++- .../effect/internals/TracingPlatformFast.java | 19 ++++-------- .../java/cats/effect/internals/Tracing.java | 29 +++++++++++++++++++ .../cats/effect/internals/TracingMode.java} | 27 ++++++++--------- .../src/main/scala/cats/effect/IO.scala | 28 +++++++++--------- .../cats/effect/internals/IOForkedStart.scala | 2 +- .../cats/effect/internals/IORunLoop.scala | 12 ++++---- 7 files changed, 70 insertions(+), 52 deletions(-) create mode 100644 core/shared/src/main/java/cats/effect/internals/Tracing.java rename core/shared/src/main/{scala/cats/effect/tracing/TracingMode.scala => java/cats/effect/internals/TracingMode.java} (62%) diff --git a/build.sbt b/build.sbt index 754d47708a..8a5070c947 100644 --- a/build.sbt +++ b/build.sbt @@ -250,7 +250,10 @@ lazy val core = crossProject(JSPlatform, JVMPlatform) // https://github.com/scala/scala/pull/6606 Nil } - } + }, + unmanagedSourceDirectories in Compile += { + baseDirectory.value.getParentFile / "shared" / "src" / "main" / "java" + }, ) .jvmConfigure(_.enablePlugins(AutomateHeaderPlugin)) .jvmConfigure(_.settings(mimaSettings)) diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java index c09390ce08..3b38322352 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java @@ -30,22 +30,13 @@ public final class TracingPlatformFast { /** - * An integer flag that sets a global tracing mode for a JVM process. - * 0 - DISABLED - * 1 - RABBIT - * 2 - SLUG - * TODO: move to enum + * A string flag that sets a global tracing mode for a JVM process. + * Acceptable values are: DISABLED, RABBIT, SLUG. */ - public static final int tracingMode = Optional.ofNullable(System.getProperty("cats.effect.tracing.mode")) + public static final TracingMode tracingMode = Optional.ofNullable(System.getProperty("cats.effect.tracing.mode")) .filter(x -> !x.isEmpty()) - .flatMap(x -> { - try { - return Optional.of(Integer.valueOf(x)); - } catch (Exception e) { - return Optional.empty(); - } - }) - .orElse(1); + .flatMap(TracingMode::fromString) + .orElse(TracingMode.RABBIT); /** * The number of trace lines to retain during tracing. If more trace diff --git a/core/shared/src/main/java/cats/effect/internals/Tracing.java b/core/shared/src/main/java/cats/effect/internals/Tracing.java new file mode 100644 index 0000000000..901801d054 --- /dev/null +++ b/core/shared/src/main/java/cats/effect/internals/Tracing.java @@ -0,0 +1,29 @@ +/* + * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers + * + * Licensed 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 cats.effect.internals; + +public class Tracing { + + private static final TracingMode globalTracingMode = TracingPlatformFast.tracingMode; + + public static final boolean isRabbitTracing = globalTracingMode == TracingMode.RABBIT; + + public static final boolean isSlugTracing = globalTracingMode == TracingMode.SLUG; + + public static final boolean isTracing = isSlugTracing || isRabbitTracing; + +} diff --git a/core/shared/src/main/scala/cats/effect/tracing/TracingMode.scala b/core/shared/src/main/java/cats/effect/internals/TracingMode.java similarity index 62% rename from core/shared/src/main/scala/cats/effect/tracing/TracingMode.scala rename to core/shared/src/main/java/cats/effect/internals/TracingMode.java index 00c31a834c..2cd58c45eb 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/TracingMode.scala +++ b/core/shared/src/main/java/cats/effect/internals/TracingMode.java @@ -14,23 +14,20 @@ * limitations under the License. */ -package cats.effect.tracing +package cats.effect.internals; -sealed abstract private[effect] class TracingMode(val tag: Int) +import java.util.Optional; -private[effect] object TracingMode { +public enum TracingMode { + DISABLED, + RABBIT, + SLUG; - case object Disabled extends TracingMode(0) - - case object Rabbit extends TracingMode(1) - - case object Slug extends TracingMode(2) - - def fromInt(value: Int): TracingMode = - value match { - case 1 => Rabbit - case 2 => Slug - case _ => Disabled + public static Optional fromString(String value) { + try { + return Optional.of(valueOf(value.toUpperCase())); + } catch(IllegalArgumentException ex) { + return Optional.empty(); + } } - } diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index f6964e7550..0ef0cce188 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -18,8 +18,8 @@ package cats package effect import cats.effect.internals._ +import cats.effect.internals.Tracing.{isRabbitTracing, isSlugTracing} import cats.effect.internals.IOPlatform.fusionMaxStackDepth -import cats.effect.internals.TracingPlatformFast.tracingMode import scala.annotation.unchecked.uncheckedVariance import scala.concurrent.duration._ @@ -105,9 +105,9 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { final def map[B](f: A => B): IO[B] = // Don't perform map fusion when tracing is enabled. // We may end up removing map fusion altogether. - if (tracingMode == 1) { + if (isRabbitTracing) { Map(this, f, 0, IOTracing.cached(TraceTag.Map, f.getClass)) - } else if (tracingMode == 2) { + } else if (isSlugTracing) { Map(this, f, 0, IOTracing.uncached(TraceTag.Map)) } else { this match { @@ -138,9 +138,9 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * never terminate on evaluation. */ final def flatMap[B](f: A => IO[B]): IO[B] = { - val trace = if (tracingMode == 1) { + val trace = if (isRabbitTracing) { IOTracing.cached(TraceTag.Bind, f.getClass) - } else if (tracingMode == 2) { + } else if (isSlugTracing) { IOTracing.uncached(TraceTag.Bind) } else { null @@ -1137,7 +1137,7 @@ object IO extends IOInstances { */ def delay[A](body: => A): IO[A] = { val nextIo = Delay(() => body) - if (tracingMode == 2) { + if (isSlugTracing) { IOTracing.decorated(nextIo, TraceTag.Delay) } else { nextIo @@ -1154,7 +1154,7 @@ object IO extends IOInstances { */ def suspend[A](thunk: => IO[A]): IO[A] = { val nextIo = Suspend(() => thunk) - if (tracingMode == 2) { + if (isSlugTracing) { IOTracing.decorated(nextIo, TraceTag.Suspend) } else { nextIo @@ -1173,7 +1173,7 @@ object IO extends IOInstances { */ def pure[A](a: A): IO[A] = { val nextIo = Pure(a) - if (tracingMode == 2) { + if (isSlugTracing) { IOTracing.decorated(nextIo, TraceTag.Pure) } else { nextIo @@ -1244,9 +1244,9 @@ object IO extends IOInstances { * @see [[asyncF]] and [[cancelable]] */ def async[A](k: (Either[Throwable, A] => Unit) => Unit): IO[A] = { - val trace = if (tracingMode == 1) { + val trace = if (isRabbitTracing) { IOTracing.cached(TraceTag.Async, k.getClass) - } else if (tracingMode == 2) { + } else if (isSlugTracing) { IOTracing.uncached(TraceTag.Async) } else { null @@ -1284,9 +1284,9 @@ object IO extends IOInstances { * @see [[async]] and [[cancelable]] */ def asyncF[A](k: (Either[Throwable, A] => Unit) => IO[Unit]): IO[A] = { - val trace = if (tracingMode == 1) { + val trace = if (isRabbitTracing) { IOTracing.cached(TraceTag.AsyncF, k.getClass) - } else if (tracingMode == 2) { + } else if (isSlugTracing) { IOTracing.uncached(TraceTag.AsyncF) } else { null @@ -1349,9 +1349,9 @@ object IO extends IOInstances { * the underlying cancelation model */ def cancelable[A](k: (Either[Throwable, A] => Unit) => CancelToken[IO]): IO[A] = { - val trace = if (tracingMode == 1) { + val trace = if (isRabbitTracing) { IOTracing.cached(TraceTag.Cancelable, k.getClass) - } else if (tracingMode == 2) { + } else if (isSlugTracing) { IOTracing.uncached(TraceTag.Cancelable) } else { null diff --git a/core/shared/src/main/scala/cats/effect/internals/IOForkedStart.scala b/core/shared/src/main/scala/cats/effect/internals/IOForkedStart.scala index 161eb62770..ba71839f45 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOForkedStart.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOForkedStart.scala @@ -53,7 +53,7 @@ private[effect] object IOForkedStart { @tailrec def detect(task: IO[_], limit: Int = 8): Boolean = if (limit > 0) { task match { - case IO.Async(k, _) => k.isInstanceOf[IOForkedStart[_]] + case IO.Async(k, _, _) => k.isInstanceOf[IOForkedStart[_]] case IO.Bind(other, _, _) => detect(other, limit - 1) case IO.Map(other, _, _, _) => detect(other, limit - 1) case IO.ContextSwitch(other, _, _) => detect(other, limit - 1) diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index 5064b4958a..02fb4a1a72 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -20,8 +20,6 @@ import cats.effect.IO import cats.effect.IO.{Async, Bind, CollectTraces, ContextSwitch, Delay, Map, Pure, RaiseError, Suspend, Trace} import cats.effect.tracing.TraceFrame -import cats.effect.internals.TracingPlatformFast.tracingMode - import scala.util.control.NonFatal private[effect] object IORunLoop { @@ -88,7 +86,7 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if ((tracingMode == 1 || tracingMode == 2) && activeCollects > 0) { + if (Tracing.isTracing && activeCollects > 0) { if (ctx eq null) ctx = IOContext() val trace = bind.trace if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) @@ -133,7 +131,7 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if ((tracingMode == 1 || tracingMode == 2) && activeCollects > 0) { + if (Tracing.isTracing && activeCollects > 0) { if (ctx eq null) ctx = IOContext() val trace = bindNext.trace if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) @@ -147,7 +145,7 @@ private[effect] object IORunLoop { // may produce trace frames e.g. IOBracket. if (ctx eq null) ctx = IOContext() if (rcb eq null) rcb = new RestartCallback(conn, cb.asInstanceOf[Callback]) - if ((tracingMode == 1 || tracingMode == 2) && activeCollects > 0) { + if (Tracing.isTracing && activeCollects > 0) { val trace = async.trace if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) } @@ -230,7 +228,7 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if ((tracingMode == 1 || tracingMode == 2) && activeCollects > 0) { + if (Tracing.isTracing && activeCollects > 0) { if (ctx eq null) ctx = IOContext() val trace = bind.trace if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) @@ -275,7 +273,7 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if ((tracingMode == 1 || tracingMode == 2) && activeCollects > 0) { + if (Tracing.isTracing && activeCollects > 0) { if (ctx eq null) ctx = IOContext() val trace = bindNext.trace if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) From e26cc665bbf1ed169ef2470165983ab53db1d27f Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Tue, 9 Jun 2020 18:19:34 -0500 Subject: [PATCH 46/78] comments --- .../main/java/cats/effect/internals/TracingPlatformFast.java | 2 +- core/shared/src/main/java/cats/effect/internals/Tracing.java | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java index 3b38322352..6cffe3186b 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java @@ -36,7 +36,7 @@ public final class TracingPlatformFast { public static final TracingMode tracingMode = Optional.ofNullable(System.getProperty("cats.effect.tracing.mode")) .filter(x -> !x.isEmpty()) .flatMap(TracingMode::fromString) - .orElse(TracingMode.RABBIT); + .orElse(TracingMode.DISABLED); /** * The number of trace lines to retain during tracing. If more trace diff --git a/core/shared/src/main/java/cats/effect/internals/Tracing.java b/core/shared/src/main/java/cats/effect/internals/Tracing.java index 901801d054..686def480e 100644 --- a/core/shared/src/main/java/cats/effect/internals/Tracing.java +++ b/core/shared/src/main/java/cats/effect/internals/Tracing.java @@ -20,6 +20,9 @@ public class Tracing { private static final TracingMode globalTracingMode = TracingPlatformFast.tracingMode; + // Provide various views of the tracing mode + // that are used by IO constructors or run-loop. + public static final boolean isRabbitTracing = globalTracingMode == TracingMode.RABBIT; public static final boolean isSlugTracing = globalTracingMode == TracingMode.SLUG; From b1054e2af6ddc6d2991e99b376a9296f974e922c Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Tue, 9 Jun 2020 18:54:44 -0500 Subject: [PATCH 47/78] Move stack trace filtering to render function --- .../effect/internals/TracingPlatformFast.java | 2 +- .../scala/cats/effect/internals/Example.scala | 10 +++------ .../cats/effect/internals/IOTracing.scala | 21 +++---------------- .../scala/cats/effect/tracing/IOTrace.scala | 16 +++++++++++--- .../cats/effect/tracing/TraceFrame.scala | 7 ++++++- .../scala/cats/effect/tracing/TraceTag.scala | 4 ---- 6 files changed, 26 insertions(+), 34 deletions(-) diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java index 6cffe3186b..3b38322352 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java @@ -36,7 +36,7 @@ public final class TracingPlatformFast { public static final TracingMode tracingMode = Optional.ofNullable(System.getProperty("cats.effect.tracing.mode")) .filter(x -> !x.isEmpty()) .flatMap(TracingMode::fromString) - .orElse(TracingMode.DISABLED); + .orElse(TracingMode.RABBIT); /** * The number of trace lines to retain during tracing. If more trace diff --git a/core/jvm/src/main/scala/cats/effect/internals/Example.scala b/core/jvm/src/main/scala/cats/effect/internals/Example.scala index 0cb4f6c627..51fbb412d5 100644 --- a/core/jvm/src/main/scala/cats/effect/internals/Example.scala +++ b/core/jvm/src/main/scala/cats/effect/internals/Example.scala @@ -48,8 +48,8 @@ object Example extends IOApp { def program2: IO[Unit] = for { - _ <- print("7") - _ <- print("8") + _ <- print("3") + _ <- print("4") } yield () def program: IO[Unit] = @@ -58,11 +58,7 @@ object Example extends IOApp { _ <- print("2") _ <- IO.shift _ <- IO.async[Int](cb => cb(Right(32))) - _ <- IO.shift.bracket(_ => - print("3") - .flatMap(_ => program2) - )(_ => IO.unit) - _ <- print("4") + _ <- IO.unit.bracket(_ => program2)(_ => IO.unit) _ <- print("5") } yield () diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index 3d47092caf..5780cc4686 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -27,7 +27,6 @@ private[effect] object IOTracing { def decorated[A](source: IO[A], traceTag: TraceTag): IO[A] = Trace(source, buildFrame(traceTag)) - // TODO: Avoid trace tag for primitive ops and rely on class def uncached(traceTag: TraceTag): TraceFrame = buildFrame(traceTag) @@ -48,12 +47,8 @@ private[effect] object IOTracing { } } - def buildFrame(traceTag: TraceTag): TraceFrame = { - val stackTrace = new Throwable().getStackTrace.toList - .dropWhile(l => classBlacklist.exists(b => l.getClassName.startsWith(b))) - - TraceFrame(traceTag, stackTrace) - } + def buildFrame(traceTag: TraceTag): TraceFrame = + TraceFrame(traceTag, new Throwable()) private[this] val incrementCollection: IO[Unit] = CollectTraces(true) @@ -73,18 +68,8 @@ private[effect] object IOTracing { } /** - * Cache for trace frames. Keys are references to: - * - lambda classes + * Cache for trace frames. Keys are references to lambda classes. */ private[this] val frameCache: ConcurrentHashMap[Class[_], TraceFrame] = new ConcurrentHashMap() - private[this] val classBlacklist = List( - "cats.effect.", - "cats.", - "sbt.", - "java.", - "sun.", - "scala." - ) - } diff --git a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala index bd008b1cfd..083568880a 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala @@ -30,9 +30,10 @@ final case class IOTrace(frames: Vector[TraceFrame], captured: Int, omitted: Int } val acc0 = s"IOTrace: $captured frames captured, $omitted omitted\n" - val acc1 = frames.foldLeft(acc0)((acc, f) => - acc + s"\t${f.tag.name} at " + f.stackTrace.headOption.map(renderStackTraceElement).getOrElse("(...)") + "\n" - ) + "\n" + val acc1 = frames.foldLeft(acc0) { (acc, f) => + val first = f.stackTrace.dropWhile(l => classBlacklist.exists(b => l.getClassName.startsWith(b))).headOption + acc + s"\t${f.tag.name} at " + first.map(renderStackTraceElement).getOrElse("(...)") + "\n" + } + "\n" acc1 } @@ -93,6 +94,15 @@ final case class IOTrace(frames: Vector[TraceFrame], captured: Int, omitted: Int case None => methodName } + private[this] val classBlacklist = List( + "cats.effect.", + "cats.", + "sbt.", + "java.", + "sun.", + "scala." + ) + } object IOTrace { diff --git a/core/shared/src/main/scala/cats/effect/tracing/TraceFrame.scala b/core/shared/src/main/scala/cats/effect/tracing/TraceFrame.scala index e651e8015e..7393c83487 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/TraceFrame.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/TraceFrame.scala @@ -16,4 +16,9 @@ package cats.effect.tracing -final case class TraceFrame(tag: TraceTag, stackTrace: List[StackTraceElement]) +final case class TraceFrame(tag: TraceTag, throwable: Throwable) { + + def stackTrace: List[StackTraceElement] = + throwable.getStackTrace().toList + +} diff --git a/core/shared/src/main/scala/cats/effect/tracing/TraceTag.scala b/core/shared/src/main/scala/cats/effect/tracing/TraceTag.scala index ff39b04036..b76e4c7a99 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/TraceTag.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/TraceTag.scala @@ -23,14 +23,10 @@ object TraceTag { val Delay = TraceTag("delay") val Suspend = TraceTag("suspend") - val Attempt = TraceTag("attempt") val Bind = TraceTag("flatMap") val Map = TraceTag("map") val Async = TraceTag("async") val AsyncF = TraceTag("asyncF") val Cancelable = TraceTag("cancelable") - - val Bracket = TraceTag("bracket") - val BracketCase = TraceTag("bracketCase") } From 1392be921eae2f63367283477c4d690b4013a5f5 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Tue, 16 Jun 2020 22:41:23 -0500 Subject: [PATCH 48/78] address pr feedback --- .../cats/effect/internals/IOTracing.scala | 4 +- .../scala/cats/effect/tracing/IOTrace.scala | 37 +++++++++++-------- 2 files changed, 23 insertions(+), 18 deletions(-) diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index 5780cc4686..1f3d61632c 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -19,7 +19,7 @@ package cats.effect.internals import java.util.concurrent.ConcurrentHashMap import cats.effect.IO -import cats.effect.IO.{CollectTraces, Pure, RaiseError, Trace} +import cats.effect.IO.{Bind, CollectTraces, Pure, RaiseError, Trace} import cats.effect.tracing.{TraceFrame, TraceTag} private[effect] object IOTracing { @@ -34,7 +34,7 @@ private[effect] object IOTracing { buildCachedFrame(traceTag, clazz) def traced[A](source: IO[A]): IO[A] = - resetTrace *> incrementCollection *> source.flatMap(DecrementTraceCollection.asInstanceOf[A => IO[A]]) + resetTrace *> incrementCollection *> Bind(source, DecrementTraceCollection.asInstanceOf[A => IO[A]], null) private def buildCachedFrame(traceTag: TraceTag, clazz: Class[_]): TraceFrame = { val cachedFr = frameCache.get(clazz) diff --git a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala index 083568880a..5a28e42bc8 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala @@ -22,7 +22,13 @@ final case class IOTrace(frames: Vector[TraceFrame], captured: Int, omitted: Int import IOTrace._ + // Number of lines to drop from the head of the stack trace. + private[this] val DropLines = 3 + def compact: String = { + val TurnRight = "╰" + val Junction = "├" + def renderStackTraceElement(ste: StackTraceElement): String = { val className = demangleClassName(ste.getClassName) val methodName = demangleMethodName(ste.getMethodName) @@ -30,9 +36,11 @@ final case class IOTrace(frames: Vector[TraceFrame], captured: Int, omitted: Int } val acc0 = s"IOTrace: $captured frames captured, $omitted omitted\n" - val acc1 = frames.foldLeft(acc0) { (acc, f) => - val first = f.stackTrace.dropWhile(l => classBlacklist.exists(b => l.getClassName.startsWith(b))).headOption - acc + s"\t${f.tag.name} at " + first.map(renderStackTraceElement).getOrElse("(...)") + "\n" + val acc1 = frames.zipWithIndex.foldLeft(acc0) { + case (acc, (f, index)) => + val junc = if (index == frames.length - 1) TurnRight else Junction + val first = f.stackTrace.dropWhile(l => classBlacklist.exists(b => l.getClassName.startsWith(b))).headOption + acc + s" $junc ${f.tag.name} at " + first.map(renderStackTraceElement).getOrElse("(...)") + "\n" } + "\n" acc1 @@ -53,7 +61,6 @@ final case class IOTrace(frames: Vector[TraceFrame], captured: Int, omitted: Int private def loop(acc: String, indent: Int, init: Boolean, rest: List[TraceFrame]): String = { val TurnRight = "╰" val InverseTurnRight = "╭" - val TurnDown = "╮" val Junction = "├" val Line = "│" @@ -68,18 +75,16 @@ final case class IOTrace(frames: Vector[TraceFrame], captured: Int, omitted: Int rest match { case k :: ks => { - val acc2 = if (init) { - InverseTurnRight + s" ${k.tag.name}\n" - } else { - Junction + s" ${k.tag.name}\n" - } - - val inner = Line + " " + TurnRight + TurnDown + "\n" - val innerLines = k.stackTrace.zipWithIndex.map { - case (ste, i) => renderStackTraceElement(ste, i == k.stackTrace.length - 1) - }.mkString - - loop(acc + acc2 + inner + innerLines + Line + "\n", indent, false, ks) + val acc2 = if (init) InverseTurnRight + s" ${k.tag.name}\n" else Junction + s" ${k.tag.name}\n" + val innerLines = k.stackTrace + .drop(DropLines) + .zipWithIndex + .map { + case (ste, i) => renderStackTraceElement(ste, i == k.stackTrace.length - 1) + } + .mkString + + loop(acc + acc2 + innerLines + Line + "\n", indent, false, ks) } case Nil => acc } From bed286291b42644ba5cb3f3fcd02f703d215ae10 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Fri, 19 Jun 2020 01:39:31 -0500 Subject: [PATCH 49/78] Fix JS cross build --- build.sbt | 5 +---- .../scala/cats/effect/internals/IOTimer.scala | 2 +- .../cats/effect/internals/Tracing.scala} | 18 +++++----------- .../effect/internals/TracingPlatform.scala | 21 +++++++++++++++++++ .../java/cats/effect/internals/Tracing.java | 10 ++++----- ...PlatformFast.java => TracingPlatform.java} | 11 +++++----- .../cats/effect/internals/IOContext.scala | 2 +- .../scala/cats/effect/tracing/IOTrace.scala | 6 +++--- .../cats/effect/laws/util/TestContext.scala | 4 ++-- 9 files changed, 43 insertions(+), 36 deletions(-) rename core/{shared/src/main/java/cats/effect/internals/TracingMode.java => js/src/main/scala/cats/effect/internals/Tracing.scala} (63%) create mode 100644 core/js/src/main/scala/cats/effect/internals/TracingPlatform.scala rename core/{shared => jvm}/src/main/java/cats/effect/internals/Tracing.java (67%) rename core/jvm/src/main/java/cats/effect/internals/{TracingPlatformFast.java => TracingPlatform.java} (79%) diff --git a/build.sbt b/build.sbt index 8a5070c947..754d47708a 100644 --- a/build.sbt +++ b/build.sbt @@ -250,10 +250,7 @@ lazy val core = crossProject(JSPlatform, JVMPlatform) // https://github.com/scala/scala/pull/6606 Nil } - }, - unmanagedSourceDirectories in Compile += { - baseDirectory.value.getParentFile / "shared" / "src" / "main" / "java" - }, + } ) .jvmConfigure(_.enablePlugins(AutomateHeaderPlugin)) .jvmConfigure(_.settings(mimaSettings)) diff --git a/core/js/src/main/scala/cats/effect/internals/IOTimer.scala b/core/js/src/main/scala/cats/effect/internals/IOTimer.scala index 10d0343223..9a99936b89 100644 --- a/core/js/src/main/scala/cats/effect/internals/IOTimer.scala +++ b/core/js/src/main/scala/cats/effect/internals/IOTimer.scala @@ -33,7 +33,7 @@ final private[internals] class IOTimer(ec: ExecutionContext) extends Timer[IO] { def sleep(timespan: FiniteDuration): IO[Unit] = IO.Async(new IOForkedStart[Unit] { - def apply(conn: IOConnection, cb: Either[Throwable, Unit] => Unit): Unit = { + def apply(conn: IOConnection, ctx: IOContext, cb: Either[Throwable, Unit] => Unit): Unit = { val task = setTimeout(timespan.toMillis, ec, new ScheduledTick(conn, cb)) // On the JVM this would need a ForwardCancelable, // but not on top of JS as we don't have concurrency diff --git a/core/shared/src/main/java/cats/effect/internals/TracingMode.java b/core/js/src/main/scala/cats/effect/internals/Tracing.scala similarity index 63% rename from core/shared/src/main/java/cats/effect/internals/TracingMode.java rename to core/js/src/main/scala/cats/effect/internals/Tracing.scala index 2cd58c45eb..f912497684 100644 --- a/core/shared/src/main/java/cats/effect/internals/TracingMode.java +++ b/core/js/src/main/scala/cats/effect/internals/Tracing.scala @@ -14,20 +14,12 @@ * limitations under the License. */ -package cats.effect.internals; +package cats.effect.internals -import java.util.Optional; +object Tracing { + val isRabbitTracing: Boolean = false -public enum TracingMode { - DISABLED, - RABBIT, - SLUG; + val isSlugTracing: Boolean = false - public static Optional fromString(String value) { - try { - return Optional.of(valueOf(value.toUpperCase())); - } catch(IllegalArgumentException ex) { - return Optional.empty(); - } - } + val isTracing: Boolean = isSlugTracing || isRabbitTracing } diff --git a/core/js/src/main/scala/cats/effect/internals/TracingPlatform.scala b/core/js/src/main/scala/cats/effect/internals/TracingPlatform.scala new file mode 100644 index 0000000000..ccf5c47ac2 --- /dev/null +++ b/core/js/src/main/scala/cats/effect/internals/TracingPlatform.scala @@ -0,0 +1,21 @@ +/* + * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers + * + * Licensed 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 cats.effect.internals + +private[effect] object TracingPlatform { + val maxTraceDepth: Int = 32 +} diff --git a/core/shared/src/main/java/cats/effect/internals/Tracing.java b/core/jvm/src/main/java/cats/effect/internals/Tracing.java similarity index 67% rename from core/shared/src/main/java/cats/effect/internals/Tracing.java rename to core/jvm/src/main/java/cats/effect/internals/Tracing.java index 686def480e..e2a59fadac 100644 --- a/core/shared/src/main/java/cats/effect/internals/Tracing.java +++ b/core/jvm/src/main/java/cats/effect/internals/Tracing.java @@ -18,14 +18,12 @@ public class Tracing { - private static final TracingMode globalTracingMode = TracingPlatformFast.tracingMode; + // Provides accessors to the tracing mode + // that are used by IO constructors or the run-loop. - // Provide various views of the tracing mode - // that are used by IO constructors or run-loop. + public static final boolean isRabbitTracing = TracingPlatform.tracingMode.equalsIgnoreCase("rabbit"); - public static final boolean isRabbitTracing = globalTracingMode == TracingMode.RABBIT; - - public static final boolean isSlugTracing = globalTracingMode == TracingMode.SLUG; + public static final boolean isSlugTracing = TracingPlatform.tracingMode.equalsIgnoreCase("slug"); public static final boolean isTracing = isSlugTracing || isRabbitTracing; diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java similarity index 79% rename from core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java rename to core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java index 3b38322352..4defb57053 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatformFast.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java @@ -22,21 +22,20 @@ * Holds platform-specific flags that control tracing behavior. * * The Scala compiler inserts a volatile bitmap access for module field accesses. - * Since several of these flags are read during IO node construction, we are opting - * to hold this flag in a Java class to bypass that and squeeze out more performance. + * Because the `tracingMode` flag is read in various IO combinators, we are opting + * to define it in a Java source file to avoid the volatile access. * * INTERNAL API. */ -public final class TracingPlatformFast { +final class TracingPlatform { /** * A string flag that sets a global tracing mode for a JVM process. * Acceptable values are: DISABLED, RABBIT, SLUG. */ - public static final TracingMode tracingMode = Optional.ofNullable(System.getProperty("cats.effect.tracing.mode")) + public static final String tracingMode = Optional.ofNullable(System.getProperty("cats.effect.tracing.mode")) .filter(x -> !x.isEmpty()) - .flatMap(TracingMode::fromString) - .orElse(TracingMode.RABBIT); + .orElse("disabled"); /** * The number of trace lines to retain during tracing. If more trace diff --git a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala index 8c0c81fb4c..d0b067bcbe 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala @@ -17,7 +17,7 @@ package cats.effect.internals import cats.effect.tracing.{IOTrace, TraceFrame} -import cats.effect.internals.TracingPlatformFast.maxTraceDepth +import cats.effect.internals.TracingPlatform.maxTraceDepth /** * INTERNAL API — Holds state related to the execution of diff --git a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala index 5a28e42bc8..5fc4136cdd 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala @@ -22,8 +22,8 @@ final case class IOTrace(frames: Vector[TraceFrame], captured: Int, omitted: Int import IOTrace._ - // Number of lines to drop from the head of the stack trace. - private[this] val DropLines = 3 + // Number of lines to drop from the top of the stack trace + private[this] val stackTraceIgnoreLines = 3 def compact: String = { val TurnRight = "╰" @@ -77,7 +77,7 @@ final case class IOTrace(frames: Vector[TraceFrame], captured: Int, omitted: Int case k :: ks => { val acc2 = if (init) InverseTurnRight + s" ${k.tag.name}\n" else Junction + s" ${k.tag.name}\n" val innerLines = k.stackTrace - .drop(DropLines) + .drop(stackTraceIgnoreLines) .zipWithIndex .map { case (ste, i) => renderStackTraceElement(ste, i == k.stackTrace.length - 1) diff --git a/laws/shared/src/main/scala/cats/effect/laws/util/TestContext.scala b/laws/shared/src/main/scala/cats/effect/laws/util/TestContext.scala index 3354140416..d46b7f8485 100644 --- a/laws/shared/src/main/scala/cats/effect/laws/util/TestContext.scala +++ b/laws/shared/src/main/scala/cats/effect/laws/util/TestContext.scala @@ -17,7 +17,7 @@ package cats.effect.laws.util import cats.effect.internals.Callback.T -import cats.effect.internals.{IOConnection, IOForkedStart, IOShift} +import cats.effect.internals.{IOConnection, IOContext, IOForkedStart, IOShift} import cats.effect._ import scala.collection.immutable.SortedSet @@ -190,7 +190,7 @@ final class TestContext private () extends ExecutionContext { self => override def shift: F[Unit] = F.liftIO(IO.Async(new IOForkedStart[Unit] { - def apply(conn: IOConnection, cb: T[Unit]): Unit = + def apply(conn: IOConnection, ctx: IOContext, cb: T[Unit]): Unit = self.execute(tick(cb)) })) From f9131b3fbce18e7b6d2b4cbe7e0c1d637c2eecd4 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Fri, 19 Jun 2020 01:54:15 -0500 Subject: [PATCH 50/78] Merge Tracing and TracingPlatform classes --- .../scala/cats/effect/internals/Tracing.scala | 25 ---------------- .../effect/internals/TracingPlatform.scala | 10 +++++-- .../java/cats/effect/internals/Tracing.java | 30 ------------------- .../effect/internals/TracingPlatform.java | 13 ++++++-- .../src/main/scala/cats/effect/IO.scala | 2 +- .../cats/effect/internals/IORunLoop.scala | 10 +++---- 6 files changed, 25 insertions(+), 65 deletions(-) delete mode 100644 core/js/src/main/scala/cats/effect/internals/Tracing.scala delete mode 100644 core/jvm/src/main/java/cats/effect/internals/Tracing.java diff --git a/core/js/src/main/scala/cats/effect/internals/Tracing.scala b/core/js/src/main/scala/cats/effect/internals/Tracing.scala deleted file mode 100644 index f912497684..0000000000 --- a/core/js/src/main/scala/cats/effect/internals/Tracing.scala +++ /dev/null @@ -1,25 +0,0 @@ -/* - * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers - * - * Licensed 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 cats.effect.internals - -object Tracing { - val isRabbitTracing: Boolean = false - - val isSlugTracing: Boolean = false - - val isTracing: Boolean = isSlugTracing || isRabbitTracing -} diff --git a/core/js/src/main/scala/cats/effect/internals/TracingPlatform.scala b/core/js/src/main/scala/cats/effect/internals/TracingPlatform.scala index ccf5c47ac2..bb5949b559 100644 --- a/core/js/src/main/scala/cats/effect/internals/TracingPlatform.scala +++ b/core/js/src/main/scala/cats/effect/internals/TracingPlatform.scala @@ -16,6 +16,12 @@ package cats.effect.internals -private[effect] object TracingPlatform { - val maxTraceDepth: Int = 32 +object TracingPlatform { + final val isRabbitTracing: Boolean = false + + final val isSlugTracing: Boolean = false + + final val isTracing: Boolean = isSlugTracing || isRabbitTracing + + final val maxTraceDepth: Int = 32 } diff --git a/core/jvm/src/main/java/cats/effect/internals/Tracing.java b/core/jvm/src/main/java/cats/effect/internals/Tracing.java deleted file mode 100644 index e2a59fadac..0000000000 --- a/core/jvm/src/main/java/cats/effect/internals/Tracing.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers - * - * Licensed 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 cats.effect.internals; - -public class Tracing { - - // Provides accessors to the tracing mode - // that are used by IO constructors or the run-loop. - - public static final boolean isRabbitTracing = TracingPlatform.tracingMode.equalsIgnoreCase("rabbit"); - - public static final boolean isSlugTracing = TracingPlatform.tracingMode.equalsIgnoreCase("slug"); - - public static final boolean isTracing = isSlugTracing || isRabbitTracing; - -} diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java index 4defb57053..7210302558 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java @@ -27,16 +27,25 @@ * * INTERNAL API. */ -final class TracingPlatform { +public final class TracingPlatform { /** * A string flag that sets a global tracing mode for a JVM process. * Acceptable values are: DISABLED, RABBIT, SLUG. + * + * This field isn't accessed by other classes; instead use one of the + * more specific accessors defined below. */ - public static final String tracingMode = Optional.ofNullable(System.getProperty("cats.effect.tracing.mode")) + private static final String tracingMode = Optional.ofNullable(System.getProperty("cats.effect.tracing.mode")) .filter(x -> !x.isEmpty()) .orElse("disabled"); + public static final boolean isRabbitTracing = tracingMode.equalsIgnoreCase("rabbit"); + + public static final boolean isSlugTracing = tracingMode.equalsIgnoreCase("slug"); + + public static final boolean isTracing = isSlugTracing || isRabbitTracing; + /** * The number of trace lines to retain during tracing. If more trace * lines are produced, then the oldest trace lines will be discarded. diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index 0ef0cce188..a9d496b491 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -18,7 +18,7 @@ package cats package effect import cats.effect.internals._ -import cats.effect.internals.Tracing.{isRabbitTracing, isSlugTracing} +import cats.effect.internals.TracingPlatform.{isRabbitTracing, isSlugTracing} import cats.effect.internals.IOPlatform.fusionMaxStackDepth import scala.annotation.unchecked.uncheckedVariance diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index 02fb4a1a72..6198bf7789 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -86,7 +86,7 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if (Tracing.isTracing && activeCollects > 0) { + if (TracingPlatform.isTracing && activeCollects > 0) { if (ctx eq null) ctx = IOContext() val trace = bind.trace if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) @@ -131,7 +131,7 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if (Tracing.isTracing && activeCollects > 0) { + if (TracingPlatform.isTracing && activeCollects > 0) { if (ctx eq null) ctx = IOContext() val trace = bindNext.trace if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) @@ -145,7 +145,7 @@ private[effect] object IORunLoop { // may produce trace frames e.g. IOBracket. if (ctx eq null) ctx = IOContext() if (rcb eq null) rcb = new RestartCallback(conn, cb.asInstanceOf[Callback]) - if (Tracing.isTracing && activeCollects > 0) { + if (TracingPlatform.isTracing && activeCollects > 0) { val trace = async.trace if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) } @@ -228,7 +228,7 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if (Tracing.isTracing && activeCollects > 0) { + if (TracingPlatform.isTracing && activeCollects > 0) { if (ctx eq null) ctx = IOContext() val trace = bind.trace if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) @@ -273,7 +273,7 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if (Tracing.isTracing && activeCollects > 0) { + if (TracingPlatform.isTracing && activeCollects > 0) { if (ctx eq null) ctx = IOContext() val trace = bindNext.trace if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) From 1e4eb730aa8b8c59d7da9ecf697d97010ff9f731 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Fri, 19 Jun 2020 16:26:20 -0500 Subject: [PATCH 51/78] Tracing tests --- build.sbt | 32 +++++- .../cats/effect/internals/IOContext.scala | 2 +- .../cats/effect/internals/IOTracing.scala | 8 +- .../scala/cats/effect/tracing/IOTrace.scala | 22 ++-- .../slug/scala/cats/effect/TracingTests.scala | 91 +++++++++++++++ .../test/scala/cats/effect/TracingTests.scala | 104 ++++++++++++++++++ 6 files changed, 243 insertions(+), 16 deletions(-) create mode 100644 tracing-tests/jvm/src/slug/scala/cats/effect/TracingTests.scala create mode 100644 tracing-tests/shared/src/test/scala/cats/effect/TracingTests.scala diff --git a/build.sbt b/build.sbt index 754d47708a..c648c42567 100644 --- a/build.sbt +++ b/build.sbt @@ -225,7 +225,7 @@ lazy val sharedSourcesSettings = Seq( lazy val root = project .in(file(".")) .disablePlugins(MimaPlugin) - .aggregate(coreJVM, coreJS, lawsJVM, lawsJS) + .aggregate(coreJVM, coreJS, lawsJVM, lawsJS, tracingTestsJVM, tracingTestsJS) .settings(skipOnPublishSettings) lazy val core = crossProject(JSPlatform, JVMPlatform) @@ -285,6 +285,36 @@ lazy val laws = crossProject(JSPlatform, JVMPlatform) lazy val lawsJVM = laws.jvm lazy val lawsJS = laws.js +lazy val SlugTest = config("slug") extend Test + +lazy val tracingTests = crossProject(JSPlatform, JVMPlatform) + .in(file("tracing-tests")) + .dependsOn(core % "compile->compile;test->test") + .settings(commonSettings: _*) + .settings( + name := "cats-effect-tracing-tests", + libraryDependencies ++= Seq( + "org.typelevel" %%% "cats-laws" % CatsVersion, + "org.typelevel" %%% "discipline-scalatest" % DisciplineScalatestVersion % Test + ) + ) + .configs(SlugTest) + .settings(inConfig(SlugTest)(Defaults.testSettings): _*) + .jvmConfigure(_.enablePlugins(AutomateHeaderPlugin)) + .jvmConfigure(_.settings(lawsMimaSettings)) + .jsConfigure(_.enablePlugins(AutomateHeaderPlugin)) + .jsConfigure(_.settings(scalaJSSettings)) + .jvmSettings( + skip.in(publish) := customScalaJSVersion.forall(_.startsWith("1.0")), + fork in Test := true, + fork in SlugTest := true, + javaOptions in Test += "-Dcats.effect.tracing.mode=rabbit", + javaOptions in SlugTest += "-Dcats.effect.tracing.mode=slug" + ) + +lazy val tracingTestsJVM = tracingTests.jvm +lazy val tracingTestsJS = tracingTests.js + lazy val benchmarksPrev = project .in(file("benchmarks/vPrev")) .settings(commonSettings ++ skipOnPublishSettings ++ sharedSourcesSettings) diff --git a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala index d0b067bcbe..ec1b0340fe 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala @@ -43,7 +43,7 @@ final private[effect] class IOContext private () { } def trace: IOTrace = - IOTrace(frames.toList.toVector, captured, omitted) + IOTrace(frames.toList, captured, omitted) } diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index 1f3d61632c..348552069a 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -34,7 +34,9 @@ private[effect] object IOTracing { buildCachedFrame(traceTag, clazz) def traced[A](source: IO[A]): IO[A] = - resetTrace *> incrementCollection *> Bind(source, DecrementTraceCollection.asInstanceOf[A => IO[A]], null) + resetTrace *> + Bind(incrementCollection, (_: Unit) => + Bind(source, DecrementTraceCollection.asInstanceOf[A => IO[A]], null), null) private def buildCachedFrame(traceTag: TraceTag, clazz: Class[_]): TraceFrame = { val cachedFr = frameCache.get(clazz) @@ -56,9 +58,9 @@ private[effect] object IOTracing { private object DecrementTraceCollection extends IOFrame[Any, IO[Any]] { override def apply(a: Any) = - decrementCollection *> Pure(a) + Bind(decrementCollection, (_: Unit) => Pure(a), null) override def recover(e: Throwable) = - decrementCollection *> RaiseError(e) + Bind(decrementCollection, (_: Unit) => RaiseError(e), null) } private[this] val resetTrace: IO[Unit] = diff --git a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala index 5fc4136cdd..fc3c8ba2ab 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala @@ -18,13 +18,10 @@ package cats.effect.tracing import cats.effect.IO -final case class IOTrace(frames: Vector[TraceFrame], captured: Int, omitted: Int) { +final case class IOTrace(frames: List[TraceFrame], captured: Int, omitted: Int) { import IOTrace._ - // Number of lines to drop from the top of the stack trace - private[this] val stackTraceIgnoreLines = 3 - def compact: String = { val TurnRight = "╰" val Junction = "├" @@ -39,7 +36,7 @@ final case class IOTrace(frames: Vector[TraceFrame], captured: Int, omitted: Int val acc1 = frames.zipWithIndex.foldLeft(acc0) { case (acc, (f, index)) => val junc = if (index == frames.length - 1) TurnRight else Junction - val first = f.stackTrace.dropWhile(l => classBlacklist.exists(b => l.getClassName.startsWith(b))).headOption + val first = f.stackTrace.dropWhile(l => stackTraceFilter.exists(b => l.getClassName.startsWith(b))).headOption acc + s" $junc ${f.tag.name} at " + first.map(renderStackTraceElement).getOrElse("(...)") + "\n" } + "\n" @@ -99,7 +96,15 @@ final case class IOTrace(frames: Vector[TraceFrame], captured: Int, omitted: Int case None => methodName } - private[this] val classBlacklist = List( +} + +private[effect] object IOTrace { + final val anonfunRegex = "^\\$+anonfun\\$+(.+)\\$+\\d+$".r + + // Number of lines to drop from the top of the stack trace + final val stackTraceIgnoreLines = 3 + + final val stackTraceFilter = List( "cats.effect.", "cats.", "sbt.", @@ -107,9 +112,4 @@ final case class IOTrace(frames: Vector[TraceFrame], captured: Int, omitted: Int "sun.", "scala." ) - -} - -object IOTrace { - private[effect] val anonfunRegex = "^\\$+anonfun\\$+(.+)\\$+\\d+$".r } diff --git a/tracing-tests/jvm/src/slug/scala/cats/effect/TracingTests.scala b/tracing-tests/jvm/src/slug/scala/cats/effect/TracingTests.scala new file mode 100644 index 0000000000..533aa8853e --- /dev/null +++ b/tracing-tests/jvm/src/slug/scala/cats/effect/TracingTests.scala @@ -0,0 +1,91 @@ +/* + * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers + * + * Licensed 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 cats.effect + +import org.scalatest.funsuite.AsyncFunSuite +import org.scalatest.matchers.should.Matchers + +import scala.concurrent.ExecutionContext + +class TracingTests extends AsyncFunSuite with Matchers { + implicit override def executionContext: ExecutionContext = ExecutionContext.Implicits.global + implicit val timer: Timer[IO] = IO.timer(executionContext) + implicit val cs: ContextShift[IO] = IO.contextShift(executionContext) + + test("IOTrace should be empty if no tracing is performed") { + val task = for { + _ <- IO.pure(1) + _ <- IO.pure(1) + t <- IO.backtrace + } yield t.frames.length + + for (r <- task.unsafeToFuture()) yield { + r shouldBe 0 + } + } + + test("IOTrace should capture relevant frames") { + val op = for { + a <- IO.pure(1) + b <- IO.pure(1) + } yield a + b + + val task = for { + _ <- op.traced + t <- IO.backtrace + } yield t.frames.length + + for (r <- task.unsafeToFuture()) yield { + r shouldBe 3 + } + } + + test("traces replace old traces") { + val op = for { + a <- IO.pure(1) + b <- IO.pure(1) + } yield a + b + + val task = for { + _ <- op.traced + _ <- IO.backtrace + _ <- op.traced + t <- IO.backtrace + } yield t.frames.length + + for (r <- task.unsafeToFuture()) yield { + r shouldBe 3 + } + } + + test("traces are preserved across asynchronous boundaries") { + val op = for { + a <- IO.pure(1) + _ <- IO.shift + b <- IO.pure(1) + } yield a + b + + val task = for { + _ <- op.traced + t <- IO.backtrace + } yield t.frames.length + + for (r <- task.unsafeToFuture()) yield { + r shouldBe 4 + } + } +} diff --git a/tracing-tests/shared/src/test/scala/cats/effect/TracingTests.scala b/tracing-tests/shared/src/test/scala/cats/effect/TracingTests.scala new file mode 100644 index 0000000000..7cd0ef5a1e --- /dev/null +++ b/tracing-tests/shared/src/test/scala/cats/effect/TracingTests.scala @@ -0,0 +1,104 @@ +/* + * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers + * + * Licensed 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 cats.effect + +import cats.effect.tracing.{IOTrace, TraceTag} +import org.scalatest.funsuite.AsyncFunSuite +import org.scalatest.matchers.should.Matchers + +import scala.concurrent.ExecutionContext + +class TracingTests extends AsyncFunSuite with Matchers { + implicit override def executionContext: ExecutionContext = ExecutionContext.Implicits.global + implicit val timer: Timer[IO] = IO.timer(executionContext) + implicit val cs: ContextShift[IO] = IO.contextShift(executionContext) + + def traced[A](io: IO[A]): IO[IOTrace] = + for { + _ <- io.traced + t <- IO.backtrace + } yield t + + test("trace is empty when no traces are captured") { + val task = for { + _ <- IO.pure(1) + _ <- IO.pure(1) + t <- IO.backtrace + } yield t + + for (r <- task.unsafeToFuture()) yield { + r.captured shouldBe 0 + } + } + + test("rabbit tracing captures map frames") { + val task = IO.pure(0).map(_ + 1).map(_ + 1) + + for (r <- traced(task).unsafeToFuture()) yield { + r.captured shouldBe 2 + r.frames.filter(_.tag == TraceTag.Map).length shouldBe 2 + } + } + + test("rabbit tracing captures bind frames") { + val task = IO.pure(0).flatMap(a => IO(a + 1)).flatMap(a => IO(a + 1)) + + for (r <- traced(task).unsafeToFuture()) yield { + r.captured shouldBe 2 + r.frames.filter(_.tag == TraceTag.Bind).length shouldBe 2 + } + } + + test("rabbit tracing captures async frames") { + val task = IO.async[Int](_(Right(0))).flatMap(a => IO(a + 1)).flatMap(a => IO(a + 1)) + + for (r <- traced(task).unsafeToFuture()) yield { + r.captured shouldBe 3 + r.frames.filter(_.tag == TraceTag.Async).length shouldBe 1 + } + } + + test("traces are preserved across asynchronous boundaries") { + val task = for { + a <- IO.pure(1) + _ <- IO.shift + b <- IO.pure(1) + } yield a + b + + for (r <- traced(task).unsafeToFuture()) yield { + r.captured shouldBe 3 + } + } + + test("traces are emptied when initiating a new trace") { + val op = for { + a <- IO.pure(1) + b <- IO.pure(1) + } yield a + b + + val task = for { + _ <- op.traced + _ <- IO.backtrace + _ <- op.traced + t <- IO.backtrace + } yield t.frames.length + + for (r <- task.unsafeToFuture()) yield { + r shouldBe 2 + } + } +} From b7a2f810b40d21e206604da5af9f6fd248cfee5a Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Fri, 19 Jun 2020 22:40:07 -0500 Subject: [PATCH 52/78] Split out rabbit and slug tracing tests --- build.sbt | 7 ++ .../slug/scala/cats/effect/TracingTests.scala | 91 ------------------- .../scala/cats/effect/SlugTracingTests.scala | 89 ++++++++++++++++++ .../cats/effect/RabbitTracingTests.scala | 62 +++++++++++++ .../test/scala/cats/effect/TracingTests.scala | 29 +----- 5 files changed, 159 insertions(+), 119 deletions(-) delete mode 100644 tracing-tests/jvm/src/slug/scala/cats/effect/TracingTests.scala create mode 100644 tracing-tests/shared/src/slug/scala/cats/effect/SlugTracingTests.scala create mode 100644 tracing-tests/shared/src/test/scala/cats/effect/RabbitTracingTests.scala diff --git a/build.sbt b/build.sbt index c648c42567..0132e3c2c3 100644 --- a/build.sbt +++ b/build.sbt @@ -300,6 +300,13 @@ lazy val tracingTests = crossProject(JSPlatform, JVMPlatform) ) .configs(SlugTest) .settings(inConfig(SlugTest)(Defaults.testSettings): _*) + .jsSettings(inConfig(SlugTest)(ScalaJSPlugin.testConfigSettings): _*) + .settings( + test in Test := (test in Test).dependsOn(test in SlugTest).value, + unmanagedSourceDirectories in SlugTest += { + baseDirectory.value.getParentFile / "shared" / "src" / "slug" / "scala" + } + ) .jvmConfigure(_.enablePlugins(AutomateHeaderPlugin)) .jvmConfigure(_.settings(lawsMimaSettings)) .jsConfigure(_.enablePlugins(AutomateHeaderPlugin)) diff --git a/tracing-tests/jvm/src/slug/scala/cats/effect/TracingTests.scala b/tracing-tests/jvm/src/slug/scala/cats/effect/TracingTests.scala deleted file mode 100644 index 533aa8853e..0000000000 --- a/tracing-tests/jvm/src/slug/scala/cats/effect/TracingTests.scala +++ /dev/null @@ -1,91 +0,0 @@ -/* - * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers - * - * Licensed 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 cats.effect - -import org.scalatest.funsuite.AsyncFunSuite -import org.scalatest.matchers.should.Matchers - -import scala.concurrent.ExecutionContext - -class TracingTests extends AsyncFunSuite with Matchers { - implicit override def executionContext: ExecutionContext = ExecutionContext.Implicits.global - implicit val timer: Timer[IO] = IO.timer(executionContext) - implicit val cs: ContextShift[IO] = IO.contextShift(executionContext) - - test("IOTrace should be empty if no tracing is performed") { - val task = for { - _ <- IO.pure(1) - _ <- IO.pure(1) - t <- IO.backtrace - } yield t.frames.length - - for (r <- task.unsafeToFuture()) yield { - r shouldBe 0 - } - } - - test("IOTrace should capture relevant frames") { - val op = for { - a <- IO.pure(1) - b <- IO.pure(1) - } yield a + b - - val task = for { - _ <- op.traced - t <- IO.backtrace - } yield t.frames.length - - for (r <- task.unsafeToFuture()) yield { - r shouldBe 3 - } - } - - test("traces replace old traces") { - val op = for { - a <- IO.pure(1) - b <- IO.pure(1) - } yield a + b - - val task = for { - _ <- op.traced - _ <- IO.backtrace - _ <- op.traced - t <- IO.backtrace - } yield t.frames.length - - for (r <- task.unsafeToFuture()) yield { - r shouldBe 3 - } - } - - test("traces are preserved across asynchronous boundaries") { - val op = for { - a <- IO.pure(1) - _ <- IO.shift - b <- IO.pure(1) - } yield a + b - - val task = for { - _ <- op.traced - t <- IO.backtrace - } yield t.frames.length - - for (r <- task.unsafeToFuture()) yield { - r shouldBe 4 - } - } -} diff --git a/tracing-tests/shared/src/slug/scala/cats/effect/SlugTracingTests.scala b/tracing-tests/shared/src/slug/scala/cats/effect/SlugTracingTests.scala new file mode 100644 index 0000000000..a1d82c7259 --- /dev/null +++ b/tracing-tests/shared/src/slug/scala/cats/effect/SlugTracingTests.scala @@ -0,0 +1,89 @@ +/* + * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers + * + * Licensed 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 cats.effect + +import cats.effect.tracing.{IOTrace, TraceTag} +import org.scalatest.funsuite.AsyncFunSuite +import org.scalatest.matchers.should.Matchers + +import scala.concurrent.ExecutionContext + +class SlugTracingTests extends AsyncFunSuite with Matchers { + implicit override def executionContext: ExecutionContext = ExecutionContext.Implicits.global + implicit val timer: Timer[IO] = IO.timer(executionContext) + implicit val cs: ContextShift[IO] = IO.contextShift(executionContext) + + def traced[A](io: IO[A]): IO[IOTrace] = + for { + _ <- io.traced + t <- IO.backtrace + } yield t + + test("slug tracing captures map frames") { + val task = IO.pure(0).map(_ + 1).map(_ + 1) + + for (r <- traced(task).unsafeToFuture()) yield { + r.captured shouldBe 3 + r.frames.filter(_.tag == TraceTag.Map).length shouldBe 2 + } + } + + test("slug tracing captures bind frames") { + val task = IO.pure(0).flatMap(a => IO(a + 1)).flatMap(a => IO(a + 1)) + + for (r <- traced(task).unsafeToFuture()) yield { + r.captured shouldBe 5 + r.frames.filter(_.tag == TraceTag.Bind).length shouldBe 2 + } + } + + test("slug tracing captures async frames") { + val task = IO.async[Int](_(Right(0))).flatMap(a => IO(a + 1)).flatMap(a => IO(a + 1)) + + for (r <- traced(task).unsafeToFuture()) yield { + r.captured shouldBe 5 + r.frames.filter(_.tag == TraceTag.Async).length shouldBe 1 + } + } + + test("slug tracing captures pure frames") { + val task = IO.pure(0).flatMap(a => IO.pure(a + 1)) + + for (r <- traced(task).unsafeToFuture()) yield { + r.captured shouldBe 3 + r.frames.filter(_.tag == TraceTag.Pure).length shouldBe 2 + } + } + + test("slug tracing captures delay frames") { + val task = IO(0).flatMap(a => IO(a + 1)) + + for (r <- traced(task).unsafeToFuture()) yield { + r.captured shouldBe 3 + r.frames.filter(_.tag == TraceTag.Delay).length shouldBe 2 + } + } + + test("slug tracing captures suspend frames") { + val task = IO.suspend(IO(1)).flatMap(a => IO.suspend(IO(a + 1))) + + for (r <- traced(task).unsafeToFuture()) yield { + r.captured shouldBe 5 + r.frames.filter(_.tag == TraceTag.Suspend).length shouldBe 2 + } + } +} diff --git a/tracing-tests/shared/src/test/scala/cats/effect/RabbitTracingTests.scala b/tracing-tests/shared/src/test/scala/cats/effect/RabbitTracingTests.scala new file mode 100644 index 0000000000..a113a81678 --- /dev/null +++ b/tracing-tests/shared/src/test/scala/cats/effect/RabbitTracingTests.scala @@ -0,0 +1,62 @@ +/* + * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers + * + * Licensed 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 cats.effect + +import cats.effect.tracing.{IOTrace, TraceTag} +import org.scalatest.funsuite.AsyncFunSuite +import org.scalatest.matchers.should.Matchers + +import scala.concurrent.ExecutionContext + +class RabbitTracingTests extends AsyncFunSuite with Matchers { + implicit override def executionContext: ExecutionContext = ExecutionContext.Implicits.global + implicit val timer: Timer[IO] = IO.timer(executionContext) + implicit val cs: ContextShift[IO] = IO.contextShift(executionContext) + + def traced[A](io: IO[A]): IO[IOTrace] = + for { + _ <- io.traced + t <- IO.backtrace + } yield t + + test("rabbit tracing captures map frames") { + val task = IO.pure(0).map(_ + 1).map(_ + 1) + + for (r <- traced(task).unsafeToFuture()) yield { + r.captured shouldBe 2 + r.frames.filter(_.tag == TraceTag.Map).length shouldBe 2 + } + } + + test("rabbit tracing captures bind frames") { + val task = IO.pure(0).flatMap(a => IO(a + 1)).flatMap(a => IO(a + 1)) + + for (r <- traced(task).unsafeToFuture()) yield { + r.captured shouldBe 2 + r.frames.filter(_.tag == TraceTag.Bind).length shouldBe 2 + } + } + + test("rabbit tracing captures async frames") { + val task = IO.async[Int](_(Right(0))).flatMap(a => IO(a + 1)).flatMap(a => IO(a + 1)) + + for (r <- traced(task).unsafeToFuture()) yield { + r.captured shouldBe 3 + r.frames.filter(_.tag == TraceTag.Async).length shouldBe 1 + } + } +} diff --git a/tracing-tests/shared/src/test/scala/cats/effect/TracingTests.scala b/tracing-tests/shared/src/test/scala/cats/effect/TracingTests.scala index 7cd0ef5a1e..1c8d10966d 100644 --- a/tracing-tests/shared/src/test/scala/cats/effect/TracingTests.scala +++ b/tracing-tests/shared/src/test/scala/cats/effect/TracingTests.scala @@ -16,7 +16,7 @@ package cats.effect -import cats.effect.tracing.{IOTrace, TraceTag} +import cats.effect.tracing.IOTrace import org.scalatest.funsuite.AsyncFunSuite import org.scalatest.matchers.should.Matchers @@ -45,33 +45,6 @@ class TracingTests extends AsyncFunSuite with Matchers { } } - test("rabbit tracing captures map frames") { - val task = IO.pure(0).map(_ + 1).map(_ + 1) - - for (r <- traced(task).unsafeToFuture()) yield { - r.captured shouldBe 2 - r.frames.filter(_.tag == TraceTag.Map).length shouldBe 2 - } - } - - test("rabbit tracing captures bind frames") { - val task = IO.pure(0).flatMap(a => IO(a + 1)).flatMap(a => IO(a + 1)) - - for (r <- traced(task).unsafeToFuture()) yield { - r.captured shouldBe 2 - r.frames.filter(_.tag == TraceTag.Bind).length shouldBe 2 - } - } - - test("rabbit tracing captures async frames") { - val task = IO.async[Int](_(Right(0))).flatMap(a => IO(a + 1)).flatMap(a => IO(a + 1)) - - for (r <- traced(task).unsafeToFuture()) yield { - r.captured shouldBe 3 - r.frames.filter(_.tag == TraceTag.Async).length shouldBe 1 - } - } - test("traces are preserved across asynchronous boundaries") { val task = for { a <- IO.pure(1) From 34e25aa48322ab0447772a9795e009087c7c4b5d Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Fri, 19 Jun 2020 23:01:59 -0500 Subject: [PATCH 53/78] IOContextTests --- .../src/main/scala/cats/effect/IO.scala | 2 +- .../cats/effect/internals/IOContext.scala | 20 +++-- .../cats/effect/internals/IORunLoop.scala | 10 +-- .../cats/effect/internals/IOTracing.scala | 5 +- .../effect/internals/IOContextTests.scala | 88 +++++++++++++++++++ 5 files changed, 112 insertions(+), 13 deletions(-) create mode 100644 core/shared/src/test/scala/cats/effect/internals/IOContextTests.scala diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index a9d496b491..6619fd4724 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -1632,7 +1632,7 @@ object IO extends IOInstances { val backtrace: IO[IOTrace] = IO.Async { (_, ctx, cb) => - cb(Right(ctx.trace)) + cb(Right(ctx.trace())) } /* -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-= */ diff --git a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala index ec1b0340fe..b6e9dc8ca3 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala @@ -26,11 +26,11 @@ import cats.effect.internals.TracingPlatform.maxTraceDepth */ final private[effect] class IOContext private () { - var frames: RingBuffer[TraceFrame] = new RingBuffer(maxTraceDepth) - var captured: Int = 0 - var omitted: Int = 0 + private var frames: RingBuffer[TraceFrame] = new RingBuffer(maxTraceDepth) + private var captured: Int = 0 + private var omitted: Int = 0 - var activeCollects: Int = 0 + private var activeCollects: Int = 0 def pushFrame(fr: TraceFrame): Unit = { captured += 1 @@ -39,12 +39,22 @@ final private[effect] class IOContext private () { def resetTrace(): Unit = { frames = new RingBuffer(maxTraceDepth) + captured = 0 omitted = 0 } - def trace: IOTrace = + def trace(): IOTrace = IOTrace(frames.toList, captured, omitted) + def activeTraces(): Int = + activeCollects + + def enterTrace(): Unit = + activeCollects += 1 + + def exitTrace(): Unit = + activeCollects -= 1 + } object IOContext { diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index 6198bf7789..82d88150f4 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -68,7 +68,7 @@ private[effect] object IORunLoop { // Can change on a context switch var conn: IOConnection = cancelable var ctx: IOContext = ctxRef - var activeCollects: Int = if (ctx ne null) ctx.activeCollects else 0 + var activeCollects: Int = if (ctx ne null) ctx.activeTraces else 0 var bFirst: Bind = bFirstRef var bRest: CallStack = bRestRef var rcb: RestartCallback = rcbRef @@ -171,10 +171,10 @@ private[effect] object IORunLoop { if (ctx eq null) ctx = IOContext() if (collect) { activeCollects += 1 - ctx.activeCollects += 1 + ctx.enterTrace() } else { activeCollects -= 1 - ctx.activeCollects -= 1 + ctx.exitTrace() } unboxed = ().asInstanceOf[AnyRef] hasUnboxed = true @@ -290,10 +290,10 @@ private[effect] object IORunLoop { if (ctx eq null) ctx = IOContext() if (collect) { activeCollects += 1 - ctx.activeCollects += 1 + ctx.enterTrace() } else { activeCollects -= 1 - ctx.activeCollects -= 1 + ctx.exitTrace() } unboxed = ().asInstanceOf[AnyRef] hasUnboxed = true diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index 348552069a..59bf59264a 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -35,8 +35,9 @@ private[effect] object IOTracing { def traced[A](source: IO[A]): IO[A] = resetTrace *> - Bind(incrementCollection, (_: Unit) => - Bind(source, DecrementTraceCollection.asInstanceOf[A => IO[A]], null), null) + Bind(incrementCollection, + (_: Unit) => Bind(source, DecrementTraceCollection.asInstanceOf[A => IO[A]], null), + null) private def buildCachedFrame(traceTag: TraceTag, clazz: Class[_]): TraceFrame = { val cachedFr = frameCache.get(clazz) diff --git a/core/shared/src/test/scala/cats/effect/internals/IOContextTests.scala b/core/shared/src/test/scala/cats/effect/internals/IOContextTests.scala new file mode 100644 index 0000000000..f71a97b4d4 --- /dev/null +++ b/core/shared/src/test/scala/cats/effect/internals/IOContextTests.scala @@ -0,0 +1,88 @@ +/* + * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers + * + * Licensed 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 cats.effect.internals + +import cats.effect.tracing.{TraceFrame, TraceTag} +import org.scalatest.matchers.should.Matchers +import org.scalatest.funsuite.AnyFunSuite + +class IOContextTests extends AnyFunSuite with Matchers { + + val maxTraceDepth: Int = cats.effect.internals.TracingPlatform.maxTraceDepth + val throwable = new Throwable() + + test("push traces") { + val ctx = IOContext() + + val t1 = TraceFrame(TraceTag.Pure, throwable) + val t2 = TraceFrame(TraceTag.Suspend, throwable) + + ctx.pushFrame(t1) + ctx.pushFrame(t2) + + val trace = ctx.trace + trace.frames shouldBe List(t1, t2) + trace.captured shouldBe 2 + trace.omitted shouldBe 0 + } + + test("track omitted frames") { + val ctx = IOContext() + + for (_ <- 0 until (maxTraceDepth + 10)) { + ctx.pushFrame(TraceFrame(TraceTag.Pure, throwable)) + } + + val trace = ctx.trace() + trace.frames.length shouldBe maxTraceDepth + trace.captured shouldBe (maxTraceDepth + 10) + trace.omitted shouldBe 10 + } + + test("reset tracing") { + val ctx = IOContext() + + val t1 = TraceFrame(TraceTag.Pure, throwable) + val t2 = TraceFrame(TraceTag.Suspend, throwable) + + ctx.pushFrame(t1) + ctx.pushFrame(t2) + + ctx.resetTrace() + + val trace = ctx.trace() + trace.frames shouldBe List() + trace.captured shouldBe 0 + trace.omitted shouldBe 0 + } + + test("track tracing regions") { + val ctx = IOContext() + + ctx.activeTraces() shouldBe 0 + + ctx.enterTrace() + ctx.activeTraces() shouldBe 1 + ctx.enterTrace() + ctx.activeTraces() shouldBe 2 + + ctx.exitTrace() + ctx.activeTraces() shouldBe 1 + ctx.exitTrace() + ctx.activeTraces() shouldBe 0 + } +} From 0819d064e46c405e5b968abdd66cc5766e5ec210 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Sat, 20 Jun 2020 02:15:59 -0500 Subject: [PATCH 54/78] Start adding documentation --- .../scala/cats/effect/internals/Example.scala | 30 +---- site/src/main/mdoc/guides/tracing.md | 106 ++++++++++++++++++ .../mdoc/{tutorial => guides}/tutorial.md | 0 .../main/resources/microsite/data/menu.yml | 14 ++- 4 files changed, 119 insertions(+), 31 deletions(-) create mode 100644 site/src/main/mdoc/guides/tracing.md rename site/src/main/mdoc/{tutorial => guides}/tutorial.md (100%) diff --git a/core/jvm/src/main/scala/cats/effect/internals/Example.scala b/core/jvm/src/main/scala/cats/effect/internals/Example.scala index 51fbb412d5..481de0fefe 100644 --- a/core/jvm/src/main/scala/cats/effect/internals/Example.scala +++ b/core/jvm/src/main/scala/cats/effect/internals/Example.scala @@ -20,31 +20,8 @@ import cats.effect.{ExitCode, IO, IOApp} object Example extends IOApp { - /* - The output of this program should be: - 1 - 2 - 3 - 7 - 8 - 4 - 5 - IOTrace: 0 omitted frames - bind at org.simpleapp.example.Example$.run (Example.scala:70) - bind at org.simpleapp.example.Example$.program (Example.scala:57) - bind at org.simpleapp.example.Example$.$anonfun$program$1 (Example.scala:58) - bind at org.simpleapp.example.Example$.$anonfun$program$2 (Example.scala:59) - bind at org.simpleapp.example.Example$.$anonfun$program$3 (Example.scala:60) - async at org.simpleapp.example.Example$.$anonfun$program$3 (Example.scala:63) - bind at org.simpleapp.example.Example$.$anonfun$program$4 (Example.scala:62) - bind at org.simpleapp.example.Example$.program2 (Example.scala:51) - map at org.simpleapp.example.Example$.$anonfun$program2$1 (Example.scala:52) - bind at org.simpleapp.example.Example$.$anonfun$program$7 (Example.scala:64) - map at org.simpleapp.example.Example$.$anonfun$program$8 (Example.scala:65) - */ - def print(msg: String): IO[Unit] = - IO.delay(println(msg)) + IO(println(msg)) def program2: IO[Unit] = for { @@ -57,17 +34,14 @@ object Example extends IOApp { _ <- print("1") _ <- print("2") _ <- IO.shift - _ <- IO.async[Int](cb => cb(Right(32))) - _ <- IO.unit.bracket(_ => program2)(_ => IO.unit) + _ <- program2 _ <- print("5") } yield () override def run(args: List[String]): IO[ExitCode] = for { _ <- IO.suspend(program).traced - _ <- IO.delay("10") trace <- IO.backtrace - _ <- trace.prettyPrint _ <- trace.compactPrint } yield ExitCode.Success diff --git a/site/src/main/mdoc/guides/tracing.md b/site/src/main/mdoc/guides/tracing.md new file mode 100644 index 0000000000..1f39185188 --- /dev/null +++ b/site/src/main/mdoc/guides/tracing.md @@ -0,0 +1,106 @@ +--- +layout: docsplus +title: "Tracing" +position: 2 +--- + + + +## Introduction + +Tracing is an advanced feature of `IO` that offers insight into the execution +graph of a fiber. This unlocks a lot of power for developers in the realm of +debugging and introspection, not only in local development environments +but also in critical production settings. + +A notable pain point of working with asynchronous code on the JVM is that +stack traces no longer provide valuable context of the execution path that +a program takes. This limitation is even more pronounced with Scala's `Future` +(pre- 2.13), where an asynchronous boundary is inserted after each operation. +`IO` suffers a similar problem, but even a synchronous `IO` program's stack +trace is polluted with the details of the `IO` run-loop. + +`IO` solves this problem by collecting a stack trace at various `IO` +operations that a fiber executes, and knitting them together to produce a more +coherent view of the fiber's execution path. For example, here is a trace of a +sample program that is running in rabbit mode: + +``` +IOTrace: 13 frames captured, 0 omitted + ├ flatMap at org.simpleapp.example.Example.run (Example.scala:67) + ├ flatMap at org.simpleapp.example.Example.program (Example.scala:57) + ├ flatMap at org.simpleapp.example.Example.program (Example.scala:58) + ├ flatMap at org.simpleapp.example.Example.program (Example.scala:59) + ├ flatMap at org.simpleapp.example.Example.program (Example.scala:60) + ├ async at org.simpleapp.example.Example.program (Example.scala:60) + ├ flatMap at org.simpleapp.example.Example.program (Example.scala:61) + ├ flatMap at org.simpleapp.example.Example.program (Example.scala:60) + ├ flatMap at org.simpleapp.example.Example.program2 (Example.scala:51) + ├ map at org.simpleapp.example.Example.program2 (Example.scala:52) + ├ map at org.simpleapp.example.Example.program (Example.scala:60) + ├ map at org.simpleapp.example.Example.program (Example.scala:62) + ╰ flatMap at org.simpleapp.example.Example.run (Example.scala:67) +``` + +However, fiber tracing isn't limited to collecting stack traces. Tracing +has many use cases that improve developer experience and aid in understanding +how our applications work (coming soon!): + +1. Asynchronous stack tracing. +2. Combinator inference. +3. Monad transformer analysis. +4. Intermediate values. +5. Thread tracking. +6. Tree rendering. +7. Fiber identity. +8. Fiber ancestry graph. +9. Asynchronous deadlock analysis. + +## Usage +Here is a sample program that demonstrates how tracing is instrumented for an +`IO` program: + +```scala +import cats.effect.{ExitCode, IO, IOApp} + +object Example extends IOApp { + + def print(msg: String): IO[Unit] = + IO(println(msg)) + + def program2: IO[Unit] = + for { + _ <- print("3") + _ <- print("4") + } yield () + + def program: IO[Unit] = + for { + _ <- print("1") + _ <- print("2") + _ <- IO.shift + _ <- program2 + _ <- print("5") + } yield () + + override def run(args: List[String]): IO[ExitCode] = + for { + _ <- IO.suspend(program).traced + trace <- IO.backtrace + _ <- trace.compactPrint + } yield ExitCode.Success + +} +``` + +The tracing mode of an application is controlled by the system property +`cats.effect.tracing.mode`. There are three tracing modes: +* `DISABLED`: No tracing is performed by the program. Negligible performance hit. +This is the default mode. +* `RABBIT`: Stack traces are collected once and cached for `map`, `flatMap` and +the various `async` combinators. <18% performance hit. This is the recommended +mode to run in production. +* `SLUG`: Stack traces are collected at every invocation of every `IO` +combinator. This is the recommended mode to run in development. + +TODO: explain the implications and capabilities of each tracing mode in more detail diff --git a/site/src/main/mdoc/tutorial/tutorial.md b/site/src/main/mdoc/guides/tutorial.md similarity index 100% rename from site/src/main/mdoc/tutorial/tutorial.md rename to site/src/main/mdoc/guides/tutorial.md diff --git a/site/src/main/resources/microsite/data/menu.yml b/site/src/main/resources/microsite/data/menu.yml index 4a6efc8ba8..f4bdd4615d 100644 --- a/site/src/main/resources/microsite/data/menu.yml +++ b/site/src/main/resources/microsite/data/menu.yml @@ -88,6 +88,14 @@ options: url: typeclasses/concurrent-effect.html menu_section: typeclasses - - title: Tutorial - url: tutorial/tutorial.html - menu_section: tutorial + - title: Guides + url: guides/ + menu_section: guides + + nested_options: + - title: Tracing + url: guides/tracing.html + menu_section: guides + - title: Tutorial + url: guides/tutorial.html + menu_section: guides From 2b9124b74b5f4901dc07b44b4da1d95990375f15 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Sat, 20 Jun 2020 03:39:28 -0500 Subject: [PATCH 55/78] Explain use cases of tracing more --- site/src/main/mdoc/guides/tracing.md | 43 +++++++++++++++++++++------- 1 file changed, 33 insertions(+), 10 deletions(-) diff --git a/site/src/main/mdoc/guides/tracing.md b/site/src/main/mdoc/guides/tracing.md index 1f39185188..418eac71c5 100644 --- a/site/src/main/mdoc/guides/tracing.md +++ b/site/src/main/mdoc/guides/tracing.md @@ -40,21 +40,44 @@ IOTrace: 13 frames captured, 0 omitted ├ map at org.simpleapp.example.Example.program (Example.scala:60) ├ map at org.simpleapp.example.Example.program (Example.scala:62) ╰ flatMap at org.simpleapp.example.Example.run (Example.scala:67) -``` +``` However, fiber tracing isn't limited to collecting stack traces. Tracing has many use cases that improve developer experience and aid in understanding how our applications work (coming soon!): -1. Asynchronous stack tracing. -2. Combinator inference. -3. Monad transformer analysis. -4. Intermediate values. -5. Thread tracking. -6. Tree rendering. -7. Fiber identity. -8. Fiber ancestry graph. -9. Asynchronous deadlock analysis. +1. Asynchronous stack tracing. This is essentially what is described above, +where stack frames are collected across asynchronous boundaries for a given +fiber. +2. Combinator inference. Stack traces can be walked to determine what +combinator was actually called by user code. For example, `void` and +`as` are combinators that are derived from `map`. +3. Intermediate values. The intermediate values that an `IO` program encounters +can be converted to a string to render. This can aid in understanding the +actions that a program takes. +4. Thread tracking. A fiber is scheduled on potentially many threads throughout +its lifetime. Knowing what thread a fiber is running on, and when it shifts +threads is a powerful tool in understanding and debugging the concurrency of an +application. +5. Tree rendering. By collecting a trace of all `IO` operations, a pretty tree +or graph can be rendered to visualize fiber execution. +6. Fiber identity. Fibers, like threads, are unique and can therefore assume an +identity. If user code can observe fiber identity, powerful observability tools +can be built on top of it. For example, another shortcoming of asynchronous +code is that it becomes tedious to correlate log messages across asynchronous +boundaries (thread IDs aren't very useful). With fiber identity, log messages +produced by a single fiber can be associated with a unique, stable identifier. +7. Fiber ancestry graph. If fibers can assume an identity, an ancestry graph +can be formed, where nodes are fibers and edges represent a fork/join +relationship. +8. Asynchronous deadlock detection. Even when working with asynchronously +blocking code, fiber deadlocks aren't impossible. Being able to detect +deadlocks or infer when a deadlock can happen makes writing concurrent code +much easier. +9. Live fiber trace dumps. Similar to JVM thread dumps, the execution status +and trace information of all fibers in an application can be extracted for +debugging purposes. +10. Monad transformer analysis. ## Usage Here is a sample program that demonstrates how tracing is instrumented for an From 197a4d06d7f65eb136315b1c4c021345d169741f Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Sun, 21 Jun 2020 00:08:30 -0500 Subject: [PATCH 56/78] Remove map fusion. --- .../src/main/scala/cats/effect/IO.scala | 29 ++++++------------- .../cats/effect/internals/IOForkedStart.scala | 2 +- .../cats/effect/internals/IORunLoop.scala | 4 +-- site/src/main/mdoc/guides/tracing.md | 4 +-- 4 files changed, 14 insertions(+), 25 deletions(-) diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index 6619fd4724..2b1347bf72 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -19,7 +19,6 @@ package effect import cats.effect.internals._ import cats.effect.internals.TracingPlatform.{isRabbitTracing, isSlugTracing} -import cats.effect.internals.IOPlatform.fusionMaxStackDepth import scala.annotation.unchecked.uncheckedVariance import scala.concurrent.duration._ @@ -102,26 +101,18 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * failures would be completely silent and `IO` references would * never terminate on evaluation. */ - final def map[B](f: A => B): IO[B] = - // Don't perform map fusion when tracing is enabled. - // We may end up removing map fusion altogether. - if (isRabbitTracing) { - Map(this, f, 0, IOTracing.cached(TraceTag.Map, f.getClass)) + final def map[B](f: A => B): IO[B] = { + val trace = if (isRabbitTracing) { + IOTracing.cached(TraceTag.Map, f.getClass) } else if (isSlugTracing) { - Map(this, f, 0, IOTracing.uncached(TraceTag.Map)) + IOTracing.uncached(TraceTag.Map) } else { - this match { - case Map(source, g, index, null) => - // Allowed to do fixed number of map operations fused before - // resetting the counter in order to avoid stack overflows; - // See `IOPlatform` for details on this maximum. - if (index != fusionMaxStackDepth) Map(source, g.andThen(f), index + 1, null) - else Map(this, f, 0, null) - case _ => - Map(this, f, 0, null) - } + null } + Map(this, f, trace) + } + /** * Monadic bind on `IO`, used for sequentially composing two `IO` * actions, where the value produced by the first `IO` is passed as @@ -1654,9 +1645,7 @@ object IO extends IOInstances { final private[effect] case class Bind[E, +A](source: IO[E], f: E => IO[A], trace: AnyRef) extends IO[A] /** Corresponds to [[IO.map]]. */ - final private[effect] case class Map[E, +A](source: IO[E], f: E => A, index: Int, trace: AnyRef) - extends IO[A] - with (E => IO[A]) { + final private[effect] case class Map[E, +A](source: IO[E], f: E => A, trace: AnyRef) extends IO[A] with (E => IO[A]) { override def apply(value: E): IO[A] = Pure(f(value)) } diff --git a/core/shared/src/main/scala/cats/effect/internals/IOForkedStart.scala b/core/shared/src/main/scala/cats/effect/internals/IOForkedStart.scala index ba71839f45..a868da3f3f 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOForkedStart.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOForkedStart.scala @@ -55,7 +55,7 @@ private[effect] object IOForkedStart { task match { case IO.Async(k, _, _) => k.isInstanceOf[IOForkedStart[_]] case IO.Bind(other, _, _) => detect(other, limit - 1) - case IO.Map(other, _, _, _) => detect(other, limit - 1) + case IO.Map(other, _, _) => detect(other, limit - 1) case IO.ContextSwitch(other, _, _) => detect(other, limit - 1) case _ => false } diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index 82d88150f4..0904315252 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -126,7 +126,7 @@ private[effect] object IORunLoop { currentIO = fa } - case bindNext @ Map(fa, _, _, _) => + case bindNext @ Map(fa, _, _) => if (bFirst ne null) { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) @@ -268,7 +268,7 @@ private[effect] object IORunLoop { currentIO = fa } - case bindNext @ Map(fa, _, _, _) => + case bindNext @ Map(fa, _, _) => if (bFirst ne null) { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) diff --git a/site/src/main/mdoc/guides/tracing.md b/site/src/main/mdoc/guides/tracing.md index 418eac71c5..54b4995a12 100644 --- a/site/src/main/mdoc/guides/tracing.md +++ b/site/src/main/mdoc/guides/tracing.md @@ -57,8 +57,8 @@ can be converted to a string to render. This can aid in understanding the actions that a program takes. 4. Thread tracking. A fiber is scheduled on potentially many threads throughout its lifetime. Knowing what thread a fiber is running on, and when it shifts -threads is a powerful tool in understanding and debugging the concurrency of an -application. +threads is a powerful tool for understanding and debugging the concurrency of +an application. 5. Tree rendering. By collecting a trace of all `IO` operations, a pretty tree or graph can be rendered to visualize fiber execution. 6. Fiber identity. Fibers, like threads, are unique and can therefore assume an From 58cc9f5d16c00583db354ecc41cd59048f51274b Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Sun, 21 Jun 2020 00:33:47 -0500 Subject: [PATCH 57/78] Format build.sbt --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index df03c156b5..a51d4f04c3 100644 --- a/build.sbt +++ b/build.sbt @@ -291,7 +291,7 @@ lazy val laws = crossProject(JSPlatform, JVMPlatform) lazy val lawsJVM = laws.jvm lazy val lawsJS = laws.js -lazy val SlugTest = config("slug") extend Test +lazy val SlugTest = config("slug").extend(Test) lazy val tracingTests = crossProject(JSPlatform, JVMPlatform) .in(file("tracing-tests")) From e5ce64dd5f04ab62cdb5c2896aaec24bf1f8f7c2 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Mon, 22 Jun 2020 23:58:51 -0500 Subject: [PATCH 58/78] More documentation and renaming --- build.sbt | 26 +++--- .../effect/internals/TracingPlatform.scala | 6 +- .../effect/internals/TracingPlatform.java | 29 ++++--- .../src/main/scala/cats/effect/IO.scala | 34 ++++---- .../cats/effect/internals/IOContext.scala | 6 +- .../cats/effect/internals/IORunLoop.scala | 29 +++---- .../cats/effect/internals/IOTracing.scala | 32 ++++---- .../scala/cats/effect/tracing/IOTrace.scala | 4 +- ...TraceFrame.scala => StackTraceFrame.scala} | 2 +- .../effect/internals/IOContextTests.scala | 12 +-- site/src/main/mdoc/guides/tracing.md | 79 +++++++++++++++---- .../cats/effect/FullStackTracingTests.scala} | 14 ++-- ...ts.scala => CachedStackTracingTests.scala} | 8 +- 13 files changed, 171 insertions(+), 110 deletions(-) rename core/shared/src/main/scala/cats/effect/tracing/{TraceFrame.scala => StackTraceFrame.scala} (91%) rename tracing-tests/shared/src/{slug/scala/cats/effect/SlugTracingTests.scala => fulltracing/scala/cats/effect/FullStackTracingTests.scala} (86%) rename tracing-tests/shared/src/test/scala/cats/effect/{RabbitTracingTests.scala => CachedStackTracingTests.scala} (89%) diff --git a/build.sbt b/build.sbt index a51d4f04c3..e68fbf044b 100644 --- a/build.sbt +++ b/build.sbt @@ -291,7 +291,7 @@ lazy val laws = crossProject(JSPlatform, JVMPlatform) lazy val lawsJVM = laws.jvm lazy val lawsJS = laws.js -lazy val SlugTest = config("slug").extend(Test) +lazy val FullTracingTest = config("fulltracing").extend(Test) lazy val tracingTests = crossProject(JSPlatform, JVMPlatform) .in(file("tracing-tests")) @@ -304,13 +304,13 @@ lazy val tracingTests = crossProject(JSPlatform, JVMPlatform) "org.typelevel" %%% "discipline-scalatest" % DisciplineScalatestVersion % Test ) ) - .configs(SlugTest) - .settings(inConfig(SlugTest)(Defaults.testSettings): _*) - .jsSettings(inConfig(SlugTest)(ScalaJSPlugin.testConfigSettings): _*) + .configs(FullTracingTest) + .settings(inConfig(FullTracingTest)(Defaults.testSettings): _*) + .jsSettings(inConfig(FullTracingTest)(ScalaJSPlugin.testConfigSettings): _*) .settings( - test in Test := (test in Test).dependsOn(test in SlugTest).value, - unmanagedSourceDirectories in SlugTest += { - baseDirectory.value.getParentFile / "shared" / "src" / "slug" / "scala" + test in Test := (test in Test).dependsOn(test in FullTracingTest).value, + unmanagedSourceDirectories in FullTracingTest += { + baseDirectory.value.getParentFile / "shared" / "src" / "fulltracing" / "scala" } ) .jvmConfigure(_.enablePlugins(AutomateHeaderPlugin)) @@ -320,9 +320,15 @@ lazy val tracingTests = crossProject(JSPlatform, JVMPlatform) .jvmSettings( skip.in(publish) := customScalaJSVersion.forall(_.startsWith("1.0")), fork in Test := true, - fork in SlugTest := true, - javaOptions in Test += "-Dcats.effect.tracing.mode=rabbit", - javaOptions in SlugTest += "-Dcats.effect.tracing.mode=slug" + fork in FullTracingTest := true, + javaOptions in Test ++= Seq( + "-Dcats.effect.tracing=true", + "-Dcats.effect.stackTracingMode=cached" + ), + javaOptions in FullTracingTest ++= Seq( + "-Dcats.effect.tracing=true", + "-Dcats.effect.stackTracingMode=full" + ) ) lazy val tracingTestsJVM = tracingTests.jvm diff --git a/core/js/src/main/scala/cats/effect/internals/TracingPlatform.scala b/core/js/src/main/scala/cats/effect/internals/TracingPlatform.scala index bb5949b559..3cd4fb1c8b 100644 --- a/core/js/src/main/scala/cats/effect/internals/TracingPlatform.scala +++ b/core/js/src/main/scala/cats/effect/internals/TracingPlatform.scala @@ -17,11 +17,11 @@ package cats.effect.internals object TracingPlatform { - final val isRabbitTracing: Boolean = false + final val isCachedStackTracing: Boolean = false - final val isSlugTracing: Boolean = false + final val isFullStackTracing: Boolean = false - final val isTracing: Boolean = isSlugTracing || isRabbitTracing + final val isStackTracing: Boolean = isFullStackTracing || isCachedStackTracing final val maxTraceDepth: Int = 32 } diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java index 7210302558..2d0ea8659b 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java @@ -30,28 +30,35 @@ public final class TracingPlatform { /** - * A string flag that sets a global tracing mode for a JVM process. - * Acceptable values are: DISABLED, RABBIT, SLUG. - * - * This field isn't accessed by other classes; instead use one of the - * more specific accessors defined below. + * Sets the global tracing mode for a JVM process. + * If this flag isn't enabled, no tracing is instrumented. */ - private static final String tracingMode = Optional.ofNullable(System.getProperty("cats.effect.tracing.mode")) + public static final boolean isTracing = Optional.ofNullable(System.getProperty("cats.effect.tracing")) .filter(x -> !x.isEmpty()) - .orElse("disabled"); + .map(Boolean::valueOf) + .orElse(false); - public static final boolean isRabbitTracing = tracingMode.equalsIgnoreCase("rabbit"); + /** + * Sets stack tracing mode for a JVM process, which controls + * how much stack trace information is captured. + * Acceptable values are: NONE, CACHED, FULL. + */ + private static final String stackTracingMode = Optional.ofNullable(System.getProperty("cats.effect.stackTracingMode")) + .filter(x -> !x.isEmpty()) + .orElse("none"); + + public static final boolean isCachedStackTracing = isTracing && stackTracingMode.equalsIgnoreCase("cached"); - public static final boolean isSlugTracing = tracingMode.equalsIgnoreCase("slug"); + public static final boolean isFullStackTracing = isTracing && stackTracingMode.equalsIgnoreCase("full"); - public static final boolean isTracing = isSlugTracing || isRabbitTracing; + public static final boolean isStackTracing = isFullStackTracing || isCachedStackTracing; /** * The number of trace lines to retain during tracing. If more trace * lines are produced, then the oldest trace lines will be discarded. * Automatically rounded up to the nearest power of 2. */ - public static final int maxTraceDepth = Optional.ofNullable(System.getProperty("cats.effect.tracing.maxTraceDepth")) + public static final int maxTraceDepth = Optional.ofNullable(System.getProperty("cats.effect.maxTraceDepth")) .filter(x -> !x.isEmpty()) .flatMap(x -> { try { diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index da0e675f9c..3c4b3d5848 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -18,7 +18,7 @@ package cats package effect import cats.effect.internals._ -import cats.effect.internals.TracingPlatform.{isRabbitTracing, isSlugTracing} +import cats.effect.internals.TracingPlatform.{isCachedStackTracing, isFullStackTracing} import scala.annotation.unchecked.uncheckedVariance import scala.concurrent.duration._ @@ -26,7 +26,7 @@ import scala.concurrent.{ExecutionContext, Future, Promise, TimeoutException} import scala.util.control.NonFatal import scala.util.{Failure, Left, Right, Success, Try} import cats.data.Ior -import cats.effect.tracing.{IOTrace, TraceFrame, TraceTag} +import cats.effect.tracing.{IOTrace, StackTraceFrame, TraceTag} /** * A pure abstraction representing the intention to perform a @@ -102,9 +102,9 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * never terminate on evaluation. */ final def map[B](f: A => B): IO[B] = { - val trace = if (isRabbitTracing) { + val trace = if (isCachedStackTracing) { IOTracing.cached(TraceTag.Map, f.getClass) - } else if (isSlugTracing) { + } else if (isFullStackTracing) { IOTracing.uncached(TraceTag.Map) } else { null @@ -129,9 +129,9 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * never terminate on evaluation. */ final def flatMap[B](f: A => IO[B]): IO[B] = { - val trace = if (isRabbitTracing) { + val trace = if (isCachedStackTracing) { IOTracing.cached(TraceTag.Bind, f.getClass) - } else if (isSlugTracing) { + } else if (isFullStackTracing) { IOTracing.uncached(TraceTag.Bind) } else { null @@ -1133,7 +1133,7 @@ object IO extends IOInstances { */ def delay[A](body: => A): IO[A] = { val nextIo = Delay(() => body) - if (isSlugTracing) { + if (isFullStackTracing) { IOTracing.decorated(nextIo, TraceTag.Delay) } else { nextIo @@ -1150,7 +1150,7 @@ object IO extends IOInstances { */ def suspend[A](thunk: => IO[A]): IO[A] = { val nextIo = Suspend(() => thunk) - if (isSlugTracing) { + if (isFullStackTracing) { IOTracing.decorated(nextIo, TraceTag.Suspend) } else { nextIo @@ -1169,7 +1169,7 @@ object IO extends IOInstances { */ def pure[A](a: A): IO[A] = { val nextIo = Pure(a) - if (isSlugTracing) { + if (isFullStackTracing) { IOTracing.decorated(nextIo, TraceTag.Pure) } else { nextIo @@ -1240,9 +1240,9 @@ object IO extends IOInstances { * @see [[asyncF]] and [[cancelable]] */ def async[A](k: (Either[Throwable, A] => Unit) => Unit): IO[A] = { - val trace = if (isRabbitTracing) { + val trace = if (isCachedStackTracing) { IOTracing.cached(TraceTag.Async, k.getClass) - } else if (isSlugTracing) { + } else if (isFullStackTracing) { IOTracing.uncached(TraceTag.Async) } else { null @@ -1280,9 +1280,9 @@ object IO extends IOInstances { * @see [[async]] and [[cancelable]] */ def asyncF[A](k: (Either[Throwable, A] => Unit) => IO[Unit]): IO[A] = { - val trace = if (isRabbitTracing) { + val trace = if (isCachedStackTracing) { IOTracing.cached(TraceTag.AsyncF, k.getClass) - } else if (isSlugTracing) { + } else if (isFullStackTracing) { IOTracing.uncached(TraceTag.AsyncF) } else { null @@ -1345,9 +1345,9 @@ object IO extends IOInstances { * the underlying cancelation model */ def cancelable[A](k: (Either[Throwable, A] => Unit) => CancelToken[IO]): IO[A] = { - val trace = if (isRabbitTracing) { + val trace = if (isCachedStackTracing) { IOTracing.cached(TraceTag.Cancelable, k.getClass) - } else if (isSlugTracing) { + } else if (isFullStackTracing) { IOTracing.uncached(TraceTag.Cancelable) } else { null @@ -1675,9 +1675,9 @@ object IO extends IOInstances { trace: AnyRef = null ) extends IO[A] - final private[effect] case class Trace[A](source: IO[A], frame: TraceFrame) extends IO[A] + final private[effect] case class Trace[A](source: IO[A], trace: StackTraceFrame) extends IO[A] - final private[effect] case class CollectTraces(collect: Boolean) extends IO[Unit] + final private[effect] case class SetTracing(collectStackTraces: Boolean) extends IO[Unit] /** * An internal state for that optimizes changes to diff --git a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala index b6e9dc8ca3..3c576fd690 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala @@ -16,7 +16,7 @@ package cats.effect.internals -import cats.effect.tracing.{IOTrace, TraceFrame} +import cats.effect.tracing.{IOTrace, StackTraceFrame} import cats.effect.internals.TracingPlatform.maxTraceDepth /** @@ -26,13 +26,13 @@ import cats.effect.internals.TracingPlatform.maxTraceDepth */ final private[effect] class IOContext private () { - private var frames: RingBuffer[TraceFrame] = new RingBuffer(maxTraceDepth) + private var frames: RingBuffer[StackTraceFrame] = new RingBuffer(maxTraceDepth) private var captured: Int = 0 private var omitted: Int = 0 private var activeCollects: Int = 0 - def pushFrame(fr: TraceFrame): Unit = { + def pushFrame(fr: StackTraceFrame): Unit = { captured += 1 if (frames.push(fr) != null) omitted += 1 } diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index bb356cb606..36e47561bb 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -17,8 +17,9 @@ package cats.effect.internals import cats.effect.IO -import cats.effect.IO.{Async, Bind, CollectTraces, ContextSwitch, Delay, Map, Pure, RaiseError, Suspend, Trace} -import cats.effect.tracing.TraceFrame +import cats.effect.IO.{Async, Bind, ContextSwitch, Delay, Map, Pure, RaiseError, SetTracing, Suspend, Trace} +import cats.effect.tracing.StackTraceFrame +import cats.effect.internals.TracingPlatform.isStackTracing import scala.util.control.NonFatal @@ -86,10 +87,10 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if (TracingPlatform.isTracing && activeCollects > 0) { + if (isStackTracing && activeCollects > 0) { if (ctx eq null) ctx = IOContext() val trace = bind.trace - if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) + if (trace ne null) ctx.pushFrame(trace.asInstanceOf[StackTraceFrame]) } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa @@ -131,10 +132,10 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if (TracingPlatform.isTracing && activeCollects > 0) { + if (isStackTracing && activeCollects > 0) { if (ctx eq null) ctx = IOContext() val trace = bindNext.trace - if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) + if (trace ne null) ctx.pushFrame(trace.asInstanceOf[StackTraceFrame]) } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa @@ -145,9 +146,9 @@ private[effect] object IORunLoop { // may produce trace frames e.g. IOBracket. if (ctx eq null) ctx = IOContext() if (rcb eq null) rcb = new RestartCallback(conn, cb.asInstanceOf[Callback]) - if (TracingPlatform.isTracing && activeCollects > 0) { + if (isStackTracing && activeCollects > 0) { val trace = async.trace - if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) + if (trace ne null) ctx.pushFrame(trace.asInstanceOf[StackTraceFrame]) } rcb.start(async, ctx, bFirst, bRest) return @@ -167,7 +168,7 @@ private[effect] object IORunLoop { ctx.pushFrame(frame) currentIO = source - case CollectTraces(collect) => + case SetTracing(collect) => if (ctx eq null) ctx = IOContext() if (collect) { activeCollects += 1 @@ -228,10 +229,10 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if (TracingPlatform.isTracing && activeCollects > 0) { + if (isStackTracing && activeCollects > 0) { if (ctx eq null) ctx = IOContext() val trace = bind.trace - if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) + if (trace ne null) ctx.pushFrame(trace.asInstanceOf[StackTraceFrame]) } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa @@ -273,10 +274,10 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if (TracingPlatform.isTracing && activeCollects > 0) { + if (isStackTracing && activeCollects > 0) { if (ctx eq null) ctx = IOContext() val trace = bindNext.trace - if (trace ne null) ctx.pushFrame(trace.asInstanceOf[TraceFrame]) + if (trace ne null) ctx.pushFrame(trace.asInstanceOf[StackTraceFrame]) } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa @@ -286,7 +287,7 @@ private[effect] object IORunLoop { ctx.pushFrame(frame) currentIO = source - case CollectTraces(collect) => + case SetTracing(collect) => if (ctx eq null) ctx = IOContext() if (collect) { activeCollects += 1 diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index 59bf59264a..603b5868c5 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -19,18 +19,18 @@ package cats.effect.internals import java.util.concurrent.ConcurrentHashMap import cats.effect.IO -import cats.effect.IO.{Bind, CollectTraces, Pure, RaiseError, Trace} -import cats.effect.tracing.{TraceFrame, TraceTag} +import cats.effect.IO.{Bind, Pure, RaiseError, SetTracing, Trace} +import cats.effect.tracing.{StackTraceFrame, TraceTag} private[effect] object IOTracing { def decorated[A](source: IO[A], traceTag: TraceTag): IO[A] = Trace(source, buildFrame(traceTag)) - def uncached(traceTag: TraceTag): TraceFrame = + def uncached(traceTag: TraceTag): StackTraceFrame = buildFrame(traceTag) - def cached(traceTag: TraceTag, clazz: Class[_]): TraceFrame = + def cached(traceTag: TraceTag, clazz: Class[_]): StackTraceFrame = buildCachedFrame(traceTag, clazz) def traced[A](source: IO[A]): IO[A] = @@ -39,23 +39,23 @@ private[effect] object IOTracing { (_: Unit) => Bind(source, DecrementTraceCollection.asInstanceOf[A => IO[A]], null), null) - private def buildCachedFrame(traceTag: TraceTag, clazz: Class[_]): TraceFrame = { - val cachedFr = frameCache.get(clazz) - if (cachedFr eq null) { - val fr = buildFrame(traceTag) - frameCache.put(clazz, fr) - fr + private def buildCachedFrame(traceTag: TraceTag, clazz: Class[_]): StackTraceFrame = { + val cf = frameCache.get(clazz) + if (cf eq null) { + val f = buildFrame(traceTag) + frameCache.put(clazz, f) + f } else { - cachedFr + cf } } - def buildFrame(traceTag: TraceTag): TraceFrame = - TraceFrame(traceTag, new Throwable()) + def buildFrame(traceTag: TraceTag): StackTraceFrame = + StackTraceFrame(traceTag, new Throwable()) - private[this] val incrementCollection: IO[Unit] = CollectTraces(true) + private[this] val incrementCollection: IO[Unit] = SetTracing(true) - private[this] val decrementCollection: IO[Unit] = CollectTraces(false) + private[this] val decrementCollection: IO[Unit] = SetTracing(false) private object DecrementTraceCollection extends IOFrame[Any, IO[Any]] { override def apply(a: Any) = @@ -73,6 +73,6 @@ private[effect] object IOTracing { /** * Cache for trace frames. Keys are references to lambda classes. */ - private[this] val frameCache: ConcurrentHashMap[Class[_], TraceFrame] = new ConcurrentHashMap() + private[this] val frameCache: ConcurrentHashMap[Class[_], StackTraceFrame] = new ConcurrentHashMap() } diff --git a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala index fc3c8ba2ab..2737964716 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala @@ -18,7 +18,7 @@ package cats.effect.tracing import cats.effect.IO -final case class IOTrace(frames: List[TraceFrame], captured: Int, omitted: Int) { +final case class IOTrace(frames: List[StackTraceFrame], captured: Int, omitted: Int) { import IOTrace._ @@ -55,7 +55,7 @@ final case class IOTrace(frames: List[TraceFrame], captured: Int, omitted: Int) def prettyPrint: IO[Unit] = IO(System.err.println(pretty)) - private def loop(acc: String, indent: Int, init: Boolean, rest: List[TraceFrame]): String = { + private def loop(acc: String, indent: Int, init: Boolean, rest: List[StackTraceFrame]): String = { val TurnRight = "╰" val InverseTurnRight = "╭" val Junction = "├" diff --git a/core/shared/src/main/scala/cats/effect/tracing/TraceFrame.scala b/core/shared/src/main/scala/cats/effect/tracing/StackTraceFrame.scala similarity index 91% rename from core/shared/src/main/scala/cats/effect/tracing/TraceFrame.scala rename to core/shared/src/main/scala/cats/effect/tracing/StackTraceFrame.scala index 7393c83487..1a8c5e8791 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/TraceFrame.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/StackTraceFrame.scala @@ -16,7 +16,7 @@ package cats.effect.tracing -final case class TraceFrame(tag: TraceTag, throwable: Throwable) { +final case class StackTraceFrame(tag: TraceTag, throwable: Throwable) { def stackTrace: List[StackTraceElement] = throwable.getStackTrace().toList diff --git a/core/shared/src/test/scala/cats/effect/internals/IOContextTests.scala b/core/shared/src/test/scala/cats/effect/internals/IOContextTests.scala index f71a97b4d4..37d166da2d 100644 --- a/core/shared/src/test/scala/cats/effect/internals/IOContextTests.scala +++ b/core/shared/src/test/scala/cats/effect/internals/IOContextTests.scala @@ -16,7 +16,7 @@ package cats.effect.internals -import cats.effect.tracing.{TraceFrame, TraceTag} +import cats.effect.tracing.{StackTraceFrame, TraceTag} import org.scalatest.matchers.should.Matchers import org.scalatest.funsuite.AnyFunSuite @@ -28,8 +28,8 @@ class IOContextTests extends AnyFunSuite with Matchers { test("push traces") { val ctx = IOContext() - val t1 = TraceFrame(TraceTag.Pure, throwable) - val t2 = TraceFrame(TraceTag.Suspend, throwable) + val t1 = StackTraceFrame(TraceTag.Pure, throwable) + val t2 = StackTraceFrame(TraceTag.Suspend, throwable) ctx.pushFrame(t1) ctx.pushFrame(t2) @@ -44,7 +44,7 @@ class IOContextTests extends AnyFunSuite with Matchers { val ctx = IOContext() for (_ <- 0 until (maxTraceDepth + 10)) { - ctx.pushFrame(TraceFrame(TraceTag.Pure, throwable)) + ctx.pushFrame(StackTraceFrame(TraceTag.Pure, throwable)) } val trace = ctx.trace() @@ -56,8 +56,8 @@ class IOContextTests extends AnyFunSuite with Matchers { test("reset tracing") { val ctx = IOContext() - val t1 = TraceFrame(TraceTag.Pure, throwable) - val t2 = TraceFrame(TraceTag.Suspend, throwable) + val t1 = StackTraceFrame(TraceTag.Pure, throwable) + val t2 = StackTraceFrame(TraceTag.Suspend, throwable) ctx.pushFrame(t1) ctx.pushFrame(t2) diff --git a/site/src/main/mdoc/guides/tracing.md b/site/src/main/mdoc/guides/tracing.md index 54b4995a12..aec8aae154 100644 --- a/site/src/main/mdoc/guides/tracing.md +++ b/site/src/main/mdoc/guides/tracing.md @@ -7,7 +7,6 @@ position: 2 ## Introduction - Tracing is an advanced feature of `IO` that offers insight into the execution graph of a fiber. This unlocks a lot of power for developers in the realm of debugging and introspection, not only in local development environments @@ -23,7 +22,7 @@ trace is polluted with the details of the `IO` run-loop. `IO` solves this problem by collecting a stack trace at various `IO` operations that a fiber executes, and knitting them together to produce a more coherent view of the fiber's execution path. For example, here is a trace of a -sample program that is running in rabbit mode: +sample program that is running in cached stack tracing mode: ``` IOTrace: 13 frames captured, 0 omitted @@ -80,10 +79,70 @@ debugging purposes. 10. Monad transformer analysis. ## Usage -Here is a sample program that demonstrates how tracing is instrumented for an -`IO` program: + +### Enable tracing +A global system property must be set on JVM in order to enable any tracing +facility: + +``` +-Dcats.effect.tracing=true +``` + +### Collecting traces +To mark a region of a program for tracing, use the `traced` combinator. After +that subprogram completes, use the `backtrace` combinator to collect the full +fiber trace. + +```scala +for { + _ <- program.traced + trace <- IO.backtrace + _ <- trace.compactPrint +} yield () +``` + +### Asynchronous stack tracing +The stack tracing mode of an application is configured by the system property +`cats.effect.tracing.mode`. There are three stack tracing modes: `DISABLED`, +`CACHED` and `FULL`. + +#### DISABLED +No tracing is instrumented by the program. This is the default stack tracing +mode. + +#### CACHED +When cached stack tracing is enabled, a stack trace is captured and cached for +every `map`, `flatMap` and `async` call in a program. + +The stack trace cache is keyed by the lambda class reference, so cached tracing +may produce inaccurate fiber traces under several scenarios: +1. Monad transformer composition +2. A named function is supplied to `map`, `async` or `flatMap` at multiple +call-sites + +When no collection is performed, we measured less than an 18% performance hit +for a completely synchronous `IO` program, so it will most likely be negligible +for any program that performs any sort of I/O. + +This is the recommended mode to run in production environments. + +#### FULL +When full stack tracing is enabled, a stack trace is captured for every +combinator traced in cached mode, but also `pure`, `delay`, `suspend` and other +derived combinators. + +This mode will incur a heavy performance hit for most programs, and is +recommended for use only in development environments. + +### Example +Here is a sample program that demonstrates how to turn on tracing in an +application. ```scala +// Specify the following flags in your JVM: +// -Dcats.effect.tracing=true +// -Dcats.effect.stackTracingMode=full + import cats.effect.{ExitCode, IO, IOApp} object Example extends IOApp { @@ -115,15 +174,3 @@ object Example extends IOApp { } ``` - -The tracing mode of an application is controlled by the system property -`cats.effect.tracing.mode`. There are three tracing modes: -* `DISABLED`: No tracing is performed by the program. Negligible performance hit. -This is the default mode. -* `RABBIT`: Stack traces are collected once and cached for `map`, `flatMap` and -the various `async` combinators. <18% performance hit. This is the recommended -mode to run in production. -* `SLUG`: Stack traces are collected at every invocation of every `IO` -combinator. This is the recommended mode to run in development. - -TODO: explain the implications and capabilities of each tracing mode in more detail diff --git a/tracing-tests/shared/src/slug/scala/cats/effect/SlugTracingTests.scala b/tracing-tests/shared/src/fulltracing/scala/cats/effect/FullStackTracingTests.scala similarity index 86% rename from tracing-tests/shared/src/slug/scala/cats/effect/SlugTracingTests.scala rename to tracing-tests/shared/src/fulltracing/scala/cats/effect/FullStackTracingTests.scala index a1d82c7259..5ceb3a2433 100644 --- a/tracing-tests/shared/src/slug/scala/cats/effect/SlugTracingTests.scala +++ b/tracing-tests/shared/src/fulltracing/scala/cats/effect/FullStackTracingTests.scala @@ -22,7 +22,7 @@ import org.scalatest.matchers.should.Matchers import scala.concurrent.ExecutionContext -class SlugTracingTests extends AsyncFunSuite with Matchers { +class FullStackTracingTests extends AsyncFunSuite with Matchers { implicit override def executionContext: ExecutionContext = ExecutionContext.Implicits.global implicit val timer: Timer[IO] = IO.timer(executionContext) implicit val cs: ContextShift[IO] = IO.contextShift(executionContext) @@ -33,7 +33,7 @@ class SlugTracingTests extends AsyncFunSuite with Matchers { t <- IO.backtrace } yield t - test("slug tracing captures map frames") { + test("full stack tracing captures map frames") { val task = IO.pure(0).map(_ + 1).map(_ + 1) for (r <- traced(task).unsafeToFuture()) yield { @@ -42,7 +42,7 @@ class SlugTracingTests extends AsyncFunSuite with Matchers { } } - test("slug tracing captures bind frames") { + test("full stack tracing captures bind frames") { val task = IO.pure(0).flatMap(a => IO(a + 1)).flatMap(a => IO(a + 1)) for (r <- traced(task).unsafeToFuture()) yield { @@ -51,7 +51,7 @@ class SlugTracingTests extends AsyncFunSuite with Matchers { } } - test("slug tracing captures async frames") { + test("full stack tracing captures async frames") { val task = IO.async[Int](_(Right(0))).flatMap(a => IO(a + 1)).flatMap(a => IO(a + 1)) for (r <- traced(task).unsafeToFuture()) yield { @@ -60,7 +60,7 @@ class SlugTracingTests extends AsyncFunSuite with Matchers { } } - test("slug tracing captures pure frames") { + test("full stack tracing captures pure frames") { val task = IO.pure(0).flatMap(a => IO.pure(a + 1)) for (r <- traced(task).unsafeToFuture()) yield { @@ -69,7 +69,7 @@ class SlugTracingTests extends AsyncFunSuite with Matchers { } } - test("slug tracing captures delay frames") { + test("full stack tracing captures delay frames") { val task = IO(0).flatMap(a => IO(a + 1)) for (r <- traced(task).unsafeToFuture()) yield { @@ -78,7 +78,7 @@ class SlugTracingTests extends AsyncFunSuite with Matchers { } } - test("slug tracing captures suspend frames") { + test("full stack tracing captures suspend frames") { val task = IO.suspend(IO(1)).flatMap(a => IO.suspend(IO(a + 1))) for (r <- traced(task).unsafeToFuture()) yield { diff --git a/tracing-tests/shared/src/test/scala/cats/effect/RabbitTracingTests.scala b/tracing-tests/shared/src/test/scala/cats/effect/CachedStackTracingTests.scala similarity index 89% rename from tracing-tests/shared/src/test/scala/cats/effect/RabbitTracingTests.scala rename to tracing-tests/shared/src/test/scala/cats/effect/CachedStackTracingTests.scala index a113a81678..a563e58f48 100644 --- a/tracing-tests/shared/src/test/scala/cats/effect/RabbitTracingTests.scala +++ b/tracing-tests/shared/src/test/scala/cats/effect/CachedStackTracingTests.scala @@ -22,7 +22,7 @@ import org.scalatest.matchers.should.Matchers import scala.concurrent.ExecutionContext -class RabbitTracingTests extends AsyncFunSuite with Matchers { +class CachedStackTracingTests extends AsyncFunSuite with Matchers { implicit override def executionContext: ExecutionContext = ExecutionContext.Implicits.global implicit val timer: Timer[IO] = IO.timer(executionContext) implicit val cs: ContextShift[IO] = IO.contextShift(executionContext) @@ -33,7 +33,7 @@ class RabbitTracingTests extends AsyncFunSuite with Matchers { t <- IO.backtrace } yield t - test("rabbit tracing captures map frames") { + test("cached stack tracing captures map frames") { val task = IO.pure(0).map(_ + 1).map(_ + 1) for (r <- traced(task).unsafeToFuture()) yield { @@ -42,7 +42,7 @@ class RabbitTracingTests extends AsyncFunSuite with Matchers { } } - test("rabbit tracing captures bind frames") { + test("cached stack tracing captures bind frames") { val task = IO.pure(0).flatMap(a => IO(a + 1)).flatMap(a => IO(a + 1)) for (r <- traced(task).unsafeToFuture()) yield { @@ -51,7 +51,7 @@ class RabbitTracingTests extends AsyncFunSuite with Matchers { } } - test("rabbit tracing captures async frames") { + test("cached stack tracing captures async frames") { val task = IO.async[Int](_(Right(0))).flatMap(a => IO(a + 1)).flatMap(a => IO(a + 1)) for (r <- traced(task).unsafeToFuture()) yield { From 4f6a37fe5b0a5a732b5b18607a4ecba4988d0377 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Tue, 23 Jun 2020 00:02:07 -0500 Subject: [PATCH 59/78] Remove map fusion configuration --- .../cats/effect/internals/IOPlatform.scala | 9 ------ .../cats/effect/internals/IOPlatform.scala | 31 ------------------- .../src/test/scala/cats/effect/IOTests.scala | 8 ----- .../test/scala/cats/effect/SyncIOTests.scala | 8 ----- 4 files changed, 56 deletions(-) diff --git a/core/js/src/main/scala/cats/effect/internals/IOPlatform.scala b/core/js/src/main/scala/cats/effect/internals/IOPlatform.scala index d79747909f..0436804fdb 100644 --- a/core/js/src/main/scala/cats/effect/internals/IOPlatform.scala +++ b/core/js/src/main/scala/cats/effect/internals/IOPlatform.scala @@ -45,15 +45,6 @@ private[effect] object IOPlatform { } } - /** - * Establishes the maximum stack depth for `IO#map` operations - * for JavaScript. - * - * The default for JavaScript is 32, from which we subtract 1 - * as an optimization. - */ - final val fusionMaxStackDepth = 31 - /** Returns `true` if the underlying platform is the JVM, * `false` if it's JavaScript. */ final val isJVM = false diff --git a/core/jvm/src/main/scala/cats/effect/internals/IOPlatform.scala b/core/jvm/src/main/scala/cats/effect/internals/IOPlatform.scala index da98b4e0e3..a94c785c34 100644 --- a/core/jvm/src/main/scala/cats/effect/internals/IOPlatform.scala +++ b/core/jvm/src/main/scala/cats/effect/internals/IOPlatform.scala @@ -73,37 +73,6 @@ private[effect] object IOPlatform { } } - /** - * Establishes the maximum stack depth for `IO#map` operations. - * - * The default is `128`, from which we subtract one as an - * optimization. This default has been reached like this: - * - * - according to official docs, the default stack size on 32-bits - * Windows and Linux was 320 KB, whereas for 64-bits it is 1024 KB - * - according to measurements chaining `Function1` references uses - * approximately 32 bytes of stack space on a 64 bits system; - * this could be lower if "compressed oops" is activated - * - therefore a "map fusion" that goes 128 in stack depth can use - * about 4 KB of stack space - * - * If this parameter becomes a problem, it can be tuned by setting - * the `cats.effect.fusionMaxStackDepth` system property when - * executing the Java VM: - * - *
-   *   java -Dcats.effect.fusionMaxStackDepth=32 \
-   *        ...
-   * 
- */ - final val fusionMaxStackDepth = - Option(System.getProperty("cats.effect.fusionMaxStackDepth", "")) - .filter(s => s != null && s.nonEmpty) - .flatMap(s => Try(s.toInt).toOption) - .filter(_ > 0) - .map(_ - 1) - .getOrElse(127) - /** * Returns `true` if the underlying platform is the JVM, * `false` if it's JavaScript. diff --git a/laws/shared/src/test/scala/cats/effect/IOTests.scala b/laws/shared/src/test/scala/cats/effect/IOTests.scala index f8f2a94d5f..56e6d65343 100644 --- a/laws/shared/src/test/scala/cats/effect/IOTests.scala +++ b/laws/shared/src/test/scala/cats/effect/IOTests.scala @@ -527,14 +527,6 @@ class IOTests extends BaseTestsSuite { io.unsafeRunSync() shouldBe 1 } - test("map is stack-safe for unsafeRunSync") { - import IOPlatform.{fusionMaxStackDepth => max} - val f = (x: Int) => x + 1 - val io = (0 until (max * 10000)).foldLeft(IO(0))((acc, _) => acc.map(f)) - - io.unsafeRunSync() shouldEqual max * 10000 - } - testAsync("parMap2 for successful values") { implicit ec => implicit val cs: ContextShift[IO] = ec.ioContextShift diff --git a/laws/shared/src/test/scala/cats/effect/SyncIOTests.scala b/laws/shared/src/test/scala/cats/effect/SyncIOTests.scala index aaa1daf0c2..a67f345a19 100644 --- a/laws/shared/src/test/scala/cats/effect/SyncIOTests.scala +++ b/laws/shared/src/test/scala/cats/effect/SyncIOTests.scala @@ -154,14 +154,6 @@ class SyncIOTests extends BaseTestsSuite { io.unsafeRunSync() shouldEqual 2 } - test("map is stack-safe for unsafeRunSync") { - import IOPlatform.{fusionMaxStackDepth => max} - val f = (x: Int) => x + 1 - val io = (0 until (max * 10000)).foldLeft(SyncIO(0))((acc, _) => acc.map(f)) - - io.unsafeRunSync() shouldEqual max * 10000 - } - testAsync("IO#redeem(throw, f) <-> IO#map") { implicit ec => check { (io: IO[Int], f: Int => Int) => io.redeem(e => throw e, f) <-> io.map(f) From 0e7aa80c6961ddda77b8556900fd0264ce4cd15c Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Wed, 8 Jul 2020 21:39:17 -0500 Subject: [PATCH 60/78] RingBuffer improvements --- .../cats/effect/internals/RingBuffer.scala | 41 ++++++++----------- .../effect/internals/RingBufferTests.scala | 2 +- 2 files changed, 17 insertions(+), 26 deletions(-) diff --git a/core/shared/src/main/scala/cats/effect/internals/RingBuffer.scala b/core/shared/src/main/scala/cats/effect/internals/RingBuffer.scala index 2ffc7237f6..f4e518c717 100644 --- a/core/shared/src/main/scala/cats/effect/internals/RingBuffer.scala +++ b/core/shared/src/main/scala/cats/effect/internals/RingBuffer.scala @@ -28,48 +28,39 @@ final private[internals] class RingBuffer[A <: AnyRef](size: Int) { private[this] val length = nextPowerOfTwo(size) private[this] val mask = length - 1 - // TODO: this can be an expensive allocation - // either construct it lazily or expand it on-demand private[this] val array: Array[AnyRef] = new Array(length) - private[this] var writeIndex: Int = 0 - private[this] var readIndex: Int = 0 + private[this] var index: Int = 0 def push(a: A): A = { - val wi = writeIndex & mask - if (writeIndex == readIndex + length) { - val old = array(wi) - array(wi) = a - // TODO: overflow at int.maxvalue? - writeIndex = writeIndex + 1 - readIndex = readIndex + 1 - old.asInstanceOf[A] - } else { - array(wi) = a - writeIndex = writeIndex + 1 - null.asInstanceOf[A] - } + val wi = index & mask + val old = array(wi).asInstanceOf[A] + array(wi) = a + index += 1 + old } + def isEmpty: Boolean = + index == 0 + def capacity: Int = length - def isEmpty: Boolean = - readIndex == writeIndex - - // TODO: expose this as an iterator instead? - def toList: List[A] = - (readIndex until writeIndex).toList + def toList: List[A] = { + val end = index + val start = Math.max(end - length, 0) + (start until end).toList .map(i => array(i & mask).asInstanceOf[A]) + } } object RingBuffer { - // TODO: bounds check at int.maxvalue ? + // N.B. this can overflow private def nextPowerOfTwo(i: Int): Int = { var n = 1 while (n < i) { - n = n * 2 + n *= 2 } n } diff --git a/core/shared/src/test/scala/cats/effect/internals/RingBufferTests.scala b/core/shared/src/test/scala/cats/effect/internals/RingBufferTests.scala index b9e7ec01d4..e10f74d049 100644 --- a/core/shared/src/test/scala/cats/effect/internals/RingBufferTests.scala +++ b/core/shared/src/test/scala/cats/effect/internals/RingBufferTests.scala @@ -40,7 +40,7 @@ class RingBufferTests extends AnyFunSuite with Matchers with TestUtils { new RingBuffer[Integer](13000).capacity shouldBe 16384 } - test("reading and writing elements") { + test("writing elements") { val buffer = new RingBuffer[Integer](4) for (i <- 0 to 3) buffer.push(i) buffer.toList shouldBe List(0, 1, 2, 3) From 442fe1b2f82766d896ec7454fa66b2e16ec31325 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Wed, 8 Jul 2020 23:41:55 -0500 Subject: [PATCH 61/78] address pr feedback --- .../effect/internals/TracingPlatform.scala | 2 +- .../effect/internals/TracingPlatform.java | 19 +-- .../scala/cats/effect/internals/Example.scala | 48 ------- .../cats/effect/internals/IOPlatform.scala | 2 +- .../src/main/scala/cats/effect/IO.scala | 7 +- .../cats/effect/internals/IOContext.scala | 21 +-- .../cats/effect/internals/IORunLoop.scala | 38 +---- .../cats/effect/internals/IOTracing.scala | 25 +--- .../scala/cats/effect/tracing/IOTrace.scala | 54 ++++--- .../effect/internals/IOContextTests.scala | 40 +----- site/src/main/mdoc/guides/tracing.md | 134 +++++++++++------- .../cats/effect/FullStackTracingTests.scala | 2 +- .../cats/effect/CachedStackTracingTests.scala | 2 +- .../test/scala/cats/effect/TracingTests.scala | 8 +- 14 files changed, 133 insertions(+), 269 deletions(-) delete mode 100644 core/jvm/src/main/scala/cats/effect/internals/Example.scala diff --git a/core/js/src/main/scala/cats/effect/internals/TracingPlatform.scala b/core/js/src/main/scala/cats/effect/internals/TracingPlatform.scala index 3cd4fb1c8b..3a9decc42e 100644 --- a/core/js/src/main/scala/cats/effect/internals/TracingPlatform.scala +++ b/core/js/src/main/scala/cats/effect/internals/TracingPlatform.scala @@ -23,5 +23,5 @@ object TracingPlatform { final val isStackTracing: Boolean = isFullStackTracing || isCachedStackTracing - final val maxTraceDepth: Int = 32 + final val traceBufferSize: Int = 32 } diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java index 2d0ea8659b..d8f9fe33f7 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java @@ -29,15 +29,6 @@ */ public final class TracingPlatform { - /** - * Sets the global tracing mode for a JVM process. - * If this flag isn't enabled, no tracing is instrumented. - */ - public static final boolean isTracing = Optional.ofNullable(System.getProperty("cats.effect.tracing")) - .filter(x -> !x.isEmpty()) - .map(Boolean::valueOf) - .orElse(false); - /** * Sets stack tracing mode for a JVM process, which controls * how much stack trace information is captured. @@ -45,11 +36,11 @@ public final class TracingPlatform { */ private static final String stackTracingMode = Optional.ofNullable(System.getProperty("cats.effect.stackTracingMode")) .filter(x -> !x.isEmpty()) - .orElse("none"); + .orElse("full"); - public static final boolean isCachedStackTracing = isTracing && stackTracingMode.equalsIgnoreCase("cached"); + public static final boolean isCachedStackTracing = stackTracingMode.equalsIgnoreCase("cached"); - public static final boolean isFullStackTracing = isTracing && stackTracingMode.equalsIgnoreCase("full"); + public static final boolean isFullStackTracing = stackTracingMode.equalsIgnoreCase("full"); public static final boolean isStackTracing = isFullStackTracing || isCachedStackTracing; @@ -58,7 +49,7 @@ public final class TracingPlatform { * lines are produced, then the oldest trace lines will be discarded. * Automatically rounded up to the nearest power of 2. */ - public static final int maxTraceDepth = Optional.ofNullable(System.getProperty("cats.effect.maxTraceDepth")) + public static final int traceBufferSize = Optional.ofNullable(System.getProperty("cats.effect.traceBufferSize")) .filter(x -> !x.isEmpty()) .flatMap(x -> { try { @@ -67,6 +58,6 @@ public final class TracingPlatform { return Optional.empty(); } }) - .orElse(64); + .orElse(128); } diff --git a/core/jvm/src/main/scala/cats/effect/internals/Example.scala b/core/jvm/src/main/scala/cats/effect/internals/Example.scala deleted file mode 100644 index 481de0fefe..0000000000 --- a/core/jvm/src/main/scala/cats/effect/internals/Example.scala +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers - * - * Licensed 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 org.simpleapp.example - -import cats.effect.{ExitCode, IO, IOApp} - -object Example extends IOApp { - - def print(msg: String): IO[Unit] = - IO(println(msg)) - - def program2: IO[Unit] = - for { - _ <- print("3") - _ <- print("4") - } yield () - - def program: IO[Unit] = - for { - _ <- print("1") - _ <- print("2") - _ <- IO.shift - _ <- program2 - _ <- print("5") - } yield () - - override def run(args: List[String]): IO[ExitCode] = - for { - _ <- IO.suspend(program).traced - trace <- IO.backtrace - _ <- trace.compactPrint - } yield ExitCode.Success - -} diff --git a/core/jvm/src/main/scala/cats/effect/internals/IOPlatform.scala b/core/jvm/src/main/scala/cats/effect/internals/IOPlatform.scala index a94c785c34..e3a378d633 100644 --- a/core/jvm/src/main/scala/cats/effect/internals/IOPlatform.scala +++ b/core/jvm/src/main/scala/cats/effect/internals/IOPlatform.scala @@ -20,7 +20,7 @@ import java.util.concurrent.locks.AbstractQueuedSynchronizer import cats.effect.IO import scala.concurrent.blocking import scala.concurrent.duration.{Duration, FiniteDuration} -import scala.util.{Either, Try} +import scala.util.Either private[effect] object IOPlatform { diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index 3c4b3d5848..19132bf75e 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -791,9 +791,6 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * */ def <&[B](another: IO[B])(implicit p: NonEmptyParallel[IO]): IO[A] = p.parProductL(this)(another) - - def traced: IO[A] = - IOTracing.traced(this) } abstract private[effect] class IOParallelNewtype extends internals.IOTimerRef with internals.IOCompanionBinaryCompat { @@ -1626,7 +1623,7 @@ object IO extends IOInstances { def contextShift(ec: ExecutionContext): ContextShift[IO] = IOContextShift(ec) - val backtrace: IO[IOTrace] = + val trace: IO[IOTrace] = IO.Async { (_, ctx, cb) => cb(Right(ctx.trace())) } @@ -1677,8 +1674,6 @@ object IO extends IOInstances { final private[effect] case class Trace[A](source: IO[A], trace: StackTraceFrame) extends IO[A] - final private[effect] case class SetTracing(collectStackTraces: Boolean) extends IO[Unit] - /** * An internal state for that optimizes changes to * [[internals.IOConnection]]. diff --git a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala index 3c576fd690..46ac4216ca 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala @@ -17,7 +17,7 @@ package cats.effect.internals import cats.effect.tracing.{IOTrace, StackTraceFrame} -import cats.effect.internals.TracingPlatform.maxTraceDepth +import cats.effect.internals.TracingPlatform.traceBufferSize /** * INTERNAL API — Holds state related to the execution of @@ -26,35 +26,18 @@ import cats.effect.internals.TracingPlatform.maxTraceDepth */ final private[effect] class IOContext private () { - private var frames: RingBuffer[StackTraceFrame] = new RingBuffer(maxTraceDepth) + private val frames: RingBuffer[StackTraceFrame] = new RingBuffer(traceBufferSize) private var captured: Int = 0 private var omitted: Int = 0 - private var activeCollects: Int = 0 - def pushFrame(fr: StackTraceFrame): Unit = { captured += 1 if (frames.push(fr) != null) omitted += 1 } - def resetTrace(): Unit = { - frames = new RingBuffer(maxTraceDepth) - captured = 0 - omitted = 0 - } - def trace(): IOTrace = IOTrace(frames.toList, captured, omitted) - def activeTraces(): Int = - activeCollects - - def enterTrace(): Unit = - activeCollects += 1 - - def exitTrace(): Unit = - activeCollects -= 1 - } object IOContext { diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index 36e47561bb..3e0c8faa80 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -17,7 +17,7 @@ package cats.effect.internals import cats.effect.IO -import cats.effect.IO.{Async, Bind, ContextSwitch, Delay, Map, Pure, RaiseError, SetTracing, Suspend, Trace} +import cats.effect.IO.{Async, Bind, ContextSwitch, Delay, Map, Pure, RaiseError, Suspend, Trace} import cats.effect.tracing.StackTraceFrame import cats.effect.internals.TracingPlatform.isStackTracing @@ -69,7 +69,6 @@ private[effect] object IORunLoop { // Can change on a context switch var conn: IOConnection = cancelable var ctx: IOContext = ctxRef - var activeCollects: Int = if (ctx ne null) ctx.activeTraces else 0 var bFirst: Bind = bFirstRef var bRest: CallStack = bRestRef var rcb: RestartCallback = rcbRef @@ -87,7 +86,7 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if (isStackTracing && activeCollects > 0) { + if (isStackTracing) { if (ctx eq null) ctx = IOContext() val trace = bind.trace if (trace ne null) ctx.pushFrame(trace.asInstanceOf[StackTraceFrame]) @@ -132,7 +131,7 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if (isStackTracing && activeCollects > 0) { + if (isStackTracing) { if (ctx eq null) ctx = IOContext() val trace = bindNext.trace if (trace ne null) ctx.pushFrame(trace.asInstanceOf[StackTraceFrame]) @@ -146,7 +145,7 @@ private[effect] object IORunLoop { // may produce trace frames e.g. IOBracket. if (ctx eq null) ctx = IOContext() if (rcb eq null) rcb = new RestartCallback(conn, cb.asInstanceOf[Callback]) - if (isStackTracing && activeCollects > 0) { + if (isStackTracing) { val trace = async.trace if (trace ne null) ctx.pushFrame(trace.asInstanceOf[StackTraceFrame]) } @@ -167,18 +166,6 @@ private[effect] object IORunLoop { if (ctx eq null) ctx = IOContext() ctx.pushFrame(frame) currentIO = source - - case SetTracing(collect) => - if (ctx eq null) ctx = IOContext() - if (collect) { - activeCollects += 1 - ctx.enterTrace() - } else { - activeCollects -= 1 - ctx.exitTrace() - } - unboxed = ().asInstanceOf[AnyRef] - hasUnboxed = true } if (hasUnboxed) { @@ -216,7 +203,6 @@ private[effect] object IORunLoop { var bFirst: Bind = null var bRest: CallStack = null var ctx: IOContext = null - var activeCollects: Int = 0 // Values from Pure and Delay are unboxed in this var, // for code reuse between Pure and Delay var hasUnboxed: Boolean = false @@ -229,7 +215,7 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if (isStackTracing && activeCollects > 0) { + if (isStackTracing) { if (ctx eq null) ctx = IOContext() val trace = bind.trace if (trace ne null) ctx.pushFrame(trace.asInstanceOf[StackTraceFrame]) @@ -274,7 +260,7 @@ private[effect] object IORunLoop { if (bRest eq null) bRest = new ArrayStack() bRest.push(bFirst) } - if (isStackTracing && activeCollects > 0) { + if (isStackTracing) { if (ctx eq null) ctx = IOContext() val trace = bindNext.trace if (trace ne null) ctx.pushFrame(trace.asInstanceOf[StackTraceFrame]) @@ -287,18 +273,6 @@ private[effect] object IORunLoop { ctx.pushFrame(frame) currentIO = source - case SetTracing(collect) => - if (ctx eq null) ctx = IOContext() - if (collect) { - activeCollects += 1 - ctx.enterTrace() - } else { - activeCollects -= 1 - ctx.exitTrace() - } - unboxed = ().asInstanceOf[AnyRef] - hasUnboxed = true - case _ => // Cannot inline the code of this method — as it would // box those vars in scala.runtime.ObjectRef! diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index 603b5868c5..b729032ecd 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -19,7 +19,7 @@ package cats.effect.internals import java.util.concurrent.ConcurrentHashMap import cats.effect.IO -import cats.effect.IO.{Bind, Pure, RaiseError, SetTracing, Trace} +import cats.effect.IO.Trace import cats.effect.tracing.{StackTraceFrame, TraceTag} private[effect] object IOTracing { @@ -33,12 +33,6 @@ private[effect] object IOTracing { def cached(traceTag: TraceTag, clazz: Class[_]): StackTraceFrame = buildCachedFrame(traceTag, clazz) - def traced[A](source: IO[A]): IO[A] = - resetTrace *> - Bind(incrementCollection, - (_: Unit) => Bind(source, DecrementTraceCollection.asInstanceOf[A => IO[A]], null), - null) - private def buildCachedFrame(traceTag: TraceTag, clazz: Class[_]): StackTraceFrame = { val cf = frameCache.get(clazz) if (cf eq null) { @@ -53,23 +47,6 @@ private[effect] object IOTracing { def buildFrame(traceTag: TraceTag): StackTraceFrame = StackTraceFrame(traceTag, new Throwable()) - private[this] val incrementCollection: IO[Unit] = SetTracing(true) - - private[this] val decrementCollection: IO[Unit] = SetTracing(false) - - private object DecrementTraceCollection extends IOFrame[Any, IO[Any]] { - override def apply(a: Any) = - Bind(decrementCollection, (_: Unit) => Pure(a), null) - override def recover(e: Throwable) = - Bind(decrementCollection, (_: Unit) => RaiseError(e), null) - } - - private[this] val resetTrace: IO[Unit] = - IO.Async { (_, ctx, cb) => - ctx.resetTrace() - cb(Right(())) - } - /** * Cache for trace frames. Keys are references to lambda classes. */ diff --git a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala index 2737964716..2c57d17b24 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala @@ -22,14 +22,13 @@ final case class IOTrace(frames: List[StackTraceFrame], captured: Int, omitted: import IOTrace._ - def compact: String = { + def compact(): String = { val TurnRight = "╰" val Junction = "├" def renderStackTraceElement(ste: StackTraceElement): String = { - val className = demangleClassName(ste.getClassName) - val methodName = demangleMethodName(ste.getMethodName) - s"$className.$methodName (${ste.getFileName}:${ste.getLineNumber})" + val methodName = demangleMethod(ste.getMethodName) + s"${ste.getClassName}.$methodName (${ste.getFileName}:${ste.getLineNumber})" } val acc0 = s"IOTrace: $captured frames captured, $omitted omitted\n" @@ -43,31 +42,32 @@ final case class IOTrace(frames: List[StackTraceFrame], captured: Int, omitted: acc1 } - def compactPrint: IO[Unit] = + def compactPrint(): IO[Unit] = IO(System.err.println(compact)) - def pretty: String = { + def pretty(maxStackTracesLines: Int = Int.MaxValue): String = { val acc0 = s"IOTrace: $captured frames captured, $omitted omitted\n" - val acc1 = acc0 + loop("", 0, true, frames.toList) + val acc1 = acc0 + loop("", 0, true, frames, maxStackTracesLines) acc1 } - def prettyPrint: IO[Unit] = - IO(System.err.println(pretty)) + def prettyPrint(maxStackTracesLines: Int = Int.MaxValue): IO[Unit] = + IO(System.err.println(pretty(maxStackTracesLines))) - private def loop(acc: String, indent: Int, init: Boolean, rest: List[StackTraceFrame]): String = { + private def loop(acc: String, + indent: Int, + init: Boolean, + rest: List[StackTraceFrame], + maxStackTraceLines: Int): String = { val TurnRight = "╰" val InverseTurnRight = "╭" val Junction = "├" val Line = "│" def renderStackTraceElement(ste: StackTraceElement, last: Boolean): String = { - val className = demangleClassName(ste.getClassName) - val methodName = demangleMethodName(ste.getMethodName) - + val methodName = demangleMethod(ste.getMethodName) val junc = if (last) TurnRight else Junction - - Line + " " + junc + s" $className.$methodName (${ste.getFileName}:${ste.getLineNumber})\n" + Line + " " + junc + s" ${ste.getClassName}.$methodName (${ste.getFileName}:${ste.getLineNumber})\n" } rest match { @@ -75,36 +75,28 @@ final case class IOTrace(frames: List[StackTraceFrame], captured: Int, omitted: val acc2 = if (init) InverseTurnRight + s" ${k.tag.name}\n" else Junction + s" ${k.tag.name}\n" val innerLines = k.stackTrace .drop(stackTraceIgnoreLines) + .take(maxStackTraceLines) .zipWithIndex .map { case (ste, i) => renderStackTraceElement(ste, i == k.stackTrace.length - 1) } .mkString - loop(acc + acc2 + innerLines + Line + "\n", indent, false, ks) + loop(acc + acc2 + innerLines + Line + "\n", indent, false, ks, maxStackTraceLines) } case Nil => acc } } - private def demangleClassName(className: String): String = - className.replaceAll("\\$", "") - - private def demangleMethodName(methodName: String): String = - anonfunRegex.findFirstMatchIn(methodName) match { - case Some(mat) => mat.group(1) - case None => methodName - } - } private[effect] object IOTrace { - final val anonfunRegex = "^\\$+anonfun\\$+(.+)\\$+\\d+$".r + private val anonfunRegex = "^\\$+anonfun\\$+(.+)\\$+\\d+$".r // Number of lines to drop from the top of the stack trace - final val stackTraceIgnoreLines = 3 + private val stackTraceIgnoreLines = 3 - final val stackTraceFilter = List( + private val stackTraceFilter = List( "cats.effect.", "cats.", "sbt.", @@ -112,4 +104,10 @@ private[effect] object IOTrace { "sun.", "scala." ) + + def demangleMethod(methodName: String): String = + anonfunRegex.findFirstMatchIn(methodName) match { + case Some(mat) => mat.group(1) + case None => methodName + } } diff --git a/core/shared/src/test/scala/cats/effect/internals/IOContextTests.scala b/core/shared/src/test/scala/cats/effect/internals/IOContextTests.scala index 37d166da2d..c6c57b8f2c 100644 --- a/core/shared/src/test/scala/cats/effect/internals/IOContextTests.scala +++ b/core/shared/src/test/scala/cats/effect/internals/IOContextTests.scala @@ -22,7 +22,7 @@ import org.scalatest.funsuite.AnyFunSuite class IOContextTests extends AnyFunSuite with Matchers { - val maxTraceDepth: Int = cats.effect.internals.TracingPlatform.maxTraceDepth + val traceBufferSize: Int = cats.effect.internals.TracingPlatform.traceBufferSize val throwable = new Throwable() test("push traces") { @@ -43,46 +43,14 @@ class IOContextTests extends AnyFunSuite with Matchers { test("track omitted frames") { val ctx = IOContext() - for (_ <- 0 until (maxTraceDepth + 10)) { + for (_ <- 0 until (traceBufferSize + 10)) { ctx.pushFrame(StackTraceFrame(TraceTag.Pure, throwable)) } val trace = ctx.trace() - trace.frames.length shouldBe maxTraceDepth - trace.captured shouldBe (maxTraceDepth + 10) + trace.frames.length shouldBe traceBufferSize + trace.captured shouldBe (traceBufferSize + 10) trace.omitted shouldBe 10 } - test("reset tracing") { - val ctx = IOContext() - - val t1 = StackTraceFrame(TraceTag.Pure, throwable) - val t2 = StackTraceFrame(TraceTag.Suspend, throwable) - - ctx.pushFrame(t1) - ctx.pushFrame(t2) - - ctx.resetTrace() - - val trace = ctx.trace() - trace.frames shouldBe List() - trace.captured shouldBe 0 - trace.omitted shouldBe 0 - } - - test("track tracing regions") { - val ctx = IOContext() - - ctx.activeTraces() shouldBe 0 - - ctx.enterTrace() - ctx.activeTraces() shouldBe 1 - ctx.enterTrace() - ctx.activeTraces() shouldBe 2 - - ctx.exitTrace() - ctx.activeTraces() shouldBe 1 - ctx.exitTrace() - ctx.activeTraces() shouldBe 0 - } } diff --git a/site/src/main/mdoc/guides/tracing.md b/site/src/main/mdoc/guides/tracing.md index aec8aae154..eab410fcf7 100644 --- a/site/src/main/mdoc/guides/tracing.md +++ b/site/src/main/mdoc/guides/tracing.md @@ -43,9 +43,10 @@ IOTrace: 13 frames captured, 0 omitted However, fiber tracing isn't limited to collecting stack traces. Tracing has many use cases that improve developer experience and aid in understanding -how our applications work (coming soon!): +how our applications work. These features are described below. A **bolded name** +indicates that the feature has been merged into master. -1. Asynchronous stack tracing. This is essentially what is described above, +1. **Asynchronous stack tracing**. This is essentially what is described above, where stack frames are collected across asynchronous boundaries for a given fiber. 2. Combinator inference. Stack traces can be walked to determine what @@ -78,53 +79,45 @@ and trace information of all fibers in an application can be extracted for debugging purposes. 10. Monad transformer analysis. -## Usage +## Asynchronous stack tracing +### Configuration +The stack tracing mode of an application is configured by the system property +`cats.effect.stackTracingMode`. There are three stack tracing modes: `DISABLED`, +`CACHED` and `FULL`. These values are case-insensitive. -### Enable tracing -A global system property must be set on JVM in order to enable any tracing -facility: +To prevent unbounded memory usage, stack traces for a fiber are accumulated +in an internal buffer as execution proceeds. If more traces are collected than +the buffer can retain, then the older traces will be overwritten. The default +size for the buffer is 128, but can be changed via the system property +`cats.effect.traceBufferSize`. +For example, to enable full stack tracing and a trace buffer size of 1024, +specify the following system properties: ``` --Dcats.effect.tracing=true -``` - -### Collecting traces -To mark a region of a program for tracing, use the `traced` combinator. After -that subprogram completes, use the `backtrace` combinator to collect the full -fiber trace. - -```scala -for { - _ <- program.traced - trace <- IO.backtrace - _ <- trace.compactPrint -} yield () +-Dcats.effect.stackTracingMode=full -Dcats.effect.traceBufferSize=1024 ``` -### Asynchronous stack tracing -The stack tracing mode of an application is configured by the system property -`cats.effect.tracing.mode`. There are three stack tracing modes: `DISABLED`, -`CACHED` and `FULL`. - #### DISABLED -No tracing is instrumented by the program. This is the default stack tracing -mode. +No tracing is instrumented by the program and so incurs negligible impact to +performance. If a trace is requested, it will be empty. #### CACHED When cached stack tracing is enabled, a stack trace is captured and cached for every `map`, `flatMap` and `async` call in a program. -The stack trace cache is keyed by the lambda class reference, so cached tracing -may produce inaccurate fiber traces under several scenarios: +The stack trace cache is indexed by the lambda class reference, so cached +tracing may produce inaccurate fiber traces under several scenarios: 1. Monad transformer composition 2. A named function is supplied to `map`, `async` or `flatMap` at multiple call-sites When no collection is performed, we measured less than an 18% performance hit for a completely synchronous `IO` program, so it will most likely be negligible -for any program that performs any sort of I/O. +for any program that performs any sort of I/O. Nonetheless, we strongly +recommend benchmarking applications that enable tracing. -This is the recommended mode to run in production environments. +This is the recommended mode to run in production environments and is enabled +by default. #### FULL When full stack tracing is enabled, a stack trace is captured for every @@ -132,44 +125,77 @@ combinator traced in cached mode, but also `pure`, `delay`, `suspend` and other derived combinators. This mode will incur a heavy performance hit for most programs, and is -recommended for use only in development environments. +recommended for use in development environments. -### Example -Here is a sample program that demonstrates how to turn on tracing in an -application. +### Requesting and rendering traces +Once the global tracing flag is configured, `IO` programs will automatically +begin collecting traces. The trace for a fiber can be accessed at any point +during its execution via the `IO.trace` combinator. This is the `IO` equivalent +of capturing a thread's stack trace. ```scala -// Specify the following flags in your JVM: -// -Dcats.effect.tracing=true +import cats.effect.IO + +def program: IO[Unit] = + for { + _ <- IO(println("Started the program")) + trace <- IO.trace + } yield () +``` + +After a fiber trace is retrieved, we can print it to the console, just like how +exception stack traces can be printed with `printStackTrace`. `compactPrint` +includes the most relevant stack trace element for each fiber operation that +was performed. `prettyPrint` includes the entire stack trace for each fiber +operation. These methods accept arguments that lets us customize how traces +are printed. + +```scala +import cats.effect.IO + +def program: IO[Unit] = + for { + _ <- IO(println("Started the program")) + trace <- IO.trace + _ <- trace.compactPrint + _ <- trace.prettyPrint() + } yield () +``` + +Keep in mind that the scope and amount of information that traces hold will +change over time as additional fiber tracing features are merged into master. + +### Complete example +Here is a sample program that demonstrates tracing in action. + +```scala +// Pass the following system property to your JVM: // -Dcats.effect.stackTracingMode=full +import cats.implicits._ import cats.effect.{ExitCode, IO, IOApp} +import scala.util.Random object Example extends IOApp { - def print(msg: String): IO[Unit] = - IO(println(msg)) - - def program2: IO[Unit] = - for { - _ <- print("3") - _ <- print("4") - } yield () - + def fib(n: Int, a: Long = 0, b: Long = 1): IO[Long] = + IO(a + b).flatMap { b2 => + if (n > 0) + fib(n - 1, b, b2) + else + IO.pure(b2) + } + def program: IO[Unit] = for { - _ <- print("1") - _ <- print("2") - _ <- IO.shift - _ <- program2 - _ <- print("5") + x <- fib(20) + _ <- IO(println(s"The 20th fibonacci number is $x")) + _ <- IO(Random.nextBoolean()).ifM(IO.raiseError(new Throwable("")), IO.unit) } yield () override def run(args: List[String]): IO[ExitCode] = for { - _ <- IO.suspend(program).traced - trace <- IO.backtrace - _ <- trace.compactPrint + _ <- program.handleErrorWith(_ => IO.trace.flatMap(_.compactPrint)) } yield ExitCode.Success } diff --git a/tracing-tests/shared/src/fulltracing/scala/cats/effect/FullStackTracingTests.scala b/tracing-tests/shared/src/fulltracing/scala/cats/effect/FullStackTracingTests.scala index 5ceb3a2433..ff9e6216ed 100644 --- a/tracing-tests/shared/src/fulltracing/scala/cats/effect/FullStackTracingTests.scala +++ b/tracing-tests/shared/src/fulltracing/scala/cats/effect/FullStackTracingTests.scala @@ -30,7 +30,7 @@ class FullStackTracingTests extends AsyncFunSuite with Matchers { def traced[A](io: IO[A]): IO[IOTrace] = for { _ <- io.traced - t <- IO.backtrace + t <- IO.trace } yield t test("full stack tracing captures map frames") { diff --git a/tracing-tests/shared/src/test/scala/cats/effect/CachedStackTracingTests.scala b/tracing-tests/shared/src/test/scala/cats/effect/CachedStackTracingTests.scala index a563e58f48..b15e30afc3 100644 --- a/tracing-tests/shared/src/test/scala/cats/effect/CachedStackTracingTests.scala +++ b/tracing-tests/shared/src/test/scala/cats/effect/CachedStackTracingTests.scala @@ -30,7 +30,7 @@ class CachedStackTracingTests extends AsyncFunSuite with Matchers { def traced[A](io: IO[A]): IO[IOTrace] = for { _ <- io.traced - t <- IO.backtrace + t <- IO.trace } yield t test("cached stack tracing captures map frames") { diff --git a/tracing-tests/shared/src/test/scala/cats/effect/TracingTests.scala b/tracing-tests/shared/src/test/scala/cats/effect/TracingTests.scala index 1c8d10966d..76d5758c9d 100644 --- a/tracing-tests/shared/src/test/scala/cats/effect/TracingTests.scala +++ b/tracing-tests/shared/src/test/scala/cats/effect/TracingTests.scala @@ -30,14 +30,14 @@ class TracingTests extends AsyncFunSuite with Matchers { def traced[A](io: IO[A]): IO[IOTrace] = for { _ <- io.traced - t <- IO.backtrace + t <- IO.trace } yield t test("trace is empty when no traces are captured") { val task = for { _ <- IO.pure(1) _ <- IO.pure(1) - t <- IO.backtrace + t <- IO.trace } yield t for (r <- task.unsafeToFuture()) yield { @@ -65,9 +65,9 @@ class TracingTests extends AsyncFunSuite with Matchers { val task = for { _ <- op.traced - _ <- IO.backtrace + _ <- IO.trace _ <- op.traced - t <- IO.backtrace + t <- IO.trace } yield t.frames.length for (r <- task.unsafeToFuture()) yield { From 4db55540cf7ac986dbbb64aa20974fed6393e1ef Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Thu, 9 Jul 2020 01:37:30 -0500 Subject: [PATCH 62/78] wip --- .../effect/benchmarks/DeepBindBenchmark.scala | 42 +++++++++---------- .../effect/internals/TracingPlatform.java | 2 +- 2 files changed, 22 insertions(+), 22 deletions(-) diff --git a/benchmarks/shared/src/main/scala/cats/effect/benchmarks/DeepBindBenchmark.scala b/benchmarks/shared/src/main/scala/cats/effect/benchmarks/DeepBindBenchmark.scala index 95811c2a37..62efa04967 100644 --- a/benchmarks/shared/src/main/scala/cats/effect/benchmarks/DeepBindBenchmark.scala +++ b/benchmarks/shared/src/main/scala/cats/effect/benchmarks/DeepBindBenchmark.scala @@ -54,26 +54,26 @@ class DeepBindBenchmark { loop(0).unsafeRunSync() } -// @Benchmark -// def delay(): Int = { -// def loop(i: Int): IO[Int] = -// for { -// j <- IO(i) -// _ <- if (j > size) IO(j) else loop(j + 1) -// } yield j -// -// loop(0).unsafeRunSync() -// } + @Benchmark + def delay(): Int = { + def loop(i: Int): IO[Int] = + for { + j <- IO(i) + _ <- if (j > size) IO(j) else loop(j + 1) + } yield j + + loop(0).unsafeRunSync() + } -// @Benchmark -// def async(): Int = { -// def loop(i: Int): IO[Int] = -// for { -// j <- IO(i) -// _ <- IO.shift -// _ <- if (j > size) IO(j) else loop(j + 1) -// } yield j -// -// loop(0).unsafeRunSync() -// } + @Benchmark + def async(): Int = { + def loop(i: Int): IO[Int] = + for { + j <- IO(i) + _ <- IO.shift + _ <- if (j > size) IO(j) else loop(j + 1) + } yield j + + loop(0).unsafeRunSync() + } } diff --git a/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java b/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java index d8f9fe33f7..099598e12b 100644 --- a/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java +++ b/core/jvm/src/main/java/cats/effect/internals/TracingPlatform.java @@ -36,7 +36,7 @@ public final class TracingPlatform { */ private static final String stackTracingMode = Optional.ofNullable(System.getProperty("cats.effect.stackTracingMode")) .filter(x -> !x.isEmpty()) - .orElse("full"); + .orElse("cached"); public static final boolean isCachedStackTracing = stackTracingMode.equalsIgnoreCase("cached"); From a277e173c67bd6353e7432ea8bcc865f9d4c88e2 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Thu, 9 Jul 2020 02:31:30 -0500 Subject: [PATCH 63/78] doc --- core/shared/src/main/scala/cats/effect/IO.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index 19132bf75e..2ae47266ca 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -1623,6 +1623,9 @@ object IO extends IOInstances { def contextShift(ec: ExecutionContext): ContextShift[IO] = IOContextShift(ec) + /** + * Returns the accumulated trace of the currently active fiber. + */ val trace: IO[IOTrace] = IO.Async { (_, ctx, cb) => cb(Right(ctx.trace())) From f6892fa7dd62373fe07635bab92a383af5362eda Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Thu, 9 Jul 2020 03:20:24 -0500 Subject: [PATCH 64/78] restore benchmarks --- .../effect/benchmarks/MapCallsBenchmark.scala | 10 +++--- .../benchmarks/ShallowBindBenchmark.scala | 34 +++++++++---------- 2 files changed, 22 insertions(+), 22 deletions(-) diff --git a/benchmarks/shared/src/main/scala/cats/effect/benchmarks/MapCallsBenchmark.scala b/benchmarks/shared/src/main/scala/cats/effect/benchmarks/MapCallsBenchmark.scala index 2474be2933..2306e30cfc 100644 --- a/benchmarks/shared/src/main/scala/cats/effect/benchmarks/MapCallsBenchmark.scala +++ b/benchmarks/shared/src/main/scala/cats/effect/benchmarks/MapCallsBenchmark.scala @@ -42,11 +42,11 @@ class MapCallsBenchmark { @Benchmark def one(): Long = test(1, 1) -// @Benchmark -// def batch30(): Long = test(12000 / 30, 30) -// -// @Benchmark -// def batch120(): Long = test(12000 / 120, 120) + @Benchmark + def batch30(): Long = test(12000 / 30, 30) + + @Benchmark + def batch120(): Long = test(12000 / 120, 120) } object MapCallsBenchmark { diff --git a/benchmarks/shared/src/main/scala/cats/effect/benchmarks/ShallowBindBenchmark.scala b/benchmarks/shared/src/main/scala/cats/effect/benchmarks/ShallowBindBenchmark.scala index 36ac6b29ee..78cddc1f32 100644 --- a/benchmarks/shared/src/main/scala/cats/effect/benchmarks/ShallowBindBenchmark.scala +++ b/benchmarks/shared/src/main/scala/cats/effect/benchmarks/ShallowBindBenchmark.scala @@ -54,21 +54,21 @@ class ShallowBindBenchmark { .unsafeRunSync() } -// @Benchmark -// def delay(): Int = { -// def loop(i: Int): IO[Int] = -// if (i < size) IO(i + 1).flatMap(loop) -// else IO(i) -// -// IO(0).flatMap(loop).unsafeRunSync() -// } -// -// @Benchmark -// def async(): Int = { -// def loop(i: Int): IO[Int] = -// if (i < size) IO.shift.flatMap(_ => IO.pure(i + 1)).flatMap(loop) -// else IO.shift.flatMap(_ => IO.pure(i)) -// -// IO(0).flatMap(loop).unsafeRunSync() -// } + @Benchmark + def delay(): Int = { + def loop(i: Int): IO[Int] = + if (i < size) IO(i + 1).flatMap(loop) + else IO(i) + + IO(0).flatMap(loop).unsafeRunSync() + } + + @Benchmark + def async(): Int = { + def loop(i: Int): IO[Int] = + if (i < size) IO.shift.flatMap(_ => IO.pure(i + 1)).flatMap(loop) + else IO.shift.flatMap(_ => IO.pure(i)) + + IO(0).flatMap(loop).unsafeRunSync() + } } From 6cbda2803853a7c728c8fa8e030f20d58d6f63d5 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Thu, 9 Jul 2020 19:32:18 -0500 Subject: [PATCH 65/78] Declare ring buffer instance variables as private[this] --- .../cats/effect/internals/IOContext.scala | 6 +++--- .../cats/effect/internals/IOTracing.scala | 20 +++++++++---------- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala index 46ac4216ca..8d3a2cf341 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala @@ -26,9 +26,9 @@ import cats.effect.internals.TracingPlatform.traceBufferSize */ final private[effect] class IOContext private () { - private val frames: RingBuffer[StackTraceFrame] = new RingBuffer(traceBufferSize) - private var captured: Int = 0 - private var omitted: Int = 0 + private[this] val frames: RingBuffer[StackTraceFrame] = new RingBuffer(traceBufferSize) + private[this] var captured: Int = 0 + private[this] var omitted: Int = 0 def pushFrame(fr: StackTraceFrame): Unit = { captured += 1 diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index b729032ecd..556dde2c95 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -24,19 +24,19 @@ import cats.effect.tracing.{StackTraceFrame, TraceTag} private[effect] object IOTracing { - def decorated[A](source: IO[A], traceTag: TraceTag): IO[A] = - Trace(source, buildFrame(traceTag)) + def decorated[A](source: IO[A], tag: TraceTag): IO[A] = + Trace(source, buildFrame(tag)) - def uncached(traceTag: TraceTag): StackTraceFrame = - buildFrame(traceTag) + def uncached(tag: TraceTag): StackTraceFrame = + buildFrame(tag) - def cached(traceTag: TraceTag, clazz: Class[_]): StackTraceFrame = - buildCachedFrame(traceTag, clazz) + def cached(tag: TraceTag, clazz: Class[_]): StackTraceFrame = + buildCachedFrame(tag, clazz) - private def buildCachedFrame(traceTag: TraceTag, clazz: Class[_]): StackTraceFrame = { + private def buildCachedFrame(tag: TraceTag, clazz: Class[_]): StackTraceFrame = { val cf = frameCache.get(clazz) if (cf eq null) { - val f = buildFrame(traceTag) + val f = buildFrame(tag) frameCache.put(clazz, f) f } else { @@ -44,8 +44,8 @@ private[effect] object IOTracing { } } - def buildFrame(traceTag: TraceTag): StackTraceFrame = - StackTraceFrame(traceTag, new Throwable()) + def buildFrame(tag: TraceTag): StackTraceFrame = + StackTraceFrame(tag, new Throwable()) /** * Cache for trace frames. Keys are references to lambda classes. From 9f396b5497a6a03d223c2ec9a6ccd10017157d70 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Thu, 9 Jul 2020 21:23:50 -0500 Subject: [PATCH 66/78] Remove TraceTag --- .../src/main/scala/cats/effect/IO.scala | 28 ++++++++-------- .../cats/effect/internals/IOTracing.scala | 12 +++---- .../scala/cats/effect/tracing/IOTrace.scala | 19 +++++++++-- .../cats/effect/tracing/StackTraceFrame.scala | 2 +- .../scala/cats/effect/tracing/TraceTag.scala | 32 ------------------- 5 files changed, 37 insertions(+), 56 deletions(-) delete mode 100644 core/shared/src/main/scala/cats/effect/tracing/TraceTag.scala diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index 2ae47266ca..4498cec78a 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -26,7 +26,7 @@ import scala.concurrent.{ExecutionContext, Future, Promise, TimeoutException} import scala.util.control.NonFatal import scala.util.{Failure, Left, Right, Success, Try} import cats.data.Ior -import cats.effect.tracing.{IOTrace, StackTraceFrame, TraceTag} +import cats.effect.tracing.{IOTrace, StackTraceFrame} /** * A pure abstraction representing the intention to perform a @@ -103,9 +103,9 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { */ final def map[B](f: A => B): IO[B] = { val trace = if (isCachedStackTracing) { - IOTracing.cached(TraceTag.Map, f.getClass) + IOTracing.cached(4, f.getClass) } else if (isFullStackTracing) { - IOTracing.uncached(TraceTag.Map) + IOTracing.uncached(4) } else { null } @@ -130,9 +130,9 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { */ final def flatMap[B](f: A => IO[B]): IO[B] = { val trace = if (isCachedStackTracing) { - IOTracing.cached(TraceTag.Bind, f.getClass) + IOTracing.cached(3, f.getClass) } else if (isFullStackTracing) { - IOTracing.uncached(TraceTag.Bind) + IOTracing.uncached(3) } else { null } @@ -1131,7 +1131,7 @@ object IO extends IOInstances { def delay[A](body: => A): IO[A] = { val nextIo = Delay(() => body) if (isFullStackTracing) { - IOTracing.decorated(nextIo, TraceTag.Delay) + IOTracing.decorated(nextIo, 1) } else { nextIo } @@ -1148,7 +1148,7 @@ object IO extends IOInstances { def suspend[A](thunk: => IO[A]): IO[A] = { val nextIo = Suspend(() => thunk) if (isFullStackTracing) { - IOTracing.decorated(nextIo, TraceTag.Suspend) + IOTracing.decorated(nextIo, 2) } else { nextIo } @@ -1167,7 +1167,7 @@ object IO extends IOInstances { def pure[A](a: A): IO[A] = { val nextIo = Pure(a) if (isFullStackTracing) { - IOTracing.decorated(nextIo, TraceTag.Pure) + IOTracing.decorated(nextIo, 0) } else { nextIo } @@ -1238,9 +1238,9 @@ object IO extends IOInstances { */ def async[A](k: (Either[Throwable, A] => Unit) => Unit): IO[A] = { val trace = if (isCachedStackTracing) { - IOTracing.cached(TraceTag.Async, k.getClass) + IOTracing.cached(5, k.getClass) } else if (isFullStackTracing) { - IOTracing.uncached(TraceTag.Async) + IOTracing.uncached(5) } else { null } @@ -1278,9 +1278,9 @@ object IO extends IOInstances { */ def asyncF[A](k: (Either[Throwable, A] => Unit) => IO[Unit]): IO[A] = { val trace = if (isCachedStackTracing) { - IOTracing.cached(TraceTag.AsyncF, k.getClass) + IOTracing.cached(6, k.getClass) } else if (isFullStackTracing) { - IOTracing.uncached(TraceTag.AsyncF) + IOTracing.uncached(6) } else { null } @@ -1343,9 +1343,9 @@ object IO extends IOInstances { */ def cancelable[A](k: (Either[Throwable, A] => Unit) => CancelToken[IO]): IO[A] = { val trace = if (isCachedStackTracing) { - IOTracing.cached(TraceTag.Cancelable, k.getClass) + IOTracing.cached(7, k.getClass) } else if (isFullStackTracing) { - IOTracing.uncached(TraceTag.Cancelable) + IOTracing.uncached(7) } else { null } diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index 556dde2c95..6a6f548c17 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -20,20 +20,20 @@ import java.util.concurrent.ConcurrentHashMap import cats.effect.IO import cats.effect.IO.Trace -import cats.effect.tracing.{StackTraceFrame, TraceTag} +import cats.effect.tracing.StackTraceFrame private[effect] object IOTracing { - def decorated[A](source: IO[A], tag: TraceTag): IO[A] = + def decorated[A](source: IO[A], tag: Int): IO[A] = Trace(source, buildFrame(tag)) - def uncached(tag: TraceTag): StackTraceFrame = + def uncached(tag: Int): StackTraceFrame = buildFrame(tag) - def cached(tag: TraceTag, clazz: Class[_]): StackTraceFrame = + def cached(tag: Int, clazz: Class[_]): StackTraceFrame = buildCachedFrame(tag, clazz) - private def buildCachedFrame(tag: TraceTag, clazz: Class[_]): StackTraceFrame = { + private def buildCachedFrame(tag: Int, clazz: Class[_]): StackTraceFrame = { val cf = frameCache.get(clazz) if (cf eq null) { val f = buildFrame(tag) @@ -44,7 +44,7 @@ private[effect] object IOTracing { } } - def buildFrame(tag: TraceTag): StackTraceFrame = + def buildFrame(tag: Int): StackTraceFrame = StackTraceFrame(tag, new Throwable()) /** diff --git a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala index 2c57d17b24..e50cb07d3b 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala @@ -36,7 +36,7 @@ final case class IOTrace(frames: List[StackTraceFrame], captured: Int, omitted: case (acc, (f, index)) => val junc = if (index == frames.length - 1) TurnRight else Junction val first = f.stackTrace.dropWhile(l => stackTraceFilter.exists(b => l.getClassName.startsWith(b))).headOption - acc + s" $junc ${f.tag.name} at " + first.map(renderStackTraceElement).getOrElse("(...)") + "\n" + acc + s" $junc ${tagToName(f.tag)} at " + first.map(renderStackTraceElement).getOrElse("(...)") + "\n" } + "\n" acc1 @@ -72,7 +72,7 @@ final case class IOTrace(frames: List[StackTraceFrame], captured: Int, omitted: rest match { case k :: ks => { - val acc2 = if (init) InverseTurnRight + s" ${k.tag.name}\n" else Junction + s" ${k.tag.name}\n" + val acc2 = if (init) InverseTurnRight + s" ${tagToName(k.tag)}\n" else Junction + s" ${tagToName(k.tag)}\n" val innerLines = k.stackTrace .drop(stackTraceIgnoreLines) .take(maxStackTraceLines) @@ -105,9 +105,22 @@ private[effect] object IOTrace { "scala." ) - def demangleMethod(methodName: String): String = + private def demangleMethod(methodName: String): String = anonfunRegex.findFirstMatchIn(methodName) match { case Some(mat) => mat.group(1) case None => methodName } + + private def tagToName(tag: Int): String = + tag match { + case 0 => "pure" + case 1 => "delay" + case 2 => "suspend" + case 3 => "flatMap" + case 4 => "map" + case 5 => "async" + case 6 => "asyncF" + case 7 => "cancelable" + case _ => "???" + } } diff --git a/core/shared/src/main/scala/cats/effect/tracing/StackTraceFrame.scala b/core/shared/src/main/scala/cats/effect/tracing/StackTraceFrame.scala index 1a8c5e8791..e86742ea47 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/StackTraceFrame.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/StackTraceFrame.scala @@ -16,7 +16,7 @@ package cats.effect.tracing -final case class StackTraceFrame(tag: TraceTag, throwable: Throwable) { +final case class StackTraceFrame(tag: Int, throwable: Throwable) { def stackTrace: List[StackTraceElement] = throwable.getStackTrace().toList diff --git a/core/shared/src/main/scala/cats/effect/tracing/TraceTag.scala b/core/shared/src/main/scala/cats/effect/tracing/TraceTag.scala deleted file mode 100644 index b76e4c7a99..0000000000 --- a/core/shared/src/main/scala/cats/effect/tracing/TraceTag.scala +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers - * - * Licensed 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 cats.effect.tracing - -final case class TraceTag(name: String) - -object TraceTag { - val Pure = TraceTag("pure") - val Delay = TraceTag("delay") - val Suspend = TraceTag("suspend") - - val Bind = TraceTag("flatMap") - val Map = TraceTag("map") - - val Async = TraceTag("async") - val AsyncF = TraceTag("asyncF") - val Cancelable = TraceTag("cancelable") -} From 1a3a946c67fdbcc5a33e3becbf8b1a659d77c610 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Thu, 9 Jul 2020 23:02:32 -0500 Subject: [PATCH 67/78] docs --- core/shared/src/main/scala/cats/effect/IO.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index 4498cec78a..570c7f13eb 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -608,10 +608,8 @@ sealed abstract class IO[+A] extends internals.IOBinaryCompat[A] { * release, along with the result of `use` * (cancellation, error or successful result) */ - def bracketCase[B](use: A => IO[B])(release: (A, ExitCase[Throwable]) => IO[Unit]): IO[B] = { - val nextIo = IOBracket(this)(use)(release) - nextIo - } + def bracketCase[B](use: A => IO[B])(release: (A, ExitCase[Throwable]) => IO[Unit]): IO[B] = + IOBracket(this)(use)(release) /** * Executes the given `finalizer` when the source is finished, @@ -1633,6 +1631,9 @@ object IO extends IOInstances { /* -=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-=-= */ /* IO's internal encoding: */ + // In the Bind, Map, and Async constructors, you will notice that traces + // are typed as an `AnyRef`. This seems to avoid a performance hit when + // tracing is disabled, specifically with the C2 JIT compiler. /** Corresponds to [[IO.pure]]. */ final private[effect] case class Pure[+A](a: A) extends IO[A] From 9adfe4140abc8bcff23ac68f83be8debc586063d Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Fri, 10 Jul 2020 01:57:14 -0500 Subject: [PATCH 68/78] Add tracing on raiseError and docs update --- .../src/main/scala/cats/effect/IO.scala | 11 ++++- .../cats/effect/internals/IOContext.scala | 7 +-- .../cats/effect/internals/IORunLoop.scala | 47 ++++++++++--------- .../cats/effect/internals/IOTracing.scala | 17 +++---- .../scala/cats/effect/tracing/IOTrace.scala | 1 + .../effect/internals/IOContextTests.scala | 12 ++--- site/src/main/mdoc/guides/tracing.md | 20 +++++--- 7 files changed, 63 insertions(+), 52 deletions(-) diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index 570c7f13eb..03808182c6 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -1382,7 +1382,14 @@ object IO extends IOInstances { * * @see [[IO#attempt]] */ - def raiseError[A](e: Throwable): IO[A] = RaiseError(e) + def raiseError[A](e: Throwable): IO[A] = { + val nextIo = RaiseError(e) + if (isFullStackTracing) { + IOTracing.decorated(nextIo, 8) + } else { + nextIo + } + } /** * Constructs an `IO` which evaluates the given `Future` and @@ -1633,7 +1640,7 @@ object IO extends IOInstances { /* IO's internal encoding: */ // In the Bind, Map, and Async constructors, you will notice that traces // are typed as an `AnyRef`. This seems to avoid a performance hit when - // tracing is disabled, specifically with the C2 JIT compiler. + // tracing is disabled, particularly with the C2 JIT compiler. /** Corresponds to [[IO.pure]]. */ final private[effect] case class Pure[+A](a: A) extends IO[A] diff --git a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala index 8d3a2cf341..4c371b69d8 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala @@ -24,7 +24,7 @@ import cats.effect.internals.TracingPlatform.traceBufferSize * an IO and should be threaded across multiple invocations * of the run-loop associated with the same fiber. */ -final private[effect] class IOContext private () { +final private[effect] class IOContext() { private[this] val frames: RingBuffer[StackTraceFrame] = new RingBuffer(traceBufferSize) private[this] var captured: Int = 0 @@ -39,8 +39,3 @@ final private[effect] class IOContext private () { IOTrace(frames.toList, captured, omitted) } - -object IOContext { - def apply(): IOContext = - new IOContext -} diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index 3e0c8faa80..8c1b803e1b 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -82,15 +82,15 @@ private[effect] object IORunLoop { while ({ currentIO match { case bind @ Bind(fa, bindNext, _) => - if (bFirst ne null) { - if (bRest eq null) bRest = new ArrayStack() - bRest.push(bFirst) - } if (isStackTracing) { - if (ctx eq null) ctx = IOContext() + if (ctx eq null) ctx = new IOContext() val trace = bind.trace if (trace ne null) ctx.pushFrame(trace.asInstanceOf[StackTraceFrame]) } + if (bFirst ne null) { + if (bRest eq null) bRest = new ArrayStack() + bRest.push(bFirst) + } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa @@ -127,15 +127,15 @@ private[effect] object IORunLoop { } case bindNext @ Map(fa, _, _) => - if (bFirst ne null) { - if (bRest eq null) bRest = new ArrayStack() - bRest.push(bFirst) - } if (isStackTracing) { - if (ctx eq null) ctx = IOContext() + if (ctx eq null) ctx = new IOContext() val trace = bindNext.trace if (trace ne null) ctx.pushFrame(trace.asInstanceOf[StackTraceFrame]) } + if (bFirst ne null) { + if (bRest eq null) bRest = new ArrayStack() + bRest.push(bFirst) + } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa @@ -143,7 +143,7 @@ private[effect] object IORunLoop { if (conn eq null) conn = IOConnection() // We need to initialize an IOContext because the continuation // may produce trace frames e.g. IOBracket. - if (ctx eq null) ctx = IOContext() + if (ctx eq null) ctx = new IOContext() if (rcb eq null) rcb = new RestartCallback(conn, cb.asInstanceOf[Callback]) if (isStackTracing) { val trace = async.trace @@ -163,7 +163,7 @@ private[effect] object IORunLoop { } case Trace(source, frame) => - if (ctx eq null) ctx = IOContext() + if (ctx eq null) ctx = new IOContext() ctx.pushFrame(frame) currentIO = source } @@ -211,15 +211,15 @@ private[effect] object IORunLoop { while ({ currentIO match { case bind @ Bind(fa, bindNext, _) => - if (bFirst ne null) { - if (bRest eq null) bRest = new ArrayStack() - bRest.push(bFirst) - } if (isStackTracing) { - if (ctx eq null) ctx = IOContext() + if (ctx eq null) ctx = new IOContext() val trace = bind.trace if (trace ne null) ctx.pushFrame(trace.asInstanceOf[StackTraceFrame]) } + if (bFirst ne null) { + if (bRest eq null) bRest = new ArrayStack() + bRest.push(bFirst) + } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa @@ -256,20 +256,21 @@ private[effect] object IORunLoop { } case bindNext @ Map(fa, _, _) => - if (bFirst ne null) { - if (bRest eq null) bRest = new ArrayStack() - bRest.push(bFirst) - } if (isStackTracing) { - if (ctx eq null) ctx = IOContext() + if (ctx eq null) ctx = new IOContext() val trace = bindNext.trace if (trace ne null) ctx.pushFrame(trace.asInstanceOf[StackTraceFrame]) } + if (bFirst ne null) { + if (bRest eq null) bRest = new ArrayStack() + bRest.push(bFirst) + } bFirst = bindNext.asInstanceOf[Bind] currentIO = fa + if (ctx eq null) ctx = new IOContext() case Trace(source, frame) => - if (ctx eq null) ctx = IOContext() + if (ctx eq null) ctx = new IOContext() ctx.pushFrame(frame) currentIO = source diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index 6a6f548c17..3f83b46119 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -34,21 +34,22 @@ private[effect] object IOTracing { buildCachedFrame(tag, clazz) private def buildCachedFrame(tag: Int, clazz: Class[_]): StackTraceFrame = { - val cf = frameCache.get(clazz) - if (cf eq null) { - val f = buildFrame(tag) - frameCache.put(clazz, f) - f + val currentFrame = frameCache.get(clazz) + if (currentFrame eq null) { + val newFrame = buildFrame(tag) + frameCache.put(clazz, newFrame) + newFrame } else { - cf + currentFrame } } - def buildFrame(tag: Int): StackTraceFrame = + private def buildFrame(tag: Int): StackTraceFrame = StackTraceFrame(tag, new Throwable()) /** - * Cache for trace frames. Keys are references to lambda classes. + * Global cache for trace frames. Keys are references to lambda classes. + * Should converge to the working set of traces very quickly for hot code paths. */ private[this] val frameCache: ConcurrentHashMap[Class[_], StackTraceFrame] = new ConcurrentHashMap() diff --git a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala index e50cb07d3b..534fea75c8 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala @@ -121,6 +121,7 @@ private[effect] object IOTrace { case 5 => "async" case 6 => "asyncF" case 7 => "cancelable" + case 8 => "raiseError" case _ => "???" } } diff --git a/core/shared/src/test/scala/cats/effect/internals/IOContextTests.scala b/core/shared/src/test/scala/cats/effect/internals/IOContextTests.scala index c6c57b8f2c..6f2e3d34af 100644 --- a/core/shared/src/test/scala/cats/effect/internals/IOContextTests.scala +++ b/core/shared/src/test/scala/cats/effect/internals/IOContextTests.scala @@ -16,7 +16,7 @@ package cats.effect.internals -import cats.effect.tracing.{StackTraceFrame, TraceTag} +import cats.effect.tracing.StackTraceFrame import org.scalatest.matchers.should.Matchers import org.scalatest.funsuite.AnyFunSuite @@ -26,10 +26,10 @@ class IOContextTests extends AnyFunSuite with Matchers { val throwable = new Throwable() test("push traces") { - val ctx = IOContext() + val ctx = new IOContext() - val t1 = StackTraceFrame(TraceTag.Pure, throwable) - val t2 = StackTraceFrame(TraceTag.Suspend, throwable) + val t1 = StackTraceFrame(0, throwable) + val t2 = StackTraceFrame(1, throwable) ctx.pushFrame(t1) ctx.pushFrame(t2) @@ -41,10 +41,10 @@ class IOContextTests extends AnyFunSuite with Matchers { } test("track omitted frames") { - val ctx = IOContext() + val ctx = new IOContext() for (_ <- 0 until (traceBufferSize + 10)) { - ctx.pushFrame(StackTraceFrame(TraceTag.Pure, throwable)) + ctx.pushFrame(StackTraceFrame(0, throwable)) } val trace = ctx.trace() diff --git a/site/src/main/mdoc/guides/tracing.md b/site/src/main/mdoc/guides/tracing.md index eab410fcf7..7c44bf9c29 100644 --- a/site/src/main/mdoc/guides/tracing.md +++ b/site/src/main/mdoc/guides/tracing.md @@ -79,6 +79,11 @@ and trace information of all fibers in an application can be extracted for debugging purposes. 10. Monad transformer analysis. +As note of caution, fiber tracing generally introduces overhead to +applications in the form of higher CPU usage, memory and GC pressure. +Always remember to performance test your applications with tracing enabled +before deploying it to a production environment! + ## Asynchronous stack tracing ### Configuration The stack tracing mode of an application is configured by the system property @@ -89,7 +94,8 @@ To prevent unbounded memory usage, stack traces for a fiber are accumulated in an internal buffer as execution proceeds. If more traces are collected than the buffer can retain, then the older traces will be overwritten. The default size for the buffer is 128, but can be changed via the system property -`cats.effect.traceBufferSize`. +`cats.effect.traceBufferSize`. Keep in mind that the buffer size will always +be rounded up to a power of 2. For example, to enable full stack tracing and a trace buffer size of 1024, specify the following system properties: @@ -111,13 +117,13 @@ tracing may produce inaccurate fiber traces under several scenarios: 2. A named function is supplied to `map`, `async` or `flatMap` at multiple call-sites -When no collection is performed, we measured less than an 18% performance hit -for a completely synchronous `IO` program, so it will most likely be negligible -for any program that performs any sort of I/O. Nonetheless, we strongly -recommend benchmarking applications that enable tracing. +We measured less than a 30% performance hit when cached tracing is enabled +for a completely synchronous `IO` program, but it will most likely be much less +for any program that performs any sort of I/O. We strongly recommend +benchmarking applications that make use of tracing. -This is the recommended mode to run in production environments and is enabled -by default. +This is the recommended mode to run in most production applications and is +enabled by default. #### FULL When full stack tracing is enabled, a stack trace is captured for every From 895ccaa2facce583ced66cf932a1a26e2a348ee7 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Fri, 10 Jul 2020 02:06:15 -0500 Subject: [PATCH 69/78] doc updates --- site/src/main/mdoc/guides/tracing.md | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/site/src/main/mdoc/guides/tracing.md b/site/src/main/mdoc/guides/tracing.md index 7c44bf9c29..507f35312c 100644 --- a/site/src/main/mdoc/guides/tracing.md +++ b/site/src/main/mdoc/guides/tracing.md @@ -50,8 +50,9 @@ indicates that the feature has been merged into master. where stack frames are collected across asynchronous boundaries for a given fiber. 2. Combinator inference. Stack traces can be walked to determine what -combinator was actually called by user code. For example, `void` and -`as` are combinators that are derived from `map`. +combinator was actually called by user code. For example, `void` and `as` are +combinators that are derived from `map`, and should appear in the fiber trace +rather than `map`. 3. Intermediate values. The intermediate values that an `IO` program encounters can be converted to a string to render. This can aid in understanding the actions that a program takes. @@ -126,12 +127,13 @@ This is the recommended mode to run in most production applications and is enabled by default. #### FULL -When full stack tracing is enabled, a stack trace is captured for every -combinator traced in cached mode, but also `pure`, `delay`, `suspend` and other -derived combinators. +When full stack tracing is enabled, a stack trace is captured for most `IO` +combinators including `pure`, `delay`, `suspend`, `raiseError` as well as those +traced in cached mode. -This mode will incur a heavy performance hit for most programs, and is -recommended for use in development environments. +Stack traces are collected *on every invocation*, so naturally most programs +will experience a significant performance hit. This mode is mainly useful for +debugging in development environments. ### Requesting and rendering traces Once the global tracing flag is configured, `IO` programs will automatically From 46a51424c8ebdfbe9e65131d2a992027b1f3a2ff Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Fri, 10 Jul 2020 02:42:52 -0500 Subject: [PATCH 70/78] fix tracing tests --- .../cats/effect/FullStackTracingTests.scala | 38 +++++++++++-------- .../cats/effect/CachedStackTracingTests.scala | 19 ++++------ .../test/scala/cats/effect/TracingTests.scala | 37 +----------------- 3 files changed, 32 insertions(+), 62 deletions(-) diff --git a/tracing-tests/shared/src/fulltracing/scala/cats/effect/FullStackTracingTests.scala b/tracing-tests/shared/src/fulltracing/scala/cats/effect/FullStackTracingTests.scala index ff9e6216ed..4379a695a8 100644 --- a/tracing-tests/shared/src/fulltracing/scala/cats/effect/FullStackTracingTests.scala +++ b/tracing-tests/shared/src/fulltracing/scala/cats/effect/FullStackTracingTests.scala @@ -16,7 +16,7 @@ package cats.effect -import cats.effect.tracing.{IOTrace, TraceTag} +import cats.effect.tracing.IOTrace import org.scalatest.funsuite.AsyncFunSuite import org.scalatest.matchers.should.Matchers @@ -28,17 +28,14 @@ class FullStackTracingTests extends AsyncFunSuite with Matchers { implicit val cs: ContextShift[IO] = IO.contextShift(executionContext) def traced[A](io: IO[A]): IO[IOTrace] = - for { - _ <- io.traced - t <- IO.trace - } yield t + io.flatMap(_ => IO.trace) test("full stack tracing captures map frames") { val task = IO.pure(0).map(_ + 1).map(_ + 1) for (r <- traced(task).unsafeToFuture()) yield { - r.captured shouldBe 3 - r.frames.filter(_.tag == TraceTag.Map).length shouldBe 2 + r.captured shouldBe 4 + r.frames.filter(_.tag == 4).length shouldBe 2 } } @@ -46,8 +43,8 @@ class FullStackTracingTests extends AsyncFunSuite with Matchers { val task = IO.pure(0).flatMap(a => IO(a + 1)).flatMap(a => IO(a + 1)) for (r <- traced(task).unsafeToFuture()) yield { - r.captured shouldBe 5 - r.frames.filter(_.tag == TraceTag.Bind).length shouldBe 2 + r.captured shouldBe 6 + r.frames.filter(_.tag == 3).length shouldBe 3 // the extra one is used to capture the trace } } @@ -55,8 +52,8 @@ class FullStackTracingTests extends AsyncFunSuite with Matchers { val task = IO.async[Int](_(Right(0))).flatMap(a => IO(a + 1)).flatMap(a => IO(a + 1)) for (r <- traced(task).unsafeToFuture()) yield { - r.captured shouldBe 5 - r.frames.filter(_.tag == TraceTag.Async).length shouldBe 1 + r.captured shouldBe 6 + r.frames.filter(_.tag == 5).length shouldBe 1 } } @@ -64,8 +61,8 @@ class FullStackTracingTests extends AsyncFunSuite with Matchers { val task = IO.pure(0).flatMap(a => IO.pure(a + 1)) for (r <- traced(task).unsafeToFuture()) yield { - r.captured shouldBe 3 - r.frames.filter(_.tag == TraceTag.Pure).length shouldBe 2 + r.captured shouldBe 4 + r.frames.filter(_.tag == 0).length shouldBe 2 } } @@ -73,17 +70,26 @@ class FullStackTracingTests extends AsyncFunSuite with Matchers { val task = IO(0).flatMap(a => IO(a + 1)) for (r <- traced(task).unsafeToFuture()) yield { - r.captured shouldBe 3 - r.frames.filter(_.tag == TraceTag.Delay).length shouldBe 2 + r.captured shouldBe 4 + r.frames.filter(_.tag == 1).length shouldBe 2 } } test("full stack tracing captures suspend frames") { val task = IO.suspend(IO(1)).flatMap(a => IO.suspend(IO(a + 1))) + for (r <- traced(task).unsafeToFuture()) yield { + r.captured shouldBe 6 + r.frames.filter(_.tag == 2).length shouldBe 2 + } + } + + test("full stack tracing captures raiseError frames") { + val task = IO(0).flatMap(_ => IO.raiseError(new Throwable())).handleErrorWith(_ => IO.unit) + for (r <- traced(task).unsafeToFuture()) yield { r.captured shouldBe 5 - r.frames.filter(_.tag == TraceTag.Suspend).length shouldBe 2 + r.frames.filter(_.tag == 8).length shouldBe 1 } } } diff --git a/tracing-tests/shared/src/test/scala/cats/effect/CachedStackTracingTests.scala b/tracing-tests/shared/src/test/scala/cats/effect/CachedStackTracingTests.scala index b15e30afc3..e0c6462ad8 100644 --- a/tracing-tests/shared/src/test/scala/cats/effect/CachedStackTracingTests.scala +++ b/tracing-tests/shared/src/test/scala/cats/effect/CachedStackTracingTests.scala @@ -16,7 +16,7 @@ package cats.effect -import cats.effect.tracing.{IOTrace, TraceTag} +import cats.effect.tracing.IOTrace import org.scalatest.funsuite.AsyncFunSuite import org.scalatest.matchers.should.Matchers @@ -28,17 +28,14 @@ class CachedStackTracingTests extends AsyncFunSuite with Matchers { implicit val cs: ContextShift[IO] = IO.contextShift(executionContext) def traced[A](io: IO[A]): IO[IOTrace] = - for { - _ <- io.traced - t <- IO.trace - } yield t + io.flatMap(_ => IO.trace) test("cached stack tracing captures map frames") { val task = IO.pure(0).map(_ + 1).map(_ + 1) for (r <- traced(task).unsafeToFuture()) yield { - r.captured shouldBe 2 - r.frames.filter(_.tag == TraceTag.Map).length shouldBe 2 + r.captured shouldBe 3 + r.frames.filter(_.tag == 4).length shouldBe 2 } } @@ -46,8 +43,8 @@ class CachedStackTracingTests extends AsyncFunSuite with Matchers { val task = IO.pure(0).flatMap(a => IO(a + 1)).flatMap(a => IO(a + 1)) for (r <- traced(task).unsafeToFuture()) yield { - r.captured shouldBe 2 - r.frames.filter(_.tag == TraceTag.Bind).length shouldBe 2 + r.captured shouldBe 3 + r.frames.filter(_.tag == 3).length shouldBe 3 // extra one is to capture the trace } } @@ -55,8 +52,8 @@ class CachedStackTracingTests extends AsyncFunSuite with Matchers { val task = IO.async[Int](_(Right(0))).flatMap(a => IO(a + 1)).flatMap(a => IO(a + 1)) for (r <- traced(task).unsafeToFuture()) yield { - r.captured shouldBe 3 - r.frames.filter(_.tag == TraceTag.Async).length shouldBe 1 + r.captured shouldBe 4 + r.frames.filter(_.tag == 5).length shouldBe 1 } } } diff --git a/tracing-tests/shared/src/test/scala/cats/effect/TracingTests.scala b/tracing-tests/shared/src/test/scala/cats/effect/TracingTests.scala index 76d5758c9d..4518089439 100644 --- a/tracing-tests/shared/src/test/scala/cats/effect/TracingTests.scala +++ b/tracing-tests/shared/src/test/scala/cats/effect/TracingTests.scala @@ -28,22 +28,7 @@ class TracingTests extends AsyncFunSuite with Matchers { implicit val cs: ContextShift[IO] = IO.contextShift(executionContext) def traced[A](io: IO[A]): IO[IOTrace] = - for { - _ <- io.traced - t <- IO.trace - } yield t - - test("trace is empty when no traces are captured") { - val task = for { - _ <- IO.pure(1) - _ <- IO.pure(1) - t <- IO.trace - } yield t - - for (r <- task.unsafeToFuture()) yield { - r.captured shouldBe 0 - } - } + io.flatMap(_ => IO.trace) test("traces are preserved across asynchronous boundaries") { val task = for { @@ -53,25 +38,7 @@ class TracingTests extends AsyncFunSuite with Matchers { } yield a + b for (r <- traced(task).unsafeToFuture()) yield { - r.captured shouldBe 3 - } - } - - test("traces are emptied when initiating a new trace") { - val op = for { - a <- IO.pure(1) - b <- IO.pure(1) - } yield a + b - - val task = for { - _ <- op.traced - _ <- IO.trace - _ <- op.traced - t <- IO.trace - } yield t.frames.length - - for (r <- task.unsafeToFuture()) yield { - r shouldBe 2 + r.captured shouldBe 4 } } } From fa90a3ab0b8b28b5bff77238f894fc507ff464f2 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Fri, 10 Jul 2020 02:51:59 -0500 Subject: [PATCH 71/78] Only run tracing tests for JVM --- build.sbt | 24 ++++++------------------ 1 file changed, 6 insertions(+), 18 deletions(-) diff --git a/build.sbt b/build.sbt index e68fbf044b..64bc948260 100644 --- a/build.sbt +++ b/build.sbt @@ -226,7 +226,7 @@ lazy val sharedSourcesSettings = Seq( lazy val root = project .in(file(".")) .disablePlugins(MimaPlugin) - .aggregate(coreJVM, coreJS, lawsJVM, lawsJS, tracingTestsJVM, tracingTestsJS) + .aggregate(coreJVM, coreJS, lawsJVM, lawsJS, tracingTests) .settings(skipOnPublishSettings) lazy val core = crossProject(JSPlatform, JVMPlatform) @@ -293,12 +293,11 @@ lazy val lawsJS = laws.js lazy val FullTracingTest = config("fulltracing").extend(Test) -lazy val tracingTests = crossProject(JSPlatform, JVMPlatform) +lazy val tracingTests = project .in(file("tracing-tests")) - .dependsOn(core % "compile->compile;test->test") - .settings(commonSettings: _*) + .dependsOn(coreJVM) + .settings(commonSettings ++ skipOnPublishSettings ++ sharedSourcesSettings) .settings( - name := "cats-effect-tracing-tests", libraryDependencies ++= Seq( "org.typelevel" %%% "cats-laws" % CatsVersion, "org.typelevel" %%% "discipline-scalatest" % DisciplineScalatestVersion % Test @@ -306,19 +305,11 @@ lazy val tracingTests = crossProject(JSPlatform, JVMPlatform) ) .configs(FullTracingTest) .settings(inConfig(FullTracingTest)(Defaults.testSettings): _*) - .jsSettings(inConfig(FullTracingTest)(ScalaJSPlugin.testConfigSettings): _*) .settings( - test in Test := (test in Test).dependsOn(test in FullTracingTest).value, unmanagedSourceDirectories in FullTracingTest += { baseDirectory.value.getParentFile / "shared" / "src" / "fulltracing" / "scala" - } - ) - .jvmConfigure(_.enablePlugins(AutomateHeaderPlugin)) - .jvmConfigure(_.settings(lawsMimaSettings)) - .jsConfigure(_.enablePlugins(AutomateHeaderPlugin)) - .jsConfigure(_.settings(scalaJSSettings)) - .jvmSettings( - skip.in(publish) := customScalaJSVersion.forall(_.startsWith("1.0")), + }, + test in Test := (test in Test).dependsOn(test in FullTracingTest).value, fork in Test := true, fork in FullTracingTest := true, javaOptions in Test ++= Seq( @@ -331,9 +322,6 @@ lazy val tracingTests = crossProject(JSPlatform, JVMPlatform) ) ) -lazy val tracingTestsJVM = tracingTests.jvm -lazy val tracingTestsJS = tracingTests.js - lazy val benchmarksPrev = project .in(file("benchmarks/vPrev")) .settings(commonSettings ++ skipOnPublishSettings ++ sharedSourcesSettings) From 10bd15f715d0840b0c2fe7aa273adc6aa424fd66 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Fri, 10 Jul 2020 02:58:37 -0500 Subject: [PATCH 72/78] Reorganize test directories --- build.sbt | 4 ++-- .../fulltracing/scala/cats/effect/FullStackTracingTests.scala | 0 .../src/test/scala/cats/effect/CachedStackTracingTests.scala | 0 .../src/test/scala/cats/effect/TracingTests.scala | 0 4 files changed, 2 insertions(+), 2 deletions(-) rename tracing-tests/{shared => }/src/fulltracing/scala/cats/effect/FullStackTracingTests.scala (100%) rename tracing-tests/{shared => }/src/test/scala/cats/effect/CachedStackTracingTests.scala (100%) rename tracing-tests/{shared => }/src/test/scala/cats/effect/TracingTests.scala (100%) diff --git a/build.sbt b/build.sbt index 64bc948260..12c4bb6483 100644 --- a/build.sbt +++ b/build.sbt @@ -296,7 +296,7 @@ lazy val FullTracingTest = config("fulltracing").extend(Test) lazy val tracingTests = project .in(file("tracing-tests")) .dependsOn(coreJVM) - .settings(commonSettings ++ skipOnPublishSettings ++ sharedSourcesSettings) + .settings(commonSettings ++ skipOnPublishSettings) .settings( libraryDependencies ++= Seq( "org.typelevel" %%% "cats-laws" % CatsVersion, @@ -307,7 +307,7 @@ lazy val tracingTests = project .settings(inConfig(FullTracingTest)(Defaults.testSettings): _*) .settings( unmanagedSourceDirectories in FullTracingTest += { - baseDirectory.value.getParentFile / "shared" / "src" / "fulltracing" / "scala" + baseDirectory.value.getParentFile / "src" / "fulltracing" / "scala" }, test in Test := (test in Test).dependsOn(test in FullTracingTest).value, fork in Test := true, diff --git a/tracing-tests/shared/src/fulltracing/scala/cats/effect/FullStackTracingTests.scala b/tracing-tests/src/fulltracing/scala/cats/effect/FullStackTracingTests.scala similarity index 100% rename from tracing-tests/shared/src/fulltracing/scala/cats/effect/FullStackTracingTests.scala rename to tracing-tests/src/fulltracing/scala/cats/effect/FullStackTracingTests.scala diff --git a/tracing-tests/shared/src/test/scala/cats/effect/CachedStackTracingTests.scala b/tracing-tests/src/test/scala/cats/effect/CachedStackTracingTests.scala similarity index 100% rename from tracing-tests/shared/src/test/scala/cats/effect/CachedStackTracingTests.scala rename to tracing-tests/src/test/scala/cats/effect/CachedStackTracingTests.scala diff --git a/tracing-tests/shared/src/test/scala/cats/effect/TracingTests.scala b/tracing-tests/src/test/scala/cats/effect/TracingTests.scala similarity index 100% rename from tracing-tests/shared/src/test/scala/cats/effect/TracingTests.scala rename to tracing-tests/src/test/scala/cats/effect/TracingTests.scala From 9fe4ece5d5a64229bf1bd87cb78ec24fb64985dc Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Fri, 10 Jul 2020 03:12:22 -0500 Subject: [PATCH 73/78] Add mima filters --- build.sbt | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index 12c4bb6483..12d7f0ee97 100644 --- a/build.sbt +++ b/build.sbt @@ -181,7 +181,21 @@ val mimaSettings = Seq( exclude[IncompatibleSignatureProblem]("cats.effect.Resource.evalTap"), // change in encoding of value classes in generic methods https://github.com/lightbend/mima/issues/423 exclude[IncompatibleSignatureProblem]("cats.effect.Blocker.apply"), - exclude[IncompatibleSignatureProblem]("cats.effect.Blocker.fromExecutorService") + exclude[IncompatibleSignatureProblem]("cats.effect.Blocker.fromExecutorService"), + // Tracing + exclude[DirectMissingMethodProblem]("cats.effect.IO#Async.apply"), + exclude[DirectMissingMethodProblem]("cats.effect.IO#Bind.apply"), + exclude[IncompatibleResultTypeProblem]("cats.effect.IO#Async.k"), + exclude[DirectMissingMethodProblem]("cats.effect.IO#Async.copy"), + exclude[IncompatibleResultTypeProblem]("cats.effect.IO#Async.copy$default$1"), + exclude[DirectMissingMethodProblem]("cats.effect.IO#Async.this"), + exclude[DirectMissingMethodProblem]("cats.effect.IO#Bind.copy"), + exclude[DirectMissingMethodProblem]("cats.effect.IO#Bind.this"), + exclude[DirectMissingMethodProblem]("cats.effect.IO#Map.index"), + exclude[IncompatibleMethTypeProblem]("cats.effect.IO#Map.copy"), + exclude[IncompatibleResultTypeProblem]("cats.effect.IO#Map.copy$default$3"), + exclude[IncompatibleMethTypeProblem]("cats.effect.IO#Map.this"), + exclude[IncompatibleMethTypeProblem]("cats.effect.IO#Map.apply") ) } ) From cb29e7ed55ed0c9f717bbc8c50ab7c3a7cbb75d4 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Fri, 10 Jul 2020 03:31:35 -0500 Subject: [PATCH 74/78] Remove unused import in synciotests --- laws/shared/src/test/scala/cats/effect/SyncIOTests.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/laws/shared/src/test/scala/cats/effect/SyncIOTests.scala b/laws/shared/src/test/scala/cats/effect/SyncIOTests.scala index a67f345a19..43e5a59566 100644 --- a/laws/shared/src/test/scala/cats/effect/SyncIOTests.scala +++ b/laws/shared/src/test/scala/cats/effect/SyncIOTests.scala @@ -17,7 +17,6 @@ package cats package effect -import cats.effect.internals.IOPlatform import cats.effect.laws.discipline.SyncEffectTests import cats.effect.laws.discipline.arbitrary._ import cats.implicits._ From c22f73163e3d2815891c487aa1a683e488ef10af Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Sat, 11 Jul 2020 02:22:20 -0500 Subject: [PATCH 75/78] Refactor for IOEvent and better printers --- .../src/main/scala/cats/effect/IO.scala | 4 +- .../cats/effect/internals/IOContext.scala | 10 +- .../cats/effect/internals/IORunLoop.scala | 16 +-- .../cats/effect/internals/IOTracing.scala | 14 +-- .../cats/effect/internals/RingBuffer.scala | 1 + .../scala/cats/effect/tracing/IOEvent.scala | 28 +++++ .../scala/cats/effect/tracing/IOTrace.scala | 100 +++++++++--------- ...TraceFrame.scala => PrintingOptions.scala} | 9 +- .../effect/internals/IOContextTests.scala | 16 +-- site/src/main/mdoc/guides/tracing.md | 2 +- .../cats/effect/FullStackTracingTests.scala | 14 +-- .../cats/effect/CachedStackTracingTests.scala | 6 +- 12 files changed, 127 insertions(+), 93 deletions(-) create mode 100644 core/shared/src/main/scala/cats/effect/tracing/IOEvent.scala rename core/shared/src/main/scala/cats/effect/tracing/{StackTraceFrame.scala => PrintingOptions.scala} (81%) diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index 03808182c6..182fdc8a71 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -26,7 +26,7 @@ import scala.concurrent.{ExecutionContext, Future, Promise, TimeoutException} import scala.util.control.NonFatal import scala.util.{Failure, Left, Right, Success, Try} import cats.data.Ior -import cats.effect.tracing.{IOTrace, StackTraceFrame} +import cats.effect.tracing.{IOTrace, IOEvent} /** * A pure abstraction representing the intention to perform a @@ -1683,7 +1683,7 @@ object IO extends IOInstances { trace: AnyRef = null ) extends IO[A] - final private[effect] case class Trace[A](source: IO[A], trace: StackTraceFrame) extends IO[A] + final private[effect] case class Trace[A](source: IO[A], trace: IOEvent) extends IO[A] /** * An internal state for that optimizes changes to diff --git a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala index 4c371b69d8..1dd37b0753 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala @@ -16,7 +16,7 @@ package cats.effect.internals -import cats.effect.tracing.{IOTrace, StackTraceFrame} +import cats.effect.tracing.{IOTrace, IOEvent} import cats.effect.internals.TracingPlatform.traceBufferSize /** @@ -26,16 +26,16 @@ import cats.effect.internals.TracingPlatform.traceBufferSize */ final private[effect] class IOContext() { - private[this] val frames: RingBuffer[StackTraceFrame] = new RingBuffer(traceBufferSize) + private[this] val events: RingBuffer[IOEvent] = new RingBuffer(traceBufferSize) private[this] var captured: Int = 0 private[this] var omitted: Int = 0 - def pushFrame(fr: StackTraceFrame): Unit = { + def pushEvent(fr: IOEvent): Unit = { captured += 1 - if (frames.push(fr) != null) omitted += 1 + if (events.push(fr) != null) omitted += 1 } def trace(): IOTrace = - IOTrace(frames.toList, captured, omitted) + IOTrace(events.toList, captured, omitted) } diff --git a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala index 8c1b803e1b..582cf439dd 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IORunLoop.scala @@ -18,7 +18,7 @@ package cats.effect.internals import cats.effect.IO import cats.effect.IO.{Async, Bind, ContextSwitch, Delay, Map, Pure, RaiseError, Suspend, Trace} -import cats.effect.tracing.StackTraceFrame +import cats.effect.tracing.IOEvent import cats.effect.internals.TracingPlatform.isStackTracing import scala.util.control.NonFatal @@ -85,7 +85,7 @@ private[effect] object IORunLoop { if (isStackTracing) { if (ctx eq null) ctx = new IOContext() val trace = bind.trace - if (trace ne null) ctx.pushFrame(trace.asInstanceOf[StackTraceFrame]) + if (trace ne null) ctx.pushEvent(trace.asInstanceOf[IOEvent]) } if (bFirst ne null) { if (bRest eq null) bRest = new ArrayStack() @@ -130,7 +130,7 @@ private[effect] object IORunLoop { if (isStackTracing) { if (ctx eq null) ctx = new IOContext() val trace = bindNext.trace - if (trace ne null) ctx.pushFrame(trace.asInstanceOf[StackTraceFrame]) + if (trace ne null) ctx.pushEvent(trace.asInstanceOf[IOEvent]) } if (bFirst ne null) { if (bRest eq null) bRest = new ArrayStack() @@ -147,7 +147,7 @@ private[effect] object IORunLoop { if (rcb eq null) rcb = new RestartCallback(conn, cb.asInstanceOf[Callback]) if (isStackTracing) { val trace = async.trace - if (trace ne null) ctx.pushFrame(trace.asInstanceOf[StackTraceFrame]) + if (trace ne null) ctx.pushEvent(trace.asInstanceOf[IOEvent]) } rcb.start(async, ctx, bFirst, bRest) return @@ -164,7 +164,7 @@ private[effect] object IORunLoop { case Trace(source, frame) => if (ctx eq null) ctx = new IOContext() - ctx.pushFrame(frame) + ctx.pushEvent(frame) currentIO = source } @@ -214,7 +214,7 @@ private[effect] object IORunLoop { if (isStackTracing) { if (ctx eq null) ctx = new IOContext() val trace = bind.trace - if (trace ne null) ctx.pushFrame(trace.asInstanceOf[StackTraceFrame]) + if (trace ne null) ctx.pushEvent(trace.asInstanceOf[IOEvent]) } if (bFirst ne null) { if (bRest eq null) bRest = new ArrayStack() @@ -259,7 +259,7 @@ private[effect] object IORunLoop { if (isStackTracing) { if (ctx eq null) ctx = new IOContext() val trace = bindNext.trace - if (trace ne null) ctx.pushFrame(trace.asInstanceOf[StackTraceFrame]) + if (trace ne null) ctx.pushEvent(trace.asInstanceOf[IOEvent]) } if (bFirst ne null) { if (bRest eq null) bRest = new ArrayStack() @@ -271,7 +271,7 @@ private[effect] object IORunLoop { case Trace(source, frame) => if (ctx eq null) ctx = new IOContext() - ctx.pushFrame(frame) + ctx.pushEvent(frame) currentIO = source case _ => diff --git a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala index 3f83b46119..7a001e1b5a 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOTracing.scala @@ -20,20 +20,20 @@ import java.util.concurrent.ConcurrentHashMap import cats.effect.IO import cats.effect.IO.Trace -import cats.effect.tracing.StackTraceFrame +import cats.effect.tracing.IOEvent private[effect] object IOTracing { def decorated[A](source: IO[A], tag: Int): IO[A] = Trace(source, buildFrame(tag)) - def uncached(tag: Int): StackTraceFrame = + def uncached(tag: Int): IOEvent = buildFrame(tag) - def cached(tag: Int, clazz: Class[_]): StackTraceFrame = + def cached(tag: Int, clazz: Class[_]): IOEvent = buildCachedFrame(tag, clazz) - private def buildCachedFrame(tag: Int, clazz: Class[_]): StackTraceFrame = { + private def buildCachedFrame(tag: Int, clazz: Class[_]): IOEvent = { val currentFrame = frameCache.get(clazz) if (currentFrame eq null) { val newFrame = buildFrame(tag) @@ -44,13 +44,13 @@ private[effect] object IOTracing { } } - private def buildFrame(tag: Int): StackTraceFrame = - StackTraceFrame(tag, new Throwable()) + private def buildFrame(tag: Int): IOEvent = + IOEvent.StackTrace(tag, new Throwable()) /** * Global cache for trace frames. Keys are references to lambda classes. * Should converge to the working set of traces very quickly for hot code paths. */ - private[this] val frameCache: ConcurrentHashMap[Class[_], StackTraceFrame] = new ConcurrentHashMap() + private[this] val frameCache: ConcurrentHashMap[Class[_], IOEvent] = new ConcurrentHashMap() } diff --git a/core/shared/src/main/scala/cats/effect/internals/RingBuffer.scala b/core/shared/src/main/scala/cats/effect/internals/RingBuffer.scala index f4e518c717..26c18fdb05 100644 --- a/core/shared/src/main/scala/cats/effect/internals/RingBuffer.scala +++ b/core/shared/src/main/scala/cats/effect/internals/RingBuffer.scala @@ -25,6 +25,7 @@ final private[internals] class RingBuffer[A <: AnyRef](size: Int) { import RingBuffer._ + // These two probably don't need to be allocated every single time, maybe in Java? private[this] val length = nextPowerOfTwo(size) private[this] val mask = length - 1 diff --git a/core/shared/src/main/scala/cats/effect/tracing/IOEvent.scala b/core/shared/src/main/scala/cats/effect/tracing/IOEvent.scala new file mode 100644 index 0000000000..624e01432c --- /dev/null +++ b/core/shared/src/main/scala/cats/effect/tracing/IOEvent.scala @@ -0,0 +1,28 @@ +/* + * Copyright (c) 2017-2019 The Typelevel Cats-effect Project Developers + * + * Licensed 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 cats.effect.tracing + +sealed abstract class IOEvent + +object IOEvent { + + final case class StackTrace(tag: Int, throwable: Throwable) extends IOEvent { + def stackTrace: List[StackTraceElement] = + throwable.getStackTrace().toList + } + +} diff --git a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala index 534fea75c8..66c25d76fb 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala @@ -18,85 +18,81 @@ package cats.effect.tracing import cats.effect.IO -final case class IOTrace(frames: List[StackTraceFrame], captured: Int, omitted: Int) { +final case class IOTrace(events: List[IOEvent], captured: Int, omitted: Int) { import IOTrace._ - def compact(): String = { + def printFiberTrace(): IO[Unit] = { + IO(System.err.println(showFiberTrace())) + } + + def showFiberTrace(): String = { val TurnRight = "╰" val Junction = "├" - def renderStackTraceElement(ste: StackTraceElement): String = { - val methodName = demangleMethod(ste.getMethodName) - s"${ste.getClassName}.$methodName (${ste.getFileName}:${ste.getLineNumber})" - } - val acc0 = s"IOTrace: $captured frames captured, $omitted omitted\n" - val acc1 = frames.zipWithIndex.foldLeft(acc0) { - case (acc, (f, index)) => - val junc = if (index == frames.length - 1) TurnRight else Junction - val first = f.stackTrace.dropWhile(l => stackTraceFilter.exists(b => l.getClassName.startsWith(b))).headOption - acc + s" $junc ${tagToName(f.tag)} at " + first.map(renderStackTraceElement).getOrElse("(...)") + "\n" - } + "\n" - - acc1 - } - - def compactPrint(): IO[Unit] = - IO(System.err.println(compact)) + val acc1 = events.zipWithIndex.map { + case (event, index) => + val junc = if (index == events.length - 1) TurnRight else Junction + val message = event match { + case ev: IOEvent.StackTrace => { + val first = bestTraceElement(ev.stackTrace) + val nameTag = tagToName(ev.tag) + val codeLine = first.map(renderStackTraceElement).getOrElse("(...)") + s"$nameTag at $codeLine" + } + } + s" $junc $message" + }.mkString(acc0, "\n", "\n") - def pretty(maxStackTracesLines: Int = Int.MaxValue): String = { - val acc0 = s"IOTrace: $captured frames captured, $omitted omitted\n" - val acc1 = acc0 + loop("", 0, true, frames, maxStackTracesLines) acc1 } - def prettyPrint(maxStackTracesLines: Int = Int.MaxValue): IO[Unit] = - IO(System.err.println(pretty(maxStackTracesLines))) - - private def loop(acc: String, - indent: Int, - init: Boolean, - rest: List[StackTraceFrame], - maxStackTraceLines: Int): String = { + def showStackTraces(maxStackTraceLines: Int = Int.MaxValue): String = { val TurnRight = "╰" val InverseTurnRight = "╭" val Junction = "├" val Line = "│" - def renderStackTraceElement(ste: StackTraceElement, last: Boolean): String = { - val methodName = demangleMethod(ste.getMethodName) - val junc = if (last) TurnRight else Junction - Line + " " + junc + s" ${ste.getClassName}.$methodName (${ste.getFileName}:${ste.getLineNumber})\n" - } + val stackTraces = events.collect { case e: IOEvent.StackTrace => e } - rest match { - case k :: ks => { - val acc2 = if (init) InverseTurnRight + s" ${tagToName(k.tag)}\n" else Junction + s" ${tagToName(k.tag)}\n" - val innerLines = k.stackTrace + val header = s"IOTrace: $captured frames captured, $omitted omitted\n" + val body = stackTraces.zipWithIndex.map { + case (st, index) => + val nameTag = tagToName(st.tag) + val op = if (index == 0) s"$InverseTurnRight $nameTag\n" else s"$Junction $nameTag\n" + val relevantLines = st.stackTrace .drop(stackTraceIgnoreLines) .take(maxStackTraceLines) + val lines = relevantLines .zipWithIndex .map { - case (ste, i) => renderStackTraceElement(ste, i == k.stackTrace.length - 1) + case (ste, i) => + val junc = if (i == relevantLines.length - 1) TurnRight else Junction + val codeLine = renderStackTraceElement(ste) + s"$Line $junc $codeLine" } - .mkString + .mkString("", "\n", "\n") - loop(acc + acc2 + innerLines + Line + "\n", indent, false, ks, maxStackTraceLines) - } - case Nil => acc - } + s"$op$lines$Line" + }.mkString("\n") + + header + body } + def printStackTraces(maxStackTracesLines: Int = Int.MaxValue): IO[Unit] = + IO(System.err.println(showStackTraces(maxStackTracesLines))) + } private[effect] object IOTrace { - private val anonfunRegex = "^\\$+anonfun\\$+(.+)\\$+\\d+$".r // Number of lines to drop from the top of the stack trace - private val stackTraceIgnoreLines = 3 + def stackTraceIgnoreLines = 3 - private val stackTraceFilter = List( + private[this] val anonfunRegex = "^\\$+anonfun\\$+(.+)\\$+\\d+$".r + + private[this] val stackTraceFilter = List( "cats.effect.", "cats.", "sbt.", @@ -105,6 +101,14 @@ private[effect] object IOTrace { "scala." ) + private def renderStackTraceElement(ste: StackTraceElement): String = { + val methodName = demangleMethod(ste.getMethodName) + s"${ste.getClassName}.$methodName (${ste.getFileName}:${ste.getLineNumber})" + } + + private def bestTraceElement(frames: List[StackTraceElement]): Option[StackTraceElement] = + frames.dropWhile(l => stackTraceFilter.exists(b => l.getClassName.startsWith(b))).headOption + private def demangleMethod(methodName: String): String = anonfunRegex.findFirstMatchIn(methodName) match { case Some(mat) => mat.group(1) diff --git a/core/shared/src/main/scala/cats/effect/tracing/StackTraceFrame.scala b/core/shared/src/main/scala/cats/effect/tracing/PrintingOptions.scala similarity index 81% rename from core/shared/src/main/scala/cats/effect/tracing/StackTraceFrame.scala rename to core/shared/src/main/scala/cats/effect/tracing/PrintingOptions.scala index e86742ea47..127cd0dd16 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/StackTraceFrame.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/PrintingOptions.scala @@ -16,9 +16,10 @@ package cats.effect.tracing -final case class StackTraceFrame(tag: Int, throwable: Throwable) { - - def stackTrace: List[StackTraceElement] = - throwable.getStackTrace().toList +final case class PrintingOptions private () +object PrintingOptions { + val Default = PrintingOptions() } + + diff --git a/core/shared/src/test/scala/cats/effect/internals/IOContextTests.scala b/core/shared/src/test/scala/cats/effect/internals/IOContextTests.scala index 6f2e3d34af..5d1488dfb1 100644 --- a/core/shared/src/test/scala/cats/effect/internals/IOContextTests.scala +++ b/core/shared/src/test/scala/cats/effect/internals/IOContextTests.scala @@ -16,7 +16,7 @@ package cats.effect.internals -import cats.effect.tracing.StackTraceFrame +import cats.effect.tracing.IOEvent import org.scalatest.matchers.should.Matchers import org.scalatest.funsuite.AnyFunSuite @@ -28,14 +28,14 @@ class IOContextTests extends AnyFunSuite with Matchers { test("push traces") { val ctx = new IOContext() - val t1 = StackTraceFrame(0, throwable) - val t2 = StackTraceFrame(1, throwable) + val t1 = IOEvent(0, throwable) + val t2 = IOEvent(1, throwable) - ctx.pushFrame(t1) - ctx.pushFrame(t2) + ctx.pushEvent(t1) + ctx.pushEvent(t2) val trace = ctx.trace - trace.frames shouldBe List(t1, t2) + trace.events shouldBe List(t1, t2) trace.captured shouldBe 2 trace.omitted shouldBe 0 } @@ -44,11 +44,11 @@ class IOContextTests extends AnyFunSuite with Matchers { val ctx = new IOContext() for (_ <- 0 until (traceBufferSize + 10)) { - ctx.pushFrame(StackTraceFrame(0, throwable)) + ctx.pushEvent(IOEvent(0, throwable)) } val trace = ctx.trace() - trace.frames.length shouldBe traceBufferSize + trace.events.length shouldBe traceBufferSize trace.captured shouldBe (traceBufferSize + 10) trace.omitted shouldBe 10 } diff --git a/site/src/main/mdoc/guides/tracing.md b/site/src/main/mdoc/guides/tracing.md index 507f35312c..fe55cd2ec1 100644 --- a/site/src/main/mdoc/guides/tracing.md +++ b/site/src/main/mdoc/guides/tracing.md @@ -166,7 +166,7 @@ def program: IO[Unit] = _ <- IO(println("Started the program")) trace <- IO.trace _ <- trace.compactPrint - _ <- trace.prettyPrint() + _ <- trace.printStackTraces() } yield () ``` diff --git a/tracing-tests/src/fulltracing/scala/cats/effect/FullStackTracingTests.scala b/tracing-tests/src/fulltracing/scala/cats/effect/FullStackTracingTests.scala index 4379a695a8..f1717b4458 100644 --- a/tracing-tests/src/fulltracing/scala/cats/effect/FullStackTracingTests.scala +++ b/tracing-tests/src/fulltracing/scala/cats/effect/FullStackTracingTests.scala @@ -35,7 +35,7 @@ class FullStackTracingTests extends AsyncFunSuite with Matchers { for (r <- traced(task).unsafeToFuture()) yield { r.captured shouldBe 4 - r.frames.filter(_.tag == 4).length shouldBe 2 + r.events.filter(_.tag == 4).length shouldBe 2 } } @@ -44,7 +44,7 @@ class FullStackTracingTests extends AsyncFunSuite with Matchers { for (r <- traced(task).unsafeToFuture()) yield { r.captured shouldBe 6 - r.frames.filter(_.tag == 3).length shouldBe 3 // the extra one is used to capture the trace + r.events.filter(_.tag == 3).length shouldBe 3 // the extra one is used to capture the trace } } @@ -53,7 +53,7 @@ class FullStackTracingTests extends AsyncFunSuite with Matchers { for (r <- traced(task).unsafeToFuture()) yield { r.captured shouldBe 6 - r.frames.filter(_.tag == 5).length shouldBe 1 + r.events.filter(_.tag == 5).length shouldBe 1 } } @@ -62,7 +62,7 @@ class FullStackTracingTests extends AsyncFunSuite with Matchers { for (r <- traced(task).unsafeToFuture()) yield { r.captured shouldBe 4 - r.frames.filter(_.tag == 0).length shouldBe 2 + r.events.filter(_.tag == 0).length shouldBe 2 } } @@ -71,7 +71,7 @@ class FullStackTracingTests extends AsyncFunSuite with Matchers { for (r <- traced(task).unsafeToFuture()) yield { r.captured shouldBe 4 - r.frames.filter(_.tag == 1).length shouldBe 2 + r.events.filter(_.tag == 1).length shouldBe 2 } } @@ -80,7 +80,7 @@ class FullStackTracingTests extends AsyncFunSuite with Matchers { for (r <- traced(task).unsafeToFuture()) yield { r.captured shouldBe 6 - r.frames.filter(_.tag == 2).length shouldBe 2 + r.events.filter(_.tag == 2).length shouldBe 2 } } @@ -89,7 +89,7 @@ class FullStackTracingTests extends AsyncFunSuite with Matchers { for (r <- traced(task).unsafeToFuture()) yield { r.captured shouldBe 5 - r.frames.filter(_.tag == 8).length shouldBe 1 + r.events.filter(_.tag == 8).length shouldBe 1 } } } diff --git a/tracing-tests/src/test/scala/cats/effect/CachedStackTracingTests.scala b/tracing-tests/src/test/scala/cats/effect/CachedStackTracingTests.scala index e0c6462ad8..02cb87fea1 100644 --- a/tracing-tests/src/test/scala/cats/effect/CachedStackTracingTests.scala +++ b/tracing-tests/src/test/scala/cats/effect/CachedStackTracingTests.scala @@ -35,7 +35,7 @@ class CachedStackTracingTests extends AsyncFunSuite with Matchers { for (r <- traced(task).unsafeToFuture()) yield { r.captured shouldBe 3 - r.frames.filter(_.tag == 4).length shouldBe 2 + r.events.filter(_.tag == 4).length shouldBe 2 } } @@ -44,7 +44,7 @@ class CachedStackTracingTests extends AsyncFunSuite with Matchers { for (r <- traced(task).unsafeToFuture()) yield { r.captured shouldBe 3 - r.frames.filter(_.tag == 3).length shouldBe 3 // extra one is to capture the trace + r.events.filter(_.tag == 3).length shouldBe 3 // extra one is to capture the trace } } @@ -53,7 +53,7 @@ class CachedStackTracingTests extends AsyncFunSuite with Matchers { for (r <- traced(task).unsafeToFuture()) yield { r.captured shouldBe 4 - r.frames.filter(_.tag == 5).length shouldBe 1 + r.events.filter(_.tag == 5).length shouldBe 1 } } } From b06b10995ffa4e4044e48c55089d82e2a0390f4d Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Sat, 11 Jul 2020 02:55:43 -0500 Subject: [PATCH 76/78] Printing options --- build.sbt | 2 +- .../src/main/scala/cats/effect/IO.scala | 2 +- .../cats/effect/internals/IOContext.scala | 2 +- .../scala/cats/effect/tracing/IOTrace.scala | 103 +++++++++--------- .../cats/effect/tracing/PrintingOptions.scala | 28 ++++- site/src/main/mdoc/guides/tracing.md | 35 +++--- .../cats/effect/FullStackTracingTests.scala | 16 +-- .../cats/effect/CachedStackTracingTests.scala | 11 +- 8 files changed, 106 insertions(+), 93 deletions(-) diff --git a/build.sbt b/build.sbt index 12d7f0ee97..6c7611566b 100644 --- a/build.sbt +++ b/build.sbt @@ -182,7 +182,7 @@ val mimaSettings = Seq( // change in encoding of value classes in generic methods https://github.com/lightbend/mima/issues/423 exclude[IncompatibleSignatureProblem]("cats.effect.Blocker.apply"), exclude[IncompatibleSignatureProblem]("cats.effect.Blocker.fromExecutorService"), - // Tracing + // Tracing - https://github.com/typelevel/cats-effect/pull/854 exclude[DirectMissingMethodProblem]("cats.effect.IO#Async.apply"), exclude[DirectMissingMethodProblem]("cats.effect.IO#Bind.apply"), exclude[IncompatibleResultTypeProblem]("cats.effect.IO#Async.k"), diff --git a/core/shared/src/main/scala/cats/effect/IO.scala b/core/shared/src/main/scala/cats/effect/IO.scala index 182fdc8a71..f534da9f62 100644 --- a/core/shared/src/main/scala/cats/effect/IO.scala +++ b/core/shared/src/main/scala/cats/effect/IO.scala @@ -26,7 +26,7 @@ import scala.concurrent.{ExecutionContext, Future, Promise, TimeoutException} import scala.util.control.NonFatal import scala.util.{Failure, Left, Right, Success, Try} import cats.data.Ior -import cats.effect.tracing.{IOTrace, IOEvent} +import cats.effect.tracing.{IOEvent, IOTrace} /** * A pure abstraction representing the intention to perform a diff --git a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala index 1dd37b0753..cdfbc129c5 100644 --- a/core/shared/src/main/scala/cats/effect/internals/IOContext.scala +++ b/core/shared/src/main/scala/cats/effect/internals/IOContext.scala @@ -16,7 +16,7 @@ package cats.effect.internals -import cats.effect.tracing.{IOTrace, IOEvent} +import cats.effect.tracing.{IOEvent, IOTrace} import cats.effect.internals.TracingPlatform.traceBufferSize /** diff --git a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala index 66c25d76fb..c86f7dd2bf 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/IOTrace.scala @@ -22,67 +22,62 @@ final case class IOTrace(events: List[IOEvent], captured: Int, omitted: Int) { import IOTrace._ - def printFiberTrace(): IO[Unit] = { - IO(System.err.println(showFiberTrace())) - } - - def showFiberTrace(): String = { - val TurnRight = "╰" - val Junction = "├" - - val acc0 = s"IOTrace: $captured frames captured, $omitted omitted\n" - val acc1 = events.zipWithIndex.map { - case (event, index) => - val junc = if (index == events.length - 1) TurnRight else Junction - val message = event match { - case ev: IOEvent.StackTrace => { - val first = bestTraceElement(ev.stackTrace) - val nameTag = tagToName(ev.tag) - val codeLine = first.map(renderStackTraceElement).getOrElse("(...)") - s"$nameTag at $codeLine" - } - } - s" $junc $message" - }.mkString(acc0, "\n", "\n") - - acc1 - } + def printFiberTrace(options: PrintingOptions = PrintingOptions.Default): IO[Unit] = + IO(System.err.println(showFiberTrace(options))) - def showStackTraces(maxStackTraceLines: Int = Int.MaxValue): String = { + def showFiberTrace(options: PrintingOptions = PrintingOptions.Default): String = { val TurnRight = "╰" val InverseTurnRight = "╭" val Junction = "├" val Line = "│" - val stackTraces = events.collect { case e: IOEvent.StackTrace => e } - - val header = s"IOTrace: $captured frames captured, $omitted omitted\n" - val body = stackTraces.zipWithIndex.map { - case (st, index) => - val nameTag = tagToName(st.tag) - val op = if (index == 0) s"$InverseTurnRight $nameTag\n" else s"$Junction $nameTag\n" - val relevantLines = st.stackTrace - .drop(stackTraceIgnoreLines) - .take(maxStackTraceLines) - val lines = relevantLines - .zipWithIndex - .map { - case (ste, i) => - val junc = if (i == relevantLines.length - 1) TurnRight else Junction - val codeLine = renderStackTraceElement(ste) - s"$Line $junc $codeLine" - } - .mkString("", "\n", "\n") - - s"$op$lines$Line" - }.mkString("\n") - - header + body - } - - def printStackTraces(maxStackTracesLines: Int = Int.MaxValue): IO[Unit] = - IO(System.err.println(showStackTraces(maxStackTracesLines))) + if (options.showFullStackTraces) { + val stackTraces = events.collect { case e: IOEvent.StackTrace => e } + + val header = s"IOTrace: $captured frames captured, $omitted omitted\n" + val body = stackTraces.zipWithIndex + .map { + case (st, index) => + val nameTag = tagToName(st.tag) + val op = if (index == 0) s"$InverseTurnRight $nameTag\n" else s"$Junction $nameTag\n" + val relevantLines = st.stackTrace + .drop(options.ignoreStackTraceLines) + .take(options.maxStackTraceLines) + val lines = relevantLines.zipWithIndex + .map { + case (ste, i) => + val junc = if (i == relevantLines.length - 1) TurnRight else Junction + val codeLine = renderStackTraceElement(ste) + s"$Line $junc $codeLine" + } + .mkString("", "\n", "\n") + + s"$op$lines$Line" + } + .mkString("\n") + + header + body + } else { + val acc0 = s"IOTrace: $captured frames captured, $omitted omitted\n" + val acc1 = events.zipWithIndex + .map { + case (event, index) => + val junc = if (index == events.length - 1) TurnRight else Junction + val message = event match { + case ev: IOEvent.StackTrace => { + val first = bestTraceElement(ev.stackTrace) + val nameTag = tagToName(ev.tag) + val codeLine = first.map(renderStackTraceElement).getOrElse("(...)") + s"$nameTag at $codeLine" + } + } + s" $junc $message" + } + .mkString(acc0, "\n", "\n") + acc1 + } + } } private[effect] object IOTrace { diff --git a/core/shared/src/main/scala/cats/effect/tracing/PrintingOptions.scala b/core/shared/src/main/scala/cats/effect/tracing/PrintingOptions.scala index 127cd0dd16..8686384a5f 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/PrintingOptions.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/PrintingOptions.scala @@ -16,10 +16,30 @@ package cats.effect.tracing -final case class PrintingOptions private () +/** + * @param showFullStackTraces Whether or not to show the entire stack trace + * @param maxStackTraceLines When `showFullStackTraces` is `true`, the maximum number of stack trace + * elements to print + * @param ignoreStackTraceLines When `showFullStackTraces` is `true`, the number of stack trace elements + * to ignore from the start + */ +final case class PrintingOptions private (showFullStackTraces: Boolean, + maxStackTraceLines: Int, + ignoreStackTraceLines: Int) { + def withShowFullStackTraces(showFullStackTraces: Boolean): PrintingOptions = + copy(showFullStackTraces = showFullStackTraces) -object PrintingOptions { - val Default = PrintingOptions() -} + def withMaxStackTraceLines(maxStackTraceLines: Int): PrintingOptions = + copy(maxStackTraceLines = maxStackTraceLines) + def withIgnoreStackTraceLines(ignoreStackTraceLines: Int): PrintingOptions = + copy(ignoreStackTraceLines = ignoreStackTraceLines) +} +object PrintingOptions { + val Default = PrintingOptions( + showFullStackTraces = false, + maxStackTraceLines = Int.MaxValue, + ignoreStackTraceLines = 3 // the number of frames to ignore because of IOTracing + ) +} diff --git a/site/src/main/mdoc/guides/tracing.md b/site/src/main/mdoc/guides/tracing.md index fe55cd2ec1..52ae266b89 100644 --- a/site/src/main/mdoc/guides/tracing.md +++ b/site/src/main/mdoc/guides/tracing.md @@ -135,28 +135,18 @@ Stack traces are collected *on every invocation*, so naturally most programs will experience a significant performance hit. This mode is mainly useful for debugging in development environments. -### Requesting and rendering traces +### Requesting and printing traces Once the global tracing flag is configured, `IO` programs will automatically begin collecting traces. The trace for a fiber can be accessed at any point during its execution via the `IO.trace` combinator. This is the `IO` equivalent of capturing a thread's stack trace. -```scala -import cats.effect.IO - -def program: IO[Unit] = - for { - _ <- IO(println("Started the program")) - trace <- IO.trace - } yield () -``` - -After a fiber trace is retrieved, we can print it to the console, just like how -exception stack traces can be printed with `printStackTrace`. `compactPrint` -includes the most relevant stack trace element for each fiber operation that -was performed. `prettyPrint` includes the entire stack trace for each fiber -operation. These methods accept arguments that lets us customize how traces -are printed. +After we have a fiber trace, we can print it to the console, not unlike how +Java exception stack traces are printed with `printStackTrace`. `printFiberTrace` +can be called to print fiber traces to the consoles. Printing behavior can be +customized by passing in a `PrintingOptions` instance. By default, a fiber trace +is rendered in a very compact presentation that includes the most relevant stack +trace element from each fiber operation. ```scala import cats.effect.IO @@ -165,8 +155,7 @@ def program: IO[Unit] = for { _ <- IO(println("Started the program")) trace <- IO.trace - _ <- trace.compactPrint - _ <- trace.printStackTraces() + _ <- trace.printFiberTrace() } yield () ``` @@ -180,12 +169,18 @@ Here is a sample program that demonstrates tracing in action. // Pass the following system property to your JVM: // -Dcats.effect.stackTracingMode=full +import cats.effect.tracing.PrintingOptions import cats.implicits._ import cats.effect.{ExitCode, IO, IOApp} + import scala.util.Random object Example extends IOApp { + val options = PrintingOptions.Default + .withShowFullStackTraces(true) + .withMaxStackTraceLines(8) + def fib(n: Int, a: Long = 0, b: Long = 1): IO[Long] = IO(a + b).flatMap { b2 => if (n > 0) @@ -203,7 +198,7 @@ object Example extends IOApp { override def run(args: List[String]): IO[ExitCode] = for { - _ <- program.handleErrorWith(_ => IO.trace.flatMap(_.compactPrint)) + _ <- program.handleErrorWith(_ => IO.trace.flatMap(_.printFiberTrace(options))) } yield ExitCode.Success } diff --git a/tracing-tests/src/fulltracing/scala/cats/effect/FullStackTracingTests.scala b/tracing-tests/src/fulltracing/scala/cats/effect/FullStackTracingTests.scala index f1717b4458..b882a12475 100644 --- a/tracing-tests/src/fulltracing/scala/cats/effect/FullStackTracingTests.scala +++ b/tracing-tests/src/fulltracing/scala/cats/effect/FullStackTracingTests.scala @@ -16,7 +16,7 @@ package cats.effect -import cats.effect.tracing.IOTrace +import cats.effect.tracing.{IOEvent, IOTrace} import org.scalatest.funsuite.AsyncFunSuite import org.scalatest.matchers.should.Matchers @@ -35,7 +35,7 @@ class FullStackTracingTests extends AsyncFunSuite with Matchers { for (r <- traced(task).unsafeToFuture()) yield { r.captured shouldBe 4 - r.events.filter(_.tag == 4).length shouldBe 2 + r.events.collect { case e: IOEvent.StackTrace => e }.filter(_.tag == 4).length shouldBe 2 } } @@ -44,7 +44,7 @@ class FullStackTracingTests extends AsyncFunSuite with Matchers { for (r <- traced(task).unsafeToFuture()) yield { r.captured shouldBe 6 - r.events.filter(_.tag == 3).length shouldBe 3 // the extra one is used to capture the trace + r.events.collect { case e: IOEvent.StackTrace => e }.filter(_.tag == 3).length shouldBe 3 // the extra one is used to capture the trace } } @@ -53,7 +53,7 @@ class FullStackTracingTests extends AsyncFunSuite with Matchers { for (r <- traced(task).unsafeToFuture()) yield { r.captured shouldBe 6 - r.events.filter(_.tag == 5).length shouldBe 1 + r.events.collect { case e: IOEvent.StackTrace => e }.filter(_.tag == 5).length shouldBe 1 } } @@ -62,7 +62,7 @@ class FullStackTracingTests extends AsyncFunSuite with Matchers { for (r <- traced(task).unsafeToFuture()) yield { r.captured shouldBe 4 - r.events.filter(_.tag == 0).length shouldBe 2 + r.events.collect { case e: IOEvent.StackTrace => e }.filter(_.tag == 0).length shouldBe 2 } } @@ -71,7 +71,7 @@ class FullStackTracingTests extends AsyncFunSuite with Matchers { for (r <- traced(task).unsafeToFuture()) yield { r.captured shouldBe 4 - r.events.filter(_.tag == 1).length shouldBe 2 + r.events.collect { case e: IOEvent.StackTrace => e }.filter(_.tag == 1).length shouldBe 2 } } @@ -80,7 +80,7 @@ class FullStackTracingTests extends AsyncFunSuite with Matchers { for (r <- traced(task).unsafeToFuture()) yield { r.captured shouldBe 6 - r.events.filter(_.tag == 2).length shouldBe 2 + r.events.collect { case e: IOEvent.StackTrace => e }.filter(_.tag == 2).length shouldBe 2 } } @@ -89,7 +89,7 @@ class FullStackTracingTests extends AsyncFunSuite with Matchers { for (r <- traced(task).unsafeToFuture()) yield { r.captured shouldBe 5 - r.events.filter(_.tag == 8).length shouldBe 1 + r.events.collect { case e: IOEvent.StackTrace => e }.filter(_.tag == 8).length shouldBe 1 } } } diff --git a/tracing-tests/src/test/scala/cats/effect/CachedStackTracingTests.scala b/tracing-tests/src/test/scala/cats/effect/CachedStackTracingTests.scala index 02cb87fea1..9938203663 100644 --- a/tracing-tests/src/test/scala/cats/effect/CachedStackTracingTests.scala +++ b/tracing-tests/src/test/scala/cats/effect/CachedStackTracingTests.scala @@ -16,7 +16,7 @@ package cats.effect -import cats.effect.tracing.IOTrace +import cats.effect.tracing.{IOEvent, IOTrace} import org.scalatest.funsuite.AsyncFunSuite import org.scalatest.matchers.should.Matchers @@ -35,7 +35,7 @@ class CachedStackTracingTests extends AsyncFunSuite with Matchers { for (r <- traced(task).unsafeToFuture()) yield { r.captured shouldBe 3 - r.events.filter(_.tag == 4).length shouldBe 2 + r.events.collect { case e: IOEvent.StackTrace => e }.filter(_.tag == 4).length shouldBe 2 } } @@ -44,7 +44,10 @@ class CachedStackTracingTests extends AsyncFunSuite with Matchers { for (r <- traced(task).unsafeToFuture()) yield { r.captured shouldBe 3 - r.events.filter(_.tag == 3).length shouldBe 3 // extra one is to capture the trace + r.events + .collect { case e: IOEvent.StackTrace => e } + .filter(_.tag == 3) + .length shouldBe 3 // extra one is to capture the trace } } @@ -53,7 +56,7 @@ class CachedStackTracingTests extends AsyncFunSuite with Matchers { for (r <- traced(task).unsafeToFuture()) yield { r.captured shouldBe 4 - r.events.filter(_.tag == 5).length shouldBe 1 + r.events.collect { case e: IOEvent.StackTrace => e }.filter(_.tag == 5).length shouldBe 1 } } } From 42aef250ede590a266f588494618210b1b726a18 Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Sat, 11 Jul 2020 13:40:40 -0500 Subject: [PATCH 77/78] Fix tests --- .../test/scala/cats/effect/internals/IOContextTests.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/shared/src/test/scala/cats/effect/internals/IOContextTests.scala b/core/shared/src/test/scala/cats/effect/internals/IOContextTests.scala index 5d1488dfb1..62b4073ad0 100644 --- a/core/shared/src/test/scala/cats/effect/internals/IOContextTests.scala +++ b/core/shared/src/test/scala/cats/effect/internals/IOContextTests.scala @@ -28,8 +28,8 @@ class IOContextTests extends AnyFunSuite with Matchers { test("push traces") { val ctx = new IOContext() - val t1 = IOEvent(0, throwable) - val t2 = IOEvent(1, throwable) + val t1 = IOEvent.StackTrace(0, throwable) + val t2 = IOEvent.StackTrace(1, throwable) ctx.pushEvent(t1) ctx.pushEvent(t2) @@ -44,7 +44,7 @@ class IOContextTests extends AnyFunSuite with Matchers { val ctx = new IOContext() for (_ <- 0 until (traceBufferSize + 10)) { - ctx.pushEvent(IOEvent(0, throwable)) + ctx.pushEvent(IOEvent.StackTrace(0, throwable)) } val trace = ctx.trace() From 7ba1b112106c96a7175793ef9c80ec6514762adb Mon Sep 17 00:00:00 2001 From: Raas Ahsan Date: Sat, 11 Jul 2020 13:55:37 -0500 Subject: [PATCH 78/78] fix build --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index 3c3ccac911..e14f0a6816 100644 --- a/build.sbt +++ b/build.sbt @@ -195,7 +195,7 @@ val mimaSettings = Seq( exclude[IncompatibleMethTypeProblem]("cats.effect.IO#Map.copy"), exclude[IncompatibleResultTypeProblem]("cats.effect.IO#Map.copy$default$3"), exclude[IncompatibleMethTypeProblem]("cats.effect.IO#Map.this"), - exclude[IncompatibleMethTypeProblem]("cats.effect.IO#Map.apply") + exclude[IncompatibleMethTypeProblem]("cats.effect.IO#Map.apply"), // revise Deferred, MVarConcurrent, LinkedLongMap - https://github.com/typelevel/cats-effect/pull/918 exclude[IncompatibleResultTypeProblem]("cats.effect.concurrent.Deferred#State#Unset.waiting"), exclude[DirectMissingMethodProblem]("cats.effect.concurrent.Deferred#State#Unset.copy"),