You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by rx...@apache.org on 2014/06/02 02:27:10 UTC

git commit: Better explanation for how to use MIMA excludes.

Repository: spark
Updated Branches:
  refs/heads/master eea3aab4f -> d17d22148


Better explanation for how to use MIMA excludes.

This patch does a few things:
1. We have a file MimaExcludes.scala exclusively for excludes.
2. The test runner tells users about that file if a test fails.
3. I've added back the excludes used from 0.9->1.0. We should keep
   these in the project as an official audit trail of times where
   we decided to make exceptions.

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

Closes #937 from pwendell/mima and squashes the following commits:

7ee0db2 [Patrick Wendell] Better explanation for how to use MIMA excludes.


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

Branch: refs/heads/master
Commit: d17d221487fa7a3af6f4af2217f1d4889ceb084d
Parents: eea3aab
Author: Patrick Wendell <pw...@gmail.com>
Authored: Sun Jun 1 17:27:05 2014 -0700
Committer: Reynold Xin <rx...@apache.org>
Committed: Sun Jun 1 17:27:05 2014 -0700

----------------------------------------------------------------------
 .gitignore                                      |  2 +-
 .rat-excludes                                   |  1 +
 dev/mima                                        | 34 +++++++++++
 dev/run-tests                                   |  3 +-
 project/MimaBuild.scala                         | 48 +++++++--------
 project/MimaExcludes.scala                      | 61 ++++++++++++++++++++
 .../apache/spark/tools/GenerateMIMAIgnore.scala |  4 +-
 7 files changed, 124 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/d17d2214/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index ad72588..b54a305 100644
--- a/.gitignore
+++ b/.gitignore
@@ -7,7 +7,7 @@
 sbt/*.jar
 .settings
 .cache
-.mima-excludes
+.generated-mima-excludes
 /build/
 work/
 out/

http://git-wip-us.apache.org/repos/asf/spark/blob/d17d2214/.rat-excludes
----------------------------------------------------------------------
diff --git a/.rat-excludes b/.rat-excludes
index 1558970..52b2dfa 100644
--- a/.rat-excludes
+++ b/.rat-excludes
@@ -3,6 +3,7 @@ target
 .project
 .classpath
 .mima-excludes
+.generated-mima-excludes
 .rat-excludes
 .*md
 derby.log

http://git-wip-us.apache.org/repos/asf/spark/blob/d17d2214/dev/mima
----------------------------------------------------------------------
diff --git a/dev/mima b/dev/mima
new file mode 100755
index 0000000..d409999
--- /dev/null
+++ b/dev/mima
@@ -0,0 +1,34 @@
+#!/usr/bin/env bash
+
+#
+# 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.
+#
+
+set -o pipefail
+
+# Go to the Spark project root directory
+FWDIR="$(cd `dirname $0`/..; pwd)"
+cd $FWDIR
+
+./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore
+echo -e "q\n" | sbt/sbt mima-report-binary-issues | grep -v -e "info.*Resolving"
+ret_val=$?
+
+if [ $ret_val != 0 ]; then
+  echo "NOTE: Exceptions to binary compatibility can be added in project/MimaExcludes.scala"
+fi
+
+exit $ret_val

http://git-wip-us.apache.org/repos/asf/spark/blob/d17d2214/dev/run-tests
----------------------------------------------------------------------
diff --git a/dev/run-tests b/dev/run-tests
index 6043f85..93d6692 100755
--- a/dev/run-tests
+++ b/dev/run-tests
@@ -81,5 +81,4 @@ fi
 echo "========================================================================="
 echo "Detecting binary incompatibilites with MiMa"
 echo "========================================================================="
-./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore
-echo -e "q\n" | sbt/sbt mima-report-binary-issues | grep -v -e "info.*Resolving" 
+dev/mima

http://git-wip-us.apache.org/repos/asf/spark/blob/d17d2214/project/MimaBuild.scala
----------------------------------------------------------------------
diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala
index 182ca76..1477809 100644
--- a/project/MimaBuild.scala
+++ b/project/MimaBuild.scala
@@ -15,21 +15,41 @@
  * limitations under the License.
  */
 
