0001 /*
0002 * Licensed to the Apache Software Foundation (ASF) under one or more
0003 * contributor license agreements. See the NOTICE file distributed with
0004 * this work for additional information regarding copyright ownership.
0005 * The ASF licenses this file to You under the Apache License, Version 2.0
0006 * (the "License"); you may not use this file except in compliance with
0007 * the License. You may obtain a copy of the License at
0008 *
0009 * http://www.apache.org/licenses/LICENSE-2.0
0010 *
0011 * Unless required by applicable law or agreed to in writing, software
0012 * distributed under the License is distributed on an "AS IS" BASIS,
0013 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
0014 * See the License for the specific language governing permissions and
0015 * limitations under the License.
0016 */
0017
0018 import java.io._
0019 import java.nio.charset.StandardCharsets.UTF_8
0020 import java.nio.file.Files
0021 import java.util.Locale
0022
0023 import scala.io.Source
0024 import scala.util.Properties
0025 import scala.collection.JavaConverters._
0026 import scala.collection.mutable.Stack
0027
0028 import sbt._
0029 import sbt.Classpaths.publishTask
0030 import sbt.Keys._
0031 import sbtunidoc.Plugin.UnidocKeys.unidocGenjavadocVersion
0032 import com.etsy.sbt.checkstyle.CheckstylePlugin.autoImport._
0033 import com.simplytyped.Antlr4Plugin._
0034 import com.typesafe.sbt.pom.{PomBuild, SbtPomKeys}
0035 import com.typesafe.tools.mima.plugin.MimaKeys
0036 import org.scalastyle.sbt.ScalastylePlugin.autoImport._
0037 import org.scalastyle.sbt.Tasks
0038
0039 import spray.revolver.RevolverPlugin._
0040
0041 object BuildCommons {
0042
0043 private val buildLocation = file(".").getAbsoluteFile.getParentFile
0044
0045 val sqlProjects@Seq(catalyst, sql, hive, hiveThriftServer, tokenProviderKafka010, sqlKafka010, avro) = Seq(
0046 "catalyst", "sql", "hive", "hive-thriftserver", "token-provider-kafka-0-10", "sql-kafka-0-10", "avro"
0047 ).map(ProjectRef(buildLocation, _))
0048
0049 val streamingProjects@Seq(streaming, streamingKafka010) =
0050 Seq("streaming", "streaming-kafka-0-10").map(ProjectRef(buildLocation, _))
0051
0052 val allProjects@Seq(
0053 core, graphx, mllib, mllibLocal, repl, networkCommon, networkShuffle, launcher, unsafe, tags, sketch, kvstore, _*
0054 ) = Seq(
0055 "core", "graphx", "mllib", "mllib-local", "repl", "network-common", "network-shuffle", "launcher", "unsafe",
0056 "tags", "sketch", "kvstore"
0057 ).map(ProjectRef(buildLocation, _)) ++ sqlProjects ++ streamingProjects
0058
0059 val optionallyEnabledProjects@Seq(kubernetes, mesos, yarn,
0060 sparkGangliaLgpl, streamingKinesisAsl,
0061 dockerIntegrationTests, hadoopCloud, kubernetesIntegrationTests) =
0062 Seq("kubernetes", "mesos", "yarn",
0063 "ganglia-lgpl", "streaming-kinesis-asl",
0064 "docker-integration-tests", "hadoop-cloud", "kubernetes-integration-tests").map(ProjectRef(buildLocation, _))
0065
0066 val assemblyProjects@Seq(networkYarn, streamingKafka010Assembly, streamingKinesisAslAssembly) =
0067 Seq("network-yarn", "streaming-kafka-0-10-assembly", "streaming-kinesis-asl-assembly")
0068 .map(ProjectRef(buildLocation, _))
0069
0070 val copyJarsProjects@Seq(assembly, examples) = Seq("assembly", "examples")
0071 .map(ProjectRef(buildLocation, _))
0072
0073 val tools = ProjectRef(buildLocation, "tools")
0074 // Root project.
0075 val spark = ProjectRef(buildLocation, "spark")
0076 val sparkHome = buildLocation
0077
0078 val testTempDir = s"$sparkHome/target/tmp"
0079
0080 val javaVersion = settingKey[String]("source and target JVM version for javac and scalac")
0081 }
0082
0083 object SparkBuild extends PomBuild {
0084
0085 import BuildCommons._
0086 import scala.collection.mutable.Map
0087
0088 val projectsMap: Map[String, Seq[Setting[_]]] = Map.empty
0089
0090 override val profiles = {
0091 val profiles = Properties.envOrNone("SBT_MAVEN_PROFILES") match {
0092 case None => Seq("sbt")
0093 case Some(v) =>
0094 v.split("(\\s+|,)").filterNot(_.isEmpty).map(_.trim.replaceAll("-P", "")).toSeq
0095 }
0096
0097 // TODO: revisit for Scala 2.13 support
0098 /*
0099 Option(System.getProperty("scala.version"))
0100 .filter(_.startsWith("2.11"))
0101 .foreach { versionString =>
0102 System.setProperty("scala-2.11", "true")
0103 }
0104 if (System.getProperty("scala-2.11") == "") {
0105 // To activate scala-2.10 profile, replace empty property value to non-empty value
0106 // in the same way as Maven which handles -Dname as -Dname=true before executes build process.
0107 // see: https://github.com/apache/maven/blob/maven-3.0.4/maven-embedder/src/main/java/org/apache/maven/cli/MavenCli.java#L1082
0108 System.setProperty("scala-2.11", "true")
0109 }
0110 */
0111 profiles
0112 }
0113
0114 Properties.envOrNone("SBT_MAVEN_PROPERTIES") match {
0115 case Some(v) =>
0116 v.split("(\\s+|,)").filterNot(_.isEmpty).map(_.split("=")).foreach(x => System.setProperty(x(0), x(1)))
0117 case _ =>
0118 }
0119
0120 override val userPropertiesMap = System.getProperties.asScala.toMap
0121
0122 lazy val MavenCompile = config("m2r") extend(Compile)
0123 lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy")
0124
0125 lazy val sparkGenjavadocSettings: Seq[sbt.Def.Setting[_]] = Seq(
0126 libraryDependencies += compilerPlugin(
0127 "com.typesafe.genjavadoc" %% "genjavadoc-plugin" % unidocGenjavadocVersion.value cross CrossVersion.full),
0128 scalacOptions ++= Seq(
0129 "-P:genjavadoc:out=" + (target.value / "java"),
0130 "-P:genjavadoc:strictVisibility=true" // hide package private types
0131 )
0132 )
0133
0134 lazy val scalaStyleRules = Project("scalaStyleRules", file("scalastyle"))
0135 .settings(
0136 libraryDependencies += "org.scalastyle" %% "scalastyle" % "1.0.0"
0137 )
0138
0139 lazy val scalaStyleOnCompile = taskKey[Unit]("scalaStyleOnCompile")
0140
0141 lazy val scalaStyleOnTest = taskKey[Unit]("scalaStyleOnTest")
0142
0143 // We special case the 'println' lint rule to only be a warning on compile, because adding
0144 // printlns for debugging is a common use case and is easy to remember to remove.
0145 val scalaStyleOnCompileConfig: String = {
0146 val in = "scalastyle-config.xml"
0147 val out = "scalastyle-on-compile.generated.xml"
0148 val replacements = Map(
0149 """customId="println" level="error"""" -> """customId="println" level="warn""""
0150 )
0151 var contents = Source.fromFile(in).getLines.mkString("\n")
0152 for ((k, v) <- replacements) {
0153 require(contents.contains(k), s"Could not rewrite '$k' in original scalastyle config.")
0154 contents = contents.replace(k, v)
0155 }
0156 new PrintWriter(out) {
0157 write(contents)
0158 close()
0159 }
0160 out
0161 }
0162
0163 // Return a cached scalastyle task for a given configuration (usually Compile or Test)
0164 private def cachedScalaStyle(config: Configuration) = Def.task {
0165 val logger = streams.value.log
0166 // We need a different cache dir per Configuration, otherwise they collide
0167 val cacheDir = target.value / s"scalastyle-cache-${config.name}"
0168 val cachedFun = FileFunction.cached(cacheDir, FilesInfo.lastModified, FilesInfo.exists) {
0169 (inFiles: Set[File]) => {
0170 val args: Seq[String] = Seq.empty
0171 val scalaSourceV = Seq(file(scalaSource.in(config).value.getAbsolutePath))
0172 val configV = (baseDirectory in ThisBuild).value / scalaStyleOnCompileConfig
0173 val configUrlV = scalastyleConfigUrl.in(config).value
0174 val streamsV = streams.in(config).value
0175 val failOnErrorV = true
0176 val failOnWarningV = false
0177 val scalastyleTargetV = scalastyleTarget.in(config).value
0178 val configRefreshHoursV = scalastyleConfigRefreshHours.in(config).value
0179 val targetV = target.in(config).value
0180 val configCacheFileV = scalastyleConfigUrlCacheFile.in(config).value
0181
0182 logger.info(s"Running scalastyle on ${name.value} in ${config.name}")
0183 Tasks.doScalastyle(args, configV, configUrlV, failOnErrorV, failOnWarningV, scalaSourceV,
0184 scalastyleTargetV, streamsV, configRefreshHoursV, targetV, configCacheFileV)
0185
0186 Set.empty
0187 }
0188 }
0189
0190 cachedFun(findFiles(scalaSource.in(config).value))
0191 }
0192
0193 private def findFiles(file: File): Set[File] = if (file.isDirectory) {
0194 file.listFiles().toSet.flatMap(findFiles) + file
0195 } else {
0196 Set(file)
0197 }
0198
0199 def enableScalaStyle: Seq[sbt.Def.Setting[_]] = Seq(
0200 scalaStyleOnCompile := cachedScalaStyle(Compile).value,
0201 scalaStyleOnTest := cachedScalaStyle(Test).value,
0202 logLevel in scalaStyleOnCompile := Level.Warn,
0203 logLevel in scalaStyleOnTest := Level.Warn,
0204 (compile in Compile) := {
0205 scalaStyleOnCompile.value
0206 (compile in Compile).value
0207 },
0208 (compile in Test) := {
0209 scalaStyleOnTest.value
0210 (compile in Test).value
0211 }
0212 )
0213
0214 lazy val sharedSettings = sparkGenjavadocSettings ++
0215 (if (sys.env.contains("NOLINT_ON_COMPILE")) Nil else enableScalaStyle) ++ Seq(
0216 exportJars in Compile := true,
0217 exportJars in Test := false,
0218 javaHome := sys.env.get("JAVA_HOME")
0219 .orElse(sys.props.get("java.home").map { p => new File(p).getParentFile().getAbsolutePath() })
0220 .map(file),
0221 incOptions := incOptions.value.withNameHashing(true),
0222 publishMavenStyle := true,
0223 unidocGenjavadocVersion := "0.15",
0224
0225 // Override SBT's default resolvers:
0226 resolvers := Seq(
0227 // Google Mirror of Maven Central, placed first so that it's used instead of flaky Maven Central.
0228 // See https://storage-download.googleapis.com/maven-central/index.html for more info.
0229 "gcs-maven-central-mirror" at "https://maven-central.storage-download.googleapis.com/maven2/",
0230 DefaultMavenRepository,
0231 Resolver.mavenLocal,
0232 Resolver.file("local", file(Path.userHome.absolutePath + "/.ivy2/local"))(Resolver.ivyStylePatterns)
0233 ),
0234 externalResolvers := resolvers.value,
0235 otherResolvers := SbtPomKeys.mvnLocalRepository(dotM2 => Seq(Resolver.file("dotM2", dotM2))).value,
0236 publishLocalConfiguration in MavenCompile :=
0237 new PublishConfiguration(None, "dotM2", packagedArtifacts.value, Seq(), ivyLoggingLevel.value),
0238 publishMavenStyle in MavenCompile := true,
0239 publishLocal in MavenCompile := publishTask(publishLocalConfiguration in MavenCompile, deliverLocal).value,
0240 publishLocalBoth := Seq(publishLocal in MavenCompile, publishLocal).dependOn.value,
0241
0242 javacOptions in (Compile, doc) ++= {
0243 val versionParts = System.getProperty("java.version").split("[+.\\-]+", 3)
0244 var major = versionParts(0).toInt
0245 if (major == 1) major = versionParts(1).toInt
0246 if (major >= 8) Seq("-Xdoclint:all", "-Xdoclint:-missing") else Seq.empty
0247 },
0248
0249 javaVersion := SbtPomKeys.effectivePom.value.getProperties.get("java.version").asInstanceOf[String],
0250
0251 javacOptions in Compile ++= Seq(
0252 "-encoding", UTF_8.name(),
0253 "-source", javaVersion.value
0254 ),
0255 // This -target and Xlint:unchecked options cannot be set in the Compile configuration scope since
0256 // `javadoc` doesn't play nicely with them; see https://github.com/sbt/sbt/issues/355#issuecomment-3817629
0257 // for additional discussion and explanation.
0258 javacOptions in (Compile, compile) ++= Seq(
0259 "-target", javaVersion.value,
0260 "-Xlint:unchecked"
0261 ),
0262
0263 scalacOptions in Compile ++= Seq(
0264 s"-target:jvm-${javaVersion.value}",
0265 "-sourcepath", (baseDirectory in ThisBuild).value.getAbsolutePath // Required for relative source links in scaladoc
0266 ),
0267
0268 // Remove certain packages from Scaladoc
0269 scalacOptions in (Compile, doc) := Seq(
0270 "-groups",
0271 "-skip-packages", Seq(
0272 "org.apache.spark.api.python",
0273 "org.apache.spark.network",
0274 "org.apache.spark.deploy",
0275 "org.apache.spark.util.collection"
0276 ).mkString(":"),
0277 "-doc-title", "Spark " + version.value.replaceAll("-SNAPSHOT", "") + " ScalaDoc"
0278 ) ++ {
0279 // Do not attempt to scaladoc javadoc comments under 2.12 since it can't handle inner classes
0280 if (scalaBinaryVersion.value == "2.12") Seq("-no-java-comments") else Seq.empty
0281 },
0282
0283 // Implements -Xfatal-warnings, ignoring deprecation warnings.
0284 // Code snippet taken from https://issues.scala-lang.org/browse/SI-8410.
0285 compile in Compile := {
0286 val analysis = (compile in Compile).value
0287 val out = streams.value
0288
0289 def logProblem(l: (=> String) => Unit, f: File, p: xsbti.Problem) = {
0290 l(f.toString + ":" + p.position.line.fold("")(_ + ":") + " " + p.message)
0291 l(p.position.lineContent)
0292 l("")
0293 }
0294
0295 var failed = 0
0296 analysis.infos.allInfos.foreach { case (k, i) =>
0297 i.reportedProblems foreach { p =>
0298 val deprecation = p.message.contains("is deprecated")
0299
0300 if (!deprecation) {
0301 failed = failed + 1
0302 }
0303
0304 val printer: (=> String) => Unit = s => if (deprecation) {
0305 out.log.warn(s)
0306 } else {
0307 out.log.error("[warn] " + s)
0308 }
0309
0310 logProblem(printer, k, p)
0311
0312 }
0313 }
0314
0315 if (failed > 0) {
0316 sys.error(s"$failed fatal warnings")
0317 }
0318 analysis
0319 }
0320 )
0321
0322 def enable(settings: Seq[Setting[_]])(projectRef: ProjectRef) = {
0323 val existingSettings = projectsMap.getOrElse(projectRef.project, Seq[Setting[_]]())
0324 projectsMap += (projectRef.project -> (existingSettings ++ settings))
0325 }
0326
0327 // Note ordering of these settings matter.
0328 /* Enable shared settings on all projects */
0329 (allProjects ++ optionallyEnabledProjects ++ assemblyProjects ++ copyJarsProjects ++ Seq(spark, tools))
0330 .foreach(enable(sharedSettings ++ DependencyOverrides.settings ++
0331 ExcludedDependencies.settings ++ Checkstyle.settings))
0332
0333 /* Enable tests settings for all projects except examples, assembly and tools */
0334 (allProjects ++ optionallyEnabledProjects).foreach(enable(TestSettings.settings))
0335
0336 val mimaProjects = allProjects.filterNot { x =>
0337 Seq(
0338 spark, hive, hiveThriftServer, catalyst, repl, networkCommon, networkShuffle, networkYarn,
0339 unsafe, tags, tokenProviderKafka010, sqlKafka010, kvstore, avro
0340 ).contains(x)
0341 }
0342
0343 mimaProjects.foreach { x =>
0344 enable(MimaBuild.mimaSettings(sparkHome, x))(x)
0345 }
0346
0347 /* Generate and pick the spark build info from extra-resources */
0348 enable(Core.settings)(core)
0349
0350 /* Unsafe settings */
0351 enable(Unsafe.settings)(unsafe)
0352
0353 /*
0354 * Set up tasks to copy dependencies during packaging. This step can be disabled in the command
0355 * line, so that dev/mima can run without trying to copy these files again and potentially
0356 * causing issues.
0357 */
0358 if (!"false".equals(System.getProperty("copyDependencies"))) {
0359 copyJarsProjects.foreach(enable(CopyDependencies.settings))
0360 }
0361
0362 /* Enable Assembly for all assembly projects */
0363 assemblyProjects.foreach(enable(Assembly.settings))
0364
0365 /* Package pyspark artifacts in a separate zip file for YARN. */
0366 enable(PySparkAssembly.settings)(assembly)
0367
0368 /* Enable unidoc only for the root spark project */
0369 enable(Unidoc.settings)(spark)
0370
0371 /* Catalyst ANTLR generation settings */
0372 enable(Catalyst.settings)(catalyst)
0373
0374 /* Spark SQL Core console settings */
0375 enable(SQL.settings)(sql)
0376
0377 /* Hive console settings */
0378 enable(Hive.settings)(hive)
0379
0380 // SPARK-14738 - Remove docker tests from main Spark build
0381 // enable(DockerIntegrationTests.settings)(dockerIntegrationTests)
0382
0383 enable(KubernetesIntegrationTests.settings)(kubernetesIntegrationTests)
0384
0385 /**
0386 * Adds the ability to run the spark shell directly from SBT without building an assembly
0387 * jar.
0388 *
0389 * Usage: `build/sbt sparkShell`
0390 */
0391 val sparkShell = taskKey[Unit]("start a spark-shell.")
0392 val sparkPackage = inputKey[Unit](
0393 s"""
0394 |Download and run a spark package.
0395 |Usage `builds/sbt "sparkPackage <group:artifact:version> <MainClass> [args]
0396 """.stripMargin)
0397 val sparkSql = taskKey[Unit]("starts the spark sql CLI.")
0398
0399 enable(Seq(
0400 connectInput in run := true,
0401 fork := true,
0402 outputStrategy in run := Some (StdoutOutput),
0403
0404 javaOptions += "-Xmx2g",
0405
0406 sparkShell := {
0407 (runMain in Compile).toTask(" org.apache.spark.repl.Main -usejavacp").value
0408 },
0409
0410 sparkPackage := {
0411 import complete.DefaultParsers._
0412 val packages :: className :: otherArgs = spaceDelimited("<group:artifact:version> <MainClass> [args]").parsed.toList
0413 val scalaRun = (runner in run).value
0414 val classpath = (fullClasspath in Runtime).value
0415 val args = Seq("--packages", packages, "--class", className, (Keys.`package` in Compile in LocalProject("core"))
0416 .value.getCanonicalPath) ++ otherArgs
0417 println(args)
0418 scalaRun.run("org.apache.spark.deploy.SparkSubmit", classpath.map(_.data), args, streams.value.log)
0419 },
0420
0421 javaOptions in Compile += "-Dspark.master=local",
0422
0423 sparkSql := {
0424 (runMain in Compile).toTask(" org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver").value
0425 }
0426 ))(assembly)
0427
0428 enable(Seq(sparkShell := sparkShell in LocalProject("assembly")))(spark)
0429
0430 // TODO: move this to its upstream project.
0431 override def projectDefinitions(baseDirectory: File): Seq[Project] = {
0432 super.projectDefinitions(baseDirectory).map { x =>
0433 if (projectsMap.exists(_._1 == x.id)) x.settings(projectsMap(x.id): _*)
0434 else x.settings(Seq[Setting[_]](): _*)
0435 } ++ Seq[Project](OldDeps.project)
0436 }
0437
0438 if (!sys.env.contains("SERIAL_SBT_TESTS")) {
0439 allProjects.foreach(enable(SparkParallelTestGrouping.settings))
0440 }
0441 }
0442
0443 object SparkParallelTestGrouping {
0444 // Settings for parallelizing tests. The basic strategy here is to run the slowest suites (or
0445 // collections of suites) in their own forked JVMs, allowing us to gain parallelism within a
0446 // SBT project. Here, we take a whitelisting approach where the default behavior is to run all
0447 // tests sequentially in a single JVM, requiring us to manually opt-in to the extra parallelism.
0448 //
0449 // There are a reasons why such a whitelist approach is good:
0450 //
0451 // 1. Launching one JVM per suite adds significant overhead for short-running suites. In
0452 // addition to JVM startup time and JIT warmup, it appears that initialization of Derby
0453 // metastores can be very slow so creating a fresh warehouse per suite is inefficient.
0454 //
0455 // 2. When parallelizing within a project we need to give each forked JVM a different tmpdir
0456 // so that the metastore warehouses do not collide. Unfortunately, it seems that there are
0457 // some tests which have an overly tight dependency on the default tmpdir, so those fragile
0458 // tests need to continue re-running in the default configuration (or need to be rewritten).
0459 // Fixing that problem would be a huge amount of work for limited payoff in most cases
0460 // because most test suites are short-running.
0461 //
0462
0463 private val testsWhichShouldRunInTheirOwnDedicatedJvm = Set(
0464 "org.apache.spark.DistributedSuite",
0465 "org.apache.spark.sql.catalyst.expressions.DateExpressionsSuite",
0466 "org.apache.spark.sql.catalyst.expressions.HashExpressionsSuite",
0467 "org.apache.spark.sql.catalyst.expressions.CastSuite",
0468 "org.apache.spark.sql.catalyst.expressions.MathExpressionsSuite",
0469 "org.apache.spark.sql.hive.HiveExternalCatalogSuite",
0470 "org.apache.spark.sql.hive.StatisticsSuite",
0471 "org.apache.spark.sql.hive.execution.HiveCompatibilitySuite",
0472 "org.apache.spark.sql.hive.client.VersionsSuite",
0473 "org.apache.spark.sql.hive.client.HiveClientVersions",
0474 "org.apache.spark.sql.hive.HiveExternalCatalogVersionsSuite",
0475 "org.apache.spark.ml.classification.LogisticRegressionSuite",
0476 "org.apache.spark.ml.classification.LinearSVCSuite",
0477 "org.apache.spark.sql.SQLQueryTestSuite",
0478 "org.apache.spark.sql.hive.thriftserver.ThriftServerQueryTestSuite",
0479 "org.apache.spark.sql.hive.thriftserver.SparkSQLEnvSuite",
0480 "org.apache.spark.sql.hive.thriftserver.ui.ThriftServerPageSuite",
0481 "org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2ListenerSuite",
0482 "org.apache.spark.sql.hive.thriftserver.ThriftServerWithSparkContextSuite",
0483 "org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite"
0484 )
0485
0486 private val DEFAULT_TEST_GROUP = "default_test_group"
0487
0488 private def testNameToTestGroup(name: String): String = name match {
0489 case _ if testsWhichShouldRunInTheirOwnDedicatedJvm.contains(name) => name
0490 case _ => DEFAULT_TEST_GROUP
0491 }
0492
0493 lazy val settings = Seq(
0494 testGrouping in Test := {
0495 val tests: Seq[TestDefinition] = (definedTests in Test).value
0496 val defaultForkOptions = ForkOptions(
0497 bootJars = Nil,
0498 javaHome = javaHome.value,
0499 connectInput = connectInput.value,
0500 outputStrategy = outputStrategy.value,
0501 runJVMOptions = (javaOptions in Test).value,
0502 workingDirectory = Some(baseDirectory.value),
0503 envVars = (envVars in Test).value
0504 )
0505 tests.groupBy(test => testNameToTestGroup(test.name)).map { case (groupName, groupTests) =>
0506 val forkOptions = {
0507 if (groupName == DEFAULT_TEST_GROUP) {
0508 defaultForkOptions
0509 } else {
0510 defaultForkOptions.copy(runJVMOptions = defaultForkOptions.runJVMOptions ++
0511 Seq(s"-Djava.io.tmpdir=${baseDirectory.value}/target/tmp/$groupName"))
0512 }
0513 }
0514 new Tests.Group(
0515 name = groupName,
0516 tests = groupTests,
0517 runPolicy = Tests.SubProcess(forkOptions))
0518 }
0519 }.toSeq
0520 )
0521 }
0522
0523 object Core {
0524 lazy val settings = Seq(
0525 resourceGenerators in Compile += Def.task {
0526 val buildScript = baseDirectory.value + "/../build/spark-build-info"
0527 val targetDir = baseDirectory.value + "/target/extra-resources/"
0528 val command = Seq("bash", buildScript, targetDir, version.value)
0529 Process(command).!!
0530 val propsFile = baseDirectory.value / "target" / "extra-resources" / "spark-version-info.properties"
0531 Seq(propsFile)
0532 }.taskValue
0533 )
0534 }
0535
0536 object Unsafe {
0537 lazy val settings = Seq(
0538 // This option is needed to suppress warnings from sun.misc.Unsafe usage
0539 javacOptions in Compile += "-XDignore.symbol.file"
0540 )
0541 }
0542
0543
0544 object DockerIntegrationTests {
0545 // This serves to override the override specified in DependencyOverrides:
0546 lazy val settings = Seq(
0547 dependencyOverrides += "com.google.guava" % "guava" % "18.0",
0548 resolvers += "DB2" at "https://app.camunda.com/nexus/content/repositories/public/",
0549 libraryDependencies += "com.oracle" % "ojdbc6" % "11.2.0.1.0" from "https://app.camunda.com/nexus/content/repositories/public/com/oracle/ojdbc6/11.2.0.1.0/ojdbc6-11.2.0.1.0.jar" // scalastyle:ignore
0550 )
0551 }
0552
0553 /**
0554 * These settings run a hardcoded configuration of the Kubernetes integration tests using
0555 * minikube. Docker images will have the "dev" tag, and will be overwritten every time the
0556 * integration tests are run. The integration tests are actually bound to the "test" phase,
0557 * so running "test" on this module will run the integration tests.
0558 *
0559 * There are two ways to run the tests:
0560 * - the "tests" task builds docker images and runs the test, so it's a little slow.
0561 * - the "run-its" task just runs the tests on a pre-built set of images.
0562 *
0563 * Note that this does not use the shell scripts that the maven build uses, which are more
0564 * configurable. This is meant as a quick way for developers to run these tests against their
0565 * local changes.
0566 */
0567 object KubernetesIntegrationTests {
0568 import BuildCommons._
0569
0570 val dockerBuild = TaskKey[Unit]("docker-imgs", "Build the docker images for ITs.")
0571 val runITs = TaskKey[Unit]("run-its", "Only run ITs, skip image build.")
0572 val imageTag = settingKey[String]("Tag to use for images built during the test.")
0573 val namespace = settingKey[String]("Namespace where to run pods.")
0574
0575 // Hack: this variable is used to control whether to build docker images. It's updated by
0576 // the tasks below in a non-obvious way, so that you get the functionality described in
0577 // the scaladoc above.
0578 private var shouldBuildImage = true
0579
0580 lazy val settings = Seq(
0581 imageTag := "dev",
0582 namespace := "default",
0583 dockerBuild := {
0584 if (shouldBuildImage) {
0585 val dockerTool = s"$sparkHome/bin/docker-image-tool.sh"
0586 val bindingsDir = s"$sparkHome/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings"
0587 val cmd = Seq(dockerTool, "-m",
0588 "-t", imageTag.value,
0589 "-p", s"$bindingsDir/python/Dockerfile",
0590 "-R", s"$bindingsDir/R/Dockerfile",
0591 "build"
0592 )
0593 val ec = Process(cmd).!
0594 if (ec != 0) {
0595 throw new IllegalStateException(s"Process '${cmd.mkString(" ")}' exited with $ec.")
0596 }
0597 }
0598 shouldBuildImage = true
0599 },
0600 runITs := Def.taskDyn {
0601 shouldBuildImage = false
0602 Def.task {
0603 (test in Test).value
0604 }
0605 }.value,
0606 test in Test := (test in Test).dependsOn(dockerBuild).value,
0607 javaOptions in Test ++= Seq(
0608 "-Dspark.kubernetes.test.deployMode=minikube",
0609 s"-Dspark.kubernetes.test.imageTag=${imageTag.value}",
0610 s"-Dspark.kubernetes.test.namespace=${namespace.value}",
0611 s"-Dspark.kubernetes.test.unpackSparkDir=$sparkHome"
0612 ),
0613 // Force packaging before building images, so that the latest code is tested.
0614 dockerBuild := dockerBuild.dependsOn(packageBin in Compile in assembly)
0615 .dependsOn(packageBin in Compile in examples).value
0616 )
0617 }
0618
0619 /**
0620 * Overrides to work around sbt's dependency resolution being different from Maven's.
0621 */
0622 object DependencyOverrides {
0623 lazy val settings = Seq(
0624 dependencyOverrides += "com.google.guava" % "guava" % "14.0.1",
0625 dependencyOverrides += "commons-io" % "commons-io" % "2.4",
0626 dependencyOverrides += "xerces" % "xercesImpl" % "2.12.0",
0627 dependencyOverrides += "jline" % "jline" % "2.14.6",
0628 dependencyOverrides += "org.apache.avro" % "avro" % "1.8.2")
0629 }
0630
0631 /**
0632 * This excludes library dependencies in sbt, which are specified in maven but are
0633 * not needed by sbt build.
0634 */
0635 object ExcludedDependencies {
0636 lazy val settings = Seq(
0637 libraryDependencies ~= { libs => libs.filterNot(_.name == "groovy-all") }
0638 )
0639 }
0640
0641 /**
0642 * Project to pull previous artifacts of Spark for generating Mima excludes.
0643 */
0644 object OldDeps {
0645
0646 lazy val project = Project("oldDeps", file("dev"), settings = oldDepsSettings)
0647
0648 lazy val allPreviousArtifactKeys = Def.settingDyn[Seq[Set[ModuleID]]] {
0649 SparkBuild.mimaProjects
0650 .map { project => MimaKeys.mimaPreviousArtifacts in project }
0651 .map(k => Def.setting(k.value))
0652 .join
0653 }
0654
0655 def oldDepsSettings() = Defaults.coreDefaultSettings ++ Seq(
0656 name := "old-deps",
0657 libraryDependencies := allPreviousArtifactKeys.value.flatten
0658 )
0659 }
0660
0661 object Catalyst {
0662 lazy val settings = antlr4Settings ++ Seq(
0663 antlr4Version in Antlr4 := SbtPomKeys.effectivePom.value.getProperties.get("antlr4.version").asInstanceOf[String],
0664 antlr4PackageName in Antlr4 := Some("org.apache.spark.sql.catalyst.parser"),
0665 antlr4GenListener in Antlr4 := true,
0666 antlr4GenVisitor in Antlr4 := true,
0667 antlr4TreatWarningsAsErrors in Antlr4 := true
0668 )
0669 }
0670
0671 object SQL {
0672 lazy val settings = Seq(
0673 initialCommands in console :=
0674 """
0675 |import org.apache.spark.SparkContext
0676 |import org.apache.spark.sql.SQLContext
0677 |import org.apache.spark.sql.catalyst.analysis._
0678 |import org.apache.spark.sql.catalyst.dsl._
0679 |import org.apache.spark.sql.catalyst.errors._
0680 |import org.apache.spark.sql.catalyst.expressions._
0681 |import org.apache.spark.sql.catalyst.plans.logical._
0682 |import org.apache.spark.sql.catalyst.rules._
0683 |import org.apache.spark.sql.catalyst.util._
0684 |import org.apache.spark.sql.execution
0685 |import org.apache.spark.sql.functions._
0686 |import org.apache.spark.sql.types._
0687 |
0688 |val sc = new SparkContext("local[*]", "dev-shell")
0689 |val sqlContext = new SQLContext(sc)
0690 |import sqlContext.implicits._
0691 |import sqlContext._
0692 """.stripMargin,
0693 cleanupCommands in console := "sc.stop()"
0694 )
0695 }
0696
0697 object Hive {
0698
0699 lazy val settings = Seq(
0700 // Specially disable assertions since some Hive tests fail them
0701 javaOptions in Test := (javaOptions in Test).value.filterNot(_ == "-ea"),
0702 // Supporting all SerDes requires us to depend on deprecated APIs, so we turn off the warnings
0703 // only for this subproject.
0704 scalacOptions := (scalacOptions map { currentOpts: Seq[String] =>
0705 currentOpts.filterNot(_ == "-deprecation")
0706 }).value,
0707 initialCommands in console :=
0708 """
0709 |import org.apache.spark.SparkContext
0710 |import org.apache.spark.sql.catalyst.analysis._
0711 |import org.apache.spark.sql.catalyst.dsl._
0712 |import org.apache.spark.sql.catalyst.errors._
0713 |import org.apache.spark.sql.catalyst.expressions._
0714 |import org.apache.spark.sql.catalyst.plans.logical._
0715 |import org.apache.spark.sql.catalyst.rules._
0716 |import org.apache.spark.sql.catalyst.util._
0717 |import org.apache.spark.sql.execution
0718 |import org.apache.spark.sql.functions._
0719 |import org.apache.spark.sql.hive._
0720 |import org.apache.spark.sql.hive.test.TestHive._
0721 |import org.apache.spark.sql.hive.test.TestHive.implicits._
0722 |import org.apache.spark.sql.types._""".stripMargin,
0723 cleanupCommands in console := "sparkContext.stop()",
0724 // Some of our log4j jars make it impossible to submit jobs from this JVM to Hive Map/Reduce
0725 // in order to generate golden files. This is only required for developers who are adding new
0726 // new query tests.
0727 fullClasspath in Test := (fullClasspath in Test).value.filterNot { f => f.toString.contains("jcl-over") }
0728 )
0729 }
0730
0731 object Assembly {
0732 import sbtassembly.AssemblyUtils._
0733 import sbtassembly.Plugin._
0734 import AssemblyKeys._
0735
0736 val hadoopVersion = taskKey[String]("The version of hadoop that spark is compiled against.")
0737
0738 lazy val settings = assemblySettings ++ Seq(
0739 test in assembly := {},
0740 hadoopVersion := {
0741 sys.props.get("hadoop.version")
0742 .getOrElse(SbtPomKeys.effectivePom.value.getProperties.get("hadoop.version").asInstanceOf[String])
0743 },
0744 jarName in assembly := {
0745 if (moduleName.value.contains("streaming-kafka-0-10-assembly")
0746 || moduleName.value.contains("streaming-kinesis-asl-assembly")) {
0747 s"${moduleName.value}-${version.value}.jar"
0748 } else {
0749 s"${moduleName.value}-${version.value}-hadoop${hadoopVersion.value}.jar"
0750 }
0751 },
0752 jarName in (Test, assembly) := s"${moduleName.value}-test-${version.value}.jar",
0753 mergeStrategy in assembly := {
0754 case m if m.toLowerCase(Locale.ROOT).endsWith("manifest.mf")
0755 => MergeStrategy.discard
0756 case m if m.toLowerCase(Locale.ROOT).matches("meta-inf.*\\.sf$")
0757 => MergeStrategy.discard
0758 case "log4j.properties" => MergeStrategy.discard
0759 case m if m.toLowerCase(Locale.ROOT).startsWith("meta-inf/services/")
0760 => MergeStrategy.filterDistinctLines
0761 case "reference.conf" => MergeStrategy.concat
0762 case _ => MergeStrategy.first
0763 }
0764 )
0765 }
0766
0767 object PySparkAssembly {
0768 import sbtassembly.Plugin._
0769 import AssemblyKeys._
0770 import java.util.zip.{ZipOutputStream, ZipEntry}
0771
0772 lazy val settings = Seq(
0773 // Use a resource generator to copy all .py files from python/pyspark into a managed directory
0774 // to be included in the assembly. We can't just add "python/" to the assembly's resource dir
0775 // list since that will copy unneeded / unwanted files.
0776 resourceGenerators in Compile += Def.macroValueI(resourceManaged in Compile map { outDir: File =>
0777 val src = new File(BuildCommons.sparkHome, "python/pyspark")
0778 val zipFile = new File(BuildCommons.sparkHome , "python/lib/pyspark.zip")
0779 zipFile.delete()
0780 zipRecursive(src, zipFile)
0781 Seq.empty[File]
0782 }).value
0783 )
0784
0785 private def zipRecursive(source: File, destZipFile: File) = {
0786 val destOutput = new ZipOutputStream(new FileOutputStream(destZipFile))
0787 addFilesToZipStream("", source, destOutput)
0788 destOutput.flush()
0789 destOutput.close()
0790 }
0791
0792 private def addFilesToZipStream(parent: String, source: File, output: ZipOutputStream): Unit = {
0793 if (source.isDirectory()) {
0794 output.putNextEntry(new ZipEntry(parent + source.getName()))
0795 for (file <- source.listFiles()) {
0796 addFilesToZipStream(parent + source.getName() + File.separator, file, output)
0797 }
0798 } else {
0799 val in = new FileInputStream(source)
0800 output.putNextEntry(new ZipEntry(parent + source.getName()))
0801 val buf = new Array[Byte](8192)
0802 var n = 0
0803 while (n != -1) {
0804 n = in.read(buf)
0805 if (n != -1) {
0806 output.write(buf, 0, n)
0807 }
0808 }
0809 output.closeEntry()
0810 in.close()
0811 }
0812 }
0813
0814 }
0815
0816 object Unidoc {
0817
0818 import BuildCommons._
0819 import sbtunidoc.Plugin._
0820 import UnidocKeys._
0821
0822 private def ignoreUndocumentedPackages(packages: Seq[Seq[File]]): Seq[Seq[File]] = {
0823 packages
0824 .map(_.filterNot(_.getName.contains("$")))
0825 .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/deploy")))
0826 .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/examples")))
0827 .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/internal")))
0828 .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/memory")))
0829 .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/network")))
0830 .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/rpc")))
0831 .map(_.filterNot(f =>
0832 f.getCanonicalPath.contains("org/apache/spark/shuffle") &&
0833 !f.getCanonicalPath.contains("org/apache/spark/shuffle/api")))
0834 .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/executor")))
0835 .map(_.filterNot(f =>
0836 f.getCanonicalPath.contains("org/apache/spark/unsafe") &&
0837 !f.getCanonicalPath.contains("org/apache/spark/unsafe/types/CalendarInterval")))
0838 .map(_.filterNot(_.getCanonicalPath.contains("python")))
0839 .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/util/collection")))
0840 .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/util/kvstore")))
0841 .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/catalyst")))
0842 .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/execution")))
0843 .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/internal")))
0844 .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/hive/test")))
0845 .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/catalog/v2/utils")))
0846 .map(_.filterNot(_.getCanonicalPath.contains("org/apache/hive")))
0847 }
0848
0849 private def ignoreClasspaths(classpaths: Seq[Classpath]): Seq[Classpath] = {
0850 classpaths
0851 .map(_.filterNot(_.data.getCanonicalPath.matches(""".*kafka-clients-0\.10.*""")))
0852 .map(_.filterNot(_.data.getCanonicalPath.matches(""".*kafka_2\..*-0\.10.*""")))
0853 }
0854
0855 val unidocSourceBase = settingKey[String]("Base URL of source links in Scaladoc.")
0856
0857 lazy val settings = scalaJavaUnidocSettings ++ Seq (
0858 publish := {},
0859
0860 unidocProjectFilter in(ScalaUnidoc, unidoc) :=
0861 inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, kubernetes,
0862 yarn, tags, streamingKafka010, sqlKafka010, avro),
0863 unidocProjectFilter in(JavaUnidoc, unidoc) :=
0864 inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, kubernetes,
0865 yarn, tags, streamingKafka010, sqlKafka010, avro),
0866
0867 unidocAllClasspaths in (ScalaUnidoc, unidoc) := {
0868 ignoreClasspaths((unidocAllClasspaths in (ScalaUnidoc, unidoc)).value)
0869 },
0870
0871 unidocAllClasspaths in (JavaUnidoc, unidoc) := {
0872 ignoreClasspaths((unidocAllClasspaths in (JavaUnidoc, unidoc)).value)
0873 },
0874
0875 // Skip actual catalyst, but include the subproject.
0876 // Catalyst is not public API and contains quasiquotes which break scaladoc.
0877 unidocAllSources in (ScalaUnidoc, unidoc) := {
0878 ignoreUndocumentedPackages((unidocAllSources in (ScalaUnidoc, unidoc)).value)
0879 },
0880
0881 // Skip class names containing $ and some internal packages in Javadocs
0882 unidocAllSources in (JavaUnidoc, unidoc) := {
0883 ignoreUndocumentedPackages((unidocAllSources in (JavaUnidoc, unidoc)).value)
0884 .map(_.filterNot(_.getCanonicalPath.contains("org/apache/hadoop")))
0885 },
0886
0887 javacOptions in (JavaUnidoc, unidoc) := Seq(
0888 "-windowtitle", "Spark " + version.value.replaceAll("-SNAPSHOT", "") + " JavaDoc",
0889 "-public",
0890 "-noqualifier", "java.lang",
0891 "-tag", """example:a:Example\:""",
0892 "-tag", """note:a:Note\:""",
0893 "-tag", "group:X",
0894 "-tag", "tparam:X",
0895 "-tag", "constructor:X",
0896 "-tag", "todo:X",
0897 "-tag", "groupname:X"
0898 ),
0899
0900 // Use GitHub repository for Scaladoc source links
0901 unidocSourceBase := s"https://github.com/apache/spark/tree/v${version.value}",
0902
0903 scalacOptions in (ScalaUnidoc, unidoc) ++= Seq(
0904 "-groups", // Group similar methods together based on the @group annotation.
0905 "-skip-packages", "org.apache.hadoop",
0906 "-sourcepath", (baseDirectory in ThisBuild).value.getAbsolutePath
0907 ) ++ (
0908 // Add links to sources when generating Scaladoc for a non-snapshot release
0909 if (!isSnapshot.value) {
0910 Opts.doc.sourceUrl(unidocSourceBase.value + "€{FILE_PATH}.scala")
0911 } else {
0912 Seq()
0913 }
0914 )
0915 )
0916 }
0917
0918 object Checkstyle {
0919 lazy val settings = Seq(
0920 checkstyleSeverityLevel := Some(CheckstyleSeverityLevel.Error),
0921 javaSource in (Compile, checkstyle) := baseDirectory.value / "src/main/java",
0922 javaSource in (Test, checkstyle) := baseDirectory.value / "src/test/java",
0923 checkstyleConfigLocation := CheckstyleConfigLocation.File("dev/checkstyle.xml"),
0924 checkstyleOutputFile := baseDirectory.value / "target/checkstyle-output.xml",
0925 checkstyleOutputFile in Test := baseDirectory.value / "target/checkstyle-output.xml"
0926 )
0927 }
0928
0929 object CopyDependencies {
0930
0931 val copyDeps = TaskKey[Unit]("copyDeps", "Copies needed dependencies to the build directory.")
0932 val destPath = (crossTarget in Compile) { _ / "jars"}
0933
0934 lazy val settings = Seq(
0935 copyDeps := {
0936 val dest = destPath.value
0937 if (!dest.isDirectory() && !dest.mkdirs()) {
0938 throw new IOException("Failed to create jars directory.")
0939 }
0940
0941 (dependencyClasspath in Compile).value.map(_.data)
0942 .filter { jar => jar.isFile() }
0943 .foreach { jar =>
0944 val destJar = new File(dest, jar.getName())
0945 if (destJar.isFile()) {
0946 destJar.delete()
0947 }
0948 Files.copy(jar.toPath(), destJar.toPath())
0949 }
0950 },
0951 crossTarget in (Compile, packageBin) := destPath.value,
0952 packageBin in Compile := (packageBin in Compile).dependsOn(copyDeps).value
0953 )
0954
0955 }
0956
0957 object TestSettings {
0958 import BuildCommons._
0959
0960 // TODO revisit for Scala 2.13 support
0961 private val scalaBinaryVersion = "2.12"
0962 /*
0963 if (System.getProperty("scala-2.11") == "true") {
0964 "2.11"
0965 } else {
0966 "2.12"
0967 }
0968 */
0969
0970 private val defaultExcludedTags = Seq("org.apache.spark.tags.ChromeUITest")
0971
0972 lazy val settings = Seq (
0973 // Fork new JVMs for tests and set Java options for those
0974 fork := true,
0975 // Setting SPARK_DIST_CLASSPATH is a simple way to make sure any child processes
0976 // launched by the tests have access to the correct test-time classpath.
0977 envVars in Test ++= Map(
0978 "SPARK_DIST_CLASSPATH" ->
0979 (fullClasspath in Test).value.files.map(_.getAbsolutePath)
0980 .mkString(File.pathSeparator).stripSuffix(File.pathSeparator),
0981 "SPARK_PREPEND_CLASSES" -> "1",
0982 "SPARK_SCALA_VERSION" -> scalaBinaryVersion,
0983 "SPARK_TESTING" -> "1",
0984 "JAVA_HOME" -> sys.env.get("JAVA_HOME").getOrElse(sys.props("java.home"))),
0985 javaOptions in Test += s"-Djava.io.tmpdir=$testTempDir",
0986 javaOptions in Test += "-Dspark.test.home=" + sparkHome,
0987 javaOptions in Test += "-Dspark.testing=1",
0988 javaOptions in Test += "-Dspark.port.maxRetries=100",
0989 javaOptions in Test += "-Dspark.master.rest.enabled=false",
0990 javaOptions in Test += "-Dspark.memory.debugFill=true",
0991 javaOptions in Test += "-Dspark.ui.enabled=false",
0992 javaOptions in Test += "-Dspark.ui.showConsoleProgress=false",
0993 javaOptions in Test += "-Dspark.unsafe.exceptionOnMemoryLeak=true",
0994 javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=false",
0995 javaOptions in Test += "-Dderby.system.durability=test",
0996 javaOptions in Test += "-Dio.netty.tryReflectionSetAccessible=true",
0997 javaOptions in Test ++= System.getProperties.asScala.filter(_._1.startsWith("spark"))
0998 .map { case (k,v) => s"-D$k=$v" }.toSeq,
0999 javaOptions in Test += "-ea",
1000 // SPARK-29282 This is for consistency between JDK8 and JDK11.
1001 javaOptions in Test ++= "-Xmx4g -Xss4m -XX:+UseParallelGC -XX:-UseDynamicNumberOfGCThreads"
1002 .split(" ").toSeq,
1003 javaOptions += "-Xmx3g",
1004 // Exclude tags defined in a system property
1005 testOptions in Test += Tests.Argument(TestFrameworks.ScalaTest,
1006 sys.props.get("test.exclude.tags").map { tags =>
1007 tags.split(",").flatMap { tag => Seq("-l", tag) }.toSeq
1008 }.getOrElse(Nil): _*),
1009 testOptions in Test += Tests.Argument(TestFrameworks.ScalaTest,
1010 sys.props.get("test.default.exclude.tags").map(tags => tags.split(",").toSeq)
1011 .map(tags => tags.filter(!_.trim.isEmpty)).getOrElse(defaultExcludedTags)
1012 .flatMap(tag => Seq("-l", tag)): _*),
1013 testOptions in Test += Tests.Argument(TestFrameworks.JUnit,
1014 sys.props.get("test.exclude.tags").map { tags =>
1015 Seq("--exclude-categories=" + tags)
1016 }.getOrElse(Nil): _*),
1017 // Show full stack trace and duration in test cases.
1018 testOptions in Test += Tests.Argument("-oDF"),
1019 testOptions in Test += Tests.Argument(TestFrameworks.JUnit, "-v", "-a"),
1020 // Enable Junit testing.
1021 libraryDependencies += "com.novocode" % "junit-interface" % "0.11" % "test",
1022 // `parallelExecutionInTest` controls whether test suites belonging to the same SBT project
1023 // can run in parallel with one another. It does NOT control whether tests execute in parallel
1024 // within the same JVM (which is controlled by `testForkedParallel`) or whether test cases
1025 // within the same suite can run in parallel (which is a ScalaTest runner option which is passed
1026 // to the underlying runner but is not a SBT-level configuration). This needs to be `true` in
1027 // order for the extra parallelism enabled by `SparkParallelTestGrouping` to take effect.
1028 // The `SERIAL_SBT_TESTS` check is here so the extra parallelism can be feature-flagged.
1029 parallelExecution in Test := { if (sys.env.contains("SERIAL_SBT_TESTS")) false else true },
1030 // Make sure the test temp directory exists.
1031 resourceGenerators in Test += Def.macroValueI(resourceManaged in Test map { outDir: File =>
1032 var dir = new File(testTempDir)
1033 if (!dir.isDirectory()) {
1034 // Because File.mkdirs() can fail if multiple callers are trying to create the same
1035 // parent directory, this code tries to create parents one at a time, and avoids
1036 // failures when the directories have been created by somebody else.
1037 val stack = new Stack[File]()
1038 while (!dir.isDirectory()) {
1039 stack.push(dir)
1040 dir = dir.getParentFile()
1041 }
1042
1043 while (stack.nonEmpty) {
1044 val d = stack.pop()
1045 require(d.mkdir() || d.isDirectory(), s"Failed to create directory $d")
1046 }
1047 }
1048 Seq.empty[File]
1049 }).value,
1050 concurrentRestrictions in Global := {
1051 // The number of concurrent test groups is empirically chosen based on experience
1052 // with Jenkins flakiness.
1053 if (sys.env.contains("SERIAL_SBT_TESTS")) (concurrentRestrictions in Global).value
1054 else Seq(Tags.limit(Tags.ForkedTestGroup, 4))
1055 }
1056 )
1057
1058 }