You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by ma...@apache.org on 2013/11/25 01:20:10 UTC

[1/3] git commit: Add graphite sink for metrics

Updated Branches:
  refs/heads/master 65de73c7f -> 859d62dc2


Add graphite sink for metrics

This adds a metrics sink for graphite.  The sink must
be configured with the host and port of a graphite node
and optionally may be configured with a prefix that will
be prepended to all metrics that are sent to graphite.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/ef85a51f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/ef85a51f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/ef85a51f

Branch: refs/heads/master
Commit: ef85a51f85c9720bc091367a0d4f80e7ed6b9778
Parents: 3d4ad84
Author: Russell Cardullo <ru...@gmail.com>
Authored: Fri Nov 8 16:36:03 2013 -0800
Committer: Russell Cardullo <ru...@gmail.com>
Committed: Fri Nov 8 16:36:03 2013 -0800

----------------------------------------------------------------------
 conf/metrics.properties.template                |  8 ++
 core/pom.xml                                    |  4 +
 .../spark/metrics/sink/GraphiteSink.scala       | 82 ++++++++++++++++++++
 docs/monitoring.md                              |  1 +
 project/SparkBuild.scala                        |  1 +
 5 files changed, 96 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ef85a51f/conf/metrics.properties.template
----------------------------------------------------------------------
diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template
index ae10f61..1c3d94e 100644
--- a/conf/metrics.properties.template
+++ b/conf/metrics.properties.template
@@ -80,6 +80,14 @@
 #     /metrics/aplications/json # App information
 #     /metrics/master/json      # Master information
 
+# org.apache.spark.metrics.sink.GraphiteSink
+#   Name:     Default:      Description:
+#   host      NONE          Hostname of Graphite server
+#   port      NONE          Port of Graphite server
+#   period    10            Poll period
+#   unit      seconds       Units of poll period
+#   prefix    EMPTY STRING  Prefix to prepend to metric name
+
 ## Examples
 # Enable JmxSink for all instances by class name
 #*.sink.jmx.class=org.apache.spark.metrics.sink.JmxSink

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ef85a51f/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index 8621d25..6af229c 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -159,6 +159,10 @@
       <artifactId>metrics-ganglia</artifactId>
     </dependency>
     <dependency>
+      <groupId>com.codahale.metrics</groupId>
+      <artifactId>metrics-graphite</artifactId>
+    </dependency>
+    <dependency>
       <groupId>org.apache.derby</groupId>
       <artifactId>derby</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ef85a51f/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala
