aboutsummaryrefslogblamecommitdiff
path: root/project/SparkBuild.scala
blob: fb229b979d41b0ea924af7039ed2b104e562f15e (plain) (tree)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16















                                                                           
 
                
                          
 
                            
                                        
 
            
                                 
                 
                                                          
                                                  
                                              
 

                                      
                     
 
                                                                     
 




                                                                   
                                                                 
          
                                                                             







                                                                                                    
 
                                                                       
                                                  
                                                                       
                                                                   
 

                                                                                                                                
                                        
 

                                                                                        


                                                
                               

                                            


                                                                                       
 
 
                                    
 

                                     
 
                                                           
 


                                                                 
                                                          
                             
                                                               
                                                                                              
                                            
     
                                                       

                                                                                                 


                                                        




                                                                                             

                                                                        
     
                            
            
   
 

                                                                     
                                      

                                         
                                 

                                                                                              
                                
                                                                                     
     
 

                                                                                        

                                                                                                                                    
                                              
     
            

   


                                                                                                             
             

   
                                                                     
 


                                                                                            




                                                                                                          
                                                            

                                  


                                                                                                    
                                                         
                              
                                            
 


                                        

                                                                                                          

                                         





                                                                                                          


                                                                                  



                                                                                

      




                                    





                                                                                                   

                                      
 
                                     
                                               


                                                                                                                          



                                                                            
                             















                                                                                 







                                                                    







                                            
   
 


                                                                                       

   

                                              
                                                                                                         
                                                                     
                                     
 

                                                                                   
 
                                                 

                                                                                                
                              
                 


                             

                                                   
 


                                 


                                                             

                                                     
 
                                                                  

                                            

                                                     
 


                                          


                                       

                             
 

                                            
                                                
 
                                                                 







                                                                                           




                                                                                   
                                                           





                                                 
                                                          


                                                                                 

      









                                                                                                                                             



                                                                                                    




                                                            




                                                                             
                                      
   
 
 
 






                                                                            



                                                  






                                                                           
   







                                                                                     


                                                                                 
                             





                                                                             
                                                                            




                                                                                






                                                                            
                                                              
                       
                             
                                                                


   




































                                                                                                  





                                                                                 











                                                                

                          

                                 

                                               





                                                             

                                                    
                                                







                                                           
   

 
             
 
                          
                                          

                                                                             






                                                                                                  
                                             





                                                             

                                                    
                                                
                                           
                                                         
                                                                   
                                                            



                                                                                                  
                                                                                                             
   
 
 
                 
                                    

                             
 

                                                                                              

                                                                                                              
                                              
                           


                                                                                                          
      
                                                                                        
                                                                                                                                                         





                                                                                            


                                                                                                
                                  



                                                                                       
                                                                                                   

                                                                                      













                                                                               
   
 
 


                             
                                                  

                          



                                                                                                  
                                                                  


                                                                               
                 


     























                                                                                                  
                         



                

 
               
 







                                                         
 


                                                                                      
                                                                               

                                                                                 


                                                                                 
                                                                               
                                                              



                                                                                        

   

                                                                                    



                                                      
                                                                                                             
                                                 
                                                                                                             



                                                                                
                                                                                   
      


                                                                           
                                                                                  







                                                                                        
                                                                                       




                                                                                              


                                                                                      

                                                                                                  
        
                                                                                                    
                                 

      
                                                      











                                                                                  
   
 
 



























                                                                                                





                                                                              
                                                                                          


   





                                                             

                                                                                    
                            
                               
                                                                                                  

                                     
                                                                                 
                                                            
                                                            
                                               
                                                         
                                                               
                                                      
                                                                  
                                                                       
                                                                           
                                                            
                                                                                         
                                              
                                 
                                                                                                       

                            








                                                                   

                                                        
                                                                            
                            
                                                                                

                                                                                          






                                                                                




                                                                 





                                                                                      


     
 
/*
 * Licensed to the Apache Software Foundation (ASF) under one or more
 * contributor license agreements.  See the NOTICE file distributed with
 * this work for additional information regarding copyright ownership.
 * The ASF licenses this file to You under the Apache License, Version 2.0
 * (the "License"); you may not use this file except in compliance with
 * the License.  You may obtain a copy of the License at
 *
 *    http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */

import java.io._
import java.nio.file.Files

import scala.util.Properties
import scala.collection.JavaConverters._

import sbt._
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._

object BuildCommons {

  private val buildLocation = file(".").getAbsoluteFile.getParentFile

