diff --git a/.mima-exclude b/.mima-exclude new file mode 100644 index 0000000000000..54a46b4f0ee9e --- /dev/null +++ b/.mima-exclude @@ -0,0 +1,360 @@ +org.apache.spark.scheduler.cluster.SparkDeploySchedulerBackend +org.apache.spark.rdd.FlatMappedRDD +org.apache.spark.storage.PutResult +org.apache.spark.mllib.recommendation.OutLinkBlock +org.apache.spark.network.netty.ShuffleSender +org.apache.spark.scheduler.TaskResult +org.apache.spark.scheduler.JobWaiter +org.apache.spark.deploy.worker.ExecutorRunner +org.apache.spark.streaming.dstream.FilteredDStream +org.apache.spark.util.SerializableBuffer +org.apache.spark.scheduler.TaskSetManager +org.apache.spark.scheduler.Stage +org.apache.spark.rdd.ZippedWithIndexRDD +org.apache.spark.scheduler.SchedulerBackend +org.apache.spark.streaming.dstream.GlommedDStream +org.apache.spark.rdd.FlatMappedValuesRDD +org.apache.spark.deploy.master.MonarchyLeaderAgent +org.apache.spark.rdd.CoGroupPartition +org.apache.spark.scheduler.SlaveLost +org.apache.spark.api.python.PythonWorkerFactory +org.apache.spark.streaming.ContextWaiter +org.apache.spark.serializer.JavaSerializationStream +org.apache.spark.rdd.CheckpointRDDPartition +org.apache.spark.scheduler.WorkerOffer +org.apache.spark.streaming.scheduler.JobCompleted +org.apache.spark.rdd.ParallelCollectionRDD +org.apache.spark.streaming.dstream.ForEachDStream +org.apache.spark.scheduler.JobFailed +org.apache.spark.graphx.impl.ReplicatedVertexView +org.apache.spark.deploy.master.ui.ApplicationPage +org.apache.spark.rdd.ShuffledRDDPartition +org.apache.spark.ui.jobs.JobProgressUI +org.apache.spark.deploy.Command +org.apache.spark.ui.jobs.StagePage +org.apache.spark.serializer.KryoDeserializationStream +org.apache.spark.scheduler.ExecutorLost +org.apache.spark.deploy.master.BlackHolePersistenceEngine +org.apache.spark.scheduler.cluster.SimrSchedulerBackend +org.apache.spark.streaming.dstream.MapPartitionedDStream +org.apache.spark.scheduler.TaskInfo +org.apache.spark.storage.BlockManagerSlaveActor +org.apache.spark.ui.env.EnvironmentUI +org.apache.spark.executor.CoarseGrainedExecutorBackend +org.apache.spark.storage.ShuffleBlockManager +org.apache.spark.rdd.NarrowCoGroupSplitDep +org.apache.spark.scheduler.JobGroupCancelled +org.apache.spark.rdd.MapPartitionsRDD +org.apache.spark.graphx.impl.EdgeTripletIterator +org.apache.spark.storage.BlockManagerMaster +org.apache.spark.partial.SumEvaluator +org.apache.spark.network.netty.FileServerHandler +org.apache.spark.GetMapOutputStatuses +org.apache.spark.tools.SparkMethod +org.apache.spark.storage.ShuffleWriterGroup +org.apache.spark.ui.jobs.StageTable +org.apache.spark.scheduler.TaskDescription +org.apache.spark.deploy.master.Master +org.apache.spark.scheduler.DAGSchedulerSource +org.apache.spark.streaming.util.Clock +org.apache.spark.deploy.master.ui.MasterWebUI +org.apache.spark.CacheManager +org.apache.spark.streaming.util.SystemClock +org.apache.spark.storage.BroadcastHelperBlockId +org.apache.spark.deploy.master.ZooKeeperLeaderElectionAgent +org.apache.spark.storage.FileSegment +org.apache.spark.api.python.PythonPartitioner +org.apache.spark.scheduler.ExecutorLossReason +org.apache.spark.network.ReceivingConnection +org.apache.spark.scheduler.Schedulable +org.apache.spark.scheduler.TaskSet +org.apache.spark.storage.BlockStore +org.apache.spark.streaming.scheduler.StreamingListenerBus +org.apache.spark.deploy.ClientArguments +org.apache.spark.metrics.MetricsConfig +org.apache.spark.serializer.SerializerManager +org.apache.spark.streaming.scheduler.NetworkInputTracker +org.apache.spark.broadcast.HttpBroadcast +org.apache.spark.executor.ExecutorURLClassLoader +org.apache.spark.scheduler.TaskSchedulerImpl +org.apache.spark.streaming.dstream.RawNetworkReceiver +org.apache.spark.util.collection.SizeTrackingAppendOnlyMap +org.apache.spark.rdd.SampledRDDPartition +org.apache.spark.storage.BlockManagerId +org.apache.spark.deploy.master.LeaderElectionAgent +org.apache.spark.streaming.dstream.StopReceiver +org.apache.spark.storage.GotBlock +org.apache.spark.scheduler.Task +org.apache.spark.rdd.CartesianPartition +org.apache.spark.storage.StorageStatus +org.apache.spark.rdd.PartitionwiseSampledRDDPartition +org.apache.spark.graphx.impl.ShuffleSerializationStream +org.apache.spark.scheduler.cluster.mesos.MesosSchedulerBackend +org.apache.spark.deploy.worker.ui.IndexPage +org.apache.spark.partial.ApproximateEvaluator +org.apache.spark.graphx.impl.ShuffleSerializerInstance +org.apache.spark.ui.jobs.ExecutorSummary +org.apache.spark.scheduler.GettingResultEvent +org.apache.spark.graphx.impl.RoutingTable +org.apache.spark.streaming.util.KillingThread +org.apache.spark.GrowableAccumulableParam +org.apache.spark.deploy.master.ApplicationInfo +org.apache.spark.deploy.worker.Worker +org.apache.spark.streaming.dstream.ReportBlock +org.apache.spark.partial.StudentTCacher +org.apache.spark.scheduler.ActiveJob +org.apache.spark.network.netty.FileServer +org.apache.spark.network.netty.ShuffleCopier +org.apache.spark.util.collection.PrimitiveVector +org.apache.spark.serializer.KryoSerializerInstance +org.apache.spark.streaming.util.RateLimitedOutputStream +org.apache.spark.network.Message +org.apache.spark.util.InnerClosureFinder +org.apache.spark.util.SerializableHyperLogLog +org.apache.spark.deploy.worker.ProcessBuilderLike +org.apache.spark.streaming.Interval +org.apache.spark.rdd.CheckpointRDD +org.apache.spark.deploy.master.SparkZooKeeperSession +org.apache.spark.graphx.impl.VertexAttributeBlock +org.apache.spark.streaming.dstream.MapValuedDStream +org.apache.spark.util.collection.ExternalAppendOnlyMap +org.apache.spark.deploy.client.AppClient +org.apache.spark.deploy.DriverDescription +org.apache.spark.scheduler.FairSchedulableBuilder +org.apache.spark.executor.MesosExecutorBackend +org.apache.spark.streaming.scheduler.ClearMetadata +org.apache.spark.storage.BroadcastBlockId +org.apache.spark.streaming.scheduler.Job +org.apache.spark.streaming.scheduler.DeregisterReceiver +org.apache.spark.serializer.KryoSerializationStream +org.apache.spark.scheduler.FIFOSchedulableBuilder +org.apache.spark.ui.jobs.JobProgressListener +org.apache.spark.TaskEndReason +org.apache.spark.storage.BlockId +org.apache.spark.streaming.scheduler.JobGenerator +org.apache.spark.deploy.master.WorkerInfo +org.apache.spark.storage.ShuffleBlockId +org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessage +org.apache.spark.storage.ThreadingTest.ConsumerThread +org.apache.spark.rdd.PartitionGroup +org.apache.spark.util.collection.AppendOnlyMap +org.apache.spark.scheduler.ExecutorExited +org.apache.spark.streaming.Checkpoint +org.apache.spark.streaming.scheduler.AddBlocks +org.apache.spark.network.netty.FileClient +org.apache.spark.scheduler.JobListener +org.apache.spark.streaming.scheduler.JobSet +org.apache.spark.scheduler.ExecutorGained +org.apache.spark.partial.CountEvaluator +org.apache.spark.scheduler.Pool +org.apache.spark.deploy.master.FileSystemPersistenceEngine +org.apache.spark.rdd.BlockRDD +org.apache.spark.network.netty.FileHeader +org.apache.spark.rdd.PartitionerAwareUnionRDDPartition +org.apache.spark.graphx.impl.VertexIdMsgSerializer +org.apache.spark.streaming.dstream.FileInputDStream +org.apache.spark.deploy.worker.ui.WorkerWebUI +org.apache.spark.metrics.MetricsSystem +org.apache.spark.scheduler.JobSubmitted +org.apache.spark.graphx.impl.MsgRDDFunctions +org.apache.spark.api.java.function.WrappedFunction3 +org.apache.spark.streaming.CheckpointWriter +org.apache.spark.storage.BlockManager +org.apache.spark.util.CompletionIterator +org.apache.spark.network.MessageChunk +org.apache.spark.rdd.MappedValuesRDD +org.apache.spark.streaming.dstream.PluggableInputDStream +org.apache.spark.rdd.ZippedWithIndexRDDPartition +org.apache.spark.streaming.receivers.ActorReceiver +org.apache.spark.serializer.JavaDeserializationStream +org.apache.spark.rdd.SubtractedRDD +org.apache.spark.streaming.dstream.SocketInputDStream +org.apache.spark.partial.GroupedSumEvaluator +org.apache.spark.broadcast.TorrentInfo +org.apache.spark.storage.BlockException +org.apache.spark.streaming.dstream.FlatMapValuedDStream +org.apache.spark.deploy.worker.DriverRunner +org.apache.spark.deploy.master.ExecutorInfo +org.apache.spark.deploy.master.ui.IndexPage +org.apache.spark.rdd.ParallelCollectionPartition +org.apache.spark.scheduler.DirectTaskResult +org.apache.spark.partial.ApproximateActionListener +org.apache.spark.streaming.DStreamGraph +org.apache.spark.partial.GroupedCountEvaluator +org.apache.spark.api.java.JavaSparkContextVarargsWorkaround +org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +org.apache.spark.streaming.dstream.StateDStream +org.apache.spark.FetchFailedException +org.apache.spark.BlockStoreShuffleFetcher +org.apache.spark.util.Clock +org.apache.spark.storage.DiskBlockManager +org.apache.spark.scheduler.SparkListenerBus +org.apache.spark.rdd.GlommedRDD +org.apache.spark.streaming.dstream.UnionDStream +org.apache.spark.scheduler.JobResult +org.apache.spark.deploy.ApplicationDescription +org.apache.spark.rdd.CoGroupSplitDep +org.apache.spark.api.python.PythonRDD +org.apache.spark.scheduler.cluster.mesos.CoarseMesosSchedulerBackend +org.apache.spark.partial.GroupedMeanEvaluator +org.apache.spark.graphx.impl.VertexBroadcastMsg +org.apache.spark.ShuffleFetcher +org.apache.spark.streaming.dstream.ShuffledDStream +org.apache.spark.scheduler.TaskScheduler +org.apache.spark.streaming.dstream.MappedDStream +org.apache.spark.storage.BlockFetchTracker +org.apache.spark.rdd.ZippedPartitionsPartition +org.apache.spark.rdd.ShuffleCoGroupSplitDep +org.apache.spark.graphx.impl.LongVertexBroadcastMsgSerializer +org.apache.spark.deploy.ExecutorDescription +org.apache.spark.MapOutputTrackerMessage +org.apache.spark.util.MetadataCleaner +org.apache.spark.HttpFileServer +org.apache.spark.streaming.util.ManualClock +org.apache.spark.streaming.scheduler.JobGeneratorEvent +org.apache.spark.scheduler.JobCancelled +org.apache.spark.streaming.scheduler.DoCheckpoint +org.apache.spark.broadcast.TorrentBroadcast +org.apache.spark.scheduler.FIFOSchedulingAlgorithm +org.apache.spark.network.ConnectionManagerId +org.apache.spark.deploy.master.PersistenceEngine +org.apache.spark.mllib.recommendation.InLinkBlock +org.apache.spark.partial.MeanEvaluator +org.apache.spark.streaming.dstream.ReportError +org.apache.spark.storage.RDDBlockId +org.apache.spark.api.java.function.WrappedFunction2 +org.apache.spark.ui.exec.ExecutorsUI +org.apache.spark.network.netty.FileServerChannelInitializer +org.apache.spark.streaming.scheduler.JobStarted +org.apache.spark.streaming.dstream.RawInputDStream +org.apache.spark.storage.GetBlock +org.apache.spark.ui.jobs.IndexPage +org.apache.spark.storage.BlockManagerSource +org.apache.spark.rdd.BlockRDDPartition +org.apache.spark.rdd.PartitionCoalescer +org.apache.spark.network.ConnectionManager +org.apache.spark.deploy.master.MasterArguments +org.apache.spark.graphx.impl.MessageToPartition +org.apache.spark.executor.ExecutorBackend +org.apache.spark.util.NextIterator +org.apache.spark.storage.BlockManagerWorker +org.apache.spark.streaming.dstream.QueueInputDStream +org.apache.spark.streaming.scheduler.JobScheduler +org.apache.spark.streaming.dstream.FlatMappedDStream +org.apache.spark.scheduler.TaskResultGetter +org.apache.spark.network.netty.FileClientChannelInitializer +org.apache.spark.rdd.MappedRDD +org.apache.spark.rdd.PartitionerAwareUnionRDD +org.apache.spark.network.BufferMessage +org.apache.spark.streaming.dstream.DStreamCheckpointData +org.apache.spark.executor.Executor +org.apache.spark.MapOutputTrackerMaster +org.apache.spark.deploy.client.AppClientListener +org.apache.spark.storage.BlockInfo +org.apache.spark.streaming.dstream.ReducedWindowedDStream +org.apache.spark.rdd.JdbcPartition +org.apache.spark.deploy.TestWorkerInfo +org.apache.spark.scheduler.BeginEvent +org.apache.spark.storage.BlockMessage +org.apache.spark.tools.SparkType +org.apache.spark.rdd.NewHadoopPartition +org.apache.spark.streaming.scheduler.JobSchedulerEvent +org.apache.spark.streaming.util.TestOutputStream +org.apache.spark.scheduler.local.LocalActor +org.apache.spark.graphx.impl.EdgePartition +org.apache.spark.scheduler.TaskSetFailed +org.apache.spark.ServerStateException +org.apache.spark.network.MessageChunkHeader +org.apache.spark.storage.DiskBlockObjectWriter +org.apache.spark.graphx.impl.VertexPartition +org.apache.spark.MapOutputTracker +org.apache.spark.rdd.CartesianRDD +org.apache.spark.storage.ThreadingTest.ProducerThread +org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +org.apache.spark.graphx.impl.IntVertexBroadcastMsgSerializer +org.apache.spark.streaming.scheduler.NetworkInputTrackerMessage +org.apache.spark.deploy.worker.WorkerSource +org.apache.spark.graphx.impl.DoubleVertexBroadcastMsgSerializer +org.apache.spark.WritableConverter +org.apache.spark.streaming.util.RecurringTimer +org.apache.spark.util.collection.PrimitiveKeyOpenHashMap +org.apache.spark.scheduler.CompletionEvent +org.apache.spark.streaming.receivers.Data +org.apache.spark.rdd.UnionPartition +org.apache.spark.broadcast.TorrentBlock +org.apache.spark.storage.TestBlockId +org.apache.spark.storage.BlockManagerMasterActor +org.apache.spark.storage.TaskResultBlockId +org.apache.spark.streaming.scheduler.ErrorReported +org.apache.spark.streaming.scheduler.RegisterReceiver +org.apache.spark.scheduler.DAGScheduler +org.apache.spark.deploy.TestMasterInfo +org.apache.spark.deploy.master.DriverInfo +org.apache.spark.ui.storage.BlockManagerUI +org.apache.spark.streaming.scheduler.GenerateJobs +org.apache.spark.storage.StreamBlockId +org.apache.spark.util.FieldAccessFinder +org.apache.spark.scheduler.local.LocalBackend +org.apache.spark.network.netty.FileClientHandler +org.apache.spark.tools.ParameterizedType +org.apache.spark.network.Connection +org.apache.spark.ui.storage.RDDPage +org.apache.spark.HttpServer +org.apache.spark.deploy.DockerId +org.apache.spark.streaming.scheduler.ClearCheckpointData +org.apache.spark.rdd.ZippedPartition +org.apache.spark.FetchFailed +org.apache.spark.serializer.JavaSerializerInstance +org.apache.spark.scheduler.SchedulableBuilder +org.apache.spark.streaming.dstream.TransformedDStream +org.apache.spark.streaming.util.FileGeneratingThread +org.apache.spark.api.java.function.WrappedFunction1 +org.apache.spark.broadcast.BroadcastManager +org.apache.spark.rdd.FilteredRDD +org.apache.spark.graphx.impl.IntAggMsgSerializer +org.apache.spark.scheduler.MapStatus +org.apache.spark.tools.BaseType +org.apache.spark.ui.jobs.PoolTable +org.apache.spark.graphx.impl.ShuffleDeserializationStream +org.apache.spark.scheduler.DAGSchedulerEvent +org.apache.spark.ui.jobs.ExecutorTable +org.apache.spark.deploy.master.MasterSource +org.apache.spark.graphx.impl.VertexBroadcastMsgRDDFunctions +org.apache.spark.deploy.worker.WorkerArguments +org.apache.spark.deploy.worker.WorkerWatcher +org.apache.spark.graphx.impl.EdgePartitionBuilder +org.apache.spark.deploy.DeployMessage +org.apache.spark.streaming.dstream.SocketReceiver +org.apache.spark.scheduler.ResultTask +org.apache.spark.rdd.RDDCheckpointData +org.apache.spark.ui.SparkUI +org.apache.spark.util.collection.OpenHashSet +org.apache.spark.streaming.dstream.NetworkReceiverMessage +org.apache.spark.ui.jobs.PoolPage +org.apache.spark.deploy.worker.Clock +org.apache.spark.scheduler.TaskLocation +org.apache.spark.deploy.worker.Sleeper +org.apache.spark.storage.TempBlockId +org.apache.spark.storage.BlockObjectWriter +org.apache.spark.streaming.dstream.WindowedDStream +org.apache.spark.util.random.XORShiftRandom +org.apache.spark.network.SendingConnection +org.apache.spark.ui.storage.IndexPage +org.apache.spark.rdd.HadoopPartition +org.apache.spark.util.Utils.CallSiteInfo +org.apache.spark.deploy.LocalSparkCluster +org.apache.spark.streaming.ObjectInputStreamWithLoader +org.apache.spark.util.IdGenerator +org.apache.spark.graphx.impl.DoubleAggMsgSerializer +org.apache.spark.ExceptionFailure +org.apache.spark.storage.BlockFetcherIterator +org.apache.spark.MapOutputTrackerMasterActor +org.apache.spark.storage.BlockMessageArray +org.apache.spark.graphx.impl.LongAggMsgSerializer +org.apache.spark.scheduler.FairSchedulingAlgorithm +org.apache.spark.scheduler.IndirectTaskResult +org.apache.spark.storage.PutBlock +org.apache.spark.util.collection.OpenHashMap +org.apache.spark.scheduler.ShuffleMapTask +org.apache.spark.util.ByteBufferInputStream +org.apache.spark.scheduler.SchedulingAlgorithm \ No newline at end of file diff --git a/bin/spark-class b/bin/spark-class index c4225a392d6da..dfa11e854dc89 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -129,7 +129,7 @@ fi # Compute classpath using external script CLASSPATH=`$FWDIR/bin/compute-classpath.sh` - +CLASSPATH="$CLASSPATH:$SPARK_TOOLS_JAR" if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then CLASSPATH="$CLASSPATH:$SPARK_TOOLS_JAR" fi diff --git a/dev/run-tests b/dev/run-tests index d65a397b4c8c7..432563e1ef845 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -32,6 +32,12 @@ echo "Running Scala style checks" echo "=========================================================================" sbt/sbt clean scalastyle +echo "=========================================================================" +echo "Running MiMa for detecting binary incompatibilites." +echo "Please see MimaBuild.scala for details." +echo "=========================================================================" +sbt/sbt mima-report-binary-issues + echo "=========================================================================" echo "Running Spark unit tests" echo "=========================================================================" diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala new file mode 100644 index 0000000000000..d83c7bf092833 --- /dev/null +++ b/project/MimaBuild.scala @@ -0,0 +1,66 @@ +import com.typesafe.tools.mima.plugin.MimaKeys.{binaryIssueFilters, previousArtifact} +import com.typesafe.tools.mima.plugin.MimaPlugin.mimaDefaultSettings +import sbt._ + +object MimaBuild { + + def ignoredABIProblems(base: File) = { + import com.typesafe.tools.mima.core._ + import com.typesafe.tools.mima.core.ProblemFilters._ + + IO.read(file(base.getAbsolutePath + "/.mima-exclude")).split("\n").map(excludePackage).toSeq ++ +// /** +// * A: Detections are semi private or likely to become semi private at some point. +// */ +// Seq(exclude[MissingClassProblem]("org.apache.spark.util.XORShiftRandom"), +// exclude[MissingClassProblem]("org.apache.spark.util.XORShiftRandom$"), +// exclude[MissingMethodProblem]("org.apache.spark.util.Utils.cloneWritables"), +// // Scheduler is not considered a public API. +// excludePackage("org.apache.spark.deploy"), +// // Was made private in 1.0 +// excludePackage("org.apache.spark.util.collection.ExternalAppendOnlyMap#DiskMapIterator"), +// excludePackage("org.apache.spark.util.collection.ExternalAppendOnlyMap#ExternalIterator"), +// exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaPairRDD.cogroupResultToJava"), +// exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaPairRDD.groupByResultToJava"), +// exclude[IncompatibleMethTypeProblem]("org.apache.spark.scheduler.TaskSchedulerImpl.handleFailedTask"), +// exclude[MissingMethodProblem]("org.apache.spark.scheduler.TaskSchedulerImpl.taskSetTaskIds"), +// exclude[IncompatibleMethTypeProblem]("org.apache.spark.scheduler.TaskSetManager.handleFailedTask"), +// exclude[MissingMethodProblem]("org.apache.spark.scheduler.TaskSetManager.removeAllRunningTasks"), +// exclude[MissingMethodProblem]("org.apache.spark.scheduler.TaskSetManager.runningTasks_="), +// exclude[MissingMethodProblem]("org.apache.spark.scheduler.DAGScheduler.lastFetchFailureTime"), +// exclude[MissingMethodProblem]("org.apache.spark.scheduler.DAGScheduler.lastFetchFailureTime_="), +// exclude[MissingMethodProblem]("org.apache.spark.storage.BlockObjectWriter.bytesWritten")) ++ +// /** +// * B: Detections are mostly false +ve. +// */ +// Seq(exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.setGenerator"), +// exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.mapPartitions"), +// exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.mapPartitions"), +// exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.mapPartitions"), +// exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.foreachPartition"), +// exclude[MissingMethodProblem]("org.apache.spark.api.python.PythonRDD.writeToStream")) ++ + /** + * Detections I am unsure about. Should be either moved to B (false +ve) or A. + */ + Seq( +// exclude[MissingClassProblem]("org.apache.spark.mllib.recommendation.MFDataGenerator$"), +// exclude[MissingClassProblem]("org.apache.spark.rdd.ClassTags"), +// exclude[MissingClassProblem]("org.apache.spark.rdd.ClassTags$"), +// exclude[MissingMethodProblem]("org.apache.spark.util.collection.ExternalAppendOnlyMap.org$apache$spark$util$collection$ExternalAppendOnlyMap$$wrapForCompression$1"), +// exclude[MissingMethodProblem]("org.apache.spark.util.collection.ExternalAppendOnlyMap.org$apache$spark$util$collection$ExternalAppendOnlyMap$$sparkConf"), +// exclude[MissingClassProblem]("org.apache.spark.mllib.recommendation.MFDataGenerator"), +// exclude[MissingClassProblem]("org.apache.spark.mllib.optimization.SquaredGradient"), +// exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.regression.LinearRegressionWithSGD.gradient"), +// exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.regression.RidgeRegressionWithSGD.gradient"), +// exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.regression.LassoWithSGD.gradient"), +// exclude[FinalClassProblem]("org.apache.spark.SparkFiles") + + ) + } + + def mimaSettings(sparkHome: File) = mimaDefaultSettings ++ Seq( + previousArtifact := None, + binaryIssueFilters ++= ignoredABIProblems(sparkHome) + ) + +} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 74bad66cfd018..ed5e598d798c2 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -22,6 +22,7 @@ import sbtassembly.Plugin._ import AssemblyKeys._ import scala.util.Properties import org.scalastyle.sbt.ScalastylePlugin.{Settings => ScalaStyleSettings} +import com.typesafe.tools.mima.plugin.MimaKeys.previousArtifact // For Sonatype publishing //import com.jsuereth.pgp.sbtplugin.PgpKeys._ @@ -121,7 +122,7 @@ object SparkBuild extends Build { lazy val allProjects = packageProjects ++ allExternalRefs ++ Seq[ProjectReference](examples, tools, assemblyProj) - def sharedSettings = Defaults.defaultSettings ++ Seq( + def sharedSettings = Defaults.defaultSettings ++ MimaBuild.mimaSettings(file(sparkHome)) ++ Seq( organization := "org.apache.spark", version := "1.0.0-incubating-SNAPSHOT", scalaVersion := "2.10.3", @@ -244,13 +245,19 @@ object SparkBuild extends Build { val excludeAsm = ExclusionRule(organization = "asm") val excludeSnappy = ExclusionRule(organization = "org.xerial.snappy") + def sparkPreviousArtifact(id: String, organization: String = "org.apache.spark", + version: String = "0.9.0-incubating", crossVersion: String = "2.10"): Option[sbt.ModuleID] = { + val fullId = if (crossVersion.isEmpty) id else id + "_" + crossVersion + Some(organization % fullId % version) // the artifact to compare binary compatibility with + } + def coreSettings = sharedSettings ++ Seq( name := "spark-core", resolvers ++= Seq( "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/", "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/" ), - + previousArtifact := sparkPreviousArtifact("spark-core"), libraryDependencies ++= Seq( "com.google.guava" % "guava" % "14.0.1", "com.google.code.findbugs" % "jsr305" % "1.3.9", @@ -289,7 +296,7 @@ object SparkBuild extends Build { publish := {} ) - def replSettings = sharedSettings ++ Seq( + def replSettings = sharedSettings ++ Seq( name := "spark-repl", libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-compiler" % v ), libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "jline" % v ), @@ -298,6 +305,7 @@ object SparkBuild extends Build { def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", + previousArtifact := sparkPreviousArtifact("spark-examples"), libraryDependencies ++= Seq( "com.twitter" %% "algebird-core" % "0.1.11", "org.apache.hbase" % "hbase" % "0.94.6" excludeAll(excludeNetty, excludeAsm), @@ -321,17 +329,20 @@ object SparkBuild extends Build { def graphxSettings = sharedSettings ++ Seq( name := "spark-graphx", + previousArtifact := sparkPreviousArtifact("spark-graphx"), libraryDependencies ++= Seq( "org.jblas" % "jblas" % "1.2.3" ) ) def bagelSettings = sharedSettings ++ Seq( - name := "spark-bagel" + name := "spark-bagel", + previousArtifact := sparkPreviousArtifact("spark-bagel") ) def mllibSettings = sharedSettings ++ Seq( name := "spark-mllib", + previousArtifact := sparkPreviousArtifact("spark-mllib"), libraryDependencies ++= Seq( "org.jblas" % "jblas" % "1.2.3" ) @@ -339,6 +350,7 @@ object SparkBuild extends Build { def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", + previousArtifact := sparkPreviousArtifact("spark-streaming"), libraryDependencies ++= Seq( "commons-io" % "commons-io" % "2.4" ) @@ -403,6 +415,7 @@ object SparkBuild extends Build { def twitterSettings() = sharedSettings ++ Seq( name := "spark-streaming-twitter", + previousArtifact := sparkPreviousArtifact("spark-streaming-twitter"), libraryDependencies ++= Seq( "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty) ) @@ -410,6 +423,7 @@ object SparkBuild extends Build { def kafkaSettings() = sharedSettings ++ Seq( name := "spark-streaming-kafka", + previousArtifact := sparkPreviousArtifact("spark-streaming-kafka"), libraryDependencies ++= Seq( "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty), "org.apache.kafka" %% "kafka" % "0.8.0" @@ -422,6 +436,7 @@ object SparkBuild extends Build { def flumeSettings() = sharedSettings ++ Seq( name := "spark-streaming-flume", + previousArtifact := sparkPreviousArtifact("spark-streaming-flume"), libraryDependencies ++= Seq( "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy) ) @@ -429,6 +444,7 @@ object SparkBuild extends Build { def zeromqSettings() = sharedSettings ++ Seq( name := "spark-streaming-zeromq", + previousArtifact := sparkPreviousArtifact("spark-streaming-zeromq"), libraryDependencies ++= Seq( "org.spark-project.akka" %% "akka-zeromq" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty) ) @@ -436,6 +452,7 @@ object SparkBuild extends Build { def mqttSettings() = streamingSettings ++ Seq( name := "spark-streaming-mqtt", + previousArtifact := sparkPreviousArtifact("spark-streaming-mqtt"), resolvers ++= Seq("Eclipse Repo" at "https://repo.eclipse.org/content/repositories/paho-releases/"), libraryDependencies ++= Seq("org.eclipse.paho" % "mqtt-client" % "0.4.0") ) diff --git a/project/plugins.sbt b/project/plugins.sbt index 914f2e05a402a..aa2baecd2fe1f 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -19,3 +19,4 @@ addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.7.4") addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.4.0") +addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.6") diff --git a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala new file mode 100644 index 0000000000000..3246db98f5df5 --- /dev/null +++ b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala @@ -0,0 +1,114 @@ +/* + * 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.tools + +import java.io.File +import java.util.jar.JarFile + +import scala.collection.mutable +import scala.collection.JavaConversions.enumerationAsScalaIterator + +/** + * Mima(TODO: Paste URL here) generates a lot of false positives as it does not detect + * private[x] as internal APIs. + */ +object GenerateMIMAIgnore { + + def classesWithPrivateWithin(packageName: String, excludePackages: Seq[String]): Set[String] = { + import scala.reflect.runtime.universe.runtimeMirror + val classLoader: ClassLoader = Thread.currentThread().getContextClassLoader + val mirror = runtimeMirror(classLoader) + val classes = Utils.getClasses(packageName, classLoader) + val privateClasses = mutable.HashSet[String]() + for (x <- classes) { + try { + // some of the classnames throw malformed class name exceptions and weird Match errors. + if (excludePackages.forall(!x.startsWith(_)) && + mirror.staticClass(x).privateWithin.toString.trim != "") { + privateClasses += x + } + } catch { + case e: Throwable => // println(e) + } + } + privateClasses.toSet + } + + def main(args: Array[String]) { + scala.tools.nsc.io.File(".mima-exclude"). + writeAll(classesWithPrivateWithin("org.apache.spark", args).mkString("\n")) + println("Created : .mima-exclude in current directory.") + } + +} + +object Utils { + + /** + * Get all classes in a package from a jar file. + */ + def getAllClasses(jarPath: String, packageName: String) = { + val jar = new JarFile(new File(jarPath)) + val enums = jar.entries().map(_.getName).filter(_.startsWith(packageName)) + val classes = mutable.HashSet[Class[_]]() + for (entry <- enums) { + if (!entry.endsWith("/") && !entry.endsWith("MANIFEST.MF") && !entry.endsWith("properties")) { + try { + classes += Class.forName(entry.trim.replaceAll(".class", "").replace('/', '.')) + } catch { + case e: Throwable => // println(e) // It may throw a few ClassNotFoundExceptions + } + } + } + classes + } + + /** + * Scans all classes accessible from the context class loader which belong to the given package + * and subpackages both from directories and jars present on the classpath. + */ + def getClasses(packageName: String, + classLoader: ClassLoader = Thread.currentThread().getContextClassLoader): Set[String] = { + val path = packageName.replace('.', '/') + val resources = classLoader.getResources(path).toArray + val jars = resources.filter(x => x.getProtocol == "jar") + .map(_.getFile.split(":")(1).split("!")(0)) + val classesFromJars = jars.map(getAllClasses(_, path)).flatten + val dirs = resources.filter(x => x.getProtocol == "file") + .map(x => new File(x.getFile.split(":")(1))) + val classFromDirs = dirs.map(findClasses(_, packageName)).flatten + (classFromDirs ++ classesFromJars).map(_.getCanonicalName).filter(_ != null).toSet + } + + private def findClasses(directory: File, packageName: String): Seq[Class[_]] = { + val classes = mutable.ArrayBuffer[Class[_]]() + if (!directory.exists()) { + return classes + } + val files = directory.listFiles() + for (file <- files) { + if (file.isDirectory) { + classes ++= findClasses(file, packageName + "." + file.getName) + } else if (file.getName.endsWith(".class")) { + classes += Class.forName(packageName + '.' + file.getName.substring(0, + file.getName.length() - 6)) + } + } + classes + } +}