From f498749274bc9f25ede7221d6bd8b3f0c3822dda Mon Sep 17 00:00:00 2001 From: Ivan Topolnjak Date: Thu, 6 Nov 2014 16:29:54 +0100 Subject: ! newrelic: major refactor of the newrelic reporter Most notable changes: - The agent connection setup is separated from the actual metrics reporting, this will be important in the near future when we start sending errors too. - The metrics subscriptions are delayed until the connection to the agent is established. - The Tick metrics buffer is only created if necessary. - Introduced the kamon.newrelic.max-initialize-retries and initialize-retry-delay settings. - External service calls via HTTP clients are reported as external services. --- kamon-newrelic/src/main/resources/reference.conf | 13 +- .../src/main/scala/kamon/newrelic/Agent.scala | 163 +++++-------- .../scala/kamon/newrelic/AgentJsonProtocol.scala | 63 ----- .../scala/kamon/newrelic/ClientPipelines.scala | 23 ++ .../kamon/newrelic/CustomMetricExtractor.scala | 33 +++ .../main/scala/kamon/newrelic/CustomMetrics.scala | 34 --- .../main/scala/kamon/newrelic/JsonProtocol.scala | 65 ++++++ .../src/main/scala/kamon/newrelic/Metric.scala | 55 +++++ .../main/scala/kamon/newrelic/MetricReporter.scala | 105 +++++++++ .../scala/kamon/newrelic/MetricTranslator.scala | 48 ---- .../src/main/scala/kamon/newrelic/NewRelic.scala | 48 +--- .../newrelic/WebTransactionMetricExtractor.scala | 81 +++++++ .../kamon/newrelic/WebTransactionMetrics.scala | 74 ------ .../src/main/scala/kamon/newrelic/package.scala | 42 +--- .../src/test/scala/kamon/newrelic/AgentSpec.scala | 253 +++++++++++++-------- .../scala/kamon/newrelic/MetricReporterSpec.scala | 156 +++++++++++++ .../src/main/resources/application.conf | 2 +- .../main/scala/test/SimpleRequestProcessor.scala | 2 +- 18 files changed, 751 insertions(+), 509 deletions(-) delete mode 100644 kamon-newrelic/src/main/scala/kamon/newrelic/AgentJsonProtocol.scala create mode 100644 kamon-newrelic/src/main/scala/kamon/newrelic/ClientPipelines.scala create mode 100644 kamon-newrelic/src/main/scala/kamon/newrelic/CustomMetricExtractor.scala delete mode 100644 kamon-newrelic/src/main/scala/kamon/newrelic/CustomMetrics.scala create mode 100644 kamon-newrelic/src/main/scala/kamon/newrelic/JsonProtocol.scala create mode 100644 kamon-newrelic/src/main/scala/kamon/newrelic/Metric.scala create mode 100644 kamon-newrelic/src/main/scala/kamon/newrelic/MetricReporter.scala delete mode 100644 kamon-newrelic/src/main/scala/kamon/newrelic/MetricTranslator.scala create mode 100644 kamon-newrelic/src/main/scala/kamon/newrelic/WebTransactionMetricExtractor.scala delete mode 100644 kamon-newrelic/src/main/scala/kamon/newrelic/WebTransactionMetrics.scala create mode 100644 kamon-newrelic/src/test/scala/kamon/newrelic/MetricReporterSpec.scala diff --git a/kamon-newrelic/src/main/resources/reference.conf b/kamon-newrelic/src/main/resources/reference.conf index 059420f9..c86e64ae 100644 --- a/kamon-newrelic/src/main/resources/reference.conf +++ b/kamon-newrelic/src/main/resources/reference.conf @@ -4,17 +4,22 @@ kamon { newrelic { + + # General ApdexT that applies to all Trace metrics reported. apdexT = 1 second + # The application name that will be shown in the New Relic dashboard. app-name = "Kamon[Development]" - license-key = e7d350b14228f3d28f35bc3140df2c3e565ea5d5 - # delay between connection attempts to NewRelic collector - retry-delay = 30 seconds + # Your New Relic license key. + license-key = e7d350b14228f3d28f35bc3140df2c3e565ea5d5 # attempts to send pending metrics in the next tick, # combining the current metrics plus the pending, after max-retry, deletes all pending metrics - max-retry = 3 + max-initialize-retries = 3 + + # delay between connection attempts to NewRelic collector + initialize-retry-delay = 30 seconds } } diff --git a/kamon-newrelic/src/main/scala/kamon/newrelic/Agent.scala b/kamon-newrelic/src/main/scala/kamon/newrelic/Agent.scala index 25fbc9db..f71ecd7f 100644 --- a/kamon-newrelic/src/main/scala/kamon/newrelic/Agent.scala +++ b/kamon-newrelic/src/main/scala/kamon/newrelic/Agent.scala @@ -18,88 +18,73 @@ package kamon.newrelic import java.util.concurrent.TimeUnit.{ MILLISECONDS ⇒ milliseconds } -import akka.actor.{ ActorLogging, Actor } +import akka.actor.{ ActorSystem, ActorLogging, Actor } import akka.event.LoggingAdapter -import org.slf4j.LoggerFactory +import akka.io.IO +import akka.util.Timeout +import kamon.Kamon +import kamon.metric.{ CollectionContext, Metrics } +import spray.can.Http import spray.json._ import scala.concurrent.{ ExecutionContext, Future } -import spray.httpx.{ SprayJsonSupport, RequestBuilding, ResponseTransformation } -import spray.httpx.encoding.Deflate +import spray.httpx.{ SprayJsonSupport, ResponseTransformation } import spray.http._ import spray.json.lenses.JsonLenses._ import java.lang.management.ManagementFactory -import spray.client.pipelining._ -import scala.util.{ Failure, Success } import spray.http.Uri.Query -import kamon.newrelic.MetricTranslator.TimeSliceMetrics import scala.concurrent.duration._ +import Agent._ -class Agent extends Actor with RequestBuilding with ResponseTransformation with SprayJsonSupport with ActorLogging { +import akka.pattern.pipe +// TODO: Setup a proper host connector with custom timeout configuration for use with this. +class Agent extends Actor with ClientPipelines with ResponseTransformation with SprayJsonSupport with ActorLogging { + import JsonProtocol._ import context.dispatcher - import Agent._ - import Retry._ - - self ! Initialize - - val agentInfo = { - val config = context.system.settings.config.getConfig("kamon.newrelic") - val appName = config.getString("app-name") - val licenseKey = config.getString("license-key") - - // Name has the format of pid@host - val runtimeName = ManagementFactory.getRuntimeMXBean.getName.split('@') - val retryDelay = FiniteDuration(config.getDuration("retry-delay", milliseconds), milliseconds) - val maxRetry = config.getInt("max-retry") - - AgentInfo(licenseKey, appName, runtimeName(1), runtimeName(0).toInt, maxRetry, retryDelay) - } + implicit val operationTimeout = Timeout(30 seconds) + val collectorClient = compressedToJsonPipeline(IO(Http)(context.system)) + val settings = buildAgentSettings(context.system) val baseQuery = Query( - "license_key" -> agentInfo.licenseKey, + "license_key" -> settings.licenseKey, "marshal_format" -> "json", "protocol_version" -> "12") - def receive: Receive = uninitialized - - def uninitialized: Receive = { - case Initialize ⇒ { - connectToCollector onComplete { - case Success(agent) ⇒ { - log.info("Agent initialized with runID: [{}] and collector: [{}]", agent.runId, agent.collector) - context become reporting(agent.runId, agent.collector) - } - case Failure(reason) ⇒ self ! InitializationFailed(reason) - } - } - case InitializationFailed(reason) ⇒ { - log.info("Initialization failed: {}, retrying in {} seconds", reason.getMessage, agentInfo.retryDelay.toSeconds) - context.system.scheduler.scheduleOnce(agentInfo.retryDelay, self, Initialize) - } - case everythingElse ⇒ //ignore - } + // Start the connection to the New Relic collector. + self ! Initialize - def reporting(runId: Long, collector: String): Receive = { - case metrics: TimeSliceMetrics ⇒ sendMetricData(runId, collector, metrics) - } + def receive: Receive = uninitialized(settings.maxRetries) - def connectToCollector: Future[Initialized] = for { - collector ← selectCollector - runId ← connect(collector, agentInfo) - } yield Initialized(runId, collector) + def uninitialized(attemptsLeft: Int): Receive = { + case Initialize ⇒ pipe(connectToCollector) to self + case Initialized(runID, collector) ⇒ + log.info("Agent initialized with runID: [{}] and collector: [{}]", runID, collector) - import AgentJsonProtocol._ + val baseCollectorUri = Uri(s"http://$collector/agent_listener/invoke_raw_method").withQuery(baseQuery) + context.actorOf(MetricReporter.props(settings, runID, baseCollectorUri), "metric-reporter") - val compressedPipeline: HttpRequest ⇒ Future[HttpResponse] = encode(Deflate) ~> sendReceive - val compressedToJsonPipeline: HttpRequest ⇒ Future[JsValue] = compressedPipeline ~> toJson + case InitializationFailed(reason) if (attemptsLeft > 0) ⇒ + log.error(reason, "Initialization failed, retrying in {} seconds", settings.retryDelay.toSeconds) + context.system.scheduler.scheduleOnce(settings.retryDelay, self, Initialize) + context become (uninitialized(attemptsLeft - 1)) - def toJson(response: HttpResponse): JsValue = response.entity.asString.parseJson + case InitializationFailed(reason) ⇒ + log.error(reason, "Giving up while trying to set up a connection with the New Relic collector.") + context.stop(self) + } + + def connectToCollector: Future[InitResult] = { + (for { + collector ← selectCollector + runId ← connect(collector, settings) + } yield Initialized(runId, collector)) recover { case error ⇒ InitializationFailed(error) } + } def selectCollector: Future[String] = { val query = ("method" -> "get_redirect_host") +: baseQuery val getRedirectHostUri = Uri("http://collector.newrelic.com/agent_listener/invoke_raw_method").withQuery(query) - compressedToJsonPipeline { + collectorClient { Post(getRedirectHostUri, JsArray()) } map { json ⇒ @@ -107,67 +92,39 @@ class Agent extends Actor with RequestBuilding with ResponseTransformation with } } - def connect(collectorHost: String, connect: AgentInfo): Future[Long] = { + def connect(collectorHost: String, connect: Settings): Future[Long] = { log.debug("Connecting to NewRelic Collector [{}]", collectorHost) val query = ("method" -> "connect") +: baseQuery val connectUri = Uri(s"http://$collectorHost/agent_listener/invoke_raw_method").withQuery(query) - compressedToJsonPipeline { + collectorClient { Post(connectUri, connect) } map { json ⇒ json.extract[Long]('return_value / 'agent_run_id) } } - - def sendMetricData(runId: Long, collector: String, metrics: TimeSliceMetrics) = { - val query = ("method" -> "metric_data") +: ("run_id" -> runId.toString) +: baseQuery - val sendMetricDataUri = Uri(s"http://$collector/agent_listener/invoke_raw_method").withQuery(query) - - withMaxAttempts(agentInfo.maxRetry, metrics, log) { currentMetrics ⇒ - compressedPipeline { - log.info("Sending metrics to NewRelic collector") - Post(sendMetricDataUri, MetricData(runId, currentMetrics)) - } - } - } } object Agent { - case class Initialize() - case class Initialized(runId: Long, collector: String) - case class InitializationFailed(reason: Throwable) - case class CollectorSelection(return_value: String) - case class AgentInfo(licenseKey: String, appName: String, host: String, pid: Int, maxRetry: Int = 0, retryDelay: FiniteDuration) - case class MetricData(runId: Long, timeSliceMetrics: TimeSliceMetrics) -} + case object Initialize + sealed trait InitResult + case class Initialized(runId: Long, collector: String) extends InitResult + case class InitializationFailed(reason: Throwable) extends InitResult + case class Settings(licenseKey: String, appName: String, host: String, pid: Int, maxRetries: Int, retryDelay: FiniteDuration, apdexT: Double) + + def buildAgentSettings(system: ActorSystem) = { + val config = system.settings.config.getConfig("kamon.newrelic") + val appName = config.getString("app-name") + val licenseKey = config.getString("license-key") + val maxRetries = config.getInt("max-initialize-retries") + val retryDelay = FiniteDuration(config.getDuration("initialize-retry-delay", milliseconds), milliseconds) + val apdexT: Double = config.getDuration("apdexT", MILLISECONDS) / 1E3 // scale to seconds. -object Retry { - - @volatile private var attempts: Int = 0 - @volatile private var pendingMetrics: Option[TimeSliceMetrics] = None - - def withMaxAttempts[T](maxRetry: Int, metrics: TimeSliceMetrics, log: LoggingAdapter)(block: TimeSliceMetrics ⇒ Future[T])(implicit executor: ExecutionContext): Unit = { - - val currentMetrics = metrics.merge(pendingMetrics) - - if (currentMetrics.metrics.nonEmpty) { - block(currentMetrics) onComplete { - case Success(_) ⇒ - pendingMetrics = None - attempts = 0 - case Failure(_) ⇒ - attempts += 1 - if (maxRetry > attempts) { - log.info("Trying to send metrics to NewRelic collector, attempt [{}] of [{}]", attempts, maxRetry) - pendingMetrics = Some(currentMetrics) - } else { - log.info("Max attempts achieved, proceeding to remove all pending metrics") - pendingMetrics = None - attempts = 0 - } - } - } + // Name has the format of 'pid'@'host' + val runtimeName = ManagementFactory.getRuntimeMXBean.getName.split('@') + + Agent.Settings(licenseKey, appName, runtimeName(1), runtimeName(0).toInt, maxRetries, retryDelay, apdexT) } } \ No newline at end of file diff --git a/kamon-newrelic/src/main/scala/kamon/newrelic/AgentJsonProtocol.scala b/kamon-newrelic/src/main/scala/kamon/newrelic/AgentJsonProtocol.scala deleted file mode 100644 index 9b3e6dea..00000000 --- a/kamon-newrelic/src/main/scala/kamon/newrelic/AgentJsonProtocol.scala +++ /dev/null @@ -1,63 +0,0 @@ -/* =================================================== - * Copyright © 2013 the kamon project - * - * 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 kamon.newrelic - -import spray.json._ -import kamon.newrelic.Agent._ - -object AgentJsonProtocol extends DefaultJsonProtocol { - - implicit object ConnectJsonWriter extends RootJsonWriter[AgentInfo] { - def write(obj: AgentInfo): JsValue = - JsArray( - JsObject( - "agent_version" -> JsString("3.1.0"), - "app_name" -> JsArray(JsString(obj.appName)), - "host" -> JsString(obj.host), - "identifier" -> JsString(s"java:${obj.appName}"), - "language" -> JsString("java"), - "pid" -> JsNumber(obj.pid))) - } - - implicit def seqWriter[T: JsonWriter] = new JsonWriter[Seq[T]] { - def write(seq: Seq[T]) = JsArray(seq.map(_.toJson).toVector) - } - - implicit object MetricDetailWriter extends JsonWriter[NewRelic.Metric] { - def write(obj: NewRelic.Metric): JsValue = { - JsArray( - JsObject( - "name" -> JsString(obj.name) // TODO Include scope - ), - JsArray( - JsNumber(obj.callCount), - JsNumber(obj.total), - JsNumber(obj.totalExclusive), - JsNumber(obj.min), - JsNumber(obj.max), - JsNumber(obj.sumOfSquares))) - } - } - - implicit object MetricDataWriter extends RootJsonWriter[MetricData] { - def write(obj: MetricData): JsValue = - JsArray( - JsNumber(obj.runId), - JsNumber(obj.timeSliceMetrics.from), - JsNumber(obj.timeSliceMetrics.to), - obj.timeSliceMetrics.metrics.values.toSeq.toJson) - } -} diff --git a/kamon-newrelic/src/main/scala/kamon/newrelic/ClientPipelines.scala b/kamon-newrelic/src/main/scala/kamon/newrelic/ClientPipelines.scala new file mode 100644 index 00000000..ca003646 --- /dev/null +++ b/kamon-newrelic/src/main/scala/kamon/newrelic/ClientPipelines.scala @@ -0,0 +1,23 @@ +package kamon.newrelic + +import akka.actor.ActorRef +import akka.util.Timeout +import spray.http.{ HttpResponse, HttpRequest } +import spray.httpx.RequestBuilding +import spray.httpx.encoding.Deflate +import spray.json._ +import spray.client.pipelining.sendReceive + +import scala.concurrent.{ ExecutionContext, Future } + +trait ClientPipelines extends RequestBuilding { + + def compressedPipeline(transport: ActorRef)(implicit ec: ExecutionContext, to: Timeout): HttpRequest ⇒ Future[HttpResponse] = + encode(Deflate) ~> sendReceive(transport) + + def compressedToJsonPipeline(transport: ActorRef)(implicit ec: ExecutionContext, to: Timeout): HttpRequest ⇒ Future[JsValue] = + compressedPipeline(transport) ~> toJson + + def toJson(response: HttpResponse): JsValue = response.entity.asString.parseJson + +} diff --git a/kamon-newrelic/src/main/scala/kamon/newrelic/CustomMetricExtractor.scala b/kamon-newrelic/src/main/scala/kamon/newrelic/CustomMetricExtractor.scala new file mode 100644 index 00000000..84472593 --- /dev/null +++ b/kamon-newrelic/src/main/scala/kamon/newrelic/CustomMetricExtractor.scala @@ -0,0 +1,33 @@ +/* + * ========================================================================================= + * Copyright © 2013 the kamon project + * + * 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 kamon.newrelic + +import kamon.metric.UserMetrics.UserMetricGroup +import kamon.metric._ +import kamon.newrelic.Agent.Settings + +object CustomMetricExtractor extends MetricExtractor { + + def extract(settings: Settings, collectionContext: CollectionContext, metrics: Map[MetricGroupIdentity, MetricGroupSnapshot]): Map[MetricID, MetricData] = { + metrics.collect { + case (mg: UserMetricGroup, groupSnapshot) ⇒ + groupSnapshot.metrics collect { + case (name, snapshot) ⇒ Metric.fromKamonMetricSnapshot(snapshot, s"Custom/${mg.name}", None, Scale.Unit) + } + }.flatten.toMap + } +} diff --git a/kamon-newrelic/src/main/scala/kamon/newrelic/CustomMetrics.scala b/kamon-newrelic/src/main/scala/kamon/newrelic/CustomMetrics.scala deleted file mode 100644 index a2b208dc..00000000 --- a/kamon-newrelic/src/main/scala/kamon/newrelic/CustomMetrics.scala +++ /dev/null @@ -1,34 +0,0 @@ -/* - * ========================================================================================= - * Copyright © 2013 the kamon project - * - * 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 kamon.newrelic - -import akka.actor.Actor -import kamon.metric.UserMetrics.UserMetricGroup -import kamon.metric._ - -trait CustomMetrics { - self: Actor ⇒ - - def collectCustomMetrics(metrics: Map[MetricGroupIdentity, MetricGroupSnapshot]): Seq[NewRelic.Metric] = { - metrics.collect { - case (mg: UserMetricGroup, groupSnapshot) ⇒ - groupSnapshot.metrics collect { - case (name, snapshot) ⇒ toNewRelicMetric(Scale.Unit)(s"Custom/${mg.name}", None, snapshot) - } - }.flatten.toSeq - } -} diff --git a/kamon-newrelic/src/main/scala/kamon/newrelic/JsonProtocol.scala b/kamon-newrelic/src/main/scala/kamon/newrelic/JsonProtocol.scala new file mode 100644 index 00000000..c573d04d --- /dev/null +++ b/kamon-newrelic/src/main/scala/kamon/newrelic/JsonProtocol.scala @@ -0,0 +1,65 @@ +/* =================================================== + * Copyright © 2013 the kamon project + * + * 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 kamon.newrelic + +import spray.json._ +import kamon.newrelic.Agent._ + +object JsonProtocol extends DefaultJsonProtocol { + + implicit object ConnectJsonWriter extends RootJsonWriter[Settings] { + def write(obj: Settings): JsValue = + JsArray( + JsObject( + "agent_version" -> JsString("3.1.0"), + "app_name" -> JsArray(JsString(obj.appName)), + "host" -> JsString(obj.host), + "identifier" -> JsString(s"java:${obj.appName}"), + "language" -> JsString("java"), + "pid" -> JsNumber(obj.pid))) + } + + implicit def seqWriter[T: JsonWriter] = new JsonWriter[Seq[T]] { + def write(seq: Seq[T]) = JsArray(seq.map(_.toJson).toVector) + } + + implicit object MetricDetailWriter extends JsonWriter[Metric] { + def write(obj: Metric): JsValue = { + val (metricID, metricData) = obj + + JsArray( + JsObject( + "name" -> JsString(metricID.name) // TODO Include scope + ), + JsArray( + JsNumber(metricData.callCount), + JsNumber(metricData.total), + JsNumber(metricData.totalExclusive), + JsNumber(metricData.min), + JsNumber(metricData.max), + JsNumber(metricData.sumOfSquares))) + } + } + + implicit object MetricBatchWriter extends RootJsonWriter[MetricBatch] { + def write(obj: MetricBatch): JsValue = + JsArray( + JsNumber(obj.runID), + JsNumber(obj.timeSliceMetrics.from), + JsNumber(obj.timeSliceMetrics.to), + obj.timeSliceMetrics.metrics.toSeq.toJson) + } +} diff --git a/kamon-newrelic/src/main/scala/kamon/newrelic/Metric.scala b/kamon-newrelic/src/main/scala/kamon/newrelic/Metric.scala new file mode 100644 index 00000000..14541483 --- /dev/null +++ b/kamon-newrelic/src/main/scala/kamon/newrelic/Metric.scala @@ -0,0 +1,55 @@ +package kamon.newrelic + +import kamon.metric.instrument.{ Counter, Histogram } +import kamon.metric.{ MetricSnapshot, Scale } + +case class MetricID(name: String, scope: Option[String]) +case class MetricData(callCount: Long, total: Double, totalExclusive: Double, min: Double, max: Double, sumOfSquares: Double) { + def merge(that: MetricData): MetricData = + MetricData( + callCount + that.callCount, + total + that.total, + totalExclusive + that.totalExclusive, + math.min(min, that.min), + math.max(max, that.max), + sumOfSquares + that.sumOfSquares) +} + +object Metric { + + def fromKamonMetricSnapshot(snapshot: MetricSnapshot, name: String, scope: Option[String], targetScale: Scale): Metric = { + snapshot match { + case hs: Histogram.Snapshot ⇒ + var total: Double = 0D + var sumOfSquares: Double = 0D + val scaledMin = Scale.convert(hs.scale, targetScale, hs.min) + val scaledMax = Scale.convert(hs.scale, targetScale, hs.max) + + hs.recordsIterator.foreach { record ⇒ + val scaledValue = Scale.convert(hs.scale, targetScale, record.level) + + total += scaledValue * record.count + sumOfSquares += (scaledValue * scaledValue) * record.count + } + + (MetricID(name, scope), MetricData(hs.numberOfMeasurements, total, total, scaledMin, scaledMax, sumOfSquares)) + + case cs: Counter.Snapshot ⇒ + (MetricID(name, scope), MetricData(cs.count, cs.count, cs.count, 0, cs.count, cs.count * cs.count)) + } + } +} + +case class TimeSliceMetrics(from: Long, to: Long, metrics: Map[MetricID, MetricData]) { + import kamon.metric.combineMaps + + def merge(that: TimeSliceMetrics): TimeSliceMetrics = { + val mergedFrom = math.min(from, that.from) + val mergedTo = math.max(to, that.to) + val mergedMetrics = combineMaps(metrics, that.metrics)((l, r) ⇒ l.merge(r)) + + TimeSliceMetrics(mergedFrom, mergedTo, mergedMetrics) + } +} + +case class MetricBatch(runID: Long, timeSliceMetrics: TimeSliceMetrics) \ No newline at end of file diff --git a/kamon-newrelic/src/main/scala/kamon/newrelic/MetricReporter.scala b/kamon-newrelic/src/main/scala/kamon/newrelic/MetricReporter.scala new file mode 100644 index 00000000..b09973ef --- /dev/null +++ b/kamon-newrelic/src/main/scala/kamon/newrelic/MetricReporter.scala @@ -0,0 +1,105 @@ +package kamon.newrelic + +import java.util.concurrent.TimeUnit + +import akka.actor.{ Props, ActorLogging, Actor } +import akka.pattern.pipe +import akka.io.IO +import akka.util.Timeout +import kamon.Kamon +import kamon.metric.Subscriptions.TickMetricSnapshot +import kamon.metric.UserMetrics.{ UserGauges, UserMinMaxCounters, UserCounters, UserHistograms } +import kamon.metric._ +import kamon.newrelic.MetricReporter.{ UnexpectedStatusCodeException, PostFailed, PostSucceeded, MetricDataPostResult } +import spray.can.Http +import spray.http.Uri +import spray.httpx.SprayJsonSupport +import spray.json.CompactPrinter + +import scala.concurrent.Future +import scala.concurrent.duration._ +import scala.util.control.NoStackTrace + +class MetricReporter(settings: Agent.Settings, runID: Long, baseUri: Uri) extends Actor + with ClientPipelines with ActorLogging with SprayJsonSupport { + + import JsonProtocol._ + import MetricReporter.Extractors + import context.dispatcher + + val metricDataQuery = ("method" -> "metric_data") +: ("run_id" -> runID.toString) +: baseUri.query + val metricDataUri = baseUri.withQuery(metricDataQuery) + + implicit val operationTimeout = Timeout(30 seconds) + val metricsExtension = Kamon(Metrics)(context.system) + val collectionContext = metricsExtension.buildDefaultCollectionContext + val collectorClient = compressedPipeline(IO(Http)(context.system)) + + val subscriber = { + val tickInterval = context.system.settings.config.getDuration("kamon.metrics.tick-interval", TimeUnit.MILLISECONDS) + if (tickInterval == 60000) + self + else + context.actorOf(TickMetricSnapshotBuffer.props(1 minute, self), "metric-buffer") + } + + // Subscribe to Trace Metrics + metricsExtension.subscribe(TraceMetrics, "*", subscriber, permanently = true) + + // Subscribe to all User Metrics + metricsExtension.subscribe(UserHistograms, "*", subscriber, permanently = true) + metricsExtension.subscribe(UserCounters, "*", subscriber, permanently = true) + metricsExtension.subscribe(UserMinMaxCounters, "*", subscriber, permanently = true) + metricsExtension.subscribe(UserGauges, "*", subscriber, permanently = true) + + def receive = reporting(None) + + def reporting(pendingMetrics: Option[TimeSliceMetrics]): Receive = { + case TickMetricSnapshot(from, to, metrics) ⇒ + val fromInSeconds = (from / 1E3).toInt + val toInSeconds = (to / 1E3).toInt + val extractedMetrics = Extractors.flatMap(_.extract(settings, collectionContext, metrics)).toMap + val tickMetrics = TimeSliceMetrics(fromInSeconds, toInSeconds, extractedMetrics) + + val metricsToReport = pendingMetrics.foldLeft(tickMetrics)((p, n) ⇒ p.merge(n)) + context become reporting(Some(metricsToReport)) + pipe(sendMetricData(metricsToReport)) to self + + case PostSucceeded ⇒ + context become (reporting(None)) + + case PostFailed(reason) ⇒ + log.error(reason, "Metric POST to the New Relic collector failed, metrics will be accumulated with the next tick.") + } + + def sendMetricData(slice: TimeSliceMetrics): Future[MetricDataPostResult] = { + log.debug("Sending [{}] metrics to New Relic for the time slice between {} and {}.", slice.metrics.size, slice.from, slice.to) + + collectorClient { + Post(metricDataUri, MetricBatch(runID, slice))(sprayJsonMarshaller(MetricBatchWriter, CompactPrinter)) + + } map { response ⇒ + if (response.status.isSuccess) + PostSucceeded + else + PostFailed(new UnexpectedStatusCodeException(s"Received unsuccessful status code [${response.status.value}] from collector.")) + } recover { case t: Throwable ⇒ PostFailed(t) } + } +} + +object MetricReporter { + val Extractors: List[MetricExtractor] = WebTransactionMetricExtractor :: CustomMetricExtractor :: Nil + + def props(settings: Agent.Settings, runID: Long, baseUri: Uri): Props = + Props(new MetricReporter(settings, runID, baseUri)) + + sealed trait MetricDataPostResult + case object PostSucceeded extends MetricDataPostResult + case class PostFailed(reason: Throwable) extends MetricDataPostResult + + class UnexpectedStatusCodeException(message: String) extends RuntimeException(message) with NoStackTrace +} + +trait MetricExtractor { + def extract(settings: Agent.Settings, collectionContext: CollectionContext, metrics: Map[MetricGroupIdentity, MetricGroupSnapshot]): Map[MetricID, MetricData] +} diff --git a/kamon-newrelic/src/main/scala/kamon/newrelic/MetricTranslator.scala b/kamon-newrelic/src/main/scala/kamon/newrelic/MetricTranslator.scala deleted file mode 100644 index 5fa571e1..00000000 --- a/kamon-newrelic/src/main/scala/kamon/newrelic/MetricTranslator.scala +++ /dev/null @@ -1,48 +0,0 @@ -/* - * ========================================================================================= - * Copyright © 2013 the kamon project - * - * 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 kamon.newrelic - -import akka.actor.{ Props, ActorRef, Actor } -import kamon.metric.Subscriptions.TickMetricSnapshot -import kamon.newrelic.MetricTranslator.TimeSliceMetrics - -class MetricTranslator(receiver: ActorRef) extends Actor - with WebTransactionMetrics with CustomMetrics { - - def receive = { - case TickMetricSnapshot(from, to, metrics) ⇒ - val fromInSeconds = (from / 1E3).toInt - val toInSeconds = (to / 1E3).toInt - val allMetrics = collectWebTransactionMetrics(metrics) ++ collectCustomMetrics(metrics) - val groupedMetrics: Map[String, NewRelic.Metric] = allMetrics.map(metric ⇒ metric.name -> metric)(collection.breakOut) // avoid intermediate tuple - - receiver ! TimeSliceMetrics(fromInSeconds, toInSeconds, groupedMetrics) - } - -} - -object MetricTranslator { - case class TimeSliceMetrics(from: Long, to: Long, metrics: Map[String, NewRelic.Metric]) { - import kamon.metric._ - - def merge(thatMetrics: Option[TimeSliceMetrics]): TimeSliceMetrics = { - thatMetrics.map(that ⇒ TimeSliceMetrics(from + that.from, to + that.to, combineMaps(metrics, that.metrics)((l, r) ⇒ l.merge(r)))).getOrElse(this) - } - } - - def props(receiver: ActorRef): Props = Props(new MetricTranslator(receiver)) -} diff --git a/kamon-newrelic/src/main/scala/kamon/newrelic/NewRelic.scala b/kamon-newrelic/src/main/scala/kamon/newrelic/NewRelic.scala index b270d228..a4be4c0b 100644 --- a/kamon-newrelic/src/main/scala/kamon/newrelic/NewRelic.scala +++ b/kamon-newrelic/src/main/scala/kamon/newrelic/NewRelic.scala @@ -16,61 +16,19 @@ package kamon.newrelic -import java.util.concurrent.TimeUnit.MILLISECONDS - import akka.actor import akka.actor._ +import akka.event.Logging import kamon.Kamon -import kamon.metric.Subscriptions.TickMetricSnapshot -import kamon.metric.UserMetrics.{ UserCounters, UserGauges, UserHistograms, UserMinMaxCounters } -import kamon.metric.{ Metrics, TickMetricSnapshotBuffer, TraceMetrics } - -import scala.concurrent.duration._ class NewRelicExtension(system: ExtendedActorSystem) extends Kamon.Extension { - val config = system.settings.config.getConfig("kamon.newrelic") - - val collectionContext = Kamon(Metrics)(system).buildDefaultCollectionContext - val metricsListener = system.actorOf(Props[NewRelicMetricsListener], "kamon-newrelic") - val apdexT: Double = config.getDuration("apdexT", MILLISECONDS) / 1E3 // scale to seconds. + val log = Logging(system, classOf[NewRelicExtension]) - Kamon(Metrics)(system).subscribe(TraceMetrics, "*", metricsListener, permanently = true) - - // Subscribe to all user metrics - Kamon(Metrics)(system).subscribe(UserHistograms, "*", metricsListener, permanently = true) - Kamon(Metrics)(system).subscribe(UserCounters, "*", metricsListener, permanently = true) - Kamon(Metrics)(system).subscribe(UserMinMaxCounters, "*", metricsListener, permanently = true) - Kamon(Metrics)(system).subscribe(UserGauges, "*", metricsListener, permanently = true) - -} - -class NewRelicMetricsListener extends Actor with ActorLogging { log.info("Starting the Kamon(NewRelic) extension") - - val agent = context.actorOf(Props[Agent], "agent") - val translator = context.actorOf(MetricTranslator.props(agent), "translator") - val buffer = context.actorOf(TickMetricSnapshotBuffer.props(1 minute, translator), "metric-buffer") - - def receive = { - case tick: TickMetricSnapshot ⇒ buffer.forward(tick) - } + val agent = system.actorOf(Props[Agent], "newrelic-agent") } object NewRelic extends ExtensionId[NewRelicExtension] with ExtensionIdProvider { def lookup(): ExtensionId[_ <: actor.Extension] = NewRelic def createExtension(system: ExtendedActorSystem): NewRelicExtension = new NewRelicExtension(system) - - case class Metric(name: String, scope: Option[String], callCount: Long, total: Double, totalExclusive: Double, - min: Double, max: Double, sumOfSquares: Double) { - - def merge(that: Metric): Metric = { - Metric(name, scope, - callCount + that.callCount, - total + that.total, - totalExclusive + that.totalExclusive, - math.min(min, that.min), - math.max(max, that.max), - sumOfSquares + that.sumOfSquares) - } - } } \ No newline at end of file diff --git a/kamon-newrelic/src/main/scala/kamon/newrelic/WebTransactionMetricExtractor.scala b/kamon-newrelic/src/main/scala/kamon/newrelic/WebTransactionMetricExtractor.scala new file mode 100644 index 00000000..cfb0e721 --- /dev/null +++ b/kamon-newrelic/src/main/scala/kamon/newrelic/WebTransactionMetricExtractor.scala @@ -0,0 +1,81 @@ +/* + * ========================================================================================= + * Copyright © 2013 the kamon project + * + * 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 kamon.newrelic + +import kamon.metric._ +import kamon.metric.TraceMetrics.ElapsedTime +import kamon.metric.instrument.Histogram +import kamon.trace.SegmentMetricIdentityLabel.HttpClient +import kamon.trace.SegmentMetricIdentity + +object WebTransactionMetricExtractor extends MetricExtractor { + + def extract(settings: Agent.Settings, collectionContext: CollectionContext, metrics: Map[MetricGroupIdentity, MetricGroupSnapshot]): Map[MetricID, MetricData] = { + val apdexBuilder = new ApdexBuilder("Apdex", None, settings.apdexT) + + // Trace metrics are recorded in nanoseconds. + var accumulatedHttpDispatcher: Histogram.Snapshot = Histogram.Snapshot.empty(Scale.Nano) + var accumulatedExternalServices: Histogram.Snapshot = Histogram.Snapshot.empty(Scale.Nano) + + val transactionMetrics = metrics.collect { + case (TraceMetrics(name), groupSnapshot) ⇒ + + groupSnapshot.metrics collect { + // Extract WebTransaction metrics and accumulate HttpDispatcher + case (ElapsedTime, snapshot: Histogram.Snapshot) ⇒ + accumulatedHttpDispatcher = accumulatedHttpDispatcher.merge(snapshot, collectionContext) + snapshot.recordsIterator.foreach { record ⇒ + apdexBuilder.record(Scale.convert(snapshot.scale, Scale.Unit, record.level), record.count) + } + + Metric.fromKamonMetricSnapshot(snapshot, s"WebTransaction/Custom/$name", None, Scale.Unit) + + // Extract all external services. + case (SegmentMetricIdentity(segmentName, label), snapshot: Histogram.Snapshot) if label.equals(HttpClient)⇒ + accumulatedExternalServices = accumulatedExternalServices.merge(snapshot, collectionContext) + + Metric.fromKamonMetricSnapshot(snapshot, s"External/$segmentName/all", None, Scale.Unit) + } + } + + val httpDispatcher = Metric.fromKamonMetricSnapshot(accumulatedHttpDispatcher, "HttpDispatcher", None, Scale.Unit) + val webTransaction = Metric.fromKamonMetricSnapshot(accumulatedHttpDispatcher, "WebTransaction", None, Scale.Unit) + val external = Metric.fromKamonMetricSnapshot(accumulatedExternalServices, "External", None, Scale.Unit) + val externalAllWeb = Metric.fromKamonMetricSnapshot(accumulatedExternalServices, "External/allWeb", None, Scale.Unit) + + Map(httpDispatcher, webTransaction, external, externalAllWeb, apdexBuilder.build) ++ transactionMetrics.flatten.toMap + } +} + +class ApdexBuilder(name: String, scope: Option[String], apdexT: Double) { + val frustratingThreshold = 4 * apdexT + + var satisfying = 0L + var tolerating = 0L + var frustrating = 0L + + def record(duration: Double, count: Long): Unit = + if (duration <= apdexT) + satisfying += count + else if (duration <= frustratingThreshold) + tolerating += count + else + frustrating += count + + // NewRelic reuses the same metric structure for recording the Apdex.. weird, but that's how it works. + def build: Metric = (MetricID(name, scope), MetricData(satisfying, tolerating, frustrating, apdexT, apdexT, 0)) +} diff --git a/kamon-newrelic/src/main/scala/kamon/newrelic/WebTransactionMetrics.scala b/kamon-newrelic/src/main/scala/kamon/newrelic/WebTransactionMetrics.scala deleted file mode 100644 index a8c54684..00000000 --- a/kamon-newrelic/src/main/scala/kamon/newrelic/WebTransactionMetrics.scala +++ /dev/null @@ -1,74 +0,0 @@ -/* - * ========================================================================================= - * Copyright © 2013 the kamon project - * - * 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 kamon.newrelic - -import kamon.metric._ -import kamon.metric.TraceMetrics.ElapsedTime -import akka.actor.Actor -import kamon.Kamon -import kamon.metric.instrument.Histogram - -trait WebTransactionMetrics { - self: Actor ⇒ - - def collectWebTransactionMetrics(metrics: Map[MetricGroupIdentity, MetricGroupSnapshot]): Seq[NewRelic.Metric] = { - val newRelicExtension = Kamon(NewRelic)(context.system) - val apdexBuilder = new ApdexBuilder("Apdex", None, newRelicExtension.apdexT) - val collectionContext = newRelicExtension.collectionContext - - // Trace metrics are recorded in nanoseconds. - var accumulatedHttpDispatcher: Histogram.Snapshot = Histogram.Snapshot.empty(Scale.Nano) - - val webTransactionMetrics = metrics.collect { - case (TraceMetrics(name), groupSnapshot) ⇒ - - groupSnapshot.metrics collect { - case (ElapsedTime, snapshot: Histogram.Snapshot) ⇒ - accumulatedHttpDispatcher = accumulatedHttpDispatcher.merge(snapshot, collectionContext) - snapshot.recordsIterator.foreach { record ⇒ - apdexBuilder.record(Scale.convert(snapshot.scale, Scale.Unit, record.level), record.count) - } - - toNewRelicMetric(Scale.Unit)(s"WebTransaction/Custom/$name", None, snapshot) - } - } - - val httpDispatcher = toNewRelicMetric(Scale.Unit)("HttpDispatcher", None, accumulatedHttpDispatcher) - val webTransaction = toNewRelicMetric(Scale.Unit)("WebTransaction", None, accumulatedHttpDispatcher) - - Seq(httpDispatcher, webTransaction, apdexBuilder.build) ++ webTransactionMetrics.flatten.toSeq - } -} - -class ApdexBuilder(name: String, scope: Option[String], apdexT: Double) { - val frustratingThreshold = 4 * apdexT - - var satisfying = 0L - var tolerating = 0L - var frustrating = 0L - - def record(duration: Double, count: Long): Unit = - if (duration <= apdexT) - satisfying += count - else if (duration <= frustratingThreshold) - tolerating += count - else - frustrating += count - - // NewRelic reuses the same metric structure for recording the Apdex.. weird, but that's how it works. - def build: NewRelic.Metric = NewRelic.Metric(name, scope, satisfying, tolerating, frustrating, apdexT, apdexT, 0) -} diff --git a/kamon-newrelic/src/main/scala/kamon/newrelic/package.scala b/kamon-newrelic/src/main/scala/kamon/newrelic/package.scala index 89a8b15b..06c3dad0 100644 --- a/kamon-newrelic/src/main/scala/kamon/newrelic/package.scala +++ b/kamon-newrelic/src/main/scala/kamon/newrelic/package.scala @@ -1,45 +1,5 @@ -/*========================================================================================= - * Copyright © 2013 the kamon project - * - * 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 kamon -import kamon.metric.instrument.{ Counter, Histogram } -import kamon.metric.{ MetricSnapshot, Scale } - package object newrelic { - - def toNewRelicMetric(scale: Scale)(name: String, scope: Option[String], snapshot: MetricSnapshot): NewRelic.Metric = { - snapshot match { - case hs: Histogram.Snapshot ⇒ - var total: Double = 0D - var sumOfSquares: Double = 0D - val scaledMin = Scale.convert(hs.scale, scale, hs.min) - val scaledMax = Scale.convert(hs.scale, scale, hs.max) - - hs.recordsIterator.foreach { record ⇒ - val scaledValue = Scale.convert(hs.scale, scale, record.level) - - total += scaledValue * record.count - sumOfSquares += (scaledValue * scaledValue) * record.count - } - - NewRelic.Metric(name, scope, hs.numberOfMeasurements, total, total, scaledMin, scaledMax, sumOfSquares) - - case cs: Counter.Snapshot ⇒ - NewRelic.Metric(name, scope, cs.count, cs.count, cs.count, 0, cs.count, cs.count * cs.count) - } - } + type Metric = (MetricID, MetricData) } diff --git a/kamon-newrelic/src/test/scala/kamon/newrelic/AgentSpec.scala b/kamon-newrelic/src/test/scala/kamon/newrelic/AgentSpec.scala index 8b61c241..7db9f2d0 100644 --- a/kamon-newrelic/src/test/scala/kamon/newrelic/AgentSpec.scala +++ b/kamon-newrelic/src/test/scala/kamon/newrelic/AgentSpec.scala @@ -16,20 +16,23 @@ package kamon.newrelic +import java.lang.management.ManagementFactory + import akka.actor.{ ActorRef, ActorSystem, Props } import akka.io.IO -import akka.testkit.TestActor.{ AutoPilot, KeepRunning } import akka.testkit._ import com.typesafe.config.ConfigFactory import kamon.AkkaExtensionSwap -import kamon.newrelic.MetricTranslator.TimeSliceMetrics import org.scalatest.{ BeforeAndAfterAll, WordSpecLike } import spray.can.Http -import spray.http.{ HttpRequest, HttpResponse, _ } - -class AgentSpec extends TestKitBase with WordSpecLike with BeforeAndAfterAll { +import spray.http._ +import spray.httpx.encoding.Deflate +import spray.httpx.{ SprayJsonSupport, RequestBuilding } +import spray.json.JsArray +import spray.json._ - import kamon.newrelic.AgentSpec._ +class AgentSpec extends TestKitBase with WordSpecLike with BeforeAndAfterAll with RequestBuilding with SprayJsonSupport { + import JsonProtocol._ implicit lazy val system: ActorSystem = ActorSystem("Agent-Spec", ConfigFactory.parseString( """ @@ -39,120 +42,180 @@ class AgentSpec extends TestKitBase with WordSpecLike with BeforeAndAfterAll { |} |kamon { | newrelic { - | retry-delay = 1 second - | max-retry = 3 + | app-name = kamon + | license-key = 1111111111 + | initialize-retry-delay = 1 second + | max-initialize-retries = 3 | } |} | """.stripMargin)) - var agent: ActorRef = _ - - setupFakeHttpManager - - "the Newrelic Agent" should { - "try to connect upon creation, retry to connect if an error occurs" in { - EventFilter.info(message = "Initialization failed: Unexpected response from HTTP transport: None, retrying in 1 seconds", occurrences = 3).intercept { - system.actorOf(Props[Agent]) - Thread.sleep(1000) - } - } - - "when everything is fine should select a NewRelic collector" in { + "the New Relic Agent" should { + "try to establish a connection to the collector upon creation" in { + val httpManager = setHttpManager(TestProbe()) + val agent = system.actorOf(Props[Agent]) + + // Request NR for a collector + httpManager.expectMsg(Deflate.encode { + Post(rawMethodUri("collector.newrelic.com", "get_redirect_host"), JsArray()) + }) + + // Receive the assigned collector + httpManager.reply(jsonResponse( + """ + | { + | "return_value": "collector-8.newrelic.com" + | } + | """.stripMargin)) + + // Connect to the collector + val (host, pid) = getHostAndPid() + httpManager.expectMsg(Deflate.encode { + Post(rawMethodUri("collector-8.newrelic.com", "connect"), + s""" + | [ + | { + | "agent_version": "3.1.0", + | "app_name": [ "kamon" ], + | "host": "$host", + | "identifier": "java:kamon", + | "language": "java", + | "pid": $pid + | } + | ] + """.stripMargin.parseJson)(sprayJsonMarshaller(JsValueFormat)) + }) + + // Receive the runID EventFilter.info(message = "Agent initialized with runID: [161221111] and collector: [collector-8.newrelic.com]", occurrences = 1).intercept { - system.actorOf(Props[Agent]) + httpManager.reply(jsonResponse( + """ + | { + | "return_value": { + | "agent_run_id": 161221111 + | } + | } + | """.stripMargin)) } } - "merge the metrics if not possible send them and do it in the next post" in { - EventFilter.info(pattern = "Trying to send metrics to NewRelic collector, attempt.*", occurrences = 2).intercept { - agent = system.actorOf(Props[Agent].withDispatcher(CallingThreadDispatcher.Id)) + "retry the connection in case it fails" in { + val httpManager = setHttpManager(TestProbe()) + val agent = system.actorOf(Props[Agent]) - for (_ ← 1 to 3) { - sendDelayedMetric(agent) - } - } - } + // Request NR for a collector + val request = httpManager.expectMsg(Deflate.encode { + Post(rawMethodUri("collector.newrelic.com", "get_redirect_host"), JsArray()) + }) - "when the connection is re-established, the metrics should be send" in { - EventFilter.info(message = "Sending metrics to NewRelic collector", occurrences = 2).intercept { - sendDelayedMetric(agent) + // Fail the request. + EventFilter[RuntimeException](start = "Initialization failed, retrying in 1 seconds", occurrences = 1).intercept { + httpManager.reply(Timedout(request)) } + + // Request NR for a collector, second attempt + httpManager.expectMsg(Deflate.encode { + Post(rawMethodUri("collector.newrelic.com", "get_redirect_host"), JsArray()) + }) + + // Receive the assigned collector + httpManager.reply(jsonResponse( + """ + | { + | "return_value": "collector-8.newrelic.com" + | } + | """.stripMargin)) + + // Connect to the collector + val (host, pid) = getHostAndPid() + httpManager.expectMsg(Deflate.encode { + Post(rawMethodUri("collector-8.newrelic.com", "connect"), + s""" + | [ + | { + | "agent_version": "3.1.0", + | "app_name": [ "kamon" ], + | "host": "$host", + | "identifier": "java:kamon", + | "language": "java", + | "pid": $pid + | } + | ] + """.stripMargin.parseJson)(sprayJsonMarshaller(JsValueFormat)) + }) + + // Receive the runID + EventFilter.info( + message = "Agent initialized with runID: [161221112] and collector: [collector-8.newrelic.com]", occurrences = 1).intercept { + + httpManager.reply(jsonResponse( + """ + | { + | "return_value": { + | "agent_run_id": 161221112 + | } + | } + | """.stripMargin)) + } } - } - def setupFakeHttpManager: Unit = { - val ConnectionAttempts = 3 // an arbitrary value only for testing purposes - val PostAttempts = 3 // if the number is achieved, the metrics should be discarded - val fakeHttpManager = TestProbe() - var attemptsToConnect: Int = 0 // should retry grab an NewRelic collector after retry-delay - var attemptsToSendMetrics: Int = 0 - - fakeHttpManager.setAutoPilot(new TestActor.AutoPilot { - def run(sender: ActorRef, msg: Any): AutoPilot = { - msg match { - case HttpRequest(_, uri, _, _, _) if rawMethodIs("get_redirect_host", uri) ⇒ - if (attemptsToConnect == ConnectionAttempts) { - sender ! jsonResponse( - """ - | { - | "return_value": "collector-8.newrelic.com" - | } - | """.stripMargin) - system.log.info("Selecting Collector") - - } else { - sender ! None - attemptsToConnect += 1 - system.log.info("Network Error or Connection Refuse") - } - - case HttpRequest(_, uri, _, _, _) if rawMethodIs("connect", uri) ⇒ - sender ! jsonResponse( - """ - | { - | "return_value": { - | "agent_run_id": 161221111 - | } - | } - | """.stripMargin) - system.log.info("Connecting") - - case HttpRequest(_, uri, _, _, _) if rawMethodIs("metric_data", uri) ⇒ - if (attemptsToSendMetrics < PostAttempts) { - sender ! None - attemptsToSendMetrics += 1 - system.log.info("Error when trying to send metrics to NewRelic collector, the metrics will be merged") - } else { - system.log.info("Sending metrics to NewRelic collector") - } + "give up the connection after max-initialize-retries" in { + val httpManager = setHttpManager(TestProbe()) + val agent = system.actorOf(Props[Agent]) + + // First attempt and two retries + for (_ ← 1 to 3) { + + // Request NR for a collector + val request = httpManager.expectMsg(Deflate.encode { + Post(rawMethodUri("collector.newrelic.com", "get_redirect_host"), JsArray()) + }) + + // Fail the request. + EventFilter[RuntimeException](start = "Initialization failed, retrying in 1 seconds", occurrences = 1).intercept { + httpManager.reply(Timedout(request)) } - KeepRunning } - def jsonResponse(json: String): HttpResponse = { - HttpResponse(entity = HttpEntity(ContentTypes.`application/json`, json)) - } + // Final retry. Request NR for a collector + val request = httpManager.expectMsg(Deflate.encode { + Post(rawMethodUri("collector.newrelic.com", "get_redirect_host"), JsArray()) + }) - def rawMethodIs(method: String, uri: Uri): Boolean = { - uri.query.get("method").filter(_ == method).isDefined + // Give up on connecting. + EventFilter[RuntimeException](message = "Giving up while trying to set up a connection with the New Relic collector.", occurrences = 1).intercept { + httpManager.reply(Timedout(request)) } - }) + } + } + def setHttpManager(probe: TestProbe): TestProbe = { AkkaExtensionSwap.swap(system, Http, new IO.Extension { - def manager: ActorRef = fakeHttpManager.ref + def manager: ActorRef = probe.ref }) + probe + } + + def rawMethodUri(host: String, methodName: String): Uri = { + Uri(s"http://$host/agent_listener/invoke_raw_method").withQuery( + "method" -> methodName, + "license_key" -> "1111111111", + "marshal_format" -> "json", + "protocol_version" -> "12") + } + + def jsonResponse(json: String): HttpResponse = { + HttpResponse(entity = HttpEntity(ContentTypes.`application/json`, json)) } - override def afterAll() { - super.afterAll() - system.shutdown() + def getHostAndPid(): (String, String) = { + val runtimeName = ManagementFactory.getRuntimeMXBean.getName.split('@') + (runtimeName(1), runtimeName(0)) } -} -object AgentSpec { - def sendDelayedMetric(agent: ActorRef, delay: Int = 1000): Unit = { - agent ! TimeSliceMetrics(100000L, 200000L, Map("Latency" -> NewRelic.Metric("Latency", None, 1000L, 2000D, 3000D, 1D, 100000D, 300D))) - Thread.sleep(delay) + implicit def JsValueFormat = new RootJsonFormat[JsValue] { + def write(value: JsValue) = value + def read(value: JsValue) = value } } \ No newline at end of file diff --git a/kamon-newrelic/src/test/scala/kamon/newrelic/MetricReporterSpec.scala b/kamon-newrelic/src/test/scala/kamon/newrelic/MetricReporterSpec.scala new file mode 100644 index 00000000..3cf4bbd0 --- /dev/null +++ b/kamon-newrelic/src/test/scala/kamon/newrelic/MetricReporterSpec.scala @@ -0,0 +1,156 @@ +/* + * ========================================================================================= + * Copyright © 2013-2014 the kamon project + * + * 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 kamon.newrelic + +import akka.actor.{ ActorRef, ActorSystem } +import akka.io.IO +import akka.testkit._ +import com.typesafe.config.ConfigFactory +import kamon.metric.{ TraceMetrics, Metrics } +import kamon.{ Kamon, AkkaExtensionSwap } +import kamon.metric.Subscriptions.TickMetricSnapshot +import org.scalatest.WordSpecLike +import spray.can.Http +import spray.http.Uri.Query +import spray.http._ +import spray.httpx.encoding.Deflate +import spray.httpx.{ RequestBuilding, SprayJsonSupport } +import scala.concurrent.duration._ + +class MetricReporterSpec extends TestKitBase with WordSpecLike with RequestBuilding with SprayJsonSupport { + import kamon.newrelic.JsonProtocol._ + + implicit lazy val system: ActorSystem = ActorSystem("metric-reporter-spec", ConfigFactory.parseString( + """ + |akka { + | loggers = ["akka.testkit.TestEventListener"] + | loglevel = "INFO" + |} + |kamon { + | metric { + | tick-interval = 1 hour + | } + |} + | + """.stripMargin)) + + val agentSettings = Agent.Settings("1111111111", "kamon", "test-host", 1, 1, 30 seconds, 1D) + val baseQuery = Query( + "license_key" -> agentSettings.licenseKey, + "marshal_format" -> "json", + "protocol_version" -> "12") + val baseCollectorUri = Uri("http://collector-1.newrelic.com/agent_listener/invoke_raw_method").withQuery(baseQuery) + + "the MetricReporter" should { + "report metrics to New Relic upon arrival" in new FakeTickSnapshotsFixture { + val httpManager = setHttpManager(TestProbe()) + val metricReporter = system.actorOf(MetricReporter.props(agentSettings, 9999, baseCollectorUri)) + + metricReporter ! firstSnapshot + httpManager.expectMsg(Deflate.encode { + HttpRequest(method = HttpMethods.POST, uri = rawMethodUri("collector-1.newrelic.com", "metric_data"), entity = compactJsonEntity( + s""" + |[9999,0,0, + |[ + | [{"name":"Apdex"},[3,0.0,0.0,1.0,1.0,0.0]], + | [{"name":"WebTransaction"},[3,0.005996544,0.005996544,0.000999424,0.002998272,0.000013983876644864]], + | [{"name":"External"}, [0, 0.0, 0.0, 0.0, 0.0, 0.0]], + | [{"name":"WebTransaction/Custom/example-trace"},[3,0.005996544,0.005996544,0.000999424,0.002998272,0.000013983876644864]], + | [{"name":"HttpDispatcher"},[3,0.005996544,0.005996544,0.000999424,0.002998272,0.000013983876644864]], + | [{"name":"External/allWeb"}, [0, 0.0, 0.0, 0.0, 0.0, 0.0]] + |] + |] + """.stripMargin)) + }) + } + + "accumulate metrics if posting fails" in new FakeTickSnapshotsFixture { + val httpManager = setHttpManager(TestProbe()) + val metricReporter = system.actorOf(MetricReporter.props(agentSettings, 9999, baseCollectorUri)) + + metricReporter ! firstSnapshot + val request = httpManager.expectMsgType[HttpRequest] + httpManager.reply(Timedout(request)) + + metricReporter ! secondSnapshot + httpManager.expectMsg(Deflate.encode { + HttpRequest(method = HttpMethods.POST, uri = rawMethodUri("collector-1.newrelic.com", "metric_data"), entity = compactJsonEntity( + s""" + |[9999,0,0, + |[ + | [{"name":"Apdex"},[6,0.0,0.0,1.0,1.0,0.0]], + | [{"name":"WebTransaction"},[6,0.02097152,0.02097152,0.000999424,0.005996544,0.000090731720998912]], + | [{"name": "External"}, [0, 0.0, 0.0, 0.0, 0.0, 0.0]], + | [{"name":"WebTransaction/Custom/example-trace"},[6,0.02097152,0.02097152,0.000999424,0.005996544,0.000090731720998912]], + | [{"name":"HttpDispatcher"},[6,0.02097152,0.02097152,0.000999424,0.005996544,0.000090731720998912]], + | [{"name": "External/allWeb"}, [0, 0.0, 0.0, 0.0, 0.0, 0.0]] + |] + |] + """.stripMargin)) + }) + } + } + /* + [9999, 0, 0, [ + [{"name": "Apdex"}, [6, 0.0, 0.0, 1.0, 1.0, 0.0]], + [{"name": "WebTransaction"}, [6, 0.02097152, 0.02097152, 0.000999424, 0.005996544, 0.000090731720998912]], + [{"name": "External"}, [0, 0.0, 0.0, 0.0, 0.0, 0.0]], + [{"name": "WebTransaction/Custom/example-trace"}, [6, 0.02097152, 0.02097152, 0.000999424, 0.005996544, 0.000090731720998912]], + [{"name": "HttpDispatcher"}, [6, 0.02097152, 0.02097152, 0.000999424, 0.005996544, 0.000090731720998912]], + [{"name": "External/allWeb"}, [0, 0.0, 0.0, 0.0, 0.0, 0.0]]]]*/ + + def setHttpManager(probe: TestProbe): TestProbe = { + AkkaExtensionSwap.swap(system, Http, new IO.Extension { + def manager: ActorRef = probe.ref + }) + probe + } + + def rawMethodUri(host: String, methodName: String): Uri = { + Uri(s"http://$host/agent_listener/invoke_raw_method").withQuery( + "method" -> methodName, + "run_id" -> "9999", + "license_key" -> "1111111111", + "marshal_format" -> "json", + "protocol_version" -> "12") + } + + def compactJsonEntity(jsonString: String): HttpEntity = { + import spray.json._ + + val compactJson = jsonString.parseJson.compactPrint + HttpEntity(ContentTypes.`application/json`, compactJson) + } + + trait FakeTickSnapshotsFixture { + val testTraceID = TraceMetrics("example-trace") + val recorder = Kamon(Metrics).register(testTraceID, TraceMetrics.Factory).get + val collectionContext = Kamon(Metrics).buildDefaultCollectionContext + + def collectRecorder = recorder.collect(collectionContext) + + recorder.elapsedTime.record(1000000) + recorder.elapsedTime.record(2000000) + recorder.elapsedTime.record(3000000) + val firstSnapshot = TickMetricSnapshot(1, 100, Map(testTraceID -> collectRecorder)) + + recorder.elapsedTime.record(6000000) + recorder.elapsedTime.record(5000000) + recorder.elapsedTime.record(4000000) + val secondSnapshot = TickMetricSnapshot(100, 200, Map(testTraceID -> collectRecorder)) + } +} \ No newline at end of file diff --git a/kamon-playground/src/main/resources/application.conf b/kamon-playground/src/main/resources/application.conf index 32f0269d..86a87439 100644 --- a/kamon-playground/src/main/resources/application.conf +++ b/kamon-playground/src/main/resources/application.conf @@ -1,5 +1,5 @@ akka { - loglevel = INFO + loglevel = DEBUG extensions = ["kamon.newrelic.NewRelic"] actor { diff --git a/kamon-playground/src/main/scala/test/SimpleRequestProcessor.scala b/kamon-playground/src/main/scala/test/SimpleRequestProcessor.scala index 878c3c8c..506e0bff 100644 --- a/kamon-playground/src/main/scala/test/SimpleRequestProcessor.scala +++ b/kamon-playground/src/main/scala/test/SimpleRequestProcessor.scala @@ -106,7 +106,7 @@ object SimpleRequestProcessor extends App with SimpleRoutingApp with RequestBuil } } ~ path("future") { - traceName("OK-Future") { + traceName("OKFuture") { dynamic { counter.increment() complete(Future { "OK" }) -- cgit v1.2.3