  val sqlProjects@Seq(catalyst, sql, hive, hiveThriftServer) = Seq(
    "catalyst", "sql", "hive", "hive-thriftserver"
  ).map(ProjectRef(buildLocation, _))

  val streamingProjects@Seq(
    streaming, streamingFlumeSink, streamingFlume, streamingKafka
  ) = Seq(
    "streaming", "streaming-flume-sink", "streaming-flume", "streaming-kafka"
  ).map(ProjectRef(buildLocation, _))

  val allProjects@Seq(
    core, graphx, mllib, repl, networkCommon, networkShuffle, launcher, unsafe, testTags, sketch, _*
  ) = Seq(
    "core", "graphx", "mllib", "repl", "network-common", "network-shuffle", "launcher", "unsafe",
    "test-tags", "sketch"
  ).map(ProjectRef(buildLocation, _)) ++ sqlProjects ++ streamingProjects

  val optionallyEnabledProjects@Seq(yarn, java8Tests, sparkGangliaLgpl,
    streamingKinesisAsl, dockerIntegrationTests) =
    Seq("yarn", "java8-tests", "ganglia-lgpl", "streaming-kinesis-asl",
      "docker-integration-tests").map(ProjectRef(buildLocation, _))

  val assemblyProjects@Seq(assembly, networkYarn, streamingFlumeAssembly, streamingKafkaAssembly, streamingKinesisAslAssembly) =
    Seq("assembly", "network-yarn", "streaming-flume-assembly", "streaming-kafka-assembly", "streaming-kinesis-asl-assembly")
      .map(ProjectRef(buildLocation, _))

  val copyJarsProjects@Seq(examples) = Seq("examples").map(ProjectRef(buildLocation, _))

  val tools = ProjectRef(buildLocation, "tools")
  // Root project.
  val spark = ProjectRef(buildLocation, "spark")
  val sparkHome = buildLocation

  val testTempDir = s"$sparkHome/target/tmp"

  val javacJVMVersion = settingKey[String]("source and target JVM version for javac")
  val scalacJVMVersion = settingKey[String]("source and target JVM version for scalac")
}

object SparkBuild extends PomBuild {

  import BuildCommons._
  import scala.collection.mutable.Map

  val projectsMap: Map[String, Seq[Setting[_]]] = Map.empty

  // Provides compatibility for older versions of the Spark build
  def backwardCompatibility = {
    import scala.collection.mutable
    var profiles: mutable.Seq[String] = mutable.Seq("sbt")
    // scalastyle:off println
    if (Properties.envOrNone("SPARK_GANGLIA_LGPL").isDefined) {
      println("NOTE: SPARK_GANGLIA_LGPL is deprecated, please use -Pspark-ganglia-lgpl flag.")
      profiles ++= Seq("spark-ganglia-lgpl")
    }
    if (Properties.envOrNone("SPARK_HIVE").isDefined) {
      println("NOTE: SPARK_HIVE is deprecated, please use -Phive and -Phive-thriftserver flags.")
      profiles ++= Seq("hive", "hive-thriftserver")
    }
    Properties.envOrNone("SPARK_HADOOP_VERSION") match {
      case Some(v) =>
        println("NOTE: SPARK_HADOOP_VERSION is deprecated, please use -Dhadoop.version=" + v)
        System.setProperty("hadoop.version", v)
      case None =>
    }
    if (Properties.envOrNone("SPARK_YARN").isDefined) {
      println("NOTE: SPARK_YARN is deprecated, please use -Pyarn flag.")
      profiles ++= Seq("yarn")
    }
    // scalastyle:on println
    profiles
  }

  override val profiles = {
    val profiles = Properties.envOrNone("SBT_MAVEN_PROFILES") match {
    case None => backwardCompatibility
    case Some(v) =>
      if (backwardCompatibility.nonEmpty)
        // scalastyle:off println
        println("Note: We ignore environment variables, when use of profile is detected in " +
          "conjunction with environment variable.")
        // scalastyle:on println
      v.split("(\\s+|,)").filterNot(_.isEmpty).map(_.trim.replaceAll("-P", "")).toSeq
    }

    if (System.getProperty("scala-2.10") == "") {
      // To activate scala-2.10 profile, replace empty property value to non-empty value
      // in the same way as Maven which handles -Dname as -Dname=true before executes build process.
      // see: https://github.com/apache/maven/blob/maven-3.0.4/maven-embedder/src/main/java/org/apache/maven/cli/MavenCli.java#L1082
      System.setProperty("scala-2.10", "true")
    }
    profiles
  }