+import com.typesafe.tools.mima.core.{MissingTypesProblem, MissingClassProblem, ProblemFilters}
+import com.typesafe.tools.mima.core.ProblemFilters._
 import com.typesafe.tools.mima.plugin.MimaKeys.{binaryIssueFilters, previousArtifact}
 import com.typesafe.tools.mima.plugin.MimaPlugin.mimaDefaultSettings
 import sbt._
 
 object MimaBuild {
+  // Exclude a single class and its corresponding object
+  def excludeClass(className: String) = {
+    Seq(
+      excludePackage(className),
+      ProblemFilters.exclude[MissingClassProblem](className),
+      ProblemFilters.exclude[MissingTypesProblem](className),
+      excludePackage(className + "$"),
+      ProblemFilters.exclude[MissingClassProblem](className + "$"),
+      ProblemFilters.exclude[MissingTypesProblem](className + "$")
+    )
+  }
+  // Exclude a Spark class, that is in the package org.apache.spark
+  def excludeSparkClass(className: String) = {
+    excludeClass("org.apache.spark." + className)
+  }
+
+  // Exclude a Spark package, that is in the package org.apache.spark
+  def excludeSparkPackage(packageName: String) = {
+    excludePackage("org.apache.spark." + packageName)
+  }
 
   def ignoredABIProblems(base: File) = {
-    import com.typesafe.tools.mima.core._
-    import com.typesafe.tools.mima.core.ProblemFilters._
 
     // Excludes placed here will be used for all Spark versions
     val defaultExcludes = Seq()
 
     // Read package-private excludes from file
-    val excludeFilePath = (base.getAbsolutePath + "/.mima-excludes")
+    val excludeFilePath = (base.getAbsolutePath + "/.generated-mima-excludes")
     val excludeFile = file(excludeFilePath)
     val ignoredClasses: Seq[String] =
       if (!excludeFile.exists()) {
@@ -38,31 +58,11 @@ object MimaBuild {
         IO.read(excludeFile).split("\n")
       }
 
-    // Exclude a single class and its corresponding object
-    def excludeClass(className: String) = {
-      Seq(
-        excludePackage(className),
-        ProblemFilters.exclude[MissingClassProblem](className),
-        ProblemFilters.exclude[MissingTypesProblem](className),
-        excludePackage(className + "$"),
-        ProblemFilters.exclude[MissingClassProblem](className + "$"),
-        ProblemFilters.exclude[MissingTypesProblem](className + "$")
-      )
-    }
-
-    // Exclude a Spark class, that is in the package org.apache.spark
-    def excludeSparkClass(className: String) = {
-      excludeClass("org.apache.spark." + className)
-    }
 
-    // Exclude a Spark package, that is in the package org.apache.spark
-    def excludeSparkPackage(packageName: String) = {
-      excludePackage("org.apache.spark." + packageName)
-    }
 
     val externalExcludeFileClasses = ignoredClasses.flatMap(excludeClass)
 
-    defaultExcludes ++ externalExcludeFileClasses
+    defaultExcludes ++ externalExcludeFileClasses ++ MimaExcludes.excludes
   }
 
   def mimaSettings(sparkHome: File) = mimaDefaultSettings ++ Seq(

http://git-wip-us.apache.org/repos/asf/spark/blob/d17d2214/project/MimaExcludes.scala
----------------------------------------------------------------------
diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala
new file mode 100644
index 0000000..ecb389d
--- /dev/null
+++ b/project/MimaExcludes.scala
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+
+import com.typesafe.tools.mima.core._
+import com.typesafe.tools.mima.core.ProblemFilters._
+
+/**
+ * Additional excludes for checking of Spark's binary compatibility.
+ *
+ * The Mima build will automatically exclude @DeveloperApi and @Experimental classes. This acts
+ * as an official audit of cases where we excluded other classes. Please use the narrowest
+ * possible exclude here. MIMA will usually tell you what exclude to use, e.g.:
+ *
+ * ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.rdd.RDD.take")
+ *
+ * It is also possible to exclude Spark classes and packages. This should be used sparingly:
+ *
+ * MimaBuild.excludeSparkClass("graphx.util.collection.GraphXPrimitiveKeyOpenHashMap")
+ */
+object MimaExcludes {
+    val excludes =
+      SparkBuild.SPARK_VERSION match {
+        case v if v.startsWith("1.1") =>
+          Seq()
+        case v if v.startsWith("1.0") =>
+          Seq(
+            MimaBuild.excludeSparkPackage("api.java"),
+            MimaBuild.excludeSparkPackage("mllib"),
+            MimaBuild.excludeSparkPackage("streaming")
+          ) ++
+          MimaBuild.excludeSparkClass("rdd.ClassTags") ++
+          MimaBuild.excludeSparkClass("util.XORShiftRandom") ++
+          MimaBuild.excludeSparkClass("graphx.EdgeRDD") ++
+          MimaBuild.excludeSparkClass("graphx.VertexRDD") ++
+          MimaBuild.excludeSparkClass("graphx.impl.GraphImpl") ++
+          MimaBuild.excludeSparkClass("graphx.impl.RoutingTable") ++
+          MimaBuild.excludeSparkClass("graphx.util.collection.PrimitiveKeyOpenHashMap") ++
+          MimaBuild.excludeSparkClass("graphx.util.collection.GraphXPrimitiveKeyOpenHashMap") ++
+          MimaBuild.excludeSparkClass("mllib.recommendation.MFDataGenerator") ++
+          MimaBuild.excludeSparkClass("mllib.optimization.SquaredGradient") ++
+          MimaBuild.excludeSparkClass("mllib.regression.RidgeRegressionWithSGD") ++
+          MimaBuild.excludeSparkClass("mllib.regression.LassoWithSGD") ++
+          MimaBuild.excludeSparkClass("mllib.regression.LinearRegressionWithSGD")
+        case _ => Seq()
+      }
+}
+

http://git-wip-us.apache.org/repos/asf/spark/blob/d17d2214/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala
----------------------------------------------------------------------
diff --git a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala
index 011db50..6a261e1 100644
--- a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala
+++ b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala
@@ -105,9 +105,9 @@ object GenerateMIMAIgnore {
   }
 
   def main(args: Array[String]) {
-    scala.tools.nsc.io.File(".mima-excludes").
+    scala.tools.nsc.io.File(".generated-mima-excludes").
       writeAll(classesPrivateWithin("org.apache.spark").mkString("\n"))
-    println("Created : .mima-excludes in current directory.")
+    println("Created : .generated-mima-excludes in current directory.")
   }