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:54:43 UTC

[1/2] git commit: SPARK-1167: Remove metrics-ganglia from default build due to LGPL issues...

Repository: spark
Updated Branches:
  refs/heads/branch-0.9 6f0db0ada -> 6cbd5800c


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.
(cherry picked from commit 16788a654246067fd966033b5dc9bc0d4c759b70)

Conflicts:

	core/pom.xml
	dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala
	dev/create-release/create-release.sh
	pom.xml
	project/SparkBuild.scala


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

Branch: refs/heads/branch-0.9
Commit: 0c919271ff77e36a83ad25245acab5d73255eff3
Parents: 6f0db0a
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:24:21 2014 -0700

----------------------------------------------------------------------
 assembly/pom.xml                                | 10 +++
 core/pom.xml                                    |  4 -
 .../apache/spark/metrics/sink/GangliaSink.scala | 82 --------------------
 docs/monitoring.md                              | 13 +++-
 extras/spark-ganglia-lgpl/pom.xml               | 45 +++++++++++
 .../apache/spark/metrics/sink/GangliaSink.scala | 82 ++++++++++++++++++++
 pom.xml                                         | 12 ++-
 project/SparkBuild.scala                        | 27 +++++--
 8 files changed, 177 insertions(+), 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/0c919271/assembly/pom.xml
----------------------------------------------------------------------
diff --git a/assembly/pom.xml b/assembly/pom.xml
index 5273309..cdae0f9 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/0c919271/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index abf4893..4901bb0 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -149,10 +149,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/0c919271/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 b924907..0000000
--- a/core/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala
+++ /dev/null
@@ -1,82 +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.ganglia.GangliaReporter
-import com.codahale.metrics.MetricRegistry
-import info.ganglia.gmetric4j.gmetric.GMetric
-
-import org.apache.spark.metrics.MetricsSystem
-
-class GangliaSink(val property: Properties, val registry: MetricRegistry) 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/0c919271/docs/monitoring.md
----------------------------------------------------------------------
diff --git a/docs/monitoring.md b/docs/monitoring.md
index 0d5eb70..2423185 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.conf.template`.
 

http://git-wip-us.apache.org/repos/asf/spark/blob/0c919271/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/0c919271/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..b924907
--- /dev/null
+++ b/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.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 java.util.Properties
+import java.util.concurrent.TimeUnit
+
+import com.codahale.metrics.ganglia.GangliaReporter
+import com.codahale.metrics.MetricRegistry
+import info.ganglia.gmetric4j.gmetric.GMetric
+
+import org.apache.spark.metrics.MetricsSystem
+
+class GangliaSink(val property: Properties, val registry: MetricRegistry) 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/0c919271/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 2a3f8fc..e53c930 100644
--- a/pom.xml
+++ b/pom.xml
@@ -743,11 +743,9 @@
         <hadoop.version>0.23.7</hadoop.version>
         <!--<hadoop.version>2.0.5-alpha</hadoop.version> -->
       </properties>
-
       <modules>
         <module>yarn</module>
       </modules>
-
     </profile>
 
     <profile>
@@ -760,7 +758,15 @@
       <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>
+
   </profiles>
 </project>

http://git-wip-us.apache.org/repos/asf/spark/blob/0c919271/project/SparkBuild.scala
----------------------------------------------------------------------
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index b8f901d..f9eeeb0 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -61,7 +61,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")
 
@@ -90,14 +90,21 @@ object SparkBuild extends Build {
     case None => DEFAULT_YARN
     case Some(v) => v.toBoolean
   }
-  lazy val hadoopClient = if (hadoopVersion.startsWith("0.20.") || hadoopVersion == "1.0.0") "hadoop-core" else "hadoop-client" 
-
-  // Conditionally include the yarn sub-project
+  lazy val hadoopClient = if (hadoopVersion.startsWith("0.20.") || hadoopVersion == "1.0.0") "hadoop-core" else "hadoop-client"
+  
+  // 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 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")
@@ -121,7 +128,7 @@ object SparkBuild extends Build {
     .dependsOn(core, mllib, graphx, bagel, streaming, externalTwitter) dependsOn(allExternal: _*)
 
   // Everything except assembly, tools 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)
 
@@ -281,7 +288,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",
         "com.twitter"              % "chill-java"       % "0.3.1",
@@ -371,6 +377,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"
+  )
+
   // Conditionally include the YARN dependencies because some tools look at all sub-projects and will complain
   // if we refer to nonexistent dependencies (e.g. hadoop-yarn-api from a Hadoop version without YARN).
   def extraYarnSettings = if(isYarnEnabled) yarnEnabledSettings else Seq()


[2/2] git commit: Log4j build fix on 0.9 branch

Posted by pw...@apache.org.
Log4j build fix on 0.9 branch

Spark should include log4j by default. Downstream packagers can exclude log4j
if they want to use another logging backend.


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

Branch: refs/heads/branch-0.9
Commit: 6cbd5800c71d961228382be57c2688af4e41b3ea
Parents: 0c91927
Author: Patrick Wendell <pw...@gmail.com>
Authored: Tue Mar 11 11:53:29 2014 -0700
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Tue Mar 11 11:53:29 2014 -0700

----------------------------------------------------------------------
 core/pom.xml | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/6cbd5800/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index 4901bb0..6b6167c 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -189,7 +189,6 @@
         <dependency>
             <groupId>org.slf4j</groupId>
             <artifactId>slf4j-log4j12</artifactId>
-            <scope>test</scope>
         </dependency>
     </dependencies>
     <build>