Tasks

One of Mill’s core abstractions is its Task Graph: this is how Mill defines, orders and caches work it needs to do, and exists independently of any support for building Scala.

Mill target graphs are primarily built using methods and macros defined on mill.define.Target, aliased as T for conciseness:

Task Cheat Sheet

The following table might help you make sense of the small collection of different Task types:

Target Command Source/Input Anonymous Task Persistent Target Worker

Cached to Disk

X

X

JSON Writable

X

X

X

X

JSON Readable

X

X

CLI Runnable

X

X

X

Takes Arguments

X

X

Cached In-Memory

X

The following is a simple self-contained example using Mill to compile Java:

build.sc (download, browse)
import mill._

def mainClass: T[Option[String]] = Some("foo.Foo")

def sources = T.source(millSourcePath / "src")
def resources = T.source(millSourcePath / "resources")

def compile = T {
  val allSources = os.walk(sources().path)
  os.proc("javac", allSources, "-d", T.dest).call()
  PathRef(T.dest)
}

def assembly = T {
  for(p <- Seq(compile(), resources())) os.copy(p.path, T.dest, mergeFolders = true)

  val mainFlags = mainClass().toSeq.flatMap(Seq("-e", _))
  os.proc("jar", "-c", mainFlags, "-f", T.dest / s"assembly.jar", ".")
    .call(cwd = T.dest)

  PathRef(T.dest / s"assembly.jar")
}

This example does not use any of Mill’s builtin support for building Java or Scala projects, and instead builds a pipeline "from scratch" using Mill tasks and javac/jar/java subprocesses. We define T.source folders, plain T{…​} targets that depend on them, and a T.command.

> ./mill show assembly
".../out/assembly.dest/assembly.jar"

> java -jar out/assembly.dest/assembly.jar i am cow
Foo.value: 31337
args: i am cow

> unzip -p out/assembly.dest/assembly.jar foo.txt
My Example Text

When you first evaluate assembly (e.g. via mill assembly at the command line), it will evaluate all the defined targets: mainClass, sources, compile, and assembly.

Subsequent invocations of mill assembly will evaluate only as much as is necessary, depending on what input sources changed:

  • If the files in sources change, it will re-evaluate compile, and assembly

  • If the files in resources change, it will only re-evaluate assembly and use the cached output of compile

Primary Tasks

There are three primary kinds of Tasks that you should care about:

Targets

build.sc (download, browse)
import mill._

def allSources = T {
  os.walk(sources().path)
    .filter(_.ext == "java")
    .map(PathRef(_))
}

def lineCount: T[Int] = T {
  println("Computing line count")
  allSources()
    .map(p => os.read.lines(p.path).size)
    .sum
}

Targets are defined using the def foo = T {…​} syntax, and dependencies on other targets are defined using foo() to extract the value from them. Apart from the foo() calls, the T {…​} block contains arbitrary code that does some work and returns a result.

If a target’s inputs change but its output does not, e.g. someone changes a comment within the source files that doesn’t affect the classfiles, then downstream targets do not re-evaluate. This is determined using the .hashCode of the Target’s return value.

> ./mill show lineCount
Computing line count
16

> ./mill show lineCount # line count already cached, doesn't need to be computed
16

The return-value of targets has to be JSON-serializable via uPickle. You can run targets directly from the command line, or use show if you want to see the JSON content or pipe it to external tools.

Each target, e.g. classFiles, is assigned a T.dest folder e.g. out/classFiles.dest/ on disk as scratch space & to store its output files , and its returned metadata is automatically JSON-serialized and stored at out/classFiles.json. If you want to return a file or a set of files as the result of a Target, write them to disk within your T.dest folder and return a PathRef() that referencing the files or folders you want to return:

def classFiles = T {
  println("Generating classfiles")

  os.proc("javac", allSources().map(_.path), "-d", T.dest)
    .call(cwd = T.dest)

  PathRef(T.dest)
}

def jar = T {
  println("Generating jar")
  os.copy(classFiles().path, T.dest, mergeFolders = true)
  os.copy(resources().path, T.dest, mergeFolders = true)

  os.proc("jar", "-cfe", T.dest / "foo.jar", "foo.Foo", ".").call(cwd = T.dest)

  PathRef(T.dest / "foo.jar")
}
> ./mill jar
Generating classfiles
Generating jar

> ./mill show jar
".../out/jar.dest/foo.jar"

Targets can depend on other targets via the foo() syntax. The graph of inter-dependent targets is evaluated in topological order; that means that the body of a target will not even begin to evaluate if one of its upstream dependencies has failed. Similar, even if the upstream targets is not used in one branch of an if condition, it will get computed regardless before the if condition is even considered.