  Properties.envOrNone("SBT_MAVEN_PROPERTIES") match {
    case Some(v) =>
      v.split("(\\s+|,)").filterNot(_.isEmpty).map(_.split("=")).foreach(x => System.setProperty(x(0), x(1)))
    case _ =>
  }

  override val userPropertiesMap = System.getProperties.asScala.toMap

  lazy val MavenCompile = config("m2r") extend(Compile)
  lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy")

  lazy val sparkGenjavadocSettings: Seq[sbt.Def.Setting[_]] = Seq(
    libraryDependencies += compilerPlugin(
      "org.spark-project" %% "genjavadoc-plugin" % unidocGenjavadocVersion.value cross CrossVersion.full),
    scalacOptions <+= target.map(t => "-P:genjavadoc:out=" + (t / "java")))

  lazy val sharedSettings = sparkGenjavadocSettings ++ Seq (
    exportJars in Compile := true,
    exportJars in Test := false,
    javaHome := sys.env.get("JAVA_HOME")
      .orElse(sys.props.get("java.home").map { p => new File(p).getParentFile().getAbsolutePath() })
      .map(file),
    incOptions := incOptions.value.withNameHashing(true),
    publishMavenStyle := true,
    unidocGenjavadocVersion := "0.9-spark0",

    // Override SBT's default resolvers:
    resolvers := Seq(
      DefaultMavenRepository,
      Resolver.mavenLocal,
      Resolver.file("local", file(Path.userHome.absolutePath + "/.ivy2/local"))(Resolver.ivyStylePatterns)
    ),
    externalResolvers := resolvers.value,
    otherResolvers <<= SbtPomKeys.mvnLocalRepository(dotM2 => Seq(Resolver.file("dotM2", dotM2))),
    publishLocalConfiguration in MavenCompile <<= (packagedArtifacts, deliverLocal, ivyLoggingLevel) map {
      (arts, _, level) => new PublishConfiguration(None, "dotM2", arts, Seq(), level)
    },
    publishMavenStyle in MavenCompile := true,
    publishLocal in MavenCompile <<= publishTask(publishLocalConfiguration in MavenCompile, deliverLocal),
    publishLocalBoth <<= Seq(publishLocal in MavenCompile, publishLocal).dependOn,

    javacOptions in (Compile, doc) ++= {
      val versionParts = System.getProperty("java.version").split("[+.\\-]+", 3)
      var major = versionParts(0).toInt
      if (major == 1) major = versionParts(1).toInt
      if (major >= 8) Seq("-Xdoclint:all", "-Xdoclint:-missing") else Seq.empty
    },

    javacJVMVersion := "1.7",
    scalacJVMVersion := "1.7",

    javacOptions in Compile ++= Seq(
      "-encoding", "UTF-8",
      "-source", javacJVMVersion.value
    ),
    // This -target option cannot be set in the Compile configuration scope since `javadoc` doesn't
    // play nicely with it; see https://github.com/sbt/sbt/issues/355#issuecomment-3817629 for
    // additional discussion and explanation.
    javacOptions in (Compile, compile) ++= Seq(
      "-target", javacJVMVersion.value
    ),

    scalacOptions in Compile ++= Seq(
      s"-target:jvm-${scalacJVMVersion.value}",
      "-sourcepath", (baseDirectory in ThisBuild).value.getAbsolutePath  // Required for relative source links in scaladoc
    ),

    // Implements -Xfatal-warnings, ignoring deprecation warnings.
    // Code snippet taken from https://issues.scala-lang.org/browse/SI-8410.
    compile in Compile := {
      val analysis = (compile in Compile).value
      val out = streams.value

      def logProblem(l: (=> String) => Unit, f: File, p: xsbti.Problem) = {
        l(f.toString + ":" + p.position.line.fold("")(_ + ":") + " " + p.message)
        l(p.position.lineContent)
        l("")
      }

      var failed = 0
      analysis.infos.allInfos.foreach { case (k, i) =>
        i.reportedProblems foreach { p =>
          val deprecation = p.message.contains("is deprecated")

          if (!deprecation) {
            failed = failed + 1
          }

          val printer: (=> String) => Unit = s => if (deprecation) {
            out.log.warn(s)
          } else {
            out.log.error("[warn] " + s)
          }

          logProblem(printer, k, p)

        }
      }

      if (failed > 0) {
        sys.error(s"$failed fatal warnings")
      }
      analysis
    }
  )

  def enable(settings: Seq[Setting[_]])(projectRef: ProjectRef) = {
    val existingSettings = projectsMap.getOrElse(projectRef.project, Seq[Setting[_]]())
    projectsMap += (projectRef.project -> (existingSettings ++ settings))
  }