new file mode 100644
index 0000000..eb1315e
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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.apache.spark.metrics.sink
+
+import com.codahale.metrics.MetricRegistry
+import com.codahale.metrics.graphite.{GraphiteReporter, Graphite}
+
+import java.util.Properties
+import java.util.concurrent.TimeUnit
+import java.net.InetSocketAddress
+
+import org.apache.spark.metrics.MetricsSystem
+
+class GraphiteSink(val property: Properties, val registry: MetricRegistry) extends Sink {
+  val GRAPHITE_DEFAULT_PERIOD = 10
+  val GRAPHITE_DEFAULT_UNIT = "SECONDS"
+  val GRAPHITE_DEFAULT_PREFIX = ""
+
+  val GRAPHITE_KEY_HOST = "host"
+  val GRAPHITE_KEY_PORT = "port"
+  val GRAPHITE_KEY_PERIOD = "period"
+  val GRAPHITE_KEY_UNIT = "unit"
+  val GRAPHITE_KEY_PREFIX = "prefix"
+
+  def propertyToOption(prop: String) = Option(property.getProperty(prop))
+
+  if (!propertyToOption(GRAPHITE_KEY_HOST).isDefined) {
+    throw new Exception("Graphite sink requires 'host' property.")
+  }
+
+  if (!propertyToOption(GRAPHITE_KEY_PORT).isDefined) {
+    throw new Exception("Graphite sink requires 'port' property.")
+  }
+
+  val host = propertyToOption(GRAPHITE_KEY_HOST).get
+  val port = propertyToOption(GRAPHITE_KEY_PORT).get.toInt
+
+  val pollPeriod = Option(property.getProperty(GRAPHITE_KEY_PERIOD)) match {
+    case Some(s) => s.toInt
+    case None => GRAPHITE_DEFAULT_PERIOD
+  }
+
+  val pollUnit = Option(property.getProperty(GRAPHITE_KEY_UNIT)) match {
+    case Some(s) => TimeUnit.valueOf(s.toUpperCase())
+    case None => TimeUnit.valueOf(GRAPHITE_DEFAULT_UNIT)
+  }
+
+  val prefix = propertyToOption(GRAPHITE_KEY_PREFIX).getOrElse(GRAPHITE_DEFAULT_PREFIX)
+
+  MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod)
+
+  val graphite: Graphite = new Graphite(new InetSocketAddress(host, port))
+
+  val reporter: GraphiteReporter = GraphiteReporter.forRegistry(registry)
+      .convertDurationsTo(TimeUnit.MILLISECONDS)
+      .convertRatesTo(TimeUnit.SECONDS)
+      .prefixedWith(prefix)
+      .build(graphite)
+
+  override def start() {
+    reporter.start(pollPeriod, pollUnit)
+  }
+
+  override def stop() {
+    reporter.stop()
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ef85a51f/docs/monitoring.md
----------------------------------------------------------------------
diff --git a/docs/monitoring.md b/docs/monitoring.md
index 5f456b9..5ed0474 100644
--- a/docs/monitoring.md
+++ b/docs/monitoring.md
@@ -50,6 +50,7 @@ Each instance can report to zero or more _sinks_. Sinks are contained in the
 * `GangliaSink`: Sends metrics to a Ganglia node or multicast group.
 * `JmxSink`: Registers metrics for viewing in a JXM console.
 * `MetricsServlet`: Adds a servlet within the existing Spark UI to serve metrics data as JSON data.
+* `GraphiteSink`: Sends metrics to a Graphite node.
 
 The syntax of the metrics configuration file is defined in an example configuration file, 
 `$SPARK_HOME/conf/metrics.conf.template`.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ef85a51f/project/SparkBuild.scala
----------------------------------------------------------------------
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 45fd30a..0bc2ca8 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -229,6 +229,7 @@ object SparkBuild extends Build {
       "com.codahale.metrics" % "metrics-jvm" % "3.0.0",
       "com.codahale.metrics" % "metrics-json" % "3.0.0",
       "com.codahale.metrics" % "metrics-ganglia" % "3.0.0",
+      "com.codahale.metrics" % "metrics-graphite" % "3.0.0",
       "com.twitter" % "chill_2.9.3" % "0.3.1",
       "com.twitter" % "chill-java" % "0.3.1"
     )


[3/3] git commit: Merge pull request #151 from russellcardullo/add-graphite-sink

Posted by ma...@apache.org.
Merge pull request #151 from russellcardullo/add-graphite-sink

Add graphite sink for metrics

This adds a metrics sink for graphite.  The sink must
be configured with the host and port of a graphite node
and optionally may be configured with a prefix that will
be prepended to all metrics that are sent to graphite.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/859d62dc
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/859d62dc
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/859d62dc

Branch: refs/heads/master
Commit: 859d62dc2a37f56f8c85138df069a359e7fabb45
Parents: 65de73c 1360f62
Author: Matei Zaharia <ma...@eecs.berkeley.edu>
Authored: Sun Nov 24 16:19:51 2013 -0800
Committer: Matei Zaharia <ma...@eecs.berkeley.edu>
Committed: Sun Nov 24 16:19:51 2013 -0800

----------------------------------------------------------------------
 conf/metrics.properties.template                |  8 ++
 core/pom.xml                                    |  4 +
 .../spark/metrics/sink/GraphiteSink.scala       | 82 ++++++++++++++++++++
 docs/monitoring.md                              |  1 +
 project/SparkBuild.scala                        |  1 +
 5 files changed, 96 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/859d62dc/project/SparkBuild.scala
----------------------------------------------------------------------


[2/3] git commit: Cleanup GraphiteSink.scala based on feedback

Posted by ma...@apache.org.
Cleanup GraphiteSink.scala based on feedback

* Reorder imports according to the style guide
* Consistently use propertyToOption in all places


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/1360f62d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/1360f62d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/1360f62d

Branch: refs/heads/master
Commit: 1360f62d15170bd295ceaba85f39401fd8109e51
Parents: ef85a51
Author: Russell Cardullo <ru...@gmail.com>
Authored: Mon Nov 18 08:37:09 2013 -0800
Committer: Russell Cardullo <ru...@gmail.com>
Committed: Mon Nov 18 08:53:39 2013 -0800

----------------------------------------------------------------------
 .../org/apache/spark/metrics/sink/GraphiteSink.scala      | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1360f62d/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala
index eb1315e..cdcfec8 100644
--- a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala
@@ -17,13 +17,13 @@
 
 package org.apache.spark.metrics.sink
 
-import com.codahale.metrics.MetricRegistry
-import com.codahale.metrics.graphite.{GraphiteReporter, Graphite}
-
 import java.util.Properties
 import java.util.concurrent.TimeUnit
 import java.net.InetSocketAddress
 
+import com.codahale.metrics.MetricRegistry
+import com.codahale.metrics.graphite.{GraphiteReporter, Graphite}
+
 import org.apache.spark.metrics.MetricsSystem
 
 class GraphiteSink(val property: Properties, val registry: MetricRegistry) extends Sink {
@@ -50,12 +50,12 @@ class GraphiteSink(val property: Properties, val registry: MetricRegistry) exten
   val host = propertyToOption(GRAPHITE_KEY_HOST).get
   val port = propertyToOption(GRAPHITE_KEY_PORT).get.toInt
 
-  val pollPeriod = Option(property.getProperty(GRAPHITE_KEY_PERIOD)) match {
+  val pollPeriod = propertyToOption(GRAPHITE_KEY_PERIOD) match {
     case Some(s) => s.toInt
     case None => GRAPHITE_DEFAULT_PERIOD
   }
 
-  val pollUnit = Option(property.getProperty(GRAPHITE_KEY_UNIT)) match {
+  val pollUnit = propertyToOption(GRAPHITE_KEY_UNIT) match {
     case Some(s) => TimeUnit.valueOf(s.toUpperCase())
     case None => TimeUnit.valueOf(GRAPHITE_DEFAULT_UNIT)
   }