The following example demonstrates this behavior, with the println defined in def largeFile running even though the largeFile() branch of the if conditional does not get used:

def largeFile = T {
  println("Finding Largest File")
  allSources()
    .map(_.path)
    .filter(_.ext == "java")
    .maxBy(os.read.lines(_).size)
}

def hugeFileName = T{
  if (lineCount() > 999) largeFile().last
  else "<no-huge-file>"
}
> ./mill show lineCount
16

> ./mill show hugeFileName # This still runs `largestFile` even though `lineCount() < 999`
Finding Largest File
"<no-huge-file>"

uPickle comes with built-in support for most Scala primitive types and builtin data structures: tuples, collections, PathRefs, etc. can be returned and automatically serialized/de-serialized as necessary. One notable exception is case classes: if you want return your own case class, you must mark it JSON-serializable by adding the following implicit to its companion object:

case class ClassFileData(totalFileSize: Long, largestFile: String)
object ClassFileData {
  implicit val rw: upickle.default.ReadWriter[ClassFileData] = upickle.default.macroRW
}

def summarizeClassFileStats = T{
  val files = os.walk(classFiles().path)
  ClassFileData(
    totalFileSize = files.map(os.size(_)).sum,
    largestFile = files.maxBy(os.size(_)).last
  )
}
> ./mill show summarizeClassFileStats
{
  "totalFileSize": ...,
  "largestFile": "..."
}

Sources

def sources = T.source { millSourcePath / "src" }
def resources = T.source { millSourcePath / "resources" }

Sources are defined using T.sources {…​}, taking one-or-more os.Paths as arguments. A Source is a subclass of Target[Seq[PathRef]]: this means that its build signature/hashCode depends not just on the path it refers to (e.g. foo/bar/baz) but also the MD5 hash of the filesystem tree under that path.

T.source and T.sources are the most common inputs to your Mill build: they watch source files and folders and cause downstream targets to re-compute if a change is detected.

Note that even though a source file changed, that does not necessarily cause all transitive downstream targets to re-compute:

> ./mill jar # Cached from earlier

> printf "\n" >> src/Foo.java # Add a newline to the end of Foo.java

> ./mill jar # Classfiles recompiled but output unchanged, jar was not rebuilt
Generating classfiles

T.sources can be overriden with super, to let you override-and-extend source lists the same way you would any other target definition:

trait Foo extends Module {
  def sourceRoots = T.sources(millSourcePath / "src")
  def sourceContents = T{
    sourceRoots()
      .flatMap(pref => os.walk(pref.path))
      .filter(_.ext == "txt")
      .sorted
      .map(os.read(_))
  }
}

trait Bar extends Foo {
  def additionalSources = T.sources(millSourcePath / "src2")
  def sourceRoots = T { super.sourceRoots() ++ additionalSources() }
}

object bar extends Bar
> ./mill show bar.sourceContents # includes both source folders
[
  "File Data From src/",
  "File Data From src2/"
]

Commands

def run(args: String*) = T.command {
  os.proc(
      "java",
      "-cp", s"${classFiles().path}:${resources().path}",
      "foo.Foo",
      args
    )
    .call(stdout = os.Inherit)
}

Defined using T.command {…​} syntax, Commands can run arbitrary code, with dependencies declared using the same foo() syntax (e.g. classFiles() above). Commands can be parametrized, but their output is not cached, so they will re-evaluate every time even if none of their inputs have changed. A command with no parameter is defined as def myCommand() = T.command {…​}. It is a compile error if () is missing.

Like Targets, a command only evaluates after all its upstream dependencies have completed, and will not begin to run if any upstream dependency has failed.

Commands are assigned the same scratch/output folder out/run.dest/ as Targets are, and its returned metadata stored at the same out/run.json path for consumption by external tools.

Commands can only be defined directly within a Module body.

Other Tasks

Anonymous Tasks

build.sc (download, browse)
import mill._, define.Task

def data = T.source(millSourcePath / "data")

def anonTask(fileName: String): Task[String] = T.task {
  os.read(data().path / fileName)
}

def helloFileData = T { anonTask("hello.txt")() }
def printFileData(fileName: String) = T.command {
  println(anonTask(fileName)())
}

You can define anonymous tasks using the T.task {…​} syntax. These are not runnable from the command-line, but can be used to share common code you find yourself repeating in Targets and Commands.

Anonymous task’s output does not need to be JSON-serializable, their output is not cached, and they can be defined with or without arguments. Unlike Targets or [_commands], anonymous tasks can be defined anywhere and passed around any way you want, until you finally make use of them within a downstream target or command.