  // Note ordering of these settings matter.
  /* Enable shared settings on all projects */
  (allProjects ++ optionallyEnabledProjects ++ assemblyProjects ++ copyJarsProjects ++ Seq(spark, tools))
    .foreach(enable(sharedSettings ++ DependencyOverrides.settings ++
      ExcludedDependencies.settings))

  /* Enable tests settings for all projects except examples, assembly and tools */
  (allProjects ++ optionallyEnabledProjects).foreach(enable(TestSettings.settings))

  val mimaProjects = allProjects.filterNot { x =>
    Seq(
      spark, hive, hiveThriftServer, catalyst, repl, networkCommon, networkShuffle, networkYarn,
      unsafe, testTags, sketch
    ).contains(x)
  }

  mimaProjects.foreach { x =>
    enable(MimaBuild.mimaSettings(sparkHome, x))(x)
  }

  /* Unsafe settings */
  enable(Unsafe.settings)(unsafe)

  /* Set up tasks to copy dependencies during packaging. */
  copyJarsProjects.foreach(enable(CopyDependencies.settings))

  /* Enable Assembly for all assembly projects */
  assemblyProjects.foreach(enable(Assembly.settings))

  /* Package pyspark artifacts in a separate zip file for YARN. */
  enable(PySparkAssembly.settings)(assembly)

  /* Enable unidoc only for the root spark project */
  enable(Unidoc.settings)(spark)

  /* Catalyst ANTLR generation settings */
  enable(Catalyst.settings)(catalyst)

  /* Spark SQL Core console settings */
  enable(SQL.settings)(sql)

  /* Hive console settings */
  enable(Hive.settings)(hive)

  enable(Flume.settings)(streamingFlumeSink)

  enable(Java8TestSettings.settings)(java8Tests)

  enable(DockerIntegrationTests.settings)(dockerIntegrationTests)

  /**
   * Adds the ability to run the spark shell directly from SBT without building an assembly
   * jar.
   *
   * Usage: `build/sbt sparkShell`
   */
  val sparkShell = taskKey[Unit]("start a spark-shell.")
  val sparkPackage = inputKey[Unit](
    s"""
       |Download and run a spark package.
       |Usage `builds/sbt "sparkPackage <group:artifact:version> <MainClass> [args]
     """.stripMargin)
  val sparkSql = taskKey[Unit]("starts the spark sql CLI.")

  enable(Seq(
    connectInput in run := true,
    fork := true,
    outputStrategy in run := Some (StdoutOutput),

    javaOptions ++= Seq("-Xmx2G", "-XX:MaxPermSize=256m"),

    sparkShell := {
      (runMain in Compile).toTask(" org.apache.spark.repl.Main -usejavacp").value
    },

    sparkPackage := {
      import complete.DefaultParsers._
      val packages :: className :: otherArgs = spaceDelimited("<group:artifact:version> <MainClass> [args]").parsed.toList
      val scalaRun = (runner in run).value
      val classpath = (fullClasspath in Runtime).value
      val args = Seq("--packages", packages, "--class", className, (Keys.`package` in Compile in "core").value.getCanonicalPath) ++ otherArgs
      println(args)
      scalaRun.run("org.apache.spark.deploy.SparkSubmit", classpath.map(_.data), args, streams.value.log)
    },

    javaOptions in Compile += "-Dspark.master=local",

    sparkSql := {
      (runMain in Compile).toTask(" org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver").value
    }
  ))(assembly)

  enable(Seq(sparkShell := sparkShell in "assembly"))(spark)

  // TODO: move this to its upstream project.
  override def projectDefinitions(baseDirectory: File): Seq[Project] = {
    super.projectDefinitions(baseDirectory).map { x =>
      if (projectsMap.exists(_._1 == x.id)) x.settings(projectsMap(x.id): _*)
      else x.settings(Seq[Setting[_]](): _*)
    } ++ Seq[Project](OldDeps.project)
  }

}

object Unsafe {
  lazy val settings = Seq(
    // This option is needed to suppress warnings from sun.misc.Unsafe usage
    javacOptions in Compile += "-XDignore.symbol.file"
  )
}

object Flume {
  lazy val settings = sbtavro.SbtAvro.avroSettings
}

object DockerIntegrationTests {
  // This serves to override the override specified in DependencyOverrides:
  lazy val settings = Seq(
    dependencyOverrides += "com.google.guava" % "guava" % "18.0"
  )
}

/**
 * Overrides to work around sbt's dependency resolution being different from Maven's.
 */
