You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by pw...@apache.org on 2014/03/11 19:17:21 UTC

git commit: SPARK-1167: Remove metrics-ganglia from default build due to LGPL issues...

Repository: spark
Updated Branches:
  refs/heads/master 2a2c9645e -> 16788a654


SPARK-1167: Remove metrics-ganglia from default build due to LGPL issues...

This patch removes Ganglia integration from the default build. It
allows users willing to link against LGPL code to use Ganglia
by adding build flags or linking against a new Spark artifact called
spark-ganglia-lgpl.

This brings Spark in line with the Apache policy on LGPL code
enumerated here:

https://www.apache.org/legal/3party.html#options-optional

Author: Patrick Wendell <pw...@gmail.com>

Closes #108 from pwendell/ganglia and squashes the following commits:

326712a [Patrick Wendell] Responding to review feedback
5f28ee4 [Patrick Wendell] SPARK-1167: Remove metrics-ganglia from default build due to LGPL issues.


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

Branch: refs/heads/master
Commit: 16788a654246067fd966033b5dc9bc0d4c759b70
Parents: 2a2c964
Author: Patrick Wendell <pw...@gmail.com>
Authored: Tue Mar 11 11:16:59 2014 -0700
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Tue Mar 11 11:16:59 2014 -0700

----------------------------------------------------------------------
 assembly/pom.xml                                | 10 +++
 core/pom.xml                                    |  4 -
 .../apache/spark/metrics/sink/GangliaSink.scala | 84 --------------------
 dev/audit-release/README.md                     | 11 +++
 .../sbt_app_core/src/main/scala/SparkApp.scala  | 15 +++-
 dev/audit-release/sbt_app_ganglia/build.sbt     | 31 ++++++++
 .../src/main/scala/SparkApp.scala               | 39 +++++++++
 dev/create-release/create-release.sh            |  4 +-
 docs/monitoring.md                              | 13 ++-
 extras/spark-ganglia-lgpl/pom.xml               | 45 +++++++++++
 .../apache/spark/metrics/sink/GangliaSink.scala | 84 ++++++++++++++++++++
 pom.xml                                         |  9 ++-
 project/SparkBuild.scala                        | 25 ++++--
 13 files changed, 274 insertions(+), 100 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/16788a65/assembly/pom.xml
----------------------------------------------------------------------
diff --git a/assembly/pom.xml b/assembly/pom.xml
index 82a5985..22bbbc5 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -159,6 +159,16 @@
       </dependencies>
     </profile>
     <profile>
+      <id>spark-ganglia-lgpl</id>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.spark</groupId>
+          <artifactId>spark-ganglia-lgpl_${scala.binary.version}</artifactId>
+          <version>${project.version}</version>
+        </dependency>
+      </dependencies>
+    </profile>
+    <profile>
       <id>bigtop-dist</id>
       <!-- This profile uses the assembly plugin to create a special "dist" package for BigTop
            that contains Spark but not the Hadoop JARs it depends on. -->

http://git-wip-us.apache.org/repos/asf/spark/blob/16788a65/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index 4d7d41a..2248f9d 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -181,10 +181,6 @@
     </dependency>
     <dependency>
       <groupId>com.codahale.metrics</groupId>
-      <artifactId>metrics-ganglia</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.codahale.metrics</groupId>
       <artifactId>metrics-graphite</artifactId>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/spark/blob/16788a65/core/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala
