Skip to content

Commit

Permalink
More cleanups for Evaluator API (#4594)
Browse files Browse the repository at this point in the history
Fixes #4094


- Inlined `EvaluatorPathsResolver` since it only has one implementation
- Renamed `Evaluator#resolveEvaluate` into just `def evaluate`,
consolidated three `def evaluate` and `def evaluateTask` methods into
`def execute
- Privatize `Evaluator#execution` to force all code paths to go through
its public APIs
- Doccomments for all exposed `Evaluator` operations
- Wrapped `(Seq[Watchable], Result[Seq[(Val, Option[(Evaluator.TaskName,
ujson.Value)])]])` in `Evaluator.Result[T]`
- Move `namesAndJson` output from `Evaluator` to `def show`, which is
where we actually use it
  • Loading branch information
lihaoyi authored Feb 20, 2025
1 parent fb7ad6d commit 038a3c3
Show file tree
Hide file tree
Showing 44 changed files with 409 additions and 390 deletions.
19 changes: 9 additions & 10 deletions bsp/worker/src/mill/bsp/worker/MillBuildServer.scala
Original file line number Diff line number Diff line change
@@ -1,17 +1,15 @@
package mill.bsp.worker

import ch.epfl.scala.bsp4j
import ch.epfl.scala.bsp4j._
import ch.epfl.scala.bsp4j.*
import com.google.gson.JsonObject

import mill.api.ExecResult
import mill.api.{ColorLogger, CompileProblemReporter, DummyTestReporter, Result, TestReporter}
import mill.bsp.{BspServerResult, Constants}
import mill.bsp.worker.Utils.{makeBuildTarget, outputPaths, sanitizeUri}
import mill.define.Segment.Label
import mill.define.{Args, Discover, ExternalModule, NamedTask, Task}
import mill.define.{Args, Discover, ExecutionResults, ExternalModule, NamedTask, Task, TaskResult}
import mill.eval.Evaluator
import mill.exec.{ExecResults, TaskResult}
import mill.main.MainModule
import mill.runner.MillBuildRootModule
import mill.scalalib.bsp.{BspModule, JvmBuildTarget, ScalaBuildTarget}
Expand All @@ -21,7 +19,7 @@ import mill.given
import java.io.PrintStream
import java.util.concurrent.CompletableFuture
import scala.concurrent.Promise
import scala.jdk.CollectionConverters._
import scala.jdk.CollectionConverters.*
import scala.reflect.ClassTag
import scala.util.chaining.scalaUtilChainingOps
import scala.util.control.NonFatal
Expand Down Expand Up @@ -267,7 +265,7 @@ private class MillBuildServer(
}.toSeq

val ids = groupList(tasksEvaluators)(_._2)(_._1)
.flatMap { case (ev, ts) => ev.evaluateValues(ts) }
.flatMap { case (ev, ts) => ev.execute(ts).values.get }
.flatten

new InverseSourcesResult(ids.asJava)
Expand Down Expand Up @@ -613,7 +611,8 @@ private class MillBuildServer(
false
)
else {
val outPaths = ev.pathsResolver.resolveDest(
val outPaths = mill.define.ExecutionPaths.resolve(
ev.outPath,
module.moduleSegments ++ Label("compile")
)
val outPathSeq = Seq(outPaths.dest, outPaths.meta, outPaths.log)
Expand Down Expand Up @@ -804,7 +803,7 @@ private class MillBuildServer(
reporter: Int => Option[CompileProblemReporter] = _ => Option.empty[CompileProblemReporter],
testReporter: TestReporter = DummyTestReporter,
logger: ColorLogger = null
): ExecResults = {
): ExecutionResults = {
val logger0 = Option(logger).getOrElse(evaluator.baseLogger)
mill.runner.MillMain.withOutLock(
noBuildLock = false,
Expand All @@ -816,13 +815,13 @@ private class MillBuildServer(
},
streams = logger0.systemStreams
) {
evaluator.execution.executeTasks(
evaluator.execute(
goals,
reporter,
testReporter,
logger0,
serialCommandExec = false
)
).executionResults
}
}
}
Expand Down
5 changes: 2 additions & 3 deletions bsp/worker/src/mill/bsp/worker/MillJavaBuildServer.scala
Original file line number Diff line number Diff line change
Expand Up @@ -37,15 +37,14 @@ private trait MillJavaBuildServer extends JavaBuildServer { this: MillBuildServe
) {
// We ignore all non-JavaModule
case (ev, state, id, m: JavaModule, (classesPath, javacOptions, bspCompileClasspath)) =>
val pathResolver = ev.pathsResolver
val options = javacOptions
val classpath =
bspCompileClasspath.map(_.resolve(pathResolver)).map(sanitizeUri)
bspCompileClasspath.map(_.resolve(ev.outPath)).map(sanitizeUri)
new JavacOptionsItem(
id,
options.asJava,
classpath.iterator.toSeq.asJava,
sanitizeUri(classesPath.resolve(pathResolver))
sanitizeUri(classesPath.resolve(ev.outPath))
)

case _ => ???
Expand Down
3 changes: 1 addition & 2 deletions bsp/worker/src/mill/bsp/worker/MillJvmBuildServer.scala
Original file line number Diff line number Diff line change
Expand Up @@ -137,12 +137,11 @@ private trait MillJvmBuildServer extends JvmBuildServer { this: MillBuildServer
}
) {
case (ev, _, id, _: JavaModule, compileClasspath) =>
val pathResolver = ev.pathsResolver

new JvmCompileClasspathItem(
id,
compileClasspath.iterator
.map(_.resolve(pathResolver))
.map(_.resolve(ev.outPath))
.map(sanitizeUri).toSeq.asJava
)
case _ => ???
Expand Down
5 changes: 2 additions & 3 deletions bsp/worker/src/mill/bsp/worker/MillScalaBuildServer.scala
Original file line number Diff line number Diff line change
Expand Up @@ -66,14 +66,13 @@ private trait MillScalaBuildServer extends ScalaBuildServer { this: MillBuildSer
m: JavaModule,
(allScalacOptions, compileClasspath, classesPathTask)
) =>
val pathResolver = ev.pathsResolver
new ScalacOptionsItem(
id,
allScalacOptions.asJava,
compileClasspath.iterator
.map(_.resolve(pathResolver))
.map(_.resolve(ev.outPath))
.map(sanitizeUri).toSeq.asJava,
sanitizeUri(classesPathTask.resolve(pathResolver))
sanitizeUri(classesPathTask.resolve(ev.outPath))
)
case _ => ???
} {
Expand Down
8 changes: 4 additions & 4 deletions bsp/worker/src/mill/bsp/worker/Utils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -12,11 +12,11 @@ import ch.epfl.scala.bsp4j.{
}
import mill.api.{CompileProblemReporter, PathRef}
import mill.api.ExecResult.{Skipped, Success}
import mill.exec.ExecResults
import mill.define.ExecutionResults
import mill.scalalib.JavaModule
import mill.scalalib.bsp.{BspBuildTarget, BspModule}

import scala.jdk.CollectionConverters._
import scala.jdk.CollectionConverters.*
import scala.util.chaining.scalaUtilChainingOps

private object Utils {
Expand Down Expand Up @@ -50,7 +50,7 @@ private object Utils {
}

// Get the execution status code given the results from Evaluator.evaluate
def getStatusCode(resultsLists: Seq[ExecResults]): StatusCode = {
def getStatusCode(resultsLists: Seq[ExecutionResults]): StatusCode = {
val statusCodes =
resultsLists.flatMap(r => r.results.keys.map(task => getStatusCodePerTask(r, task)).toSeq)
if (statusCodes.contains(StatusCode.ERROR)) StatusCode.ERROR
Expand Down Expand Up @@ -107,7 +107,7 @@ private object Utils {
}

private def getStatusCodePerTask(
results: ExecResults,
results: ExecutionResults,
task: mill.define.Task[?]
): StatusCode = {
results.results(task).result match {
Expand Down
4 changes: 2 additions & 2 deletions contrib/jmh/test/src/mill/contrib/jmh/JmhModuleTest.scala
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@ package mill
package contrib.jmh

import mill.define.Discover
import mill.exec.ExecutionPaths
import mill.define.ExecutionPaths
import mill.scalalib.ScalaModule
import mill.testkit.UnitTester
import mill.testkit.TestBaseModule
Expand All @@ -23,7 +23,7 @@ object JmhModuleTest extends TestSuite {
def tests = Tests {
test("jmh") {
test("listJmhBenchmarks") - UnitTester(jmh, testModuleSourcesPath).scoped { eval =>
val paths = ExecutionPaths.resolveDestPaths(eval.outPath, jmh.listJmhBenchmarks())
val paths = ExecutionPaths.resolve(eval.outPath, jmh.listJmhBenchmarks())
val outFile = paths.dest / "benchmarks.out"
val Right(result) = eval(jmh.listJmhBenchmarks("-o", outFile.toString)): @unchecked
val expected =
Expand Down
Original file line number Diff line number Diff line change
@@ -1,10 +1,8 @@
package mill.exec
package mill.define

import mill.api.*

import mill.define.*

trait ExecResults {
trait ExecutionResults {
def rawValues: Seq[ExecResult[Val]]
def evaluated: Seq[Task[?]]
def failing: Map[Task[?], Seq[ExecResult.Failing[Val]]]
Expand Down
Original file line number Diff line number Diff line change
@@ -1,7 +1,4 @@
package mill.exec

import mill.api.internal
import mill.define.{NamedTask, Segment, Segments}
package mill.define

import java.util.regex.Matcher

Expand All @@ -12,27 +9,22 @@ object ExecutionPaths {
def apply(dest: os.Path, meta: os.Path, log: os.Path): ExecutionPaths =
new ExecutionPaths(dest, meta, log)

@internal
private[mill] def makeSegmentStrings(segments: Segments): Seq[String] = segments.value.flatMap {
case Segment.Label(s) => Seq(s)
case Segment.Cross(values) => values.map(_.toString)
}
def resolveDestPaths(
workspacePath: os.Path,
def resolve(
outPath: os.Path,
segments: Segments
): ExecutionPaths = {
val segmentStrings = makeSegmentStrings(segments)
val targetPath = workspacePath / segmentStrings.map(sanitizePathSegment)
val segmentStrings = segments.parts
val targetPath = outPath / segmentStrings.map(sanitizePathSegment)
ExecutionPaths(
targetPath / os.up / s"${targetPath.last}.dest",
targetPath / os.up / s"${targetPath.last}.json",
targetPath / os.up / s"${targetPath.last}.log"
)
}
def resolveDestPaths(
workspacePath: os.Path,
def resolve(
outPath: os.Path,
task: NamedTask[?]
): ExecutionPaths = resolveDestPaths(workspacePath, task.ctx.segments)
): ExecutionPaths = resolve(outPath, task.ctx.segments)

// case-insensitive match on reserved names
private val ReservedWinNames =
Expand Down
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
package mill.exec
package mill.define

import mill.api.ExecResult

Expand Down
Original file line number Diff line number Diff line change
@@ -1,8 +1,8 @@
package mill.exec
package mill.define

import utest.*

object EvaluatorPathsTests extends TestSuite {
object ExecutionPathsTests extends TestSuite {

override def tests: Tests = Tests {
test("sanitizedPathSegment") {
Expand Down
53 changes: 24 additions & 29 deletions core/exec/src/mill/exec/Execution.scala
Original file line number Diff line number Diff line change
Expand Up @@ -15,23 +15,23 @@ import scala.concurrent._
* Core logic of evaluating tasks, without any user-facing helper methods
*/
private[mill] case class Execution(
val baseLogger: ColorLogger,
val chromeProfileLogger: ChromeProfileLogger,
val profileLogger: ProfileLogger,
val home: os.Path,
val workspace: os.Path,
val outPath: os.Path,
val externalOutPath: os.Path,
val rootModule: BaseModule,
val classLoaderSigHash: Int,
val classLoaderIdentityHash: Int,
val workerCache: mutable.Map[Segments, (Int, Val)],
val env: Map[String, String],
val failFast: Boolean,
val threadCount: Option[Int],
val methodCodeHashSignatures: Map[String, Int],
val systemExit: Int => Nothing,
val exclusiveSystemStreams: SystemStreams
baseLogger: ColorLogger,
chromeProfileLogger: JsonArrayLogger.ChromeProfile,
profileLogger: JsonArrayLogger.Profile,
home: os.Path,
workspace: os.Path,
outPath: os.Path,
externalOutPath: os.Path,
rootModule: BaseModule,
classLoaderSigHash: Int,
classLoaderIdentityHash: Int,
workerCache: mutable.Map[Segments, (Int, Val)],
env: Map[String, String],
failFast: Boolean,
threadCount: Option[Int],
methodCodeHashSignatures: Map[String, Int],
systemExit: Int => Nothing,
exclusiveSystemStreams: SystemStreams
) extends GroupExecution with AutoCloseable {

def withBaseLogger(newBaseLogger: ColorLogger) = this.copy(baseLogger = newBaseLogger)
Expand All @@ -47,15 +47,15 @@ private[mill] case class Execution(
testReporter: TestReporter = DummyTestReporter,
logger: ColorLogger = baseLogger,
serialCommandExec: Boolean = false
): ExecResults = logger.withPromptUnpaused {
): Execution.Results = logger.withPromptUnpaused {
os.makeDir.all(outPath)

PathRef.validatedPaths.withValue(new PathRef.ValidatedPaths()) {
val ec =
if (effectiveThreadCount == 1) ExecutionContexts.RunNow
else new ExecutionContexts.ThreadPool(effectiveThreadCount)

try evaluate0(goals, logger, reporter, testReporter, ec, serialCommandExec)
try execute0(goals, logger, reporter, testReporter, ec, serialCommandExec)
finally ec.close()
}
}
Expand All @@ -75,14 +75,14 @@ private[mill] case class Execution(
failing
}

private def evaluate0(
private def execute0(
goals: Seq[Task[?]],
logger: ColorLogger,
reporter: Int => Option[CompileProblemReporter] = _ => Option.empty[CompileProblemReporter],
testReporter: TestReporter = DummyTestReporter,
ec: mill.api.Ctx.Fork.Impl,
serialCommandExec: Boolean
): ExecResults = {
): Execution.Results = {
os.makeDir.all(outPath)

val threadNumberer = new ThreadNumberer()
Expand Down Expand Up @@ -260,12 +260,7 @@ private[mill] case class Execution(

Execution.Results(
goals.toIndexedSeq.map(results(_).map(_._1).result),
// result of flatMap may contain non-distinct entries,
// so we manually clean it up before converting to a `Strict.Agg`
// see https://github.com/com-lihaoyi/mill/issues/2958
Seq.from(
finishedOptsMap.values.flatMap(_.toSeq.flatMap(_.newEvaluated)).iterator.distinct
),
finishedOptsMap.values.flatMap(_.toSeq.flatMap(_.newEvaluated)).toSeq,
getFailing(plan.sortedGroups, results).items().map { case (k, v) => (k, v.toSeq) }.toMap,
results.map { case (k, v) => (k, v.map(_._1)) }
)
Expand All @@ -292,10 +287,10 @@ private[mill] object Execution {
}
.toMap
}
case class Results(
private[Execution] case class Results(
rawValues: Seq[ExecResult[Val]],
evaluated: Seq[Task[?]],
failing: Map[Task[?], Seq[ExecResult.Failing[Val]]],
results: Map[Task[?], TaskResult[Val]]
) extends ExecResults
) extends mill.define.ExecutionResults
}
2 changes: 1 addition & 1 deletion core/exec/src/mill/exec/ExecutionLogs.scala
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,7 @@ import mill.internal.SpanningForest
import java.util.concurrent.ConcurrentHashMap
import scala.jdk.CollectionConverters.EnumerationHasAsScala

private[mill] object ExecutionLogs {
private object ExecutionLogs {
def logDependencyTree(
interGroupDeps: Map[Task[?], Seq[Task[?]]],
indexToTerminal: Array[Task[?]],
Expand Down
16 changes: 0 additions & 16 deletions core/exec/src/mill/exec/ExecutionPathsResolver.scala

This file was deleted.

Loading

0 comments on commit 038a3c3

Please sign in to comment.