mirror of https://github.com/sbt/sbt.git
cleanup/rework related to Settings/InputParser
- drop fillThis: handle in injectStreams instead
- simplify InputParser construction (at the expense of implementation simplicity)
- split out ScopeKey/initialization parts of Setting with separate Initialize trait
+ makes Apply obsolete
+ makes the Initialize trait properly composable
+ this allowed splitting the InputParser definition into
an Initialize for parsing and one for the action
- implement test-only
- inject resolved scope
This commit is contained in:
parent
cbdabc1383
commit
c9b932acd1
|
|
@ -65,10 +65,18 @@ object Act
|
|||
structure.data.get(key.scope, key.key) match
|
||||
{
|
||||
case None => failure("Invalid setting or task")
|
||||
case Some(input: InputTask[_]) => applyTask(s, structure, input.parser, show)
|
||||
case Some(input: InputStatic[_]) => applyTask(s, structure, input.parser(s), show)
|
||||
case Some(input: InputDynamic[_]) => applyDynamicTask(s, structure, input, show)
|
||||
case Some(task: Task[_]) => applyTask(s, structure, success(task), show)
|
||||
case Some(v) => success(() => { logger(s).info(v.toString); s})
|
||||
}
|
||||
def applyDynamicTask[I](s: State, structure: Load.BuildStructure, input: InputDynamic[I], show: Boolean): Parser[() => State] =
|
||||
Command.applyEffect(input parser s) { parsed =>
|
||||
import EvaluateTask._
|
||||
val result = withStreams(structure){ str => runTask(input.task)(nodeView(s, str, parsed)) }
|
||||
processResult(result, logger(s), show)
|
||||
s
|
||||
}
|
||||
def applyTask(s: State, structure: Load.BuildStructure, p: Parser[Task[_]], show: Boolean): Parser[() => State] =
|
||||
Command.applyEffect(p) { t =>
|
||||
import EvaluateTask._
|
||||
|
|
|
|||
|
|
@ -11,7 +11,7 @@ package sbt
|
|||
import scala.annotation.tailrec
|
||||
import collection.mutable
|
||||
import Compile.{Compilers,Inputs}
|
||||
import Project.{AppConfig, Config, ScopedKey, Setting, ThisProject, ThisProjectRef}
|
||||
import Project.{AppConfig, Config, ScopedKey, ScopeLocal, Setting, ThisProject, ThisProjectRef}
|
||||
import TypeFunctions.{Endo,Id}
|
||||
import tools.nsc.reporters.ConsoleReporter
|
||||
|
||||
|
|
@ -107,11 +107,16 @@ object EvaluateTask
|
|||
|
||||
val (state, dummyState) = dummy[State]("state")
|
||||
val (streamsManager, dummyStreamsManager) = dummy[Streams]("streams-manager")
|
||||
val streams = TaskKey[TaskStreams]("streams", true)
|
||||
val streams = TaskKey[TaskStreams]("streams")
|
||||
val resolvedScoped = SettingKey[ScopedKey[_]]("resolved-scoped")
|
||||
// will be cast to the appropriate type when passed to an InputTask implementation
|
||||
// this assumes there is only one InputTask finally selected and will need to be
|
||||
// revisited for aggregating InputTasks
|
||||
private[sbt] val (parseResult, dummyParseResult) = dummy[Any]("$parse-result")
|
||||
|
||||
def injectSettings: Seq[Project.Setting[_]] = Seq(
|
||||
(state in Scope.GlobalScope) :== dummyState,
|
||||
(streamsManager in Scope.GlobalScope) :== dummyStreamsManager
|
||||
(state in Scope.GlobalScope) ::= dummyState,
|
||||
(streamsManager in Scope.GlobalScope) ::= dummyStreamsManager
|
||||
)
|
||||
|
||||
def dummy[T](name: String): (TaskKey[T], Task[T]) = (TaskKey[T](name), dummyTask(name))
|
||||
|
|
@ -143,8 +148,8 @@ object EvaluateTask
|
|||
for( t <- structure.data.get(resolvedScope, taskKey.key)) yield
|
||||
(t, nodeView(state, streams))
|
||||
}
|
||||
def nodeView(state: State, streams: Streams): Execute.NodeView[Task] =
|
||||
Transform(dummyStreamsManager :^: dummyState :^: KNil, streams :+: state :+: HNil)
|
||||
def nodeView(state: State, streams: Streams, parsed: Any = ()): Execute.NodeView[Task] =
|
||||
Transform(dummyParseResult :^: dummyStreamsManager :^: dummyState :^: KNil, parsed :+: streams :+: state :+: HNil)
|
||||
|
||||
def runTask[Task[_] <: AnyRef, T](root: Task[T], checkCycles: Boolean = false, maxWorkers: Int = SystemProcessors)(implicit taskToNode: Execute.NodeView[Task]): Result[T] =
|
||||
{
|
||||
|
|
@ -165,17 +170,20 @@ object EvaluateTask
|
|||
error("Task did not complete successfully")
|
||||
}
|
||||
|
||||
val injectStreams = (scoped: ScopedKey[_]) =>
|
||||
// if the return type Seq[Setting[_]] is not explicitly given, scalac hangs
|
||||
val injectStreams: ScopedKey[_] => Seq[Setting[_]] = scoped =>
|
||||
if(scoped.key == streams.key)
|
||||
{
|
||||
val scope = Scope.fillTaskAxis(scoped.scope, scoped.key)
|
||||
Seq(streams in scope <<= streamsManager map { mgr =>
|
||||
val stream = mgr(ScopedKey(scope, scoped.key))
|
||||
Seq(streams in scoped.scope <<= streamsManager map { mgr =>
|
||||
val stream = mgr(scoped)
|
||||
stream.open()
|
||||
stream
|
||||
})
|
||||
}
|
||||
else Nil
|
||||
else if(scoped.key == resolvedScoped.key)
|
||||
Seq(resolvedScoped in scoped.scope :== scoped)
|
||||
else if(scoped.key == parseResult.key)
|
||||
Seq(parseResult in scoped.scope ::= dummyParseResult)
|
||||
else
|
||||
Nil
|
||||
}
|
||||
object Index
|
||||
{
|
||||
|
|
@ -248,7 +256,7 @@ object Load
|
|||
val loaded = resolveProjects(load(rootBase, s, config))
|
||||
val projects = loaded.units
|
||||
lazy val rootEval = lazyEval(loaded.units(loaded.root).unit)
|
||||
val settings = config.injectSettings ++ buildConfigurations(loaded, getRootProject(projects), rootEval)
|
||||
val settings = finalTransforms(config.injectSettings ++ buildConfigurations(loaded, getRootProject(projects), rootEval))
|
||||
val delegates = config.delegates(loaded)
|
||||
val data = Project.makeSettings(settings, delegates, config.scopeLocal)
|
||||
val index = structureIndex(data)
|
||||
|
|
@ -256,6 +264,24 @@ object Load
|
|||
(rootEval, new BuildStructure(projects, loaded.root, settings, data, index, streams, delegates, config.scopeLocal))
|
||||
}
|
||||
|
||||
// map dependencies on the special tasks so that the scope is the same as the defining key
|
||||
// additionally, set the task axis to the defining key if it is not set
|
||||
def finalTransforms(ss: Seq[Setting[_]]): Seq[Setting[_]] =
|
||||
{
|
||||
import EvaluateTask.{parseResult, resolvedScoped, streams}
|
||||
def isSpecial(key: AttributeKey[_]) = key == streams.key || key == resolvedScoped.key || key == parseResult.key
|
||||
def mapSpecial(to: ScopedKey[_]) = new (ScopedKey ~> ScopedKey){ def apply[T](key: ScopedKey[T]) =
|
||||
if(isSpecial(key.key))
|
||||
{
|
||||
val replaced = Scope.replaceThis(to.scope)(key.scope)
|
||||
val scope = if(key.key == resolvedScoped.key) replaced else Scope.fillTaskAxis(replaced, to.key)
|
||||
ScopedKey(scope, key.key)
|
||||
}
|
||||
else key
|
||||
}
|
||||
ss.map(s => s mapReferenced mapSpecial(s.key) )
|
||||
}
|
||||
|
||||
def structureIndex(settings: Settings[Scope]): StructureIndex =
|
||||
new StructureIndex(Index.stringToKeyMap(settings), Index.taskToKeyMap(settings), KeyIndex(settings.allKeys( (s,k) => ScopedKey(s,k))))
|
||||
|
||||
|
|
@ -541,8 +567,8 @@ object Load
|
|||
def referenced(definition: Project): Seq[ProjectRef] = definition.inherits ++ definition.aggregate ++ definition.dependencies.map(_.project)
|
||||
|
||||
|
||||
final class BuildStructure(val units: Map[URI, LoadedBuildUnit], val root: URI, val settings: Seq[Setting[_]], val data: Settings[Scope], val index: StructureIndex, val streams: Streams, val delegates: Scope => Seq[Scope], val scopeLocal: ScopedKey[_] => Seq[Setting[_]])
|
||||
final class LoadBuildConfiguration(val stagingDirectory: File, val classpath: Seq[File], val loader: ClassLoader, val compilers: Compilers, val evalPluginDef: (BuildStructure, State) => (Seq[File], Analysis), val delegates: LoadedBuild => Scope => Seq[Scope], val scopeLocal: ScopedKey[_] => Seq[Setting[_]], val injectSettings: Seq[Setting[_]], val log: Logger)
|
||||
final class BuildStructure(val units: Map[URI, LoadedBuildUnit], val root: URI, val settings: Seq[Setting[_]], val data: Settings[Scope], val index: StructureIndex, val streams: Streams, val delegates: Scope => Seq[Scope], val scopeLocal: ScopeLocal)
|
||||
final class LoadBuildConfiguration(val stagingDirectory: File, val classpath: Seq[File], val loader: ClassLoader, val compilers: Compilers, val evalPluginDef: (BuildStructure, State) => (Seq[File], Analysis), val delegates: LoadedBuild => Scope => Seq[Scope], val scopeLocal: ScopeLocal, val injectSettings: Seq[Setting[_]], val log: Logger)
|
||||
// information that is not original, but can be reconstructed from the rest of BuildStructure
|
||||
final class StructureIndex(val keyMap: Map[String, AttributeKey[_]], val taskToKey: Map[Task[_], ScopedKey[Task[_]]], val keyIndex: KeyIndex)
|
||||
|
||||
|
|
|
|||
|
|
@ -4,18 +4,20 @@
|
|||
package sbt
|
||||
|
||||
import java.io.File
|
||||
import Scoped.Apply
|
||||
import Scope.{GlobalScope,ThisScope}
|
||||
import Project.{AppConfig, Config, Setting, ThisProject, ThisProjectRef}
|
||||
import Project.{AppConfig, Config, Initialize, ScopedKey, Setting, ThisProject, ThisProjectRef}
|
||||
import Configurations.{Compile => CompileConf, Test => TestConf}
|
||||
import Command.HistoryPath
|
||||
import EvaluateTask.streams
|
||||
import EvaluateTask.{resolvedScoped, streams}
|
||||
import complete._
|
||||
import inc.Analysis
|
||||
import std.TaskExtra._
|
||||
import scala.xml.{Node => XNode,NodeSeq}
|
||||
import org.apache.ivy.core.module.{descriptor, id}
|
||||
import descriptor.ModuleDescriptor, id.ModuleRevisionId
|
||||
import org.scalatools.testing.Framework
|
||||
import Types._
|
||||
|
||||
|
||||
object Keys
|
||||
{
|
||||
|
|
@ -84,6 +86,7 @@ object Keys
|
|||
val DefinedTests = TaskKey[Seq[TestDefinition]]("defined-tests")
|
||||
val ExecuteTests = TaskKey[Test.Output]("execute-tests")
|
||||
val TestTask = TaskKey[Unit]("test")
|
||||
val TestOnly = InputKey[Unit]("test-only")
|
||||
val TestOptions = TaskKey[Seq[TestOption]]("test-options")
|
||||
val TestFrameworks = SettingKey[Seq[TestFramework]]("test-frameworks")
|
||||
val TestListeners = TaskKey[Iterable[TestReportListener]]("test-listeners")
|
||||
|
|
@ -166,26 +169,26 @@ object Default
|
|||
|
||||
final class RichFileSetting(s: ScopedSetting[File]) extends RichFileBase
|
||||
{
|
||||
def /(c: String): Apply[File] = s { _ / c }
|
||||
def /(c: String): Initialize[File] = s { _ / c }
|
||||
protected[this] def map0(f: PathFinder => PathFinder) = s(file => finder(f)(file :: Nil))
|
||||
}
|
||||
final class RichFilesSetting(s: ScopedSetting[Seq[File]]) extends RichFileBase
|
||||
{
|
||||
def /(s: String): Apply[Seq[File]] = map0 { _ / s }
|
||||
def /(s: String): Initialize[Seq[File]] = map0 { _ / s }
|
||||
protected[this] def map0(f: PathFinder => PathFinder) = s(finder(f))
|
||||
}
|
||||
sealed abstract class RichFileBase
|
||||
{
|
||||
def *(filter: FileFilter): Apply[Seq[File]] = map0 { _ * filter }
|
||||
def **(filter: FileFilter): Apply[Seq[File]] = map0 { _ ** filter }
|
||||
protected[this] def map0(f: PathFinder => PathFinder): Apply[Seq[File]]
|
||||
def *(filter: FileFilter): Initialize[Seq[File]] = map0 { _ * filter }
|
||||
def **(filter: FileFilter): Initialize[Seq[File]] = map0 { _ ** filter }
|
||||
protected[this] def map0(f: PathFinder => PathFinder): Initialize[Seq[File]]
|
||||
protected[this] def finder(f: PathFinder => PathFinder): Seq[File] => Seq[File] =
|
||||
in => f(in).getFiles.toSeq
|
||||
}
|
||||
def configSrcSub(key: ScopedSetting[File]): Apply[File] = (key, Config) { (src, conf) => src / nameForSrc(conf.name) }
|
||||
def configSrcSub(key: ScopedSetting[File]): Initialize[File] = (key, Config) { (src, conf) => src / nameForSrc(conf.name) }
|
||||
def nameForSrc(config: String) = if(config == "compile") "main" else config
|
||||
def prefix(config: String) = if(config == "compile") "" else config + "-"
|
||||
def toSeq[T](key: ScopedSetting[T]): Apply[Seq[T]] = key( _ :: Nil)
|
||||
def toSeq[T](key: ScopedSetting[T]): Initialize[Seq[T]] = key( _ :: Nil)
|
||||
|
||||
def extractAnalysis[T](a: Attributed[T]): (T, Analysis) =
|
||||
(a.data, a.metadata get Command.Analysis getOrElse Analysis.Empty)
|
||||
|
|
@ -255,7 +258,8 @@ object Default
|
|||
DiscoveredMainClasses <<= CompileTask map discoverMainClasses,
|
||||
Runner <<= ScalaInstance( si => new Run(si) ),
|
||||
SelectMainClass <<= DiscoveredMainClasses map selectRunMain,
|
||||
MainClass :== SelectMainClass,
|
||||
MainClass in RunTask :== SelectMainClass,
|
||||
MainClass <<= DiscoveredMainClasses map selectPackageMain,
|
||||
RunTask <<= runTask(FullClasspath, MainClass in RunTask),
|
||||
ScaladocOptions <<= ScalacOptions(identity),
|
||||
DocTask <<= docTask,
|
||||
|
|
@ -276,17 +280,30 @@ object Default
|
|||
LoadedTestFrameworks <<= (TestFrameworks, streams, TestLoader) map { (frameworks, s, loader) =>
|
||||
frameworks.flatMap(f => f.create(loader, s.log).map( x => (f,x)).toIterable).toMap
|
||||
},
|
||||
DefinedTests <<= (LoadedTestFrameworks, CompileTask) map { (frameworkMap, analysis) =>
|
||||
Test.discover(frameworkMap.values.toSeq, analysis)._1
|
||||
DefinedTests <<= (LoadedTestFrameworks, CompileTask, streams) map { (frameworkMap, analysis, s) =>
|
||||
val tests = Test.discover(frameworkMap.values.toSeq, analysis)._1
|
||||
IO.writeLines(s.text(CompletionsID), tests.map(_.name).distinct)
|
||||
tests
|
||||
},
|
||||
TestListeners <<= (streams in TestTask) map ( s => TestLogger(s.log) :: Nil ),
|
||||
TestOptions <<= TestListeners map { listeners => Test.Listeners(listeners) :: Nil },
|
||||
ExecuteTests <<= (streams in TestTask, LoadedTestFrameworks, TestOptions, TestLoader, DefinedTests, CopyResources) flatMap {
|
||||
(s, frameworkMap, options, loader, discovered, _) => Test(frameworkMap, loader, discovered, options, s.log)
|
||||
},
|
||||
TestTask <<= (ExecuteTests, streams) map { (results, s) => Test.showResults(s.log, results) }
|
||||
TestTask <<= (ExecuteTests, streams) map { (results, s) => Test.showResults(s.log, results) },
|
||||
TestOnly <<= testOnly
|
||||
)
|
||||
|
||||
def testOnly =
|
||||
InputTask(resolvedScoped(testOnlyParser)) ( result =>
|
||||
(streams, LoadedTestFrameworks, TestOptions, TestLoader, DefinedTests, CopyResources, result) flatMap {
|
||||
case (s, frameworks, opts, loader, discovered, _, (tests, frameworkOptions)) =>
|
||||
val modifiedOpts = Test.Filter(if(tests.isEmpty) _ => true else tests.toSet ) +: Test.Argument(frameworkOptions : _*) +: opts
|
||||
Test(frameworks, loader, discovered, modifiedOpts, s.log) map { results =>
|
||||
Test.showResults(s.log, results)
|
||||
}
|
||||
}
|
||||
)
|
||||
lazy val packageDefaults = packageBase ++ inConfig(CompileConf)(packageConfig) ++ inConfig(TestConf)(packageConfig)
|
||||
|
||||
lazy val packageBase = Seq(
|
||||
|
|
@ -314,7 +331,7 @@ object Default
|
|||
}
|
||||
def jarPath = JarPath <<= (Target, JarName, NameToString) { (t, n, toString) => t / toString(n) }
|
||||
|
||||
def packageTasks(key: TaskKey[sbt.Package.Configuration], tpeString: String, mappings: Apply[Task[Seq[(File,String)]]]) =
|
||||
def packageTasks(key: TaskKey[sbt.Package.Configuration], tpeString: String, mappings: Initialize[Task[Seq[(File,String)]]]) =
|
||||
inTask(key)( Seq(
|
||||
key in ThisScope.copy(task = Global) <<= packageTask,
|
||||
Mappings <<= mappings,
|
||||
|
|
@ -323,7 +340,7 @@ object Default
|
|||
CacheDirectory <<= CacheDirectory / key.key.label,
|
||||
jarPath
|
||||
))
|
||||
def packageTask: Apply[Task[sbt.Package.Configuration]] =
|
||||
def packageTask: Initialize[Task[sbt.Package.Configuration]] =
|
||||
(JarPath, Mappings, PackageOptions, CacheDirectory, streams) map { (jar, srcs, options, cacheDir, s) =>
|
||||
val config = new sbt.Package.Configuration(srcs, jar, options)
|
||||
sbt.Package(config, cacheDir, s.log)
|
||||
|
|
@ -332,19 +349,18 @@ object Default
|
|||
|
||||
def selectRunMain(classes: Seq[String]): Option[String] =
|
||||
sbt.SelectMainClass(Some(SimpleReader readLine _), classes)
|
||||
def selectPackageMain(classes: Seq[String]): Option[String] =
|
||||
sbt.SelectMainClass(None, classes)
|
||||
|
||||
def runTask(classpath: ScopedTask[Classpath], mainClass: ScopedTask[Option[String]]): Apply[InputTask[Unit]] =
|
||||
(classpath.setting, mainClass.setting, Runner, streams.setting, CopyResources.setting) { (cpTask, mainTask, runner, sTask, copy) =>
|
||||
import Types._
|
||||
InputTask(complete.Parsers.spaceDelimited("<arg>")) { args =>
|
||||
(cpTask :^: mainTask :^: sTask :^: copy :^: KNil) map { case cp :+: main :+: s :+: _ :+: HNil =>
|
||||
val mainClass = main getOrElse error("No main class detected.")
|
||||
runner.run(mainClass, data(cp), args, s.log) foreach error
|
||||
}
|
||||
def runTask(classpath: ScopedTask[Classpath], mainClass: ScopedTask[Option[String]]): Initialize[InputTask[Unit]] =
|
||||
InputTask(_ => complete.Parsers.spaceDelimited("<arg>")) { result =>
|
||||
(classpath, mainClass, Runner, streams, CopyResources, result) map { (cp, main, runner, s, _, args) =>
|
||||
val mainClass = main getOrElse error("No main class detected.")
|
||||
runner.run(mainClass, data(cp), args, s.log) foreach error
|
||||
}
|
||||
}
|
||||
|
||||
def docTask: Apply[Task[File]] =
|
||||
def docTask: Initialize[Task[File]] =
|
||||
(CompileInputs, streams, DocDirectory, Config, ScaladocOptions) map { (in, s, target, config, options) =>
|
||||
val d = new Scaladoc(in.config.maxErrors, in.compilers.scalac)
|
||||
d(nameForSrc(config.name), in.config.sources, in.config.classpath, target, options)(s.log)
|
||||
|
|
@ -383,6 +399,28 @@ object Default
|
|||
mappings
|
||||
}
|
||||
|
||||
def testOnlyParser(resolved: ScopedKey[_]): State => Parser[(Seq[String],Seq[String])] =
|
||||
{ state =>
|
||||
import DefaultParsers._
|
||||
def distinctParser(exs: Set[String]): Parser[Seq[String]] =
|
||||
token(Space ~> NotSpace.examples(exs)).flatMap(ex => distinctParser(exs - ex).map(ex +: _)) ?? Nil
|
||||
val tests = savedLines(state, resolved, DefinedTests)
|
||||
val selectTests = distinctParser(tests.toSet) // todo: proper IDs
|
||||
val options = (Space ~> "--" ~> spaceDelimited("<arg>")) ?? Nil
|
||||
selectTests ~ options
|
||||
}
|
||||
def savedLines(state: State, reader: ScopedKey[_], readFrom: Scoped): Seq[String] =
|
||||
{
|
||||
val structure = Project.structure(state)
|
||||
structure.data.definingScope(reader.scope, readFrom.key) match {
|
||||
case Some(defined) =>
|
||||
val key = ScopedKey(Scope.fillTaskAxis(defined, readFrom.key), readFrom.key)
|
||||
structure.streams.use(reader){ ts => IO.readLines(ts.readText(key, CompletionsID)) }
|
||||
case None => Nil
|
||||
}
|
||||
}
|
||||
|
||||
val CompletionsID = "completions"
|
||||
|
||||
// lazy val projectConsole = task { Console.sbtDefault(info.compileInputs, this)(ConsoleLogger()) }
|
||||
|
||||
|
|
@ -426,7 +464,7 @@ object Classpaths
|
|||
import Default._
|
||||
import Attributed.{blank, blankSeq}
|
||||
|
||||
def concat[T](a: ScopedTaskable[Seq[T]], b: ScopedTaskable[Seq[T]]): Apply[Task[Seq[T]]] = (a,b) map (_ ++ _)
|
||||
def concat[T](a: ScopedTaskable[Seq[T]], b: ScopedTaskable[Seq[T]]): Initialize[Task[Seq[T]]] = (a,b) map (_ ++ _)
|
||||
|
||||
val configSettings: Seq[Project.Setting[_]] = Seq(
|
||||
ExternalDependencyClasspath <<= concat(UnmanagedClasspath, ManagedClasspath),
|
||||
|
|
@ -445,7 +483,9 @@ object Classpaths
|
|||
val publishSettings: Seq[Project.Setting[_]] = Seq(
|
||||
PublishMavenStyle in GlobalScope :== true,
|
||||
PackageToPublish <<= defaultPackageTasks.dependOn,
|
||||
DeliverDepends := (PublishMavenStyle, MakePom.setting, PackageToPublish.setting) { (mavenStyle, makePom, ptp) => if(mavenStyle) makePom else ptp },
|
||||
DeliverDepends <<= (PublishMavenStyle, MakePom.setting, PackageToPublish.setting) { (mavenStyle, makePom, ptp) =>
|
||||
if(mavenStyle) makePom.map(_ => ()) else ptp
|
||||
},
|
||||
MakePom <<= (IvyModule, MakePomConfig, PackageToPublish) map { (ivyModule, makePomConfig, _) => IvyActions.makePom(ivyModule, makePomConfig); makePomConfig.file },
|
||||
Deliver <<= deliver(PublishConfig),
|
||||
DeliverLocal <<= deliver(PublishLocalConfig),
|
||||
|
|
@ -502,13 +542,12 @@ object Classpaths
|
|||
)
|
||||
|
||||
|
||||
def deliver(config: TaskKey[PublishConfiguration]): Apply[Task[Unit]] =
|
||||
def deliver(config: TaskKey[PublishConfiguration]): Initialize[Task[Unit]] =
|
||||
(IvyModule, config, DeliverDepends) map { (ivyModule, config, _) => IvyActions.deliver(ivyModule, config) }
|
||||
def publish(config: TaskKey[PublishConfiguration], deliverKey: TaskKey[_]): Apply[Task[Unit]] =
|
||||
def publish(config: TaskKey[PublishConfiguration], deliverKey: TaskKey[_]): Initialize[Task[Unit]] =
|
||||
(IvyModule, config, deliverKey) map { (ivyModule, config, _) => IvyActions.publish(ivyModule, config) }
|
||||
|
||||
import Cache._
|
||||
import Types._
|
||||
import CacheIvy.{classpathFormat, publishIC, updateIC}
|
||||
|
||||
def cachedUpdate(cacheFile: File, module: IvySbt#Module, config: UpdateConfiguration, log: Logger): Map[String, Seq[File]] =
|
||||
|
|
@ -560,7 +599,7 @@ object Classpaths
|
|||
p.dependencies flatMap { dep => (ProjectID in dep.project) get data map { _.copy(configurations = dep.configuration) } }
|
||||
}
|
||||
|
||||
def depMap: Apply[Task[Map[ModuleRevisionId, ModuleDescriptor]]] =
|
||||
def depMap: Initialize[Task[Map[ModuleRevisionId, ModuleDescriptor]]] =
|
||||
(ThisProject, ThisProjectRef, Data) flatMap { (root, rootRef, data) =>
|
||||
val dependencies = (p: (ProjectRef, Project)) => p._2.dependencies.flatMap(pr => ThisProject in pr.project get data map { (pr.project, _) })
|
||||
depMap(Dag.topologicalSort((rootRef,root))(dependencies).dropRight(1), data)
|
||||
|
|
@ -574,16 +613,16 @@ object Classpaths
|
|||
}).toMap
|
||||
}
|
||||
|
||||
def projectResolver: Apply[Task[Resolver]] =
|
||||
def projectResolver: Initialize[Task[Resolver]] =
|
||||
ProjectDescriptors map { m =>
|
||||
new RawRepository(new ProjectResolver("inter-project", m))
|
||||
}
|
||||
|
||||
def analyzed[T](data: T, analysis: Analysis) = Attributed.blank(data).put(Command.Analysis, analysis)
|
||||
def makeProducts: Apply[Task[Classpath]] =
|
||||
def makeProducts: Initialize[Task[Classpath]] =
|
||||
(CompileTask, CompileInputs) map { (analysis, i) => analyzed(i.config.classesDirectory, analysis) :: Nil }
|
||||
|
||||
def internalDependencies: Apply[Task[Classpath]] =
|
||||
def internalDependencies: Initialize[Task[Classpath]] =
|
||||
(ThisProjectRef, ThisProject, Config, Data) flatMap internalDependencies0
|
||||
|
||||
def internalDependencies0(projectRef: ProjectRef, project: Project, conf: Configuration, data: Settings[Scope]): Task[Classpath] =
|
||||
|
|
|
|||
|
|
@ -102,6 +102,10 @@ object Project extends Init[Scope]
|
|||
val f = mapScope(g)
|
||||
ss.map(_ mapKey f mapReferenced f)
|
||||
}
|
||||
def transformRef(g: Scope => Scope, ss: Seq[Setting[_]]): Seq[Setting[_]] = {
|
||||
val f = mapScope(g)
|
||||
ss.map(_ mapReferenced f)
|
||||
}
|
||||
def translateUninitialized[T](f: => T): T =
|
||||
try { f } catch { case u: Project.Uninitialized =>
|
||||
val msg = "Uninitialized reference to " + display(u.key) + " from " + display(u.refKey)
|
||||
|
|
|
|||
|
|
@ -8,46 +8,80 @@ package sbt
|
|||
import Types._
|
||||
import std.TaskExtra._
|
||||
import Task._
|
||||
import Project.{ScopedKey, Setting}
|
||||
import Project.{Initialize, ScopedKey, Setting, setting}
|
||||
import complete.Parser
|
||||
import java.io.File
|
||||
import java.net.URI
|
||||
|
||||
sealed trait InputTask[T] {
|
||||
def parser: Parser[Task[T]]
|
||||
sealed trait InputTask[T]
|
||||
private final class InputStatic[T](val parser: State => Parser[Task[T]]) extends InputTask[T]
|
||||
private sealed trait InputDynamic[T] extends InputTask[T]
|
||||
{
|
||||
type Result
|
||||
def parser: State => Parser[Result]
|
||||
def task: Task[T]
|
||||
}
|
||||
object InputTask {
|
||||
def apply[T](p: Parser[Task[T]]): InputTask[T] = new InputTask[T] { def parser = p }
|
||||
def apply[I,T](p: Parser[I])(c: I => Task[T]): InputTask[T] = apply(p map c)
|
||||
object InputTask
|
||||
{
|
||||
def static[T](p: Parser[Task[T]]): InputTask[T] = free(_ => p)
|
||||
def static[I,T](p: Parser[I])(c: I => Task[T]): InputTask[T] = static(p map c)
|
||||
|
||||
def free[T](p: State => Parser[Task[T]]): InputTask[T] = new InputStatic[T](p)
|
||||
def free[I,T](p: State => Parser[I])(c: I => Task[T]): InputTask[T] = free(s => p(s) map c)
|
||||
|
||||
def separate[I,T](p: State => Parser[I])(action: Initialize[I => Task[T]]): Initialize[InputTask[T]] =
|
||||
separate(Project value p)(action)
|
||||
def separate[I,T](p: Initialize[State => Parser[I]])(action: Initialize[I => Task[T]]): Initialize[InputTask[T]] =
|
||||
p.zipWith(action)((parser, act) => free(parser)(act))
|
||||
|
||||
|
||||
// This interface allows the Parser to be constructed using other Settings, but not Tasks (which is desired).
|
||||
// The action can be constructed using Settings and Tasks and with the parse result injected into a Task.
|
||||
// This is the ugly part, requiring hooks in injectStreams and Act to handle the dummy task for the parse result.
|
||||
// However, this is results in a minimal interface to the full capabilities of an InputTask
|
||||
def apply[I,T](p: Initialize[State => Parser[I]])(action: TaskKey[I] => Initialize[Task[T]]): Initialize[InputTask[T]] =
|
||||
{
|
||||
val key = EvaluateTask.parseResult.asInstanceOf[TaskKey[I]]
|
||||
p.zipWith(action(key)) { (parserF, act) =>
|
||||
new InputDynamic[T]
|
||||
{
|
||||
type Result = I
|
||||
def parser = parserF
|
||||
def task = act
|
||||
}
|
||||
}
|
||||
}
|
||||
def apply[I,T](p: State => Parser[I])(action: TaskKey[I] => Initialize[Task[T]]): Initialize[InputTask[T]] =
|
||||
apply(Project.value(p))(action)
|
||||
}
|
||||
|
||||
sealed trait Scoped { def scope: Scope; def fillThis: Boolean; def key: AttributeKey[_] }
|
||||
sealed trait Scoped { def scope: Scope; def key: AttributeKey[_] }
|
||||
sealed trait ScopedTaskable[T] extends Scoped
|
||||
sealed trait ScopedSetting[T] extends ScopedTaskable[T] { def key: AttributeKey[T] }
|
||||
sealed trait ScopedTask[T] extends ScopedTaskable[T] { def key: AttributeKey[Task[T]] }
|
||||
sealed trait ScopedInput[T] extends Scoped { def key: AttributeKey[InputTask[T]]; def fillThis = false }
|
||||
sealed trait ScopedInput[T] extends Scoped { def key: AttributeKey[InputTask[T]] }
|
||||
|
||||
sealed trait Key[T] extends Scoped { final def scope: Scope = Scope(This,This,This,This) }
|
||||
final class SettingKey[T] private(val key: AttributeKey[T], val fillThis: Boolean) extends Key[T] with ScopedSetting[T]
|
||||
final class TaskKey[T] private(val key: AttributeKey[Task[T]], val fillThis: Boolean) extends Key[T] with ScopedTask[T]
|
||||
final class SettingKey[T] private(val key: AttributeKey[T]) extends Key[T] with ScopedSetting[T]
|
||||
final class TaskKey[T] private(val key: AttributeKey[Task[T]]) extends Key[T] with ScopedTask[T]
|
||||
final class InputKey[T] private(val key: AttributeKey[InputTask[T]]) extends Key[InputTask[T]] with ScopedInput[T]
|
||||
|
||||
object Scoped
|
||||
{
|
||||
implicit def richSettingScoped[T](s: ScopedSetting[T]): RichSettingScoped[T] = new RichSettingScoped[T](s.scope, s.key, s.fillThis)
|
||||
implicit def richTaskScoped[T](s: ScopedTask[T]): RichTaskScoped[T] = new RichTaskScoped[T](s.scope, s.key, s.fillThis)
|
||||
implicit def richSettingScoped[T](s: ScopedSetting[T]): RichSettingScoped[T] = new RichSettingScoped[T](s.scope, s.key)
|
||||
implicit def richTaskScoped[T](s: ScopedTask[T]): RichTaskScoped[T] = new RichTaskScoped[T](s.scope, s.key)
|
||||
implicit def richInputScoped[T](s: ScopedInput[T]): RichInputScoped[T] = new RichInputScoped[T](s.scope, s.key)
|
||||
implicit def richSettingListScoped[T](s: ScopedSetting[Seq[T]]): RichSettingList[T] = new RichSettingList[T](s.scope, s.key, s.fillThis)
|
||||
implicit def richListTaskScoped[T](s: ScopedTask[Seq[T]]): RichListTask[T] = new RichListTask[T](s.scope, s.key, s.fillThis)
|
||||
implicit def richSettingListScoped[T](s: ScopedSetting[Seq[T]]): RichSettingList[T] = new RichSettingList[T](s.scope, s.key)
|
||||
implicit def richListTaskScoped[T](s: ScopedTask[Seq[T]]): RichListTask[T] = new RichListTask[T](s.scope, s.key)
|
||||
|
||||
implicit def settingScoping[T](s: SettingKey[T]): ScopingSetting[T, ScopedSetting[T]] =
|
||||
new ScopingSetting(s.key, scope => scopedSetting(scope, s.key, s.fillThis))
|
||||
new ScopingSetting(s.key, scope => scopedSetting(scope, s.key))
|
||||
|
||||
implicit def inputScoping[T](s: InputKey[T]): ScopingSetting[InputTask[T], ScopedInput[T]] =
|
||||
new ScopingSetting(s.key, scope => scopedInput(scope, s.key))
|
||||
|
||||
implicit def taskScoping[T](s: TaskKey[T]): ScopingSetting[Task[T], ScopedTask[T]] =
|
||||
new ScopingSetting(s.key, scope => scopedTask(scope, s.key, s.fillThis))
|
||||
new ScopingSetting(s.key, scope => scopedTask(scope, s.key))
|
||||
|
||||
final class ScopingSetting[T, Result](val key: AttributeKey[T], app0: Scope => Result)
|
||||
{
|
||||
|
|
@ -63,61 +97,60 @@ object Scoped
|
|||
def in(p: ScopeAxis[ProjectRef], c: ScopeAxis[ConfigKey], t: ScopeAxis[AttributeKey[_]]): Result = in( Scope(p, c, t, This) )
|
||||
}
|
||||
|
||||
private[this] def scopedSetting[T](s: Scope, k: AttributeKey[T], fb: Boolean): ScopedSetting[T] = new ScopedSetting[T] { val scope = s; val key = k; val fillThis = fb }
|
||||
private[this] def scopedSetting[T](s: Scope, k: AttributeKey[T]): ScopedSetting[T] = new ScopedSetting[T] { val scope = s; val key = k }
|
||||
private[this] def scopedInput[T](s: Scope, k: AttributeKey[InputTask[T]]): ScopedInput[T] = new ScopedInput[T] { val scope = s; val key = k }
|
||||
private[this] def scopedTask[T](s: Scope, k: AttributeKey[Task[T]], fb: Boolean): ScopedTask[T] = new ScopedTask[T] { val scope = s; val key = k; val fillThis = fb }
|
||||
private[this] def scopedTask[T](s: Scope, k: AttributeKey[Task[T]]): ScopedTask[T] = new ScopedTask[T] { val scope = s; val key = k }
|
||||
|
||||
final class RichSettingList[S](scope: Scope, key: AttributeKey[Seq[S]], val fillThis: Boolean)
|
||||
final class RichSettingList[S](scope: Scope, key: AttributeKey[Seq[S]])
|
||||
{
|
||||
def += (value: => S): Setting[Seq[S]] = ++=(value :: Nil)
|
||||
def ++=(values: => Seq[S]): Setting[Seq[S]] = (new RichSettingScoped(scope, key, fillThis)) ~= (_ ++ values )
|
||||
def ++=(values: => Seq[S]): Setting[Seq[S]] = (new RichSettingScoped(scope, key)) ~= (_ ++ values )
|
||||
}
|
||||
final class RichListTask[S](scope: Scope, key: AttributeKey[Task[Seq[S]]], val fillThis: Boolean)
|
||||
final class RichListTask[S](scope: Scope, key: AttributeKey[Task[Seq[S]]])
|
||||
{
|
||||
def += (value: => S): Setting[Task[Seq[S]]] = ++=(value :: Nil)
|
||||
def ++=(values: => Seq[S]): Setting[Task[Seq[S]]] = (new RichTaskScoped(scope, key, fillThis)) ~= (_ ++ values )
|
||||
def ++=(values: => Seq[S]): Setting[Task[Seq[S]]] = (new RichTaskScoped(scope, key)) ~= (_ ++ values )
|
||||
}
|
||||
sealed abstract class RichBaseScoped[S]
|
||||
{
|
||||
def scope: Scope
|
||||
def key: AttributeKey[S]
|
||||
def fillThis: Boolean
|
||||
protected final val scoped = ScopedKey(scope, key)
|
||||
|
||||
final def :==(value: S): Setting[S] = :=(value)
|
||||
final def :==(value: SettingKey[S]): Setting[S] = <<=(value(identity))
|
||||
final def := (value: => S): Setting[S] = Project.value(scoped)(value)
|
||||
final def := (value: => S): Setting[S] = setting(scoped, Project.value(value))
|
||||
final def ~= (f: S => S): Setting[S] = Project.update(scoped)(f)
|
||||
final def <<= (app: Apply[S]): Setting[S] = app toSetting scoped
|
||||
final def <<= (app: Initialize[S]): Setting[S] = setting(scoped, app)
|
||||
|
||||
final def apply[T](f: S => T): Apply[T] = Apply.single(scoped, fillThis)(f)
|
||||
final def apply[T](f: S => T): Initialize[T] = Apply.single(scoped)(f)
|
||||
|
||||
final def get(settings: Settings[Scope]): Option[S] = settings.get(scope, key)
|
||||
}
|
||||
final class RichInputScoped[T](val scope: Scope, val key: AttributeKey[InputTask[T]]) extends RichBaseScoped[InputTask[T]] { def fillThis = false }
|
||||
final class RichSettingScoped[S](val scope: Scope, val key: AttributeKey[S], val fillThis: Boolean) extends RichBaseScoped[S]
|
||||
final class RichInputScoped[T](val scope: Scope, val key: AttributeKey[InputTask[T]]) extends RichBaseScoped[InputTask[T]]
|
||||
final class RichSettingScoped[S](val scope: Scope, val key: AttributeKey[S]) extends RichBaseScoped[S]
|
||||
{
|
||||
def map[T](f: S => T): Apply[Task[T]] = flatMap(s => task(f(s)) )
|
||||
def flatMap[T](f: S => Task[T]): Apply[Task[T]] = Apply.single(scoped, fillThis)(f)
|
||||
def map[T](f: S => T): Initialize[Task[T]] = flatMap(s => task(f(s)) )
|
||||
def flatMap[T](f: S => Task[T]): Initialize[Task[T]] = Apply.single(scoped)(f)
|
||||
}
|
||||
final class RichTaskScoped[S](scope: Scope, key: AttributeKey[Task[S]], fillThis: Boolean)
|
||||
final class RichTaskScoped[S](scope: Scope, key: AttributeKey[Task[S]])
|
||||
{
|
||||
type ScS = Setting[Task[S]]
|
||||
def :==(value: S): ScS = :=(value)
|
||||
def :==(value: Task[S]): ScS = Project.value(scoped)( value )
|
||||
def := (value: => S): ScS = :==(task(value))
|
||||
def :== (v: TaskKey[S]): ScS = Project.app(scoped, ScopedKey(scope, v.key) :^: KNil)(_.head)
|
||||
def ::=(value: Task[S]): ScS = Project.setting(scoped, Project.value( value ))
|
||||
def := (value: => S): ScS = ::=(task(value))
|
||||
def :== (v: TaskKey[S]): ScS = Project.setting(scoped, Project.app(ScopedKey(scope, v.key) :^: KNil)(_.head) )
|
||||
def :== (v: SettingKey[S]): ScS = <<=( v(const))
|
||||
def ~= (f: S => S): ScS = Project.update(scoped)( _ map f )
|
||||
|
||||
def <<= (app: App[S]): ScS = app toSetting scoped
|
||||
def <<= (app: App[S]): ScS = Project.setting(scoped, app)
|
||||
|
||||
def setting: ScopedSetting[Task[S]] = scopedSetting(scope, key, fillThis)
|
||||
def setting: ScopedSetting[Task[S]] = scopedSetting(scope, key)
|
||||
def get(settings: Settings[Scope]): Option[Task[S]] = settings.get(scope, key)
|
||||
|
||||
type App[T] = Apply[Task[T]]
|
||||
type App[T] = Initialize[Task[T]]
|
||||
private[this] def scoped = ScopedKey(scope, key)
|
||||
private[this] def mk[T](onTask: Task[S] => Task[T]): App[T] = Apply.single(scoped, fillThis)(onTask)
|
||||
private[this] def mk[T](onTask: Task[S] => Task[T]): App[T] = Apply.single(scoped)(onTask)
|
||||
|
||||
def flatMapR[T](f: Result[S] => Task[T]): App[T] = mk(_ flatMapR f)
|
||||
def flatMap[T](f: S => Task[T]): App[T] = flatMapR(f compose successM)
|
||||
|
|
@ -133,7 +166,7 @@ object Scoped
|
|||
|
||||
def dependsOn(tasks: ScopedTask[_]*): App[S] =
|
||||
{
|
||||
val in = KCons(scopedTask(scope, key, fillThis), KList.fromList(tasks))
|
||||
val in = KCons(scopedTask(scope, key), KList.fromList(tasks))
|
||||
Apply.tasks(in) { kl =>
|
||||
val KCons(h,t) = KList.kcons(kl)
|
||||
h.dependsOn(t.toList :_*)
|
||||
|
|
@ -144,7 +177,7 @@ object Scoped
|
|||
implicit def richTaskSeq(in: Seq[ScopedTask[_]]): RichTaskSeq = new RichTaskSeq(in)
|
||||
final class RichTaskSeq(keys: Seq[ScopedTask[_]])
|
||||
{
|
||||
def dependOn: Apply[Task[Unit]] = Apply.tasks(KList.fromList(keys)) { kl => nop.dependsOn(kl.toList :_*) }
|
||||
def dependOn: Initialize[Task[Unit]] = Apply.tasks(KList.fromList(keys)) { kl => nop.dependsOn(kl.toList :_*) }
|
||||
}
|
||||
|
||||
/*
|
||||
|
|
@ -179,10 +212,10 @@ object Scoped
|
|||
key match
|
||||
{
|
||||
case ss: ScopedSetting[H] => prependSetting(ss)
|
||||
case st: ScopedTask[H] => prependTask(scopedSetting(st.scope, st.key, st.fillThis))
|
||||
case st: ScopedTask[H] => prependTask(scopedSetting(st.scope, st.key))
|
||||
}
|
||||
|
||||
def combine[D[_],S](c: Combine[D], f: Results[HLv] => D[S]): Apply[Task[S]] =
|
||||
def combine[D[_],S](c: Combine[D], f: Results[HLv] => D[S]): Initialize[Task[S]] =
|
||||
Apply(settings)(hls => c(tasks(hls))(hlt => f(expand(hls, hlt))) )
|
||||
}
|
||||
type RedHL[HL <: HList] = Reduced[_,_,HL]
|
||||
|
|
@ -224,7 +257,7 @@ object Scoped
|
|||
|
||||
sealed abstract class RichTaskables[In <: HList](keys: KList[ScopedTaskable, In])
|
||||
{
|
||||
type App[T] = Apply[Task[T]]
|
||||
type App[T] = Initialize[Task[T]]
|
||||
type Fun[M[_],Ret]
|
||||
protected def convertH[Ret](f: Fun[Id,Ret]): In => Ret
|
||||
protected def convertK[M[_],Ret](f: Fun[M,Ret]): KList[M,In] => Ret
|
||||
|
|
@ -290,7 +323,7 @@ object Scoped
|
|||
implicit def richTaskableKeys[HL <: HList](in: KList[ScopedTaskable, HL]): RichTaskableKeys[HL] = new RichTaskableKeys(in)
|
||||
final class RichTaskableKeys[In <: HList](keys: KList[ScopedTaskable, In])
|
||||
{
|
||||
type App[T] = Apply[Task[T]]
|
||||
type App[T] = Initialize[Task[T]]
|
||||
private[this] val red = reduced(keys)
|
||||
|
||||
def flatMap[T](f: In => Task[T]): App[T] = flatMapR(f compose allM)
|
||||
|
|
@ -305,27 +338,23 @@ object Scoped
|
|||
implicit def t3ToApp3[A,B,C](t3: (ScopedSetting[A], ScopedSetting[B], ScopedSetting[C]) ): Apply3[A,B,C] = new Apply3(t3)
|
||||
implicit def t4ToApp4[A,B,C,D](t4: (ScopedSetting[A], ScopedSetting[B], ScopedSetting[C], ScopedSetting[D]) ): Apply4[A,B,C,D] = new Apply4(t4)
|
||||
implicit def t5ToApp5[A,B,C,D,E](t5: (ScopedSetting[A], ScopedSetting[B], ScopedSetting[C], ScopedSetting[D], ScopedSetting[E]) ): Apply5[A,B,C,D,E] = new Apply5(t5)
|
||||
|
||||
final class Apply[T] private(val toSetting: ScopedKey[T] => Setting[T])
|
||||
implicit def t6ToApp6[A,B,C,D,E,F](t6: (ScopedSetting[A], ScopedSetting[B], ScopedSetting[C], ScopedSetting[D], ScopedSetting[E], ScopedSetting[F]) ): Apply6[A,B,C,D,E,F] = new Apply6(t6)
|
||||
implicit def t7ToApp7[A,B,C,D,E,F,G](t7: (ScopedSetting[A], ScopedSetting[B], ScopedSetting[C], ScopedSetting[D], ScopedSetting[E], ScopedSetting[F], ScopedSetting[G]) ): Apply7[A,B,C,D,E,F,G] = new Apply7(t7)
|
||||
|
||||
object Apply
|
||||
{
|
||||
def single[I,T](in: ScopedKey[I], fillThis: Boolean)(f: I => T): Apply[T] =
|
||||
mk(scoped => fill(scoped, in, fillThis) :^: KNil)(hl => f(hl.head))
|
||||
private[this] def mk[H <: HList, T](in: ScopedKey[_] => KList[ScopedKey, H])(f: H => T): Apply[T] =
|
||||
new Apply[T](scoped => Project.app(scoped, in(scoped))(f) )
|
||||
|
||||
def apply[HL <: HList, T](in: KList[ScopedSetting, HL])(f: HL => T): Apply[T] = mk(scoped => in transform ssToSK(scoped))(f)
|
||||
def tasks[HL <: HList, T](in: KList[ScopedTask, HL])(f: KList[Task, HL] => T): Apply[T] =
|
||||
def single[I,T](in: ScopedKey[I])(f: I => T): Initialize[T] =
|
||||
Project.app(in :^: KNil)(hl => f(hl.head))
|
||||
|
||||
def apply[HL <: HList, T](in: KList[ScopedSetting, HL])(f: HL => T): Initialize[T] =
|
||||
Project.app(in transform ssToSK)(f)
|
||||
def tasks[HL <: HList, T](in: KList[ScopedTask, HL])(f: KList[Task, HL] => T): Initialize[T] =
|
||||
{
|
||||
val kapp = new Project.KApp[HL, Task, T]
|
||||
new Apply[T](scoped => kapp(scoped, in transform stToSK(scoped))(f) )
|
||||
kapp(in transform stToSK)(f)
|
||||
}
|
||||
|
||||
private def fill[I](scoped: ScopedKey[_], in: ScopedKey[I], fillThis: Boolean): ScopedKey[I] =
|
||||
if(fillThis) Project.resolveThis(Scope.fillTaskAxis(scoped.scope,scoped.key))(in) else in
|
||||
private def ssToSK(scoped: ScopedKey[_]) = new (ScopedSetting ~> ScopedKey) { def apply[T](sk: ScopedSetting[T]) = fill(scoped, new ScopedKey(sk.scope, sk.key), sk.fillThis) }
|
||||
private def stToSK(scoped: ScopedKey[_]) = new (ScopedTask ~> ScopedTaskKey) { def apply[T](st: ScopedTask[T]) = fill(scoped, new ScopedKey(st.scope, st.key), st.fillThis) }
|
||||
private val ssToSK = new (ScopedSetting ~> ScopedKey) { def apply[T](sk: ScopedSetting[T]) = new ScopedKey(sk.scope, sk.key) }
|
||||
private val stToSK = new (ScopedTask ~> ScopedTaskKey) { def apply[T](st: ScopedTask[T]) = new ScopedKey(st.scope, st.key) }
|
||||
private type ScopedTaskKey[T] = ScopedKey[Task[T]]
|
||||
}
|
||||
|
||||
|
|
@ -345,6 +374,14 @@ object Scoped
|
|||
def apply[T](f: (A,B,C,D,E) => T) =
|
||||
Apply(t5._1 :^: t5._2 :^: t5._3 :^: t5._4 :^: t5._5 :^: KNil){ case a :+: b :+: c :+: d :+: e :+: HNil => f(a,b,c,d,e) }
|
||||
}
|
||||
final class Apply6[A,B,C,D,E,F](t6: (ScopedSetting[A], ScopedSetting[B], ScopedSetting[C], ScopedSetting[D], ScopedSetting[E], ScopedSetting[F])) {
|
||||
def apply[T](z: (A,B,C,D,E,F) => T) =
|
||||
Apply(t6._1 :^: t6._2 :^: t6._3 :^: t6._4 :^: t6._5 :^: t6._6 :^: KNil){ case a :+: b :+: c :+: d :+: e :+: f :+: HNil => z(a,b,c,d,e, f) }
|
||||
}
|
||||
final class Apply7[A,B,C,D,E,F,G](t7: (ScopedSetting[A], ScopedSetting[B], ScopedSetting[C], ScopedSetting[D], ScopedSetting[E], ScopedSetting[F], ScopedSetting[G])) {
|
||||
def apply[T](z: (A,B,C,D,E,F,G) => T) =
|
||||
Apply(t7._1 :^: t7._2 :^: t7._3 :^: t7._4 :^: t7._5 :^: t7._6 :^: t7._7 :^: KNil){ case a :+: b :+: c :+: d :+: e :+: f :+: g :+: HNil => z(a,b,c,d,e,f,g) }
|
||||
}
|
||||
|
||||
/*def unresolved(scope: Scope): Seq[String] = unresolvedProject(scope.project) ++ unresolvedThis(scope)
|
||||
def unresolvedProject(ps: ScopeAxis[ProjectRef]): Seq[String] = ps match {
|
||||
|
|
@ -371,17 +408,17 @@ object InputKey
|
|||
}
|
||||
object TaskKey
|
||||
{
|
||||
def apply[T](label: String, fillThis: Boolean = false): TaskKey[T] =
|
||||
apply( AttributeKey[Task[T]](label), fillThis )
|
||||
def apply[T](label: String): TaskKey[T] =
|
||||
apply( AttributeKey[Task[T]](label) )
|
||||
|
||||
def apply[T](akey: AttributeKey[Task[T]], fillThis: Boolean): TaskKey[T] =
|
||||
new TaskKey[T](akey, fillThis)
|
||||
def apply[T](akey: AttributeKey[Task[T]]): TaskKey[T] =
|
||||
new TaskKey[T](akey)
|
||||
}
|
||||
object SettingKey
|
||||
{
|
||||
def apply[T](label: String, fillThis: Boolean = false): SettingKey[T] =
|
||||
apply( AttributeKey[T](label), fillThis )
|
||||
def apply[T](label: String): SettingKey[T] =
|
||||
apply( AttributeKey[T](label) )
|
||||
|
||||
def apply[T](akey: AttributeKey[T], fillThis: Boolean): SettingKey[T] =
|
||||
new SettingKey[T](akey, fillThis)
|
||||
def apply[T](akey: AttributeKey[T]): SettingKey[T] =
|
||||
new SettingKey[T](akey)
|
||||
}
|
||||
|
|
@ -13,5 +13,5 @@ abstract class BasicProject
|
|||
def javapCompiledTask(conf: Configuration): Task[Unit] =
|
||||
javapTask(taskData(fullClasspath(conf)), buildScalaInstance)
|
||||
|
||||
// lazy val test-only, test-quick, test-failed, javap, javap-quick, jetty-{run,stop,restart}, prepare-webapp, watch paths, Ivy settings loaded and printed
|
||||
// lazy val test-quick, test-failed, javap, javap-quick, jetty-{run,stop,restart}, prepare-webapp, watch paths
|
||||
}
|
||||
|
|
|
|||
|
|
@ -47,17 +47,19 @@ trait Init[Scope]
|
|||
type ScopedMap = IMap[ScopedKey, SettingSeq]
|
||||
type CompiledMap = Map[ScopedKey[_], Compiled]
|
||||
type MapScoped = ScopedKey ~> ScopedKey
|
||||
type ScopeLocal = ScopedKey[_] => Seq[Setting[_]]
|
||||
|
||||
def value[T](key: ScopedKey[T])(value: => T): Setting[T] = new Value(key, value _)
|
||||
def update[T](key: ScopedKey[T])(f: T => T): Setting[T] = app(key, key :^: KNil)(h => f(h.head))
|
||||
def app[HL <: HList, T](key: ScopedKey[T], inputs: KList[ScopedKey, HL])(f: HL => T): Setting[T] = new Apply(key, f, inputs)
|
||||
def uniform[S,T](key: ScopedKey[T], inputs: Seq[ScopedKey[S]])(f: Seq[S] => T): Setting[T] = new Uniform(key, f, inputs)
|
||||
def kapp[HL <: HList, M[_], T](key: ScopedKey[T], inputs: KList[({type l[t] = ScopedKey[M[t]]})#l, HL])(f: KList[M, HL] => T): Setting[T] = new KApply[HL, M, T](key, f, inputs)
|
||||
def setting[T](key: ScopedKey[T], init: Initialize[T]): Setting[T] = new Setting[T](key, init)
|
||||
def value[T](value: => T): Initialize[T] = new Value(value _)
|
||||
def update[T](key: ScopedKey[T])(f: T => T): Setting[T] = new Setting[T](key, app(key :^: KNil)(hl => f(hl.head)))
|
||||
def app[HL <: HList, T](inputs: KList[ScopedKey, HL])(f: HL => T): Initialize[T] = new Apply(f, inputs)
|
||||
def uniform[S,T](inputs: Seq[ScopedKey[S]])(f: Seq[S] => T): Initialize[T] = new Uniform(f, inputs)
|
||||
def kapp[HL <: HList, M[_], T](inputs: KList[({type l[t] = ScopedKey[M[t]]})#l, HL])(f: KList[M, HL] => T): Initialize[T] = new KApply[HL, M, T](f, inputs)
|
||||
|
||||
// the following is a temporary workaround for the "... cannot be instantiated from ..." bug, which renders 'kapp' above unusable outside this source file
|
||||
class KApp[HL <: HList, M[_], T] {
|
||||
type Composed[S] = ScopedKey[M[S]]
|
||||
def apply(key: ScopedKey[T], inputs: KList[Composed, HL])(f: KList[M, HL] => T): Setting[T] = new KApply[HL, M, T](key, f, inputs)
|
||||
def apply(inputs: KList[Composed, HL])(f: KList[M, HL] => T): Initialize[T] = new KApply[HL, M, T](f, inputs)
|
||||
}
|
||||
|
||||
def empty(implicit delegates: Scope => Seq[Scope]): Settings[Scope] = new Settings0(Map.empty, delegates)
|
||||
|
|
@ -67,7 +69,7 @@ trait Init[Scope]
|
|||
def getValue[T](s: Settings[Scope], k: ScopedKey[T]) = s.get(k.scope, k.key).get
|
||||
def asFunction[T](s: Settings[Scope]): ScopedKey[T] => T = k => getValue(s, k)
|
||||
|
||||
def compiled(init: Seq[Setting[_]])(implicit delegates: Scope => Seq[Scope], scopeLocal: ScopedKey[_] => Seq[Setting[_]]): CompiledMap =
|
||||
def compiled(init: Seq[Setting[_]])(implicit delegates: Scope => Seq[Scope], scopeLocal: ScopeLocal): CompiledMap =
|
||||
{
|
||||
// prepend per-scope settings
|
||||
val withLocal = addLocal(init)(scopeLocal)
|
||||
|
|
@ -78,7 +80,7 @@ trait Init[Scope]
|
|||
// merge Seq[Setting[_]] into Compiled
|
||||
compile(dMap)
|
||||
}
|
||||
def make(init: Seq[Setting[_]])(implicit delegates: Scope => Seq[Scope], scopeLocal: ScopedKey[_] => Seq[Setting[_]]): Settings[Scope] =
|
||||
def make(init: Seq[Setting[_]])(implicit delegates: Scope => Seq[Scope], scopeLocal: ScopeLocal): Settings[Scope] =
|
||||
{
|
||||
val cMap = compiled(init)(delegates, scopeLocal)
|
||||
// order the initializations. cyclic references are detected here.
|
||||
|
|
@ -105,7 +107,7 @@ trait Init[Scope]
|
|||
def append[T](ss: Seq[Setting[T]], s: Setting[T]): Seq[Setting[T]] =
|
||||
if(s.definitive) s :: Nil else ss :+ s
|
||||
|
||||
def addLocal(init: Seq[Setting[_]])(implicit scopeLocal: ScopedKey[_] => Seq[Setting[_]]): Seq[Setting[_]] =
|
||||
def addLocal(init: Seq[Setting[_]])(implicit scopeLocal: ScopeLocal): Seq[Setting[_]] =
|
||||
init.flatMap( _.dependsOn flatMap scopeLocal ) ++ init
|
||||
|
||||
def delegate(sMap: ScopedMap)(implicit delegates: Scope => Seq[Scope]): ScopedMap =
|
||||
|
|
@ -138,15 +140,9 @@ trait Init[Scope]
|
|||
|
||||
private[this] def applySetting[T](map: Settings[Scope], setting: Setting[T]): Settings[Scope] =
|
||||
{
|
||||
def execK[HL <: HList, M[_]](a: KApply[HL, M, T]) =
|
||||
map.set(a.key.scope, a.key.key, a.f(a.inputs.transform[M]( nestCon[ScopedKey, Id, M](asTransform(map)) )) )
|
||||
setting match
|
||||
{
|
||||
case s: Value[T] => map.set(s.key.scope, s.key.key, s.value())
|
||||
case u: Uniform[s, T] => map.set(u.key.scope, u.key.key, u.f(u.inputs map asFunction(map)) )
|
||||
case a: Apply[hl, T] => map.set(a.key.scope, a.key.key, a.f(a.inputs down asTransform(map) ) )
|
||||
case ka: KApply[hl, m, T] => execK[hl, m](ka) // separate method needed to workaround bug where m is not recognized as higher-kinded in inline version
|
||||
}
|
||||
val value = setting.init.get(map)
|
||||
val key = setting.key
|
||||
map.set(key.scope, key.key, value)
|
||||
}
|
||||
|
||||
final class Uninitialized(val key: ScopedKey[_], val refKey: ScopedKey[_], msg: String) extends Exception(msg)
|
||||
|
|
@ -154,42 +150,58 @@ trait Init[Scope]
|
|||
new Uninitialized(key, refKey, "Reference to uninitialized setting " + key.key.label + " (in " + key.scope + ") from " + refKey.key.label +" (in " + refKey.scope + ")")
|
||||
final class Compiled(val dependencies: Iterable[ScopedKey[_]], val eval: Settings[Scope] => Settings[Scope])
|
||||
|
||||
sealed trait Setting[T]
|
||||
sealed trait Initialize[T]
|
||||
{
|
||||
def key: ScopedKey[T]
|
||||
def definitive: Boolean
|
||||
def dependsOn: Seq[ScopedKey[_]]
|
||||
def mapReferenced(g: MapScoped): Setting[T]
|
||||
def mapKey(g: MapScoped): Setting[T]
|
||||
def map[S](g: T => S): Initialize[S]
|
||||
def mapReferenced(g: MapScoped): Initialize[T]
|
||||
def zip[S](o: Initialize[S]): Initialize[(T,S)] = zipWith(o)((x,y) => (x,y))
|
||||
def zipWith[S,U](o: Initialize[S])(f: (T,S) => U): Initialize[U] = new Joined[T,S,U](this, o, f)
|
||||
def get(map: Settings[Scope]): T
|
||||
}
|
||||
private[this] final class Value[T](val key: ScopedKey[T], val value: () => T) extends Setting[T]
|
||||
final class Setting[T](val key: ScopedKey[T], val init: Initialize[T])
|
||||
{
|
||||
def definitive: Boolean = !init.dependsOn.contains(key)
|
||||
def dependsOn: Seq[ScopedKey[_]] = remove(init.dependsOn, key)
|
||||
def mapReferenced(g: MapScoped): Setting[T] = new Setting(key, init mapReferenced g)
|
||||
def mapKey(g: MapScoped): Setting[T] = new Setting(g(key), init)
|
||||
}
|
||||
|
||||
private[this] final class Joined[S,T,U](a: Initialize[S], b: Initialize[T], f: (S,T) => U) extends Initialize[U]
|
||||
{
|
||||
def dependsOn = a.dependsOn ++ b.dependsOn
|
||||
def mapReferenced(g: MapScoped) = new Joined(a mapReferenced g, b mapReferenced g, f)
|
||||
def map[Z](g: U => Z) = new Joined[S,T,Z](a, b, (s,t) => g(f(s,t)))
|
||||
def get(map: Settings[Scope]): U = f(a get map, b get map)
|
||||
}
|
||||
private[this] final class Value[T](value: () => T) extends Initialize[T]
|
||||
{
|
||||
def definitive = true
|
||||
def dependsOn = Nil
|
||||
def mapReferenced(g: MapScoped) = this
|
||||
def mapKey(g: MapScoped): Setting[T] = new Value(g(key), value)
|
||||
def map[S](g: T => S) = new Value[S](() => g(value()))
|
||||
def get(map: Settings[Scope]): T = value()
|
||||
}
|
||||
private[this] final class Apply[HL <: HList, T](val key: ScopedKey[T], val f: HL => T, val inputs: KList[ScopedKey, HL]) extends Setting[T]
|
||||
private[this] final class Apply[HL <: HList, T](val f: HL => T, val inputs: KList[ScopedKey, HL]) extends Initialize[T]
|
||||
{
|
||||
def definitive = !inputs.toList.contains(key)
|
||||
def dependsOn = remove(inputs.toList, key)
|
||||
def mapReferenced(g: MapScoped) = new Apply(key, f, inputs transform g)
|
||||
def mapKey(g: MapScoped): Setting[T] = new Apply(g(key), f, inputs)
|
||||
def dependsOn = inputs.toList
|
||||
def mapReferenced(g: MapScoped) = new Apply(f, inputs transform g)
|
||||
def map[S](g: T => S) = new Apply(g compose f, inputs)
|
||||
def get(map: Settings[Scope]) = f(inputs down asTransform(map) )
|
||||
}
|
||||
private[this] final class KApply[HL <: HList, M[_], T](val key: ScopedKey[T], val f: KList[M, HL] => T, val inputs: KList[({type l[t] = ScopedKey[M[t]]})#l, HL]) extends Setting[T]
|
||||
private[this] final class KApply[HL <: HList, M[_], T](val f: KList[M, HL] => T, val inputs: KList[({type l[t] = ScopedKey[M[t]]})#l, HL]) extends Initialize[T]
|
||||
{
|
||||
def definitive = !inputs.toList.contains(key)
|
||||
def dependsOn = remove(unnest(inputs.toList), key)
|
||||
def mapReferenced(g: MapScoped) = new KApply[HL, M, T](key, f, inputs.transform[({type l[t] = ScopedKey[M[t]]})#l]( nestCon(g) ) )
|
||||
def mapKey(g: MapScoped): Setting[T] = new KApply[HL, M, T](g(key), f, inputs)
|
||||
def dependsOn = unnest(inputs.toList)
|
||||
def mapReferenced(g: MapScoped) = new KApply[HL, M, T](f, inputs.transform[({type l[t] = ScopedKey[M[t]]})#l]( nestCon(g) ) )
|
||||
def map[S](g: T => S) = new KApply[HL, M, S](g compose f, inputs)
|
||||
def get(map: Settings[Scope]) = f(inputs.transform[M]( nestCon[ScopedKey, Id, M](asTransform(map)) ))
|
||||
private[this] def unnest(l: List[ScopedKey[M[T]] forSome { type T }]): List[ScopedKey[_]] = l.asInstanceOf[List[ScopedKey[_]]]
|
||||
}
|
||||
private[this] final class Uniform[S, T](val key: ScopedKey[T], val f: Seq[S] => T, val inputs: Seq[ScopedKey[S]]) extends Setting[T]
|
||||
private[this] final class Uniform[S, T](val f: Seq[S] => T, val inputs: Seq[ScopedKey[S]]) extends Initialize[T]
|
||||
{
|
||||
def definitive = !inputs.contains(key)
|
||||
def dependsOn = remove(inputs, key)
|
||||
def mapReferenced(g: MapScoped) = new Uniform(key, f, inputs map g.fn[S])
|
||||
def mapKey(g: MapScoped): Setting[T] = new Uniform(g(key), f, inputs)
|
||||
def dependsOn = inputs
|
||||
def mapReferenced(g: MapScoped) = new Uniform(f, inputs map g.fn[S])
|
||||
def map[S](g: T => S) = new Uniform(g compose f, inputs)
|
||||
def get(map: Settings[Scope]) = f(inputs map asFunction(map))
|
||||
}
|
||||
private def remove[T](s: Seq[T], v: T) = s filterNot (_ == v)
|
||||
}
|
||||
|
|
|
|||
Loading…
Reference in New Issue