@@ -28,13 +28,13 @@ import scala.collection.mutable.Stack
2828import sbt ._
2929import sbt .Classpaths .publishTask
3030import sbt .Keys ._
31- import sbtunidoc .Plugin .UnidocKeys .unidocGenjavadocVersion
3231import com .etsy .sbt .checkstyle .CheckstylePlugin .autoImport ._
3332import com .simplytyped .Antlr4Plugin ._
3433import com .typesafe .sbt .pom .{PomBuild , SbtPomKeys }
3534import com .typesafe .tools .mima .plugin .MimaKeys
3635import org .scalastyle .sbt .ScalastylePlugin .autoImport ._
3736import org .scalastyle .sbt .Tasks
37+ import sbtassembly .AssemblyPlugin .autoImport ._
3838
3939import spray .revolver .RevolverPlugin ._
4040
@@ -83,6 +83,8 @@ object BuildCommons {
8383object SparkBuild extends PomBuild {
8484
8585 import BuildCommons ._
86+ import sbtunidoc .GenJavadocPlugin
87+ import sbtunidoc .GenJavadocPlugin .autoImport ._
8688 import scala .collection .mutable .Map
8789
8890 val projectsMap : Map [String , Seq [Setting [_]]] = Map .empty
@@ -106,13 +108,10 @@ object SparkBuild extends PomBuild {
106108 override val userPropertiesMap = System .getProperties.asScala.toMap
107109
108110 lazy val MavenCompile = config(" m2r" ) extend(Compile )
109- lazy val publishLocalBoth = TaskKey [Unit ](" publish-local " , " publish local for m2 and ivy" )
111+ lazy val publishLocalBoth = TaskKey [Unit ](" localPublish " , " publish local for m2 and ivy" , KeyRanks . ATask )
110112
111- lazy val sparkGenjavadocSettings : Seq [sbt.Def .Setting [_]] = Seq (
112- libraryDependencies += compilerPlugin(
113- " com.typesafe.genjavadoc" %% " genjavadoc-plugin" % unidocGenjavadocVersion.value cross CrossVersion .full),
113+ lazy val sparkGenjavadocSettings : Seq [sbt.Def .Setting [_]] = GenJavadocPlugin .projectSettings ++ Seq (
114114 scalacOptions ++= Seq (
115- " -P:genjavadoc:out=" + (target.value / " java" ),
116115 " -P:genjavadoc:strictVisibility=true" // hide package private types
117116 )
118117 )
@@ -157,7 +156,7 @@ object SparkBuild extends PomBuild {
157156 val scalaSourceV = Seq (file(scalaSource.in(config).value.getAbsolutePath))
158157 val configV = (baseDirectory in ThisBuild ).value / scalaStyleOnCompileConfig
159158 val configUrlV = scalastyleConfigUrl.in(config).value
160- val streamsV = streams.in(config).value
159+ val streamsV = ( streams.in(config).value: @ sbtUnchecked)
161160 val failOnErrorV = true
162161 val failOnWarningV = false
163162 val scalastyleTargetV = scalastyleTarget.in(config).value
@@ -204,7 +203,6 @@ object SparkBuild extends PomBuild {
204203 javaHome := sys.env.get(" JAVA_HOME" )
205204 .orElse(sys.props.get(" java.home" ).map { p => new File (p).getParentFile().getAbsolutePath() })
206205 .map(file),
207- incOptions := incOptions.value.withNameHashing(true ),
208206 publishMavenStyle := true ,
209207 unidocGenjavadocVersion := " 0.16" ,
210208
@@ -219,10 +217,12 @@ object SparkBuild extends PomBuild {
219217 ),
220218 externalResolvers := resolvers.value,
221219 otherResolvers := SbtPomKeys .mvnLocalRepository(dotM2 => Seq (Resolver .file(" dotM2" , dotM2))).value,
222- publishLocalConfiguration in MavenCompile :=
223- new PublishConfiguration (None , " dotM2" , packagedArtifacts.value, Seq (), ivyLoggingLevel.value),
220+ publishLocalConfiguration in MavenCompile := PublishConfiguration ()
221+ .withResolverName(" dotM2" )
222+ .withArtifacts(packagedArtifacts.value.toVector)
223+ .withLogging(ivyLoggingLevel.value),
224224 publishMavenStyle in MavenCompile := true ,
225- publishLocal in MavenCompile := publishTask(publishLocalConfiguration in MavenCompile , deliverLocal ).value,
225+ publishLocal in MavenCompile := publishTask(publishLocalConfiguration in MavenCompile ).value,
226226 publishLocalBoth := Seq (publishLocal in MavenCompile , publishLocal).dependOn.value,
227227
228228 javacOptions in (Compile , doc) ++= {
@@ -251,6 +251,8 @@ object SparkBuild extends PomBuild {
251251 " -sourcepath" , (baseDirectory in ThisBuild ).value.getAbsolutePath // Required for relative source links in scaladoc
252252 ),
253253
254+ SbtPomKeys .profiles := profiles,
255+
254256 // Remove certain packages from Scaladoc
255257 scalacOptions in (Compile , doc) := Seq (
256258 " -groups" ,
@@ -273,14 +275,15 @@ object SparkBuild extends PomBuild {
273275 val out = streams.value
274276
275277 def logProblem (l : (=> String ) => Unit , f : File , p : xsbti.Problem ) = {
276- l(f.toString + " :" + p.position.line.fold(" " )(_ + " :" ) + " " + p.message)
278+ val jmap = new java.util.function.Function [Integer , String ]() {override def apply (i : Integer ): String = {i.toString}}
279+ l(f.toString + " :" + p.position.line.map[String ](jmap.apply).map(_ + " :" ).orElse(" " ) + " " + p.message)
277280 l(p.position.lineContent)
278281 l(" " )
279282 }
280283
281284 var failed = 0
282- analysis.infos.allInfos.foreach { case (k, i) =>
283- i.reportedProblems foreach { p =>
285+ analysis.asInstanceOf [sbt.internal.inc. Analysis ]. infos.allInfos.foreach { case (k, i) =>
286+ i.getReportedProblems foreach { p =>
284287 val deprecation = p.message.contains(" deprecated" )
285288
286289 if (! deprecation) {
@@ -302,7 +305,10 @@ object SparkBuild extends PomBuild {
302305 sys.error(s " $failed fatal warnings " )
303306 }
304307 analysis
305- }
308+ },
309+ // disable Mima check for all modules,
310+ // to be enabled in specific ones that have previous artifacts
311+ MimaKeys .mimaFailOnNoPrevious := false
306312 )
307313
308314 def enable (settings : Seq [Setting [_]])(projectRef : ProjectRef ) = {
@@ -411,7 +417,7 @@ object SparkBuild extends PomBuild {
411417 }
412418 ))(assembly)
413419
414- enable(Seq (sparkShell := sparkShell in LocalProject (" assembly" )))(spark)
420+ enable(Seq (sparkShell := ( sparkShell in LocalProject (" assembly" )).value ))(spark)
415421
416422 // TODO: move this to its upstream project.
417423 override def projectDefinitions (baseDirectory : File ): Seq [Project ] = {
@@ -485,20 +491,20 @@ object SparkParallelTestGrouping {
485491 testGrouping in Test := {
486492 val tests : Seq [TestDefinition ] = (definedTests in Test ).value
487493 val defaultForkOptions = ForkOptions (
488- bootJars = Nil ,
489494 javaHome = javaHome.value,
490- connectInput = connectInput.value,
491495 outputStrategy = outputStrategy.value,
492- runJVMOptions = (javaOptions in Test ).value ,
496+ bootJars = Vector .empty[java.io. File ] ,
493497 workingDirectory = Some (baseDirectory.value),
498+ runJVMOptions = (javaOptions in Test ).value.toVector,
499+ connectInput = connectInput.value,
494500 envVars = (envVars in Test ).value
495501 )
496502 tests.groupBy(test => testNameToTestGroup(test.name)).map { case (groupName, groupTests) =>
497503 val forkOptions = {
498504 if (groupName == DEFAULT_TEST_GROUP ) {
499505 defaultForkOptions
500506 } else {
501- defaultForkOptions.copy(runJVMOptions = defaultForkOptions.runJVMOptions ++
507+ defaultForkOptions.withRunJVMOptions( defaultForkOptions.runJVMOptions ++
502508 Seq (s " -Djava.io.tmpdir= ${baseDirectory.value}/target/tmp/ $groupName" ))
503509 }
504510 }
@@ -512,6 +518,7 @@ object SparkParallelTestGrouping {
512518}
513519
514520object Core {
521+ import scala .sys .process .Process
515522 lazy val settings = Seq (
516523 resourceGenerators in Compile += Def .task {
517524 val buildScript = baseDirectory.value + " /../build/spark-build-info"
@@ -557,6 +564,7 @@ object DockerIntegrationTests {
557564 */
558565object KubernetesIntegrationTests {
559566 import BuildCommons ._
567+ import scala .sys .process .Process
560568
561569 val dockerBuild = TaskKey [Unit ](" docker-imgs" , " Build the docker images for ITs." )
562570 val runITs = TaskKey [Unit ](" run-its" , " Only run ITs, skip image build." )
@@ -634,7 +642,9 @@ object ExcludedDependencies {
634642 */
635643object OldDeps {
636644
637- lazy val project = Project (" oldDeps" , file(" dev" ), settings = oldDepsSettings)
645+ lazy val project = Project (" oldDeps" , file(" dev" ))
646+ .settings(oldDepsSettings)
647+ .disablePlugins(com.typesafe.sbt.pom.PomReaderPlugin )
638648
639649 lazy val allPreviousArtifactKeys = Def .settingDyn[Seq [Set [ModuleID ]]] {
640650 SparkBuild .mimaProjects
@@ -650,7 +660,10 @@ object OldDeps {
650660}
651661
652662object Catalyst {
653- lazy val settings = antlr4Settings ++ Seq (
663+ import com .simplytyped .Antlr4Plugin
664+ import com .simplytyped .Antlr4Plugin .autoImport ._
665+
666+ lazy val settings = Antlr4Plugin .projectSettings ++ Seq (
654667 antlr4Version in Antlr4 := SbtPomKeys .effectivePom.value.getProperties.get(" antlr4.version" ).asInstanceOf [String ],
655668 antlr4PackageName in Antlr4 := Some (" org.apache.spark.sql.catalyst.parser" ),
656669 antlr4GenListener in Antlr4 := true ,
@@ -660,6 +673,9 @@ object Catalyst {
660673}
661674
662675object SQL {
676+
677+ import sbtavro .SbtAvro .autoImport ._
678+
663679 lazy val settings = Seq (
664680 initialCommands in console :=
665681 """
@@ -681,8 +697,10 @@ object SQL {
681697 |import sqlContext.implicits._
682698 |import sqlContext._
683699 """ .stripMargin,
684- cleanupCommands in console := " sc.stop()"
700+ cleanupCommands in console := " sc.stop()" ,
701+ Test / avroGenerate := (Compile / avroGenerate).value
685702 )
703+
686704}
687705
688706object Hive {
@@ -721,27 +739,27 @@ object Hive {
721739
722740object Assembly {
723741 import sbtassembly .AssemblyUtils ._
724- import sbtassembly .Plugin ._
725- import AssemblyKeys ._
742+ import sbtassembly .AssemblyPlugin .autoImport ._
726743
727744 val hadoopVersion = taskKey[String ](" The version of hadoop that spark is compiled against." )
728745
729- lazy val settings = assemblySettings ++ Seq (
746+ lazy val settings = baseAssemblySettings ++ Seq (
730747 test in assembly := {},
731748 hadoopVersion := {
732749 sys.props.get(" hadoop.version" )
733750 .getOrElse(SbtPomKeys .effectivePom.value.getProperties.get(" hadoop.version" ).asInstanceOf [String ])
734751 },
735- jarName in assembly := {
752+ assemblyJarName in assembly := {
753+ lazy val hdpVersion = hadoopVersion.value
736754 if (moduleName.value.contains(" streaming-kafka-0-10-assembly" )
737755 || moduleName.value.contains(" streaming-kinesis-asl-assembly" )) {
738756 s " ${moduleName.value}- ${version.value}.jar "
739757 } else {
740- s " ${moduleName.value}- ${version.value}-hadoop ${hadoopVersion.value }.jar "
758+ s " ${moduleName.value}- ${version.value}-hadoop ${hdpVersion }.jar "
741759 }
742760 },
743- jarName in (Test , assembly) := s " ${moduleName.value}-test- ${version.value}.jar " ,
744- mergeStrategy in assembly := {
761+ assemblyJarName in (Test , assembly) := s " ${moduleName.value}-test- ${version.value}.jar " ,
762+ assemblyMergeStrategy in assembly := {
745763 case m if m.toLowerCase(Locale .ROOT ).endsWith(" manifest.mf" )
746764 => MergeStrategy .discard
747765 case m if m.toLowerCase(Locale .ROOT ).matches(" meta-inf.*\\ .sf$" )
@@ -756,8 +774,7 @@ object Assembly {
756774}
757775
758776object PySparkAssembly {
759- import sbtassembly .Plugin ._
760- import AssemblyKeys ._
777+ import sbtassembly .AssemblyPlugin .autoImport ._
761778 import java .util .zip .{ZipOutputStream , ZipEntry }
762779
763780 lazy val settings = Seq (
@@ -807,8 +824,13 @@ object PySparkAssembly {
807824object Unidoc {
808825
809826 import BuildCommons ._
810- import sbtunidoc .Plugin ._
811- import UnidocKeys ._
827+ import sbtunidoc .BaseUnidocPlugin
828+ import sbtunidoc .JavaUnidocPlugin
829+ import sbtunidoc .ScalaUnidocPlugin
830+ import sbtunidoc .BaseUnidocPlugin .autoImport ._
831+ import sbtunidoc .GenJavadocPlugin .autoImport ._
832+ import sbtunidoc .JavaUnidocPlugin .autoImport ._
833+ import sbtunidoc .ScalaUnidocPlugin .autoImport ._
812834
813835 private def ignoreUndocumentedPackages (packages : Seq [Seq [File ]]): Seq [Seq [File ]] = {
814836 packages
@@ -838,6 +860,7 @@ object Unidoc {
838860 .map(_.filterNot(_.getCanonicalPath.contains(" org/apache/spark/sql/catalog/v2/utils" )))
839861 .map(_.filterNot(_.getCanonicalPath.contains(" org/apache/hive" )))
840862 .map(_.filterNot(_.getCanonicalPath.contains(" org/apache/spark/sql/v2/avro" )))
863+ .map(_.filterNot(_.getCanonicalPath.contains(" SSLOptions" )))
841864 }
842865
843866 private def ignoreClasspaths (classpaths : Seq [Classpath ]): Seq [Classpath ] = {
@@ -848,7 +871,10 @@ object Unidoc {
848871
849872 val unidocSourceBase = settingKey[String ](" Base URL of source links in Scaladoc." )
850873
851- lazy val settings = scalaJavaUnidocSettings ++ Seq (
874+ lazy val settings = BaseUnidocPlugin .projectSettings ++
875+ ScalaUnidocPlugin .projectSettings ++
876+ JavaUnidocPlugin .projectSettings ++
877+ Seq (
852878 publish := {},
853879
854880 unidocProjectFilter in(ScalaUnidoc , unidoc) :=
0 commit comments