While an anonymous task foo's own output is not cached, if it is used in a downstream target baz and the upstream target bar hasn’t changed, baz's cached output will be used and foo's evaluation will be skipped altogether.

> ./mill show helloFileData
"Hello"

> ./mill printFileData hello.txt
Hello

> ./mill printFileData world.txt
World!

Inputs

build.sc (download, browse)
import mill._

def myInput = T.input {
  os.proc("git", "rev-parse", "HEAD").call(cwd = T.workspace)
    .out
    .text()
    .trim()
}

A generalization of Sources, T.inputs are tasks that re-evaluate every time (unlike Anonymous Tasks), containing an arbitrary block of code.

Inputs can be used to force re-evaluation of some external property that may affect your build. For example, if I have a Target bar that calls out to git to compute the latest commit hash and message directly, that target does not have any Task inputs and so will never re-compute even if the external git status changes:

def gitStatusTarget = T {
  "v-" +
  os.proc("git", "log", "-1", "--pretty=format:%h-%B ")
    .call(cwd = T.workspace)
    .out
    .text()
    .trim()
}
> git init .
> git commit --allow-empty -m "Initial-Commit"

> ./mill show gitStatusTarget
"v-...-Initial-Commit"

> git commit --allow-empty -m "Second-Commit"

> ./mill show gitStatusTarget # Mill didn't pick up the git change!
"v-...-Initial-Commit"

gitStatusTarget will not know that git rev-parse can change, and will not know to re-evaluate when your git log does change. This means gitStatusTarget will continue to use any previously cached value, and gitStatusTarget's output will be out of date!

To fix this, you can wrap your git log in a T.input:

def gitStatusInput = T.input {
  os.proc("git", "log", "-1", "--pretty=format:%h-%B ")
    .call(cwd = T.workspace)
    .out
    .text()
    .trim()
}
def gitStatusTarget2 = T { "v-" + gitStatusInput() }

This makes gitStatusInput to always re-evaluate every build, and only if the output of gitStatusInput changes will gitStatusTarget2 re-compute

> git commit --allow-empty -m "Initial-Commit"

> ./mill show gitStatusTarget2
"v-...-Initial-Commit"

> git commit --allow-empty -m "Second-Commit"

> ./mill show gitStatusTarget2 # Mill picked up git change
"v-...-Second-Commit"

Note that because T.inputs re-evaluate every time, you should ensure that the code you put in T.input runs quickly. Ideally it should just be a simple check "did anything change?" and any heavy-lifting should be delegated to downstream targets where it can be cached if possible.

Persistent Targets

Persistent targets defined using T.persistent are similar to normal Targets, except their T.dest folder is not cleared before every evaluation. This makes them useful for caching things on disk in a more fine-grained manner than Mill’s own Target-level caching.

Below is a semi-realistic example of using a T.persistent target:

build.sc (download, browse)
import mill._, scalalib._
import java.util.Arrays
import java.io.ByteArrayOutputStream
import java.util.zip.GZIPOutputStream

def data = T.source(millSourcePath / "data")

def compressedData = T.persistent{
  println("Evaluating compressedData")
  os.makeDir.all(T.dest / "cache")
  os.remove.all(T.dest / "compressed")

  for(p <- os.list(data().path)){
    val compressedPath = T.dest / "compressed" / s"${p.last}.gz"
    val bytes = os.read.bytes(p)
    val hash = Arrays.hashCode(bytes)
    val cachedPath = T.dest / "cache" / hash.toHexString
    if (!os.exists(cachedPath)) {
      println("Compressing: " + p.last)
      os.write(cachedPath, compressBytes(bytes))
    } else {
      println("Reading Cached from disk: " + p.last)
    }
    os.copy(cachedPath, compressedPath, createFolders = true)
  }

  os.list(T.dest / "compressed").map(PathRef(_))
}

def compressBytes(input: Array[Byte]) = {
  val bos = new ByteArrayOutputStream(input.length)
  val gzip = new GZIPOutputStream(bos)
  gzip.write(input)
  gzip.close()
  bos.toByteArray
}

In this example, we implement a compressedData target that takes a folder of files in inputData and compresses them, while maintaining a cache of compressed contents for each file. That means that if the inputData folder is modified, but some files remain unchanged, those files would not be unnecessarily re-compressed when compressedData evaluates.

Since persistent targets have long-lived state on disk that lives beyond a single evaluation, this raises the possibility of the disk contents getting into a bad state and causing all future evaluations to fail. It is left up to the person implementing the T.persistent to ensure their implementation is eventually consistent. You can also use mill clean to manually purge the disk contents to start fresh.

