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 2016/03/17 07:02:28 UTC

spark git commit: [SPARK-13948] MiMa check should catch if the visibility changes to private

Repository: spark
Updated Branches:
  refs/heads/master 5faba9fac -> 82066a166


[SPARK-13948] MiMa check should catch if the visibility changes to private

MiMa excludes are currently generated using both the current Spark version's classes and Spark 1.2.0's classes, but this doesn't make sense: we should only be ignoring classes which were `private` in the previous Spark version, not classes which became private in the current version.

This patch updates `dev/mima` to only generate excludes with respect to the previous artifacts that MiMa checks against. It also updates `MimaBuild` so that `excludeClass` only applies directly to the class being excluded and not to its companion object (since a class and its companion object can have different accessibility).

Author: Josh Rosen <jo...@databricks.com>

Closes #11774 from JoshRosen/SPARK-13948.


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

Branch: refs/heads/master
Commit: 82066a166768399eada42f3d65150becf43320b3
Parents: 5faba9f
Author: Josh Rosen <jo...@databricks.com>
Authored: Wed Mar 16 23:02:25 2016 -0700
Committer: Reynold Xin <rx...@databricks.com>
Committed: Wed Mar 16 23:02:25 2016 -0700

----------------------------------------------------------------------
 dev/mima                   | 18 +++++++-----------
 project/MimaBuild.scala    |  7 ++-----
 project/MimaExcludes.scala | 19 +++++++++++++++++++
 project/SparkBuild.scala   | 24 ++++++++++++++----------
 4 files changed, 42 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/82066a16/dev/mima
----------------------------------------------------------------------
diff --git a/dev/mima b/dev/mima
index c8e2df6..ea746e6 100755
--- a/dev/mima
+++ b/dev/mima
@@ -24,21 +24,17 @@ set -e
 FWDIR="$(cd "`dirname "$0"`"/..; pwd)"
 cd "$FWDIR"
 
+SPARK_PROFILES="-Pyarn -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Phive"
 TOOLS_CLASSPATH="$(build/sbt "export tools/fullClasspath" | tail -n1)"
+OLD_DEPS_CLASSPATH="$(build/sbt $SPARK_PROFILES "export oldDeps/fullClasspath" | tail -n1)"
 
 rm -f .generated-mima*
 
-generate_mima_ignore() {
-    java \
-      -XX:MaxPermSize=1g \
-      -Xmx2g \
-      -cp "$TOOLS_CLASSPATH:$1" \
-      org.apache.spark.tools.GenerateMIMAIgnore
-}
-
-SPARK_PROFILES="-Pyarn -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Phive"
-generate_mima_ignore "$(build/sbt $SPARK_PROFILES "export assembly/fullClasspath" | tail -n1)"
-generate_mima_ignore "$(build/sbt $SPARK_PROFILES "export oldDeps/fullClasspath" | tail -n1)"
+java \
+  -XX:MaxPermSize=1g \
+  -Xmx2g \
+  -cp "$TOOLS_CLASSPATH:$OLD_DEPS_CLASSPATH" \
+  org.apache.spark.tools.GenerateMIMAIgnore
 
 echo -e "q\n" | build/sbt mimaReportBinaryIssues | grep -v -e "info.*Resolving"
 ret_val=$?

http://git-wip-us.apache.org/repos/asf/spark/blob/82066a16/project/MimaBuild.scala
----------------------------------------------------------------------
diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala
index 4adf64a..acf7b89 100644
--- a/project/MimaBuild.scala
+++ b/project/MimaBuild.scala
@@ -42,14 +42,11 @@ object MimaBuild {
       ProblemFilters.exclude[IncompatibleFieldTypeProblem](fullName)
     )
 
-  // Exclude a single class and its corresponding object
+  // Exclude a single class
   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 + "$")
+      ProblemFilters.exclude[MissingTypesProblem](className)
     )
 
   // Exclude a Spark class, that is in the package org.apache.spark

