aboutsummaryrefslogtreecommitdiff
path: root/kamon-core/src/test/scala/kamon
diff options
context:
space:
mode:
Diffstat (limited to 'kamon-core/src/test/scala/kamon')
-rw-r--r--kamon-core/src/test/scala/kamon/instrumentation/akka/ActorSystemMessageInstrumentationSpec.scala8
-rw-r--r--kamon-core/src/test/scala/kamon/instrumentation/akka/AskPatternInstrumentationSpec.scala1
-rw-r--r--kamon-core/src/test/scala/kamon/instrumentation/akka/RemotingInstrumentationSpec.scala171
-rw-r--r--kamon-core/src/test/scala/kamon/metric/ActorMetricsSpec.scala2
-rw-r--r--kamon-core/src/test/scala/kamon/metric/RouterMetricsSpec.scala8
-rw-r--r--kamon-core/src/test/scala/kamon/metric/TraceMetricsSpec.scala21
-rw-r--r--kamon-core/src/test/scala/kamon/metric/instrument/HistogramSpec.scala7
-rw-r--r--kamon-core/src/test/scala/kamon/trace/TraceContextManipulationSpec.scala24
8 files changed, 36 insertions, 206 deletions
diff --git a/kamon-core/src/test/scala/kamon/instrumentation/akka/ActorSystemMessageInstrumentationSpec.scala b/kamon-core/src/test/scala/kamon/instrumentation/akka/ActorSystemMessageInstrumentationSpec.scala
index 47867c55..d79ccbe0 100644
--- a/kamon-core/src/test/scala/kamon/instrumentation/akka/ActorSystemMessageInstrumentationSpec.scala
+++ b/kamon-core/src/test/scala/kamon/instrumentation/akka/ActorSystemMessageInstrumentationSpec.scala
@@ -3,7 +3,7 @@ package kamon.instrumentation.akka
import akka.actor.SupervisorStrategy.{ Escalate, Restart, Resume, Stop }
import akka.actor._
import akka.testkit.{ ImplicitSender, TestKit }
-import kamon.trace.TraceRecorder
+import kamon.trace.{ EmptyTraceContext, TraceRecorder }
import org.scalatest.WordSpecLike
import scala.concurrent.duration._
@@ -59,7 +59,7 @@ class ActorSystemMessageInstrumentationSpec extends TestKit(ActorSystem("actor-s
// Ensure we didn't tie the actor with the context
supervisor ! "context"
- expectMsg(None)
+ expectMsg(EmptyTraceContext)
}
"the actor is restarted" in {
@@ -76,7 +76,7 @@ class ActorSystemMessageInstrumentationSpec extends TestKit(ActorSystem("actor-s
// Ensure we didn't tie the actor with the context
supervisor ! "context"
- expectMsg(None)
+ expectMsg(EmptyTraceContext)
}
"the actor is stopped" in {
@@ -142,7 +142,7 @@ class ActorSystemMessageInstrumentationSpec extends TestKit(ActorSystem("actor-s
class Child extends Actor {
def receive = {
- case "fail" ⇒ 1 / 0
+ case "fail" ⇒ throw new ArithmeticException("Division by zero.")
case "context" ⇒ sender ! TraceRecorder.currentContext
}
diff --git a/kamon-core/src/test/scala/kamon/instrumentation/akka/AskPatternInstrumentationSpec.scala b/kamon-core/src/test/scala/kamon/instrumentation/akka/AskPatternInstrumentationSpec.scala
index d914ffe8..17312ba3 100644
--- a/kamon-core/src/test/scala/kamon/instrumentation/akka/AskPatternInstrumentationSpec.scala
+++ b/kamon-core/src/test/scala/kamon/instrumentation/akka/AskPatternInstrumentationSpec.scala
@@ -54,7 +54,6 @@ class AskPatternInstrumentationSpec extends TestKitBase with WordSpecLike with M
}
val capturedCtx = warn.asInstanceOf[TraceContextAware].traceContext
- capturedCtx should be('defined)
capturedCtx should equal(testTraceContext)
}
}
diff --git a/kamon-core/src/test/scala/kamon/instrumentation/akka/RemotingInstrumentationSpec.scala b/kamon-core/src/test/scala/kamon/instrumentation/akka/RemotingInstrumentationSpec.scala
deleted file mode 100644
index bc25e2d4..00000000
--- a/kamon-core/src/test/scala/kamon/instrumentation/akka/RemotingInstrumentationSpec.scala
+++ /dev/null
@@ -1,171 +0,0 @@
-package kamon.instrumentation.akka
-
-import akka.actor.SupervisorStrategy.Resume
-import akka.actor._
-import akka.remote.RemoteScope
-import akka.routing.RoundRobinRouter
-import akka.testkit.{ ImplicitSender, TestKitBase }
-import akka.util.Timeout
-import com.typesafe.config.ConfigFactory
-import kamon.trace.TraceRecorder
-import org.scalatest.{ Matchers, WordSpecLike }
-import akka.pattern.{ ask, pipe }
-import scala.concurrent.duration._
-import scala.util.control.NonFatal
-
-class RemotingInstrumentationSpec extends TestKitBase with WordSpecLike with Matchers {
- implicit def self = testActor
-
- implicit lazy val system: ActorSystem = ActorSystem("remoting-spec-local-system", ConfigFactory.parseString(
- """
- |akka {
- | actor {
- | provider = "akka.remote.RemoteActorRefProvider"
- | }
- | remote {
- | enabled-transports = ["akka.remote.netty.tcp"]
- | netty.tcp {
- | hostname = "127.0.0.1"
- | port = 2552
- | }
- | }
- |}
- """.stripMargin))
-
- val remoteSystem: ActorSystem = ActorSystem("remoting-spec-remote-system", ConfigFactory.parseString(
- """
- |akka {
- | actor {
- | provider = "akka.remote.RemoteActorRefProvider"
- | }
- | remote {
- | enabled-transports = ["akka.remote.netty.tcp"]
- | netty.tcp {
- | hostname = "127.0.0.1"
- | port = 2553
- | }
- | }
- |}
- """.stripMargin))
-
- val RemoteSystemAddress = AddressFromURIString("akka.tcp://remoting-spec-remote-system@127.0.0.1:2553")
-
- "The Remoting instrumentation" should {
- "propagate the TraceContext when creating a new remote actor" in {
- TraceRecorder.withNewTraceContext("deploy-remote-actor", Some("deploy-remote-actor-1")) {
- system.actorOf(TraceTokenReplier.remoteProps(Some(testActor), RemoteSystemAddress), "remote-deploy-fixture")
- }
-
- expectMsg("name=deploy-remote-actor|token=deploy-remote-actor-1|isOpen=true")
- }
-
- "propagate the TraceContext when sending a message to a remotely deployed actor" in {
- val remoteRef = system.actorOf(TraceTokenReplier.remoteProps(None, RemoteSystemAddress), "remote-message-fixture")
-
- TraceRecorder.withNewTraceContext("message-remote-actor", Some("message-remote-actor-1")) {
- remoteRef ! "reply-trace-token"
- }
-
- expectMsg("name=message-remote-actor|token=message-remote-actor-1|isOpen=true")
- }
-
- "propagate the TraceContext when pipe or ask a message to a remotely deployed actor" in {
- implicit val ec = system.dispatcher
- implicit val askTimeout = Timeout(10 seconds)
- val remoteRef = system.actorOf(TraceTokenReplier.remoteProps(None, RemoteSystemAddress), "remote-ask-and-pipe-fixture")
-
- TraceRecorder.withNewTraceContext("ask-and-pipe-remote-actor", Some("ask-and-pipe-remote-actor-1")) {
- (remoteRef ? "reply-trace-token") pipeTo (testActor)
- }
-
- expectMsg("name=ask-and-pipe-remote-actor|token=ask-and-pipe-remote-actor-1|isOpen=true")
- }
-
- "propagate the TraceContext when sending a message to an ActorSelection" in {
- remoteSystem.actorOf(TraceTokenReplier.props(None), "actor-selection-target-a")
- remoteSystem.actorOf(TraceTokenReplier.props(None), "actor-selection-target-b")
- val selection = system.actorSelection(RemoteSystemAddress + "/user/actor-selection-target-*")
-
- TraceRecorder.withNewTraceContext("message-remote-actor-selection", Some("message-remote-actor-selection-1")) {
- selection ! "reply-trace-token"
- }
-
- // one for each selected actor
- expectMsg("name=message-remote-actor-selection|token=message-remote-actor-selection-1|isOpen=true")
- expectMsg("name=message-remote-actor-selection|token=message-remote-actor-selection-1|isOpen=true")
- }
-
- "propagate the TraceContext a remotely supervised child fails" in {
- val supervisor = system.actorOf(Props(new SupervisorOfRemote(testActor, RemoteSystemAddress)))
-
- TraceRecorder.withNewTraceContext("remote-supervision", Some("remote-supervision-1")) {
- supervisor ! "fail"
- }
-
- expectMsg("name=remote-supervision|token=remote-supervision-1|isOpen=true")
- }
-
- "propagate the TraceContext when sending messages to remote routees of a router" in {
- remoteSystem.actorOf(TraceTokenReplier.props(None), "remote-routee")
- val routees = Vector[String](RemoteSystemAddress + "/user/remote-routee")
- val router = system.actorOf(Props.empty.withRouter(RoundRobinRouter(routees = routees)))
-
- TraceRecorder.withNewTraceContext("remote-routee", Some("remote-routee-1")) {
- router ! "reply-trace-token"
- }
-
- expectMsg("name=remote-routee|token=remote-routee-1|isOpen=true")
- }
- }
-
-}
-
-class TraceTokenReplier(creationTraceContextListener: Option[ActorRef]) extends Actor with ActorLogging {
- creationTraceContextListener map { recipient ⇒
- recipient ! currentTraceContextInfo
- }
-
- def receive = {
- case "fail" ⇒
- 1 / 0
- case "reply-trace-token" ⇒
- log.info("Sending back the TT: " + TraceRecorder.currentContext.map(_.token).getOrElse("unavailable"))
- sender ! currentTraceContextInfo
- }
-
- def currentTraceContextInfo: String = {
- TraceRecorder.currentContext.map { context ⇒
- s"name=${context.name}|token=${context.token}|isOpen=${context.isOpen}"
- }.getOrElse("unavailable")
- }
-}
-
-object TraceTokenReplier {
- def props(creationTraceContextListener: Option[ActorRef]): Props =
- Props(new TraceTokenReplier(creationTraceContextListener))
-
- def remoteProps(creationTraceContextListener: Option[ActorRef], remoteAddress: Address): Props = {
- Props(new TraceTokenReplier(creationTraceContextListener))
- .withDeploy(Deploy(scope = RemoteScope(remoteAddress)))
- }
-}
-
-class SupervisorOfRemote(traceContextListener: ActorRef, remoteAddress: Address) extends Actor {
- val supervisedChild = context.actorOf(TraceTokenReplier.remoteProps(None, remoteAddress), "remotely-supervised-child")
-
- def receive = {
- case "fail" ⇒ supervisedChild ! "fail"
- }
-
- override def supervisorStrategy: SupervisorStrategy = OneForOneStrategy() {
- case NonFatal(throwable) ⇒
- traceContextListener ! currentTraceContextInfo
- Resume
- }
-
- def currentTraceContextInfo: String = {
- TraceRecorder.currentContext.map { context ⇒
- s"name=${context.name}|token=${context.token}|isOpen=${context.isOpen}"
- }.getOrElse("unavailable")
- }
-}
diff --git a/kamon-core/src/test/scala/kamon/metric/ActorMetricsSpec.scala b/kamon-core/src/test/scala/kamon/metric/ActorMetricsSpec.scala
index 21e0bbba..006366ba 100644
--- a/kamon-core/src/test/scala/kamon/metric/ActorMetricsSpec.scala
+++ b/kamon-core/src/test/scala/kamon/metric/ActorMetricsSpec.scala
@@ -192,7 +192,7 @@ class ActorMetricsSpec extends TestKitBase with WordSpecLike with Matchers {
class ActorMetricsTestActor extends Actor {
def receive = {
case Discard ⇒
- case Fail ⇒ 1 / 0
+ case Fail ⇒ throw new ArithmeticException("Division by zero.")
case Ping ⇒ sender ! Pong
case TrackTimings(sendTimestamp, sleep) ⇒ {
val dequeueTimestamp = System.nanoTime()
diff --git a/kamon-core/src/test/scala/kamon/metric/RouterMetricsSpec.scala b/kamon-core/src/test/scala/kamon/metric/RouterMetricsSpec.scala
index 8b33d216..6585a618 100644
--- a/kamon-core/src/test/scala/kamon/metric/RouterMetricsSpec.scala
+++ b/kamon-core/src/test/scala/kamon/metric/RouterMetricsSpec.scala
@@ -30,9 +30,7 @@ import org.scalatest.{ Matchers, WordSpecLike }
import scala.concurrent.duration._
-class RouterMetricsSpec extends TestKitBase with WordSpecLike with Matchers {
- implicit def self = testActor
-
+class RouterMetricsSpec extends TestKitBase with WordSpecLike with Matchers with ImplicitSender {
implicit lazy val system: ActorSystem = ActorSystem("router-metrics-spec", ConfigFactory.parseString(
"""
|kamon.metrics {
@@ -132,15 +130,13 @@ class RouterMetricsSpec extends TestKitBase with WordSpecLike with Matchers {
def createTestRouter(name: String): ActorRef = system.actorOf(Props[RouterMetricsTestActor]
.withRouter(RoundRobinRouter(nrOfInstances = 5)), name)
-
- def takeSnapshotOf(amr: RouterMetricsRecorder): RouterMetricSnapshot = amr.collect(collectionContext)
}
}
class RouterMetricsTestActor extends Actor {
def receive = {
case Discard ⇒
- case Fail ⇒ 1 / 0
+ case Fail ⇒ throw new ArithmeticException("Division by zero.")
case Ping ⇒ sender ! Pong
case RouterTrackTimings(sendTimestamp, sleep) ⇒ {
val dequeueTimestamp = System.nanoTime()
diff --git a/kamon-core/src/test/scala/kamon/metric/TraceMetricsSpec.scala b/kamon-core/src/test/scala/kamon/metric/TraceMetricsSpec.scala
index 23977971..7468f59c 100644
--- a/kamon-core/src/test/scala/kamon/metric/TraceMetricsSpec.scala
+++ b/kamon-core/src/test/scala/kamon/metric/TraceMetricsSpec.scala
@@ -5,8 +5,7 @@ import akka.testkit.{ ImplicitSender, TestKitBase }
import com.typesafe.config.ConfigFactory
import kamon.Kamon
import kamon.metric.TraceMetrics.TraceMetricsSnapshot
-import kamon.trace.TraceContext.SegmentIdentity
-import kamon.trace.TraceRecorder
+import kamon.trace.{ SegmentMetricIdentity, TraceRecorder }
import org.scalatest.{ Matchers, WordSpecLike }
class TraceMetricsSpec extends TestKitBase with WordSpecLike with Matchers {
@@ -55,39 +54,37 @@ class TraceMetricsSpec extends TestKitBase with WordSpecLike with Matchers {
"record the elapsed time for segments that occur inside a given trace" in {
TraceRecorder.withNewTraceContext("trace-with-segments") {
- val segmentHandle = TraceRecorder.startSegment(TraceMetricsTestSegment("test-segment"))
- segmentHandle.get.finish()
+ val segment = TraceRecorder.currentContext.startSegment("test-segment", "test-label")
+ segment.finish()
TraceRecorder.finish()
}
val snapshot = takeSnapshotOf("trace-with-segments")
snapshot.elapsedTime.numberOfMeasurements should be(1)
snapshot.segments.size should be(1)
- snapshot.segments(TraceMetricsTestSegment("test-segment")).numberOfMeasurements should be(1)
+ snapshot.segments(SegmentMetricIdentity("test-segment", "test-label")).numberOfMeasurements should be(1)
}
"record the elapsed time for segments that finish after their correspondent trace has finished" in {
- val segmentHandle = TraceRecorder.withNewTraceContext("closing-segment-after-trace") {
- val sh = TraceRecorder.startSegment(TraceMetricsTestSegment("test-segment"))
+ val segment = TraceRecorder.withNewTraceContext("closing-segment-after-trace") {
+ val s = TraceRecorder.currentContext.startSegment("test-segment", "test-label")
TraceRecorder.finish()
- sh
+ s
}
val beforeFinishSegmentSnapshot = takeSnapshotOf("closing-segment-after-trace")
beforeFinishSegmentSnapshot.elapsedTime.numberOfMeasurements should be(1)
beforeFinishSegmentSnapshot.segments.size should be(0)
- segmentHandle.get.finish()
+ segment.finish()
val afterFinishSegmentSnapshot = takeSnapshotOf("closing-segment-after-trace")
afterFinishSegmentSnapshot.elapsedTime.numberOfMeasurements should be(0)
afterFinishSegmentSnapshot.segments.size should be(1)
- afterFinishSegmentSnapshot.segments(TraceMetricsTestSegment("test-segment")).numberOfMeasurements should be(1)
+ afterFinishSegmentSnapshot.segments(SegmentMetricIdentity("test-segment", "test-label")).numberOfMeasurements should be(1)
}
}
- case class TraceMetricsTestSegment(name: String) extends SegmentIdentity
-
def takeSnapshotOf(traceName: String): TraceMetricsSnapshot = {
val recorder = Kamon(Metrics).register(TraceMetrics(traceName), TraceMetrics.Factory)
val collectionContext = Kamon(Metrics).buildDefaultCollectionContext
diff --git a/kamon-core/src/test/scala/kamon/metric/instrument/HistogramSpec.scala b/kamon-core/src/test/scala/kamon/metric/instrument/HistogramSpec.scala
index cefdf0f4..c3060d4a 100644
--- a/kamon-core/src/test/scala/kamon/metric/instrument/HistogramSpec.scala
+++ b/kamon-core/src/test/scala/kamon/metric/instrument/HistogramSpec.scala
@@ -57,7 +57,7 @@ class HistogramSpec extends WordSpec with Matchers {
}
"produce a snapshot" which {
- "supports min, max and numberOfMeasurements operations" in new HistogramFixture {
+ "supports min, max, percentile, sum and numberOfMeasurements operations" in new HistogramFixture {
histogram.record(100)
histogram.record(200, count = 200)
histogram.record(300)
@@ -67,7 +67,12 @@ class HistogramSpec extends WordSpec with Matchers {
snapshot.min should equal(100L +- 1L)
snapshot.max should equal(900L +- 9L)
+ snapshot.percentile(50.0D) should be(200)
+ snapshot.percentile(99.5D) should be(300)
+ snapshot.percentile(99.9D) should be(900)
+ snapshot.sum should be(41300)
snapshot.numberOfMeasurements should be(203)
+
}
"can be merged with another snapshot" in new MultipleHistogramFixture {
diff --git a/kamon-core/src/test/scala/kamon/trace/TraceContextManipulationSpec.scala b/kamon-core/src/test/scala/kamon/trace/TraceContextManipulationSpec.scala
index d073f68e..206fbd4e 100644
--- a/kamon-core/src/test/scala/kamon/trace/TraceContextManipulationSpec.scala
+++ b/kamon-core/src/test/scala/kamon/trace/TraceContextManipulationSpec.scala
@@ -3,7 +3,6 @@ package kamon.trace
import akka.actor.ActorSystem
import akka.testkit.TestKitBase
import com.typesafe.config.ConfigFactory
-import kamon.trace.TraceContext.SegmentIdentity
import org.scalatest.{ Matchers, WordSpecLike }
class TraceContextManipulationSpec extends TestKitBase with WordSpecLike with Matchers {
@@ -39,7 +38,7 @@ class TraceContextManipulationSpec extends TestKitBase with WordSpecLike with Ma
"allow starting a trace within a specified block of code, and only within that block of code" in {
val createdContext = TraceRecorder.withNewTraceContext("start-context") {
TraceRecorder.currentContext should not be empty
- TraceRecorder.currentContext.get
+ TraceRecorder.currentContext
}
TraceRecorder.currentContext shouldBe empty
@@ -49,7 +48,7 @@ class TraceContextManipulationSpec extends TestKitBase with WordSpecLike with Ma
"allow starting a trace within a specified block of code, providing a trace-token and only within that block of code" in {
val createdContext = TraceRecorder.withNewTraceContext("start-context-with-token", Some("token-1")) {
TraceRecorder.currentContext should not be empty
- TraceRecorder.currentContext.get
+ TraceRecorder.currentContext
}
TraceRecorder.currentContext shouldBe empty
@@ -71,7 +70,7 @@ class TraceContextManipulationSpec extends TestKitBase with WordSpecLike with Ma
"allow renaming a trace" in {
val createdContext = TraceRecorder.withNewTraceContext("trace-before-rename") {
TraceRecorder.rename("renamed-trace")
- TraceRecorder.currentContext.get
+ TraceRecorder.currentContext
}
TraceRecorder.currentContext shouldBe empty
@@ -80,17 +79,22 @@ class TraceContextManipulationSpec extends TestKitBase with WordSpecLike with Ma
"allow creating a segment within a trace" in {
val createdContext = TraceRecorder.withNewTraceContext("trace-with-segments") {
- val segmentHandle = TraceRecorder.startSegment(TraceManipulationTestSegment("segment-1"))
-
- TraceRecorder.currentContext.get
+ val segment = TraceRecorder.currentContext.startSegment("segment-1", "segment-1-label")
+ TraceRecorder.currentContext
}
TraceRecorder.currentContext shouldBe empty
createdContext.name shouldBe ("trace-with-segments")
-
}
- }
- case class TraceManipulationTestSegment(name: String) extends SegmentIdentity
+ "allow renaming a segment" in {
+ TraceRecorder.withNewTraceContext("trace-with-renamed-segment") {
+ val segment = TraceRecorder.currentContext.startSegment("original-segment-name", "segment-label")
+ segment.name should be("original-segment-name")
+ segment.rename("new-segment-name")
+ segment.name should be("new-segment-name")
+ }
+ }
+ }
}