> ./mill show compressedData
Evaluating compressedData
Compressing: hello.txt
Compressing: world.txt
[
  ".../hello.txt.gz",
  ".../world.txt.gz"
]

> ./mill compressedData # when no input changes, compressedData does not evaluate at all

> sed -i 's/Hello/HELLO/g' data/hello.txt

> ./mill compressedData # when one input file changes, only that file is re-compressed
Compressing: hello.txt
Reading Cached from disk: world.txt

> ./mill clean compressedData

> ./mill compressedData
Evaluating compressedData
Compressing: hello.txt
Compressing: world.txt

Workers

Mill workers defined using T.worker are long-lived in-memory objects that can persistent across multiple evaluations. These are similar to persistent targets in that they let you cache things, but the fact that they let you cache the worker object in-memory allows for greater performance and flexibility: you are no longer limited to caching only serializable data and paying the cost of serializing it to disk every evaluation. This example uses a Worker to provide simple in-memory caching for compressed files.

build.sc (download, browse)
import mill._, scalalib._
import java.util.Arrays
import java.io.ByteArrayOutputStream
import java.util.zip.GZIPOutputStream

def data = T.source(millSourcePath / "data")

def compressWorker = T.worker{ new CompressWorker(T.dest) }

def compressedData = T{
  println("Evaluating compressedData")
  for(p <- os.list(data().path)){
    os.write(
      T.dest / s"${p.last}.gz",
      compressWorker().compress(p.last, os.read.bytes(p))
    )
  }
  os.list(T.dest).map(PathRef(_))
}

class CompressWorker(dest: os.Path){
  val cache = collection.mutable.Map.empty[Int, Array[Byte]]
  def compress(name: String, bytes: Array[Byte]): Array[Byte] = {
    val hash = Arrays.hashCode(bytes)
    if (!cache.contains(hash)) {
      val cachedPath = dest / hash.toHexString
      if (!os.exists(cachedPath)) {
        println("Compressing: " + name)
        cache(hash) = compressBytes(bytes)
        os.write(cachedPath, cache(hash))
      }else{
        println("Cached from disk: " + name)
        cache(hash) = os.read.bytes(cachedPath)
      }
    }else {
      println("Cached from memory: " + name)
    }
    cache(hash)
  }
}

def compressBytes(input: Array[Byte]) = {
  val bos = new ByteArrayOutputStream(input.length)
  val gzip = new GZIPOutputStream(bos)
  gzip.write(input)
  gzip.close()
  bos.toByteArray
}

Common things to put in workers include:

  1. References to third-party daemon processes, e.g. Webpack or wkhtmltopdf, which perform their own in-memory caching

  2. Classloaders containing plugin code, to avoid classpath conflicts while also avoiding classloading cost every time the code is executed

Workers live as long as the Mill process. By default, consecutive mill commands in the same folder will re-use the same Mill process and workers, unless --no-server is passed which will terminate the Mill process and workers after every command. Commands run repeatedly using --watch will also preserve the workers between them.

Workers can also make use of their T.dest folder as a cache that persist when the worker shuts down, as a second layer of caching. The example usage below demonstrates how using the --no-server flag will make the worker read from its disk cache, where it would have normally read from its in-memory cache

> ./mill show compressedData
Evaluating compressedData
Compressing: hello.txt
Compressing: world.txt
[
  ".../hello.txt.gz",
  "...world.txt.gz"
]

> ./mill compressedData # when no input changes, compressedData does not evaluate at all

> sed -i 's/Hello/HELLO/g' data/hello.txt

> ./mill compressedData # not --no-server, we read the data from memory
Compressing: hello.txt
Cached from memory: world.txt

> ./mill compressedData # --no-server, we read the data from disk
Compressing: hello.txt
Cached from disk: world.txt

Mill uses workers to manage long-lived instances of the Zinc Incremental Scala Compiler and the Scala.js Optimizer. This lets us keep them in-memory with warm caches and fast incremental execution.

Autoclosable Workers

As Workers may also hold limited resources, it may be necessary to free up these resources once a worker is no longer needed. This is especially the case, when your worker tasks depends on other tasks and these tasks change, as Mill will then also create a new worker instance.

To implement resource cleanup, your worker can implement java.lang.AutoCloseable. Once the worker is no longer needed, Mill will call the close() method on it before any newer version of this worker is created.

import mill._
import java.lang.AutoCloseable

class MyWorker() extends AutoCloseable {
  // ...
  override def close() = { /* cleanup and free resources */ }
}

def myWorker = T.worker { new MyWorker() }