deleted file mode 100644
index cd37317..0000000
--- a/core/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * 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 java.util.Properties
-import java.util.concurrent.TimeUnit
-
-import com.codahale.metrics.MetricRegistry
-import com.codahale.metrics.ganglia.GangliaReporter
-import info.ganglia.gmetric4j.gmetric.GMetric
-
-import org.apache.spark.SecurityManager
-import org.apache.spark.metrics.MetricsSystem
-
-class GangliaSink(val property: Properties, val registry: MetricRegistry,
-    securityMgr: SecurityManager) extends Sink {
-  val GANGLIA_KEY_PERIOD = "period"
-  val GANGLIA_DEFAULT_PERIOD = 10
-
-  val GANGLIA_KEY_UNIT = "unit"
-  val GANGLIA_DEFAULT_UNIT = TimeUnit.SECONDS
-
-  val GANGLIA_KEY_MODE = "mode"
-  val GANGLIA_DEFAULT_MODE = GMetric.UDPAddressingMode.MULTICAST
-
-  // TTL for multicast messages. If listeners are X hops away in network, must be at least X.
-  val GANGLIA_KEY_TTL = "ttl"
-  val GANGLIA_DEFAULT_TTL = 1
-
-  val GANGLIA_KEY_HOST = "host"
-  val GANGLIA_KEY_PORT = "port"
-
-  def propertyToOption(prop: String) = Option(property.getProperty(prop))
-
-  if (!propertyToOption(GANGLIA_KEY_HOST).isDefined) {
-    throw new Exception("Ganglia sink requires 'host' property.")
-  }
-
-  if (!propertyToOption(GANGLIA_KEY_PORT).isDefined) {
-    throw new Exception("Ganglia sink requires 'port' property.")
-  }
-
-  val host = propertyToOption(GANGLIA_KEY_HOST).get
-  val port = propertyToOption(GANGLIA_KEY_PORT).get.toInt
-  val ttl = propertyToOption(GANGLIA_KEY_TTL).map(_.toInt).getOrElse(GANGLIA_DEFAULT_TTL)
-  val mode = propertyToOption(GANGLIA_KEY_MODE)
-    .map(u => GMetric.UDPAddressingMode.valueOf(u.toUpperCase)).getOrElse(GANGLIA_DEFAULT_MODE)
-  val pollPeriod = propertyToOption(GANGLIA_KEY_PERIOD).map(_.toInt)
-    .getOrElse(GANGLIA_DEFAULT_PERIOD)
-  val pollUnit = propertyToOption(GANGLIA_KEY_UNIT).map(u => TimeUnit.valueOf(u.toUpperCase))
-    .getOrElse(GANGLIA_DEFAULT_UNIT)
-
-  MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod)
-
-  val ganglia = new GMetric(host, port, mode, ttl)
-  val reporter: GangliaReporter = GangliaReporter.forRegistry(registry)
-      .convertDurationsTo(TimeUnit.MILLISECONDS)
-      .convertRatesTo(TimeUnit.SECONDS)
-      .build(ganglia)
-
-  override def start() {
-    reporter.start(pollPeriod, pollUnit)
-  }
-
-  override def stop() {
-    reporter.stop()
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/spark/blob/16788a65/dev/audit-release/README.md
----------------------------------------------------------------------
diff --git a/dev/audit-release/README.md b/dev/audit-release/README.md
new file mode 100644
index 0000000..2437a98
--- /dev/null
+++ b/dev/audit-release/README.md
@@ -0,0 +1,11 @@
+# Test Application Builds
+This directory includes test applications which are built when auditing releases. You can
+run them locally by setting appropriate environment variables.
+
+```
+$ cd sbt_app_core
+$ SCALA_VERSION=2.10.3 \
+  SPARK_VERSION=1.0.0-SNAPSHOT \
+  SPARK_RELEASE_REPOSITORY=file:///home/patrick/.ivy2/local \
+  sbt run
+```

http://git-wip-us.apache.org/repos/asf/spark/blob/16788a65/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala
----------------------------------------------------------------------
diff --git a/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala b/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala
index d49de8b..53fe432 100644
--- a/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala
+++ b/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala
@@ -17,6 +17,8 @@
 
 package main.scala
 
+import scala.util.Try
+
 import org.apache.spark.SparkContext
 import org.apache.spark.SparkContext._
 
@@ -31,6 +33,17 @@ object SimpleApp {
       println("Failed to parse log files with Spark")
       System.exit(-1)
     }
-    println("Test succeeded")
+
+    // Regression test for SPARK-1167: Remove metrics-ganglia from default build due to LGPL issue
+    val foundConsole = Try(Class.forName("org.apache.spark.metrics.sink.ConsoleSink")).isSuccess
+    val foundGanglia = Try(Class.forName("org.apache.spark.metrics.sink.GangliaSink")).isSuccess
+    if (!foundConsole) {
+      println("Console sink not loaded via spark-core")
+      System.exit(-1)
+    }
+    if (foundGanglia) {
+      println("Ganglia sink was loaded via spark-core")
+      System.exit(-1)
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/16788a65/dev/audit-release/sbt_app_ganglia/build.sbt
----------------------------------------------------------------------
diff --git a/dev/audit-release/sbt_app_ganglia/build.sbt b/dev/audit-release/sbt_app_ganglia/build.sbt
new file mode 100644
index 0000000..55db675
--- /dev/null
+++ b/dev/audit-release/sbt_app_ganglia/build.sbt
@@ -0,0 +1,31 @@
+//
+// 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.
+//
+
+name := "Ganglia Test"
+
+version := "1.0"
+
+scalaVersion := System.getenv.get("SCALA_VERSION")
+
+libraryDependencies += "org.apache.spark" %% "spark-core" % System.getenv.get("SPARK_VERSION")
+
+libraryDependencies += "org.apache.spark" %% "spark-ganglia-lgpl" % System.getenv.get("SPARK_VERSION")
+
+resolvers ++= Seq(
+  "Spark Release Repository" at System.getenv.get("SPARK_RELEASE_REPOSITORY"),
+  "Akka Repository" at "http://repo.akka.io/releases/",
+  "Spray Repository" at "http://repo.spray.cc/")

http://git-wip-us.apache.org/repos/asf/spark/blob/16788a65/dev/audit-release/sbt_app_ganglia/src/main/scala/SparkApp.scala
----------------------------------------------------------------------
diff --git a/dev/audit-release/sbt_app_ganglia/src/main/scala/SparkApp.scala b/dev/audit-release/sbt_app_ganglia/src/main/scala/SparkApp.scala
new file mode 100644
index 0000000..0be8e64
--- /dev/null
+++ b/dev/audit-release/sbt_app_ganglia/src/main/scala/SparkApp.scala
@@ -0,0 +1,39 @@
+/*
+ * 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 main.scala
+
+import scala.util.Try
+
+import org.apache.spark.SparkContext
+import org.apache.spark.SparkContext._
+
+object SimpleApp {
+  def main(args: Array[String]) {
+    // Regression test for SPARK-1167: Remove metrics-ganglia from default build due to LGPL issue
+    val foundConsole = Try(Class.forName("org.apache.spark.metrics.sink.ConsoleSink")).isSuccess
+    val foundGanglia = Try(Class.forName("org.apache.spark.metrics.sink.GangliaSink")).isSuccess
+    if (!foundConsole) {
+      println("Console sink not loaded via spark-core")
+      System.exit(-1)
+    }
+    if (!foundGanglia) {
+      println("Ganglia sink not loaded via spark-ganglia-lgpl")
+      System.exit(-1)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/16788a65/dev/create-release/create-release.sh
----------------------------------------------------------------------
diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh
index b9088ea..995106f 100755
--- a/dev/create-release/create-release.sh
+++ b/dev/create-release/create-release.sh
@@ -49,14 +49,14 @@ mvn -DskipTests \
   -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \
   -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \
   -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \
-  -Pyarn \
+  -Pyarn -Pspark-ganglia-lgpl \
   -Dtag=$GIT_TAG -DautoVersionSubmodules=true \
   --batch-mode release:prepare
 
 mvn -DskipTests \
   -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \
   -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \
-  -Pyarn \
+  -Pyarn -Pspark-ganglia-lgpl\
   release:perform
 
 rm -rf spark

http://git-wip-us.apache.org/repos/asf/spark/blob/16788a65/docs/monitoring.md
----------------------------------------------------------------------
diff --git a/docs/monitoring.md b/docs/monitoring.md
index e9b1d2b..15bfb04 100644
--- a/docs/monitoring.md
+++ b/docs/monitoring.md
@@ -48,11 +48,22 @@ Each instance can report to zero or more _sinks_. Sinks are contained in the
 
 * `ConsoleSink`: Logs metrics information to the console.
 * `CSVSink`: Exports metrics data to CSV files at regular intervals.
-* `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.
 
+Spark also supports a Ganglia sink which is not included in the default build due to
+licensing restrictions:
+
+* `GangliaSink`: Sends metrics to a Ganglia node or multicast group.
+
+To install the `GangliaSink` you'll need to perform a custom build of Spark. _**Note that
+by embedding this library you will include [LGPL](http://www.gnu.org/copyleft/lesser.html)-licensed 
+code in your Spark package**_. For sbt users, set the 
+`SPARK_GANGLIA_LGPL` environment variable before building. For Maven users, enable 
+the `-Pspark-ganglia-lgpl` profile. In addition to modifying the cluster's Spark build
+user applications will need to link to the `spark-ganglia-lgpl` artifact.
+
 The syntax of the metrics configuration file is defined in an example configuration file, 
 `$SPARK_HOME/conf/metrics.properties.template`.
 

http://git-wip-us.apache.org/repos/asf/spark/blob/16788a65/extras/spark-ganglia-lgpl/pom.xml
----------------------------------------------------------------------
diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml
new file mode 100644
index 0000000..11ac827
--- /dev/null
+++ b/extras/spark-ganglia-lgpl/pom.xml
@@ -0,0 +1,45 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+~ 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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.spark</groupId>
+    <artifactId>spark-parent</artifactId>
+    <version>1.0.0-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+
+  <!-- Ganglia integration is not included by default due to LGPL-licensed code -->
+  <groupId>org.apache.spark</groupId>
+  <artifactId>spark-ganglia-lgpl_2.10</artifactId>
+  <packaging>jar</packaging>
+  <name>Spark Ganglia Integration</name>
+  
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-core_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>com.codahale.metrics</groupId>
+      <artifactId>metrics-ganglia</artifactId>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/spark/blob/16788a65/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala
----------------------------------------------------------------------
diff --git a/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala b/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala
new file mode 100644
index 0000000..cd37317
--- /dev/null
+++ b/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala
@@ -0,0 +1,84 @@
+/*
+ * 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 java.util.Properties
+import java.util.concurrent.TimeUnit
+
+import com.codahale.metrics.MetricRegistry
+import com.codahale.metrics.ganglia.GangliaReporter
+import info.ganglia.gmetric4j.gmetric.GMetric
+
+import org.apache.spark.SecurityManager
+import org.apache.spark.metrics.MetricsSystem
+
+class GangliaSink(val property: Properties, val registry: MetricRegistry,
+    securityMgr: SecurityManager) extends Sink {
+  val GANGLIA_KEY_PERIOD = "period"
+  val GANGLIA_DEFAULT_PERIOD = 10
+
+  val GANGLIA_KEY_UNIT = "unit"
+  val GANGLIA_DEFAULT_UNIT = TimeUnit.SECONDS
+
+  val GANGLIA_KEY_MODE = "mode"
+  val GANGLIA_DEFAULT_MODE = GMetric.UDPAddressingMode.MULTICAST
+
+  // TTL for multicast messages. If listeners are X hops away in network, must be at least X.
+  val GANGLIA_KEY_TTL = "ttl"
+  val GANGLIA_DEFAULT_TTL = 1
+
+  val GANGLIA_KEY_HOST = "host"
+  val GANGLIA_KEY_PORT = "port"
+
+  def propertyToOption(prop: String) = Option(property.getProperty(prop))
+
+  if (!propertyToOption(GANGLIA_KEY_HOST).isDefined) {
+    throw new Exception("Ganglia sink requires 'host' property.")
+  }
+
+  if (!propertyToOption(GANGLIA_KEY_PORT).isDefined) {
+    throw new Exception("Ganglia sink requires 'port' property.")
+  }
+
+  val host = propertyToOption(GANGLIA_KEY_HOST).get
+  val port = propertyToOption(GANGLIA_KEY_PORT).get.toInt
+  val ttl = propertyToOption(GANGLIA_KEY_TTL).map(_.toInt).getOrElse(GANGLIA_DEFAULT_TTL)
+  val mode = propertyToOption(GANGLIA_KEY_MODE)
+    .map(u => GMetric.UDPAddressingMode.valueOf(u.toUpperCase)).getOrElse(GANGLIA_DEFAULT_MODE)
+  val pollPeriod = propertyToOption(GANGLIA_KEY_PERIOD).map(_.toInt)
+    .getOrElse(GANGLIA_DEFAULT_PERIOD)
+  val pollUnit = propertyToOption(GANGLIA_KEY_UNIT).map(u => TimeUnit.valueOf(u.toUpperCase))
+    .getOrElse(GANGLIA_DEFAULT_UNIT)
+
+  MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod)
+
+  val ganglia = new GMetric(host, port, mode, ttl)
+  val reporter: GangliaReporter = GangliaReporter.forRegistry(registry)
+      .convertDurationsTo(TimeUnit.MILLISECONDS)
+      .convertRatesTo(TimeUnit.SECONDS)
+      .build(ganglia)
+
+  override def start() {
+    reporter.start(pollPeriod, pollUnit)
+  }
+
+  override def stop() {
+    reporter.stop()
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/spark/blob/16788a65/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index f0c877d..986626f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -756,12 +756,19 @@
         <hadoop.version>0.23.7</hadoop.version>
         <!--<hadoop.version>2.0.5-alpha</hadoop.version> -->
       </properties>
-
       <modules>
         <module>yarn</module>
       </modules>
+    </profile>
 
+    <!-- Ganglia integration is not included by default due to LGPL-licensed code -->
+    <profile>
+      <id>spark-ganglia-lgpl</id>
+      <modules>
+        <module>extras/spark-ganglia-lgpl</module>
+      </modules>
     </profile>
+
     <profile>
       <id>java8-tests</id>
       <build>

http://git-wip-us.apache.org/repos/asf/spark/blob/16788a65/project/SparkBuild.scala
----------------------------------------------------------------------
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 8fa220c..b0c3bf2 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -65,7 +65,7 @@ object SparkBuild extends Build {
   lazy val mllib = Project("mllib", file("mllib"), settings = mllibSettings) dependsOn(core)
 
   lazy val assemblyProj = Project("assembly", file("assembly"), settings = assemblyProjSettings)
-    .dependsOn(core, graphx, bagel, mllib, repl, streaming) dependsOn(maybeYarn: _*)
+    .dependsOn(core, graphx, bagel, mllib, repl, streaming) dependsOn(maybeYarn: _*) dependsOn(maybeGanglia: _*)
 
   lazy val assembleDeps = TaskKey[Unit]("assemble-deps", "Build assembly of dependencies and packages Spark projects")
 
@@ -91,19 +91,26 @@ object SparkBuild extends Build {
   lazy val hadoopClient = if (hadoopVersion.startsWith("0.20.") || hadoopVersion == "1.0.0") "hadoop-core" else "hadoop-client"
   val maybeAvro = if (hadoopVersion.startsWith("0.23.") && isYarnEnabled) Seq("org.apache.avro" % "avro" % "1.7.4") else Seq()
 
-  // Conditionally include the java 8 sub-project
+  // Include Ganglia integration if the user has enabled Ganglia
+  // This is isolated from the normal build due to LGPL-licensed code in the library
+  lazy val isGangliaEnabled = Properties.envOrNone("SPARK_GANGLIA_LGPL").isDefined
+  lazy val gangliaProj = Project("spark-ganglia-lgpl", file("extras/spark-ganglia-lgpl"), settings = gangliaSettings).dependsOn(core)
+  val maybeGanglia: Seq[ClasspathDependency] = if (isGangliaEnabled) Seq(gangliaProj) else Seq()
+  val maybeGangliaRef: Seq[ProjectReference] = if (isGangliaEnabled) Seq(gangliaProj) else Seq()
+
+  // Include the Java 8 project if the JVM version is 8+
   lazy val javaVersion = System.getProperty("java.specification.version")
   lazy val isJava8Enabled = javaVersion.toDouble >= "1.8".toDouble
   val maybeJava8Tests = if (isJava8Enabled) Seq[ProjectReference](java8Tests) else Seq[ProjectReference]()
   lazy val java8Tests = Project("java8-tests", file("extras/java8-tests"), settings = java8TestsSettings).
     dependsOn(core) dependsOn(streaming % "compile->compile;test->test")
 
-  // Conditionally include the yarn sub-project
+  // Include the YARN project if the user has enabled YARN
   lazy val yarnAlpha = Project("yarn-alpha", file("yarn/alpha"), settings = yarnAlphaSettings) dependsOn(core)
   lazy val yarn = Project("yarn", file("yarn/stable"), settings = yarnSettings) dependsOn(core)
 
-  lazy val maybeYarn = if (isYarnEnabled) Seq[ClasspathDependency](if (isNewHadoop) yarn else yarnAlpha) else Seq[ClasspathDependency]()
-  lazy val maybeYarnRef = if (isYarnEnabled) Seq[ProjectReference](if (isNewHadoop) yarn else yarnAlpha) else Seq[ProjectReference]()
+  lazy val maybeYarn: Seq[ClasspathDependency] = if (isYarnEnabled) Seq(if (isNewHadoop) yarn else yarnAlpha) else Seq()
+  lazy val maybeYarnRef: Seq[ProjectReference] = if (isYarnEnabled) Seq(if (isNewHadoop) yarn else yarnAlpha) else Seq()
 
   lazy val externalTwitter = Project("external-twitter", file("external/twitter"), settings = twitterSettings)
     .dependsOn(streaming % "compile->compile;test->test")
@@ -127,7 +134,7 @@ object SparkBuild extends Build {
     .dependsOn(core, mllib, graphx, bagel, streaming, externalTwitter) dependsOn(allExternal: _*)
 
   // Everything except assembly, tools, java8Tests and examples belong to packageProjects
-  lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graphx) ++ maybeYarnRef
+  lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graphx) ++ maybeYarnRef ++ maybeGangliaRef
 
   lazy val allProjects = packageProjects ++ allExternalRefs ++
     Seq[ProjectReference](examples, tools, assemblyProj) ++ maybeJava8Tests
@@ -296,7 +303,6 @@ object SparkBuild extends Build {
         "com.codahale.metrics"       % "metrics-core"     % "3.0.0",
         "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"            % "0.3.1" excludeAll(excludeAsm),
         "com.twitter"                % "chill-java"       % "0.3.1" excludeAll(excludeAsm),
@@ -384,6 +390,11 @@ object SparkBuild extends Build {
     name := "spark-yarn"
   )
 
+  def gangliaSettings = sharedSettings ++ Seq(
+    name := "spark-ganglia-lgpl",
+    libraryDependencies += "com.codahale.metrics" % "metrics-ganglia" % "3.0.0"
+  )
+
   def java8TestsSettings = sharedSettings ++ Seq(
     name := "java8-tests",
     javacOptions := Seq("-target", "1.8", "-source", "1.8"),