http://git-wip-us.apache.org/repos/asf/spark/blob/82066a16/project/MimaExcludes.scala
----------------------------------------------------------------------
diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala
index 5e8d13b..17ecb5f 100644
--- a/project/MimaExcludes.scala
+++ b/project/MimaExcludes.scala
@@ -224,6 +224,7 @@ object MimaExcludes {
         ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.streaming.receiver.ActorReceiver$Supervisor")
       ) ++ Seq(
         // SPARK-12348 Remove deprecated Streaming APIs.
+        ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.streaming.dstream.DStream.foreach"),
         ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions"),
         ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions$default$4"),
         ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.awaitTermination"),
@@ -540,6 +541,24 @@ object MimaExcludes {
         // SPARK-13927: add row/column iterator to local matrices
         ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.linalg.Matrix.rowIter"),
         ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.linalg.Matrix.colIter")
+      ) ++ Seq(
+        // SPARK-13948: MiMa Check should catch if the visibility change to `private`
+        // TODO(josh): Some of these may be legitimate incompatibilities; we should follow up before the 2.0.0 release
+        ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.Dataset.toDS"),
+        ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.sources.OutputWriterFactory.newInstance"),
+        ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.util.RpcUtils.askTimeout"),
+        ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.util.RpcUtils.lookupTimeout"),
+        ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.UnaryTransformer.transform"),
+        ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.DecisionTreeClassifier.train"),
+        ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.LogisticRegression.train"),
+        ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.DecisionTreeRegressor.train"),
+        ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.Dataset.groupBy"),
+        ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.Dataset.groupBy"),
+        ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.Dataset.select"),
+        ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.Dataset.toDF"),
+        ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.Logging.initializeLogIfNecessary"),
+        ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerEvent.logEvent"),
+        ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.OutputWriterFactory.newInstance")
       )
     case v if v.startsWith("1.6") =>
       Seq(

http://git-wip-us.apache.org/repos/asf/spark/blob/82066a16/project/SparkBuild.scala
----------------------------------------------------------------------
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index f76cda0..dbe98d1 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -26,6 +26,7 @@ import sbt.Classpaths.publishTask
 import sbt.Keys._
 import sbtunidoc.Plugin.UnidocKeys.unidocGenjavadocVersion
 import com.typesafe.sbt.pom.{PomBuild, SbtPomKeys}
+import com.typesafe.tools.mima.plugin.MimaKeys
 
 import spray.revolver.RevolverPlugin._
 
@@ -247,12 +248,14 @@ object SparkBuild extends PomBuild {
   /* Enable tests settings for all projects except examples, assembly and tools */
   (allProjects ++ optionallyEnabledProjects).foreach(enable(TestSettings.settings))
 
-  allProjects.filterNot { x =>
+  val mimaProjects = allProjects.filterNot { x =>
     Seq(
       spark, hive, hiveThriftServer, catalyst, repl, networkCommon, networkShuffle, networkYarn,
       unsafe, testTags, sketch
     ).contains(x)
-  }.foreach { x =>
+  }
+
+  mimaProjects.foreach { x =>
     enable(MimaBuild.mimaSettings(sparkHome, x))(x)
   }
 
@@ -371,22 +374,23 @@ object ExcludedDependencies {
 }
 
 /**
- * Following project only exists to pull previous artifacts of Spark for generating
- * Mima ignores. For more information see: SPARK 2071
+ * Project to pull previous artifacts of Spark for generating Mima excludes.
  */
 object OldDeps {
 
   lazy val project = Project("oldDeps", file("dev"), settings = oldDepsSettings)
 
+  lazy val allPreviousArtifactKeys = Def.settingDyn[Seq[Option[ModuleID]]] {
+    SparkBuild.mimaProjects
+      .map { project => MimaKeys.previousArtifact in project }
+      .map(k => Def.setting(k.value))
+      .join
+  }
+
   def oldDepsSettings() = Defaults.coreDefaultSettings ++ Seq(
     name := "old-deps",
     scalaVersion := "2.10.5",
-    libraryDependencies := Seq(
-      "spark-streaming",
-      "spark-mllib",
-      "spark-graphx",
-      "spark-core"
-    ).map(id => "org.apache.spark" % (id + "_2.11") % "1.2.0")
+    libraryDependencies := allPreviousArtifactKeys.value.flatten
   )
 }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org