object DependencyOverrides {
  lazy val settings = Seq(
    dependencyOverrides += "com.google.guava" % "guava" % "14.0.1")
}

/**
  This excludes library dependencies in sbt, which are specified in maven but are
  not needed by sbt build.
  */
object ExcludedDependencies {
  lazy val settings = Seq(
    libraryDependencies ~= { libs => libs.filterNot(_.name == "groovy-all") }
  )
}

/**
 * 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 := allPreviousArtifactKeys.value.flatten
  )
}

object Catalyst {
  lazy val settings = Seq(
    // ANTLR code-generation step.
    //
    // This has been heavily inspired by com.github.stefri.sbt-antlr (0.5.3). It fixes a number of
    // build errors in the current plugin.
    // Create Parser from ANTLR grammar files.
    sourceGenerators in Compile += Def.task {
      val log = streams.value.log

      val grammarFileNames = Seq(
        "SparkSqlLexer.g",
        "SparkSqlParser.g")
      val sourceDir = (sourceDirectory in Compile).value / "antlr3"
      val targetDir = (sourceManaged in Compile).value

      // Create default ANTLR Tool.
      val antlr = new org.antlr.Tool

      // Setup input and output directories.
      antlr.setInputDirectory(sourceDir.getPath)
      antlr.setOutputDirectory(targetDir.getPath)
      antlr.setForceRelativeOutput(true)
      antlr.setMake(true)

      // Add grammar files.
      grammarFileNames.flatMap(gFileName => (sourceDir ** gFileName).get).foreach { gFilePath =>
        val relGFilePath = (gFilePath relativeTo sourceDir).get.getPath
        log.info("ANTLR: Grammar file '%s' detected.".format(relGFilePath))
        antlr.addGrammarFile(relGFilePath)
        // We will set library directory multiple times here. However, only the
        // last one has effect. Because the grammar files are located under the same directory,
        // We assume there is only one library directory.
        antlr.setLibDirectory(gFilePath.getParent)
      }

      // Generate the parser.
      antlr.process()
      val errorState = org.antlr.tool.ErrorManager.getErrorState
      if (errorState.errors > 0) {
        sys.error("ANTLR: Caught %d build errors.".format(errorState.errors))
      } else if (errorState.warnings > 0) {
        sys.error("ANTLR: Caught %d build warnings.".format(errorState.warnings))
      }

      // Return all generated java files.
      (targetDir ** "*.java").get.toSeq
    }.taskValue,
    // Include ANTLR tokens files.
    resourceGenerators in Compile += Def.task {
      ((sourceManaged in Compile).value ** "*.tokens").get.toSeq
    }.taskValue
  )
}

object SQL {
  lazy val settings = Seq(
    initialCommands in console :=
      """
        |import org.apache.spark.SparkContext
        |import org.apache.spark.sql.SQLContext
        |import org.apache.spark.sql.catalyst.analysis._
        |import org.apache.spark.sql.catalyst.dsl._
        |import org.apache.spark.sql.catalyst.errors._
        |import org.apache.spark.sql.catalyst.expressions._
        |import org.apache.spark.sql.catalyst.plans.logical._
        |import org.apache.spark.sql.catalyst.rules._
        |import org.apache.spark.sql.catalyst.util._
        |import org.apache.spark.sql.execution
        |import org.apache.spark.sql.functions._
        |import org.apache.spark.sql.types._
        |
        |val sc = new SparkContext("local[*]", "dev-shell")
        |val sqlContext = new SQLContext(sc)
        |import sqlContext.implicits._
        |import sqlContext._
      """.stripMargin,
    cleanupCommands in console := "sc.stop()"
  )
}

object Hive {

  lazy val settings = Seq(
    javaOptions += "-XX:MaxPermSize=256m",
    // Specially disable assertions since some Hive tests fail them
    javaOptions in Test := (javaOptions in Test).value.filterNot(_ == "-ea"),
    // Supporting all SerDes requires us to depend on deprecated APIs, so we turn off the warnings
    // only for this subproject.
    scalacOptions <<= scalacOptions map { currentOpts: Seq[String] =>
      currentOpts.filterNot(_ == "-deprecation")
    },
    initialCommands in console :=
      """
        |import org.apache.spark.SparkContext
        |import org.apache.spark.sql.catalyst.analysis._
        |import org.apache.spark.sql.catalyst.dsl._
        |import org.apache.spark.sql.catalyst.errors._
        |import org.apache.spark.sql.catalyst.expressions._
        |import org.apache.spark.sql.catalyst.plans.logical._
        |import org.apache.spark.sql.catalyst.rules._
        |import org.apache.spark.sql.catalyst.util._
        |import org.apache.spark.sql.execution
        |import org.apache.spark.sql.functions._
        |import org.apache.spark.sql.hive._
        |import org.apache.spark.sql.hive.test.TestHive._
        |import org.apache.spark.sql.hive.test.TestHive.implicits._
        |import org.apache.spark.sql.types._""".stripMargin,
    cleanupCommands in console := "sparkContext.stop()",
    // Some of our log4j jars make it impossible to submit jobs from this JVM to Hive Map/Reduce
    // in order to generate golden files.  This is only required for developers who are adding new
    // new query tests.
    fullClasspath in Test := (fullClasspath in Test).value.filterNot { f => f.toString.contains("jcl-over") }
  )
}

object Assembly {
  import sbtassembly.AssemblyUtils._
  import sbtassembly.Plugin._
  import AssemblyKeys._

  val hadoopVersion = taskKey[String]("The version of hadoop that spark is compiled against.")

  val deployDatanucleusJars = taskKey[Unit]("Deploy datanucleus jars to the spark/lib_managed/jars directory")

  lazy val settings = assemblySettings ++ Seq(
    test in assembly := {},
    hadoopVersion := {
      sys.props.get("hadoop.version")
        .getOrElse(SbtPomKeys.effectivePom.value.getProperties.get("hadoop.version").asInstanceOf[String])
    },
    jarName in assembly <<= (version, moduleName, hadoopVersion) map { (v, mName, hv) =>
      if (mName.contains("streaming-flume-assembly") || mName.contains("streaming-kafka-assembly") || mName.contains("streaming-kinesis-asl-assembly")) {
        // This must match the same name used in maven (see external/kafka-assembly/pom.xml)
        s"${mName}-${v}.jar"
      } else {
        s"${mName}-${v}-hadoop${hv}.jar"
      }
    },
    jarName in (Test, assembly) <<= (version, moduleName, hadoopVersion) map { (v, mName, hv) =>
      s"${mName}-test-${v}.jar"
    },
    mergeStrategy in assembly := {
      case PathList("org", "datanucleus", xs @ _*)             => MergeStrategy.discard
      case m if m.toLowerCase.endsWith("manifest.mf")          => MergeStrategy.discard
      case m if m.toLowerCase.matches("meta-inf.*\\.sf$")      => MergeStrategy.discard
      case "log4j.properties"                                  => MergeStrategy.discard
      case m if m.toLowerCase.startsWith("meta-inf/services/") => MergeStrategy.filterDistinctLines
      case "reference.conf"                                    => MergeStrategy.concat
      case _                                                   => MergeStrategy.first
    },
    deployDatanucleusJars := {
      val jars: Seq[File] = (fullClasspath in assembly).value.map(_.data)
        .filter(_.getPath.contains("org.datanucleus"))
      var libManagedJars = new File(BuildCommons.sparkHome, "lib_managed/jars")
      libManagedJars.mkdirs()
      jars.foreach { jar =>
        val dest = new File(libManagedJars, jar.getName)
        if (!dest.exists()) {
          Files.copy(jar.toPath, dest.toPath)
        }
      }
    },
    assembly <<= assembly.dependsOn(deployDatanucleusJars)
  )
}

object PySparkAssembly {
  import sbtassembly.Plugin._
  import AssemblyKeys._
  import java.util.zip.{ZipOutputStream, ZipEntry}

  lazy val settings = Seq(
    // Use a resource generator to copy all .py files from python/pyspark into a managed directory
    // to be included in the assembly. We can't just add "python/" to the assembly's resource dir
    // list since that will copy unneeded / unwanted files.
    resourceGenerators in Compile <+= resourceManaged in Compile map { outDir: File =>
      val src = new File(BuildCommons.sparkHome, "python/pyspark")
      val zipFile = new File(BuildCommons.sparkHome , "python/lib/pyspark.zip")
      zipFile.delete()
      zipRecursive(src, zipFile)
      Seq[File]()
    }
  )

  private def zipRecursive(source: File, destZipFile: File) = {
    val destOutput = new ZipOutputStream(new FileOutputStream(destZipFile))
    addFilesToZipStream("", source, destOutput)
    destOutput.flush()
    destOutput.close()
  }

  private def addFilesToZipStream(parent: String, source: File, output: ZipOutputStream): Unit = {
    if (source.isDirectory()) {
      output.putNextEntry(new ZipEntry(parent + source.getName()))
      for (file <- source.listFiles()) {
        addFilesToZipStream(parent + source.getName() + File.separator, file, output)
      }
    } else {
      val in = new FileInputStream(source)
      output.putNextEntry(new ZipEntry(parent + source.getName()))
      val buf = new Array[Byte](8192)
      var n = 0
      while (n != -1) {
        n = in.read(buf)
        if (n != -1) {
          output.write(buf, 0, n)
        }
      }
      output.closeEntry()
      in.close()
    }
  }

}

object Unidoc {

  import BuildCommons._
  import sbtunidoc.Plugin._
  import UnidocKeys._

  // for easier specification of JavaDoc package groups
  private def packageList(names: String*): String = {
    names.map(s => "org.apache.spark." + s).mkString(":")
  }

  private def ignoreUndocumentedPackages(packages: Seq[Seq[File]]): Seq[Seq[File]] = {
    packages
      .map(_.filterNot(_.getName.contains("$")))
      .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/deploy")))
      .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/examples")))
      .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/memory")))
      .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/network")))
      .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/shuffle")))
      .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/executor")))
      .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/unsafe")))
      .map(_.filterNot(_.getCanonicalPath.contains("python")))
      .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/util/collection")))
      .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/catalyst")))
      .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/execution")))
      .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/hive/test")))
  }

  val unidocSourceBase = settingKey[String]("Base URL of source links in Scaladoc.")

  lazy val settings = scalaJavaUnidocSettings ++ Seq (
    publish := {},

    unidocProjectFilter in(ScalaUnidoc, unidoc) :=
      inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, yarn, testTags),
    unidocProjectFilter in(JavaUnidoc, unidoc) :=
      inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, yarn, testTags),

    // Skip actual catalyst, but include the subproject.
    // Catalyst is not public API and contains quasiquotes which break scaladoc.
    unidocAllSources in (ScalaUnidoc, unidoc) := {
      ignoreUndocumentedPackages((unidocAllSources in (ScalaUnidoc, unidoc)).value)
    },

    // Skip class names containing $ and some internal packages in Javadocs
    unidocAllSources in (JavaUnidoc, unidoc) := {
      ignoreUndocumentedPackages((unidocAllSources in (JavaUnidoc, unidoc)).value)
    },

    // Javadoc options: create a window title, and group key packages on index page
    javacOptions in doc := Seq(
      "-windowtitle", "Spark " + version.value.replaceAll("-SNAPSHOT", "") + " JavaDoc",
      "-public",
      "-group", "Core Java API", packageList("api.java", "api.java.function"),
      "-group", "Spark Streaming", packageList(
        "streaming.api.java", "streaming.flume", "streaming.kafka", "streaming.kinesis"
      ),
      "-group", "MLlib", packageList(
        "mllib.classification", "mllib.clustering", "mllib.evaluation.binary", "mllib.linalg",
        "mllib.linalg.distributed", "mllib.optimization", "mllib.rdd", "mllib.recommendation",
        "mllib.regression", "mllib.stat", "mllib.tree", "mllib.tree.configuration",
        "mllib.tree.impurity", "mllib.tree.model", "mllib.util",
        "mllib.evaluation", "mllib.feature", "mllib.random", "mllib.stat.correlation",
        "mllib.stat.test", "mllib.tree.impl", "mllib.tree.loss",
        "ml", "ml.attribute", "ml.classification", "ml.clustering", "ml.evaluation", "ml.feature",
        "ml.param", "ml.recommendation", "ml.regression", "ml.tuning"
      ),
      "-group", "Spark SQL", packageList("sql.api.java", "sql.api.java.types", "sql.hive.api.java"),
      "-noqualifier", "java.lang"
    ),

    // Use GitHub repository for Scaladoc source links
    unidocSourceBase := s"https://github.com/apache/spark/tree/v${version.value}",

    scalacOptions in (ScalaUnidoc, unidoc) ++= Seq(
      "-groups" // Group similar methods together based on the @group annotation.
    ) ++ (
      // Add links to sources when generating Scaladoc for a non-snapshot release
      if (!isSnapshot.value) {
        Opts.doc.sourceUrl(unidocSourceBase.value + "€{FILE_PATH}.scala")
      } else {
        Seq()
      }
    )
  )
}

object CopyDependencies {

  val copyDeps = TaskKey[Unit]("copyDeps", "Copies needed dependencies to the build directory.")
  val destPath = (crossTarget in Compile) / "jars"

  lazy val settings = Seq(
    copyDeps := {
      val dest = destPath.value
      if (!dest.isDirectory() && !dest.mkdirs()) {
        throw new IOException("Failed to create jars directory.")
      }

      (dependencyClasspath in Compile).value.map(_.data)
        .filter { jar => jar.isFile() }
        .foreach { jar =>
          val destJar = new File(dest, jar.getName())
          if (destJar.isFile()) {
            destJar.delete()
          }
          Files.copy(jar.toPath(), destJar.toPath())
        }
    },
    crossTarget in (Compile, packageBin) := destPath.value,
    packageBin in Compile <<= (packageBin in Compile).dependsOn(copyDeps)
  )

}

object Java8TestSettings {
  import BuildCommons._

  lazy val settings = Seq(
    javacJVMVersion := "1.8",
    // Targeting Java 8 bytecode is only supported in Scala 2.11.4 and higher:
    scalacJVMVersion := (if (System.getProperty("scala-2.10") == "true") "1.7" else "1.8")
  )
}

object TestSettings {
  import BuildCommons._

  lazy val settings = Seq (
    // Fork new JVMs for tests and set Java options for those
    fork := true,
    // Setting SPARK_DIST_CLASSPATH is a simple way to make sure any child processes
    // launched by the tests have access to the correct test-time classpath.
    envVars in Test ++= Map(
      "SPARK_DIST_CLASSPATH" ->
        (fullClasspath in Test).value.files.map(_.getAbsolutePath).mkString(":").stripSuffix(":"),
      "SPARK_PREPEND_CLASSES" -> "1",
      "SPARK_TESTING" -> "1",
      "JAVA_HOME" -> sys.env.get("JAVA_HOME").getOrElse(sys.props("java.home"))),
    javaOptions in Test += s"-Djava.io.tmpdir=$testTempDir",
    javaOptions in Test += "-Dspark.test.home=" + sparkHome,
    javaOptions in Test += "-Dspark.testing=1",
    javaOptions in Test += "-Dspark.port.maxRetries=100",
    javaOptions in Test += "-Dspark.master.rest.enabled=false",
    javaOptions in Test += "-Dspark.ui.enabled=false",
    javaOptions in Test += "-Dspark.ui.showConsoleProgress=false",
    javaOptions in Test += "-Dspark.unsafe.exceptionOnMemoryLeak=true",
    javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true",
    javaOptions in Test += "-Dderby.system.durability=test",
    javaOptions in Test ++= System.getProperties.asScala.filter(_._1.startsWith("spark"))
      .map { case (k,v) => s"-D$k=$v" }.toSeq,
    javaOptions in Test += "-ea",
    javaOptions in Test ++= "-Xmx3g -Xss4096k -XX:PermSize=128M -XX:MaxNewSize=256m -XX:MaxPermSize=1g"
      .split(" ").toSeq,
    javaOptions += "-Xmx3g",
    // Exclude tags defined in a system property
    testOptions in Test += Tests.Argument(TestFrameworks.ScalaTest,
      sys.props.get("test.exclude.tags").map { tags =>
        tags.split(",").flatMap { tag => Seq("-l", tag) }.toSeq
      }.getOrElse(Nil): _*),
    testOptions in Test += Tests.Argument(TestFrameworks.JUnit,
      sys.props.get("test.exclude.tags").map { tags =>
        Seq("--exclude-categories=" + tags)
      }.getOrElse(Nil): _*),
    // Show full stack trace and duration in test cases.
    testOptions in Test += Tests.Argument("-oDF"),
    testOptions in Test += Tests.Argument(TestFrameworks.JUnit, "-v", "-a"),
    // Enable Junit testing.
    libraryDependencies += "com.novocode" % "junit-interface" % "0.11" % "test",
    // Only allow one test at a time, even across projects, since they run in the same JVM
    parallelExecution in Test := false,
    // Make sure the test temp directory exists.
    resourceGenerators in Test <+= resourceManaged in Test map { outDir: File =>
      if (!new File(testTempDir).isDirectory()) {
        require(new File(testTempDir).mkdirs())
      }
      Seq[File]()
    },
    concurrentRestrictions in Global += Tags.limit(Tags.Test, 1),
    // Remove certain packages from Scaladoc
    scalacOptions in (Compile, doc) := Seq(
      "-groups",
      "-skip-packages", Seq(
        "org.apache.spark.api.python",
        "org.apache.spark.network",
        "org.apache.spark.deploy",
        "org.apache.spark.util.collection"
      ).mkString(":"),
      "-doc-title", "Spark " + version.value.replaceAll("-SNAPSHOT", "") + " ScalaDoc"
    )
  )

}