Re: Registering custom metrics

2015-01-08 Thread Gerard Maas
Very interesting approach. Thanks for sharing it!

On Thu, Jan 8, 2015 at 5:30 PM, Enno Shioji eshi...@gmail.com wrote:

 FYI I found this approach by Ooyala.

 /** Instrumentation for Spark based on accumulators.
   *
   * Usage:
   * val instrumentation = new SparkInstrumentation(example.metrics)
   * val numReqs = sc.accumulator(0L)
   * instrumentation.source.registerDailyAccumulator(numReqs, numReqs)
   * instrumentation.register()
   *
   * Will create and report the following metrics:
   * - Gauge with total number of requests (daily)
   * - Meter with rate of requests
   *
   * @param prefix prefix for all metrics that will be reported by this 
 Instrumentation
   */

 https://gist.github.com/ibuenros/9b94736c2bad2f4b8e23
 ᐧ

 On Mon, Jan 5, 2015 at 2:56 PM, Enno Shioji eshi...@gmail.com wrote:

 Hi Gerard,

 Thanks for the answer! I had a good look at it, but I couldn't figure out
 whether one can use that to emit metrics from your application code.

 Suppose I wanted to monitor the rate of bytes I produce, like so:

 stream
 .map { input =
   val bytes = produce(input)
   // metricRegistry.meter(some.metrics).mark(bytes.length)
   bytes
 }
 .saveAsTextFile(text)

 Is there a way to achieve this with the MetricSystem?


 ᐧ

 On Mon, Jan 5, 2015 at 10:24 AM, Gerard Maas gerard.m...@gmail.com
 wrote:

 Hi,

 Yes, I managed to create a register custom metrics by creating an
  implementation  of org.apache.spark.metrics.source.Source and
 registering it to the metrics subsystem.
 Source is [Spark] private, so you need to create it under a org.apache.spark
 package. In my case, I'm dealing with Spark Streaming metrics, and I
 created my CustomStreamingSource under org.apache.spark.streaming as I
 also needed access to some [Streaming] private components.

 Then, you register your new metric Source on the Spark's metric system,
 like so:

 SparkEnv.get.metricsSystem.registerSource(customStreamingSource)

 And it will get reported to the metrics Sync active on your system. By
 default, you can access them through the metric endpoint:
 http://driver-host:ui-port/metrics/json

 I hope this helps.

 -kr, Gerard.






 On Tue, Dec 30, 2014 at 3:32 PM, eshioji eshi...@gmail.com wrote:

 Hi,

 Did you find a way to do this / working on this?
 Am trying to find a way to do this as well, but haven't been able to
 find a
 way.



 --
 View this message in context:
 http://apache-spark-developers-list.1001551.n3.nabble.com/Registering-custom-metrics-tp9030p9968.html
 Sent from the Apache Spark Developers List mailing list archive at
 Nabble.com.

 -
 To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org
 For additional commands, e-mail: dev-h...@spark.apache.org







Re: Registering custom metrics

2015-01-05 Thread Gerard Maas
Hi,

Yes, I managed to create a register custom metrics by creating an
 implementation  of org.apache.spark.metrics.source.Source and registering
it to the metrics subsystem.
Source is [Spark] private, so you need to create it under a org.apache.spark
package. In my case, I'm dealing with Spark Streaming metrics, and I
created my CustomStreamingSource under org.apache.spark.streaming as I also
needed access to some [Streaming] private components.

Then, you register your new metric Source on the Spark's metric system,
like so:

SparkEnv.get.metricsSystem.registerSource(customStreamingSource)

And it will get reported to the metrics Sync active on your system. By
default, you can access them through the metric endpoint:
http://driver-host:ui-port/metrics/json

I hope this helps.

-kr, Gerard.






On Tue, Dec 30, 2014 at 3:32 PM, eshioji eshi...@gmail.com wrote:

 Hi,

 Did you find a way to do this / working on this?
 Am trying to find a way to do this as well, but haven't been able to find a
 way.



 --
 View this message in context:
 http://apache-spark-developers-list.1001551.n3.nabble.com/Registering-custom-metrics-tp9030p9968.html
 Sent from the Apache Spark Developers List mailing list archive at
 Nabble.com.

 -
 To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org
 For additional commands, e-mail: dev-h...@spark.apache.org




Re: Registering custom metrics

2015-01-05 Thread Enno Shioji
Hi Gerard,

Thanks for the answer! I had a good look at it, but I couldn't figure out
whether one can use that to emit metrics from your application code.

Suppose I wanted to monitor the rate of bytes I produce, like so:

stream
.map { input =
  val bytes = produce(input)
  // metricRegistry.meter(some.metrics).mark(bytes.length)
  bytes
}
.saveAsTextFile(text)

Is there a way to achieve this with the MetricSystem?


ᐧ

On Mon, Jan 5, 2015 at 10:24 AM, Gerard Maas gerard.m...@gmail.com wrote:

 Hi,

 Yes, I managed to create a register custom metrics by creating an
  implementation  of org.apache.spark.metrics.source.Source and
 registering it to the metrics subsystem.
 Source is [Spark] private, so you need to create it under a org.apache.spark
 package. In my case, I'm dealing with Spark Streaming metrics, and I
 created my CustomStreamingSource under org.apache.spark.streaming as I
 also needed access to some [Streaming] private components.

 Then, you register your new metric Source on the Spark's metric system,
 like so:

 SparkEnv.get.metricsSystem.registerSource(customStreamingSource)

 And it will get reported to the metrics Sync active on your system. By
 default, you can access them through the metric endpoint:
 http://driver-host:ui-port/metrics/json

 I hope this helps.

 -kr, Gerard.






 On Tue, Dec 30, 2014 at 3:32 PM, eshioji eshi...@gmail.com wrote:

 Hi,

 Did you find a way to do this / working on this?
 Am trying to find a way to do this as well, but haven't been able to find
 a
 way.



 --
 View this message in context:
 http://apache-spark-developers-list.1001551.n3.nabble.com/Registering-custom-metrics-tp9030p9968.html
 Sent from the Apache Spark Developers List mailing list archive at
 Nabble.com.

 -
 To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org
 For additional commands, e-mail: dev-h...@spark.apache.org





Re: Registering custom metrics

2014-12-30 Thread eshioji
Hi,

Did you find a way to do this / working on this?
Am trying to find a way to do this as well, but haven't been able to find a
way.



--
View this message in context: 
http://apache-spark-developers-list.1001551.n3.nabble.com/Registering-custom-metrics-tp9030p9968.html
Sent from the Apache Spark Developers List mailing list archive at Nabble.com.

-
To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org
For additional commands, e-mail: dev-h...@spark.apache.org