about summary refs log tree commit diff
path: root/tvix/build/src
diff options
context:
space:
mode:
Diffstat (limited to 'tvix/build/src')
-rw-r--r--tvix/build/src/bin/tvix-build.rs54
-rw-r--r--tvix/build/src/buildservice/build_request.rs131
-rw-r--r--tvix/build/src/buildservice/dummy.rs5
-rw-r--r--tvix/build/src/buildservice/from_addr.rs42
-rw-r--r--tvix/build/src/buildservice/grpc.rs7
-rw-r--r--tvix/build/src/buildservice/mod.rs9
-rw-r--r--tvix/build/src/buildservice/oci.rs266
-rw-r--r--tvix/build/src/lib.rs2
-rw-r--r--tvix/build/src/oci/bundle.rs144
-rw-r--r--tvix/build/src/oci/mod.rs16
-rw-r--r--tvix/build/src/oci/spec.rs307
-rw-r--r--tvix/build/src/proto/grpc_buildservice_wrapper.rs4
-rw-r--r--tvix/build/src/proto/mod.rs211
13 files changed, 1113 insertions, 85 deletions
diff --git a/tvix/build/src/bin/tvix-build.rs b/tvix/build/src/bin/tvix-build.rs
index 07d7e30dfda5..ffdbc3ca4148 100644
--- a/tvix/build/src/bin/tvix-build.rs
+++ b/tvix/build/src/bin/tvix-build.rs
@@ -1,13 +1,10 @@
-use std::sync::Arc;
-
 use clap::Parser;
 use clap::Subcommand;
 use tokio_listener::Listener;
 use tokio_listener::SystemOptions;
 use tokio_listener::UserOptions;
 use tonic::{self, transport::Server};
-use tracing::{info, Level};
-use tracing_subscriber::prelude::*;
+use tracing::info;
 use tvix_build::{
     buildservice,
     proto::{build_service_server::BuildServiceServer, GRPCBuildServiceWrapper},
@@ -20,12 +17,14 @@ use tvix_build::proto::FILE_DESCRIPTOR_SET;
 #[cfg(feature = "tonic-reflection")]
 use tvix_castore::proto::FILE_DESCRIPTOR_SET as CASTORE_FILE_DESCRIPTOR_SET;
 
+use mimalloc::MiMalloc;
+
+#[global_allocator]
+static GLOBAL: MiMalloc = MiMalloc;
+
 #[derive(Parser)]
 #[command(author, version, about, long_about = None)]
 struct Cli {
-    #[arg(long)]
-    log_level: Option<Level>,
-
     #[command(subcommand)]
     command: Commands,
 }
@@ -48,19 +47,12 @@ enum Commands {
 }
 
 #[tokio::main]
-async fn main() -> Result<(), Box<dyn std::error::Error>> {
+async fn main() -> Result<(), Box<dyn std::error::Error + Send + Sync>> {
     let cli = Cli::parse();
 
-    // configure log settings
-    let level = cli.log_level.unwrap_or(Level::INFO);
-
-    tracing_subscriber::registry()
-        .with(
-            tracing_subscriber::fmt::Layer::new()
-                .with_writer(std::io::stderr.with_max_level(level))
-                .pretty(),
-        )
-        .init();
+    tvix_tracing::TracingBuilder::default()
+        .enable_progressbar()
+        .build()?;
 
     match cli.command {
         Commands::Daemon {
@@ -73,12 +65,9 @@ async fn main() -> Result<(), Box<dyn std::error::Error>> {
             let blob_service = blobservice::from_addr(&blob_service_addr).await?;
             let directory_service = directoryservice::from_addr(&directory_service_addr).await?;
 
-            let build_service = buildservice::from_addr(
-                &build_service_addr,
-                Arc::from(blob_service),
-                Arc::from(directory_service),
-            )
-            .await?;
+            let build_service =
+                buildservice::from_addr(&build_service_addr, blob_service, directory_service)
+                    .await?;
 
             let listen_address = listen_address
                 .unwrap_or_else(|| "[::]:8000".to_string())
@@ -94,11 +83,18 @@ async fn main() -> Result<(), Box<dyn std::error::Error>> {
 
             #[cfg(feature = "tonic-reflection")]
             {
-                let reflection_svc = tonic_reflection::server::Builder::configure()
-                    .register_encoded_file_descriptor_set(CASTORE_FILE_DESCRIPTOR_SET)
-                    .register_encoded_file_descriptor_set(FILE_DESCRIPTOR_SET)
-                    .build()?;
-                router = router.add_service(reflection_svc);
+                router = router.add_service(
+                    tonic_reflection::server::Builder::configure()
+                        .register_encoded_file_descriptor_set(CASTORE_FILE_DESCRIPTOR_SET)
+                        .register_encoded_file_descriptor_set(FILE_DESCRIPTOR_SET)
+                        .build_v1alpha()?,
+                );
+                router = router.add_service(
+                    tonic_reflection::server::Builder::configure()
+                        .register_encoded_file_descriptor_set(CASTORE_FILE_DESCRIPTOR_SET)
+                        .register_encoded_file_descriptor_set(FILE_DESCRIPTOR_SET)
+                        .build_v1()?,
+                );
             }
 
             info!(listen_address=%listen_address, "listening");
diff --git a/tvix/build/src/buildservice/build_request.rs b/tvix/build/src/buildservice/build_request.rs
new file mode 100644
index 000000000000..4d53ee55096c
--- /dev/null
+++ b/tvix/build/src/buildservice/build_request.rs
@@ -0,0 +1,131 @@
+use std::collections::{BTreeMap, HashSet};
+use std::path::PathBuf;
+
+use bytes::Bytes;
+use tvix_castore::{Node, PathComponent};
+/// A BuildRequest describes the request of something to be run on the builder.
+/// It is distinct from an actual \[Build\] that has already happened, or might be
+/// currently ongoing.
+///
+/// A BuildRequest can be seen as a more normalized version of a Derivation
+/// (parsed from A-Term), "writing out" some of the Nix-internal details about
+/// how e.g. environment variables in the build are set.
+///
+/// Nix has some impurities when building a Derivation, for example the --cores option
+/// ends up as an environment variable in the build, that's not part of the ATerm.
+///
+/// As of now, we serialize this into the BuildRequest, so builders can stay dumb.
+/// This might change in the future.
+///
+/// There's also a big difference when it comes to how inputs are modelled:
+///
+/// * Nix only uses store path (strings) to describe the inputs.
+///   As store paths can be input-addressed, a certain store path can contain
+///   different contents (as not all store paths are binary reproducible).
+///   This requires that for every input-addressed input, the builder has access
+///   to either the input's deriver (and needs to build it) or else a trusted
+///   source for the built input.
+///   to upload input-addressed paths, requiring the trusted users concept.
+/// * tvix-build records a list of tvix.castore.v1.Node as inputs.
+///   These map from the store path base name to their contents, relieving the
+///   builder from having to "trust" any input-addressed paths, contrary to Nix.
+///
+/// While this approach gives a better hermeticity, it has one downside:
+/// A BuildRequest can only be sent once the contents of all its inputs are known.
+///
+/// As of now, we're okay to accept this, but it prevents uploading an
+/// entirely-non-IFD subgraph of BuildRequests eagerly.
+#[derive(Default, Debug, Clone, PartialEq)]
+pub struct BuildRequest {
+    /// The list of all root nodes that should be visible in `inputs_dir` at the
+    /// time of the build.
+    /// As all references are content-addressed, no additional signatures are
+    /// needed to substitute / make these available in the build environment.
+    pub inputs: BTreeMap<PathComponent, Node>,
+    /// The command (and its args) executed as the build script.
+    /// In the case of a Nix derivation, this is usually
+    /// \["/path/to/some-bash/bin/bash", "-e", "/path/to/some/builder.sh"\].
+    pub command_args: Vec<String>,
+    /// The working dir of the command, relative to the build root.
+    /// "build", in the case of Nix.
+    /// This MUST be a clean relative path, without any ".", "..", or superfluous
+    /// slashes.
+    pub working_dir: PathBuf,
+    /// A list of "scratch" paths, relative to the build root.
+    /// These will be write-able during the build.
+    /// \[build, nix/store\] in the case of Nix.
+    /// These MUST be clean relative paths, without any ".", "..", or superfluous
+    /// slashes, and sorted.
+    pub scratch_paths: Vec<PathBuf>,
+    /// The path where the castore input nodes will be located at,
+    /// "nix/store" in case of Nix.
+    /// Builds might also write into here (Nix builds do that).
+    /// This MUST be a clean relative path, without any ".", "..", or superfluous
+    /// slashes.
+    pub inputs_dir: PathBuf,
+    /// The list of output paths the build is expected to produce,
+    /// relative to the root.
+    /// If the path is not produced, the build is considered to have failed.
+    /// These MUST be clean relative paths, without any ".", "..", or superfluous
+    /// slashes, and sorted.
+    pub outputs: Vec<PathBuf>,
+    /// The list of environment variables and their values that should be set
+    /// inside the build environment.
+    /// This includes both environment vars set inside the derivation, as well as
+    /// more "ephemeral" ones like NIX_BUILD_CORES, controlled by the `--cores`
+    /// CLI option of `nix-build`.
+    /// For now, we consume this as an option when turning a Derivation into a BuildRequest,
+    /// similar to how Nix has a `--cores` option.
+    /// We don't want to bleed these very nix-specific sandbox impl details into
+    /// (dumber) builders if we don't have to.
+    /// Environment variables are sorted by their keys.
+    pub environment_vars: Vec<EnvVar>,
+    /// A set of constraints that need to be satisfied on a build host before a
+    /// Build can be started.
+    pub constraints: HashSet<BuildConstraints>,
+    /// Additional (small) files and their contents that should be placed into the
+    /// build environment, but outside inputs_dir.
+    /// Used for passAsFile and structuredAttrs in Nix.
+    pub additional_files: Vec<AdditionalFile>,
+    /// If this is an non-empty list, all paths in `outputs` are scanned for these.
+    /// For Nix, `refscan_needles` would be populated with the nixbase32 hash parts of
+    /// every input store path and output store path. The latter is necessary to scan
+    /// for references between multi-output derivations.
+    pub refscan_needles: Vec<String>,
+}
+
+#[derive(Debug, Clone, PartialEq)]
+pub struct EnvVar {
+    /// name of the environment variable. Must not contain =.
+    pub key: String,
+    pub value: Bytes,
+}
+/// BuildConstraints represents certain conditions that must be fulfilled
+/// inside the build environment to be able to build this.
+/// Constraints can be things like required architecture and minimum amount of memory.
+/// The required input paths are *not* represented in here, because it
+/// wouldn't be hermetic enough - see the comment around inputs too.
+#[derive(Debug, Clone, PartialEq, Eq, Hash)]
+pub enum BuildConstraints {
+    /// The system that's needed to execute the build.
+    /// Must not be empty.
+    System(String),
+    /// The amount of memory required to be available for the build, in bytes.
+    MinMemory(u64),
+    /// An absolute path that need to be available in the build
+    /// environment, like `/dev/kvm`.
+    /// This is distinct from the castore nodes in inputs.
+    /// These MUST be clean absolute paths, without any ".", "..", or superfluous
+    /// slashes, and sorted.
+    AvailableReadOnlyPath(PathBuf),
+    /// Whether the build should be able to access the network.
+    NetworkAccess,
+    /// Whether to provide a /bin/sh inside the build environment, usually a static bash.
+    ProvideBinSh,
+}
+
+#[derive(Debug, Clone, PartialEq)]
+pub struct AdditionalFile {
+    pub path: PathBuf,
+    pub contents: Bytes,
+}
diff --git a/tvix/build/src/buildservice/dummy.rs b/tvix/build/src/buildservice/dummy.rs
index d20444755e73..3368201376ed 100644
--- a/tvix/build/src/buildservice/dummy.rs
+++ b/tvix/build/src/buildservice/dummy.rs
@@ -2,7 +2,8 @@ use tonic::async_trait;
 use tracing::instrument;
 
 use super::BuildService;
-use crate::proto::{Build, BuildRequest};
+use crate::buildservice::BuildRequest;
+use crate::proto;
 
 #[derive(Default)]
 pub struct DummyBuildService {}
@@ -10,7 +11,7 @@ pub struct DummyBuildService {}
 #[async_trait]
 impl BuildService for DummyBuildService {
     #[instrument(skip(self), ret, err)]
-    async fn do_build(&self, _request: BuildRequest) -> std::io::Result<Build> {
+    async fn do_build(&self, _request: BuildRequest) -> std::io::Result<proto::Build> {
         Err(std::io::Error::new(
             std::io::ErrorKind::Other,
             "builds are not supported with DummyBuildService",
diff --git a/tvix/build/src/buildservice/from_addr.rs b/tvix/build/src/buildservice/from_addr.rs
index cc5403edefff..c88eef2bbdb8 100644
--- a/tvix/build/src/buildservice/from_addr.rs
+++ b/tvix/build/src/buildservice/from_addr.rs
@@ -2,22 +2,27 @@ use super::{grpc::GRPCBuildService, BuildService, DummyBuildService};
 use tvix_castore::{blobservice::BlobService, directoryservice::DirectoryService};
 use url::Url;
 
+#[cfg(target_os = "linux")]
+use super::oci::OCIBuildService;
+
 /// Constructs a new instance of a [BuildService] from an URI.
 ///
 /// The following schemes are supported by the following services:
 /// - `dummy://` ([DummyBuildService])
+/// - `oci://` ([OCIBuildService])
 /// - `grpc+*://` ([GRPCBuildService])
 ///
 /// As some of these [BuildService] need to talk to a [BlobService] and
 /// [DirectoryService], these also need to be passed in.
+#[cfg_attr(target_os = "macos", allow(unused_variables))]
 pub async fn from_addr<BS, DS>(
     uri: &str,
-    _blob_service: BS,
-    _directory_service: DS,
+    blob_service: BS,
+    directory_service: DS,
 ) -> std::io::Result<Box<dyn BuildService>>
 where
-    BS: AsRef<dyn BlobService> + Send + Sync + Clone + 'static,
-    DS: AsRef<dyn DirectoryService> + Send + Sync + Clone + 'static,
+    BS: BlobService + Send + Sync + Clone + 'static,
+    DS: DirectoryService + Send + Sync + Clone + 'static,
 {
     let url = Url::parse(uri)
         .map_err(|e| std::io::Error::other(format!("unable to parse url: {}", e)))?;
@@ -25,6 +30,21 @@ where
     Ok(match url.scheme() {
         // dummy doesn't care about parameters.
         "dummy" => Box::<DummyBuildService>::default(),
+        #[cfg(target_os = "linux")]
+        "oci" => {
+            // oci wants a path in which it creates bundles.
+            if url.path().is_empty() {
+                Err(std::io::Error::other("oci needs a bundle dir as path"))?
+            }
+
+            // TODO: make sandbox shell and rootless_uid_gid
+
+            Box::new(OCIBuildService::new(
+                url.path().into(),
+                blob_service,
+                directory_service,
+            ))
+        }
         scheme => {
             if scheme.starts_with("grpc+") {
                 let client = crate::proto::build_service_client::BuildServiceClient::new(
@@ -47,15 +67,21 @@ where
 
 #[cfg(test)]
 mod tests {
-    use std::sync::Arc;
-
     use super::from_addr;
     use rstest::rstest;
+    use std::sync::Arc;
+    #[cfg(target_os = "linux")]
+    use std::sync::LazyLock;
+    #[cfg(target_os = "linux")]
+    use tempfile::TempDir;
     use tvix_castore::{
         blobservice::{BlobService, MemoryBlobService},
         directoryservice::{DirectoryService, MemoryDirectoryService},
     };
 
+    #[cfg(target_os = "linux")]
+    static TMPDIR_OCI_1: LazyLock<TempDir> = LazyLock::new(|| TempDir::new().unwrap());
+
     #[rstest]
     /// This uses an unsupported scheme.
     #[case::unsupported_scheme("http://foo.example/test", false)]
@@ -73,6 +99,10 @@ mod tests {
     #[case::grpc_valid_https_host_without_port("grpc+https://localhost", true)]
     /// Correct scheme to connect to localhost over http, but with additional path, which is invalid.
     #[case::grpc_invalid_host_and_path("grpc+http://localhost/some-path", false)]
+    /// This configures OCI, but doesn't specify the bundle path
+    #[cfg_attr(target_os = "linux", case::oci_missing_bundle_dir("oci://", false))]
+    /// This configures OCI, specifying the bundle path
+    #[cfg_attr(target_os = "linux", case::oci_bundle_path(&format!("oci://{}", TMPDIR_OCI_1.path().to_str().unwrap()), true))]
     #[tokio::test]
     async fn test_from_addr(#[case] uri_str: &str, #[case] exp_succeed: bool) {
         let blob_service: Arc<dyn BlobService> = Arc::from(MemoryBlobService::default());
diff --git a/tvix/build/src/buildservice/grpc.rs b/tvix/build/src/buildservice/grpc.rs
index 9d22d8397abf..14f06f0ee3e6 100644
--- a/tvix/build/src/buildservice/grpc.rs
+++ b/tvix/build/src/buildservice/grpc.rs
@@ -1,6 +1,7 @@
 use tonic::{async_trait, transport::Channel};
 
-use crate::proto::{build_service_client::BuildServiceClient, Build, BuildRequest};
+use crate::buildservice::BuildRequest;
+use crate::proto::{self, build_service_client::BuildServiceClient};
 
 use super::BuildService;
 
@@ -17,10 +18,10 @@ impl GRPCBuildService {
 
 #[async_trait]
 impl BuildService for GRPCBuildService {
-    async fn do_build(&self, request: BuildRequest) -> std::io::Result<Build> {
+    async fn do_build(&self, request: BuildRequest) -> std::io::Result<proto::Build> {
         let mut client = self.client.clone();
         client
-            .do_build(request)
+            .do_build(Into::<proto::BuildRequest>::into(request))
             .await
             .map(|resp| resp.into_inner())
             .map_err(std::io::Error::other)
diff --git a/tvix/build/src/buildservice/mod.rs b/tvix/build/src/buildservice/mod.rs
index a61d782919b9..b12db6b95d13 100644
--- a/tvix/build/src/buildservice/mod.rs
+++ b/tvix/build/src/buildservice/mod.rs
@@ -1,16 +1,21 @@
 use tonic::async_trait;
 
-use crate::proto::{Build, BuildRequest};
+use crate::proto;
 
+pub mod build_request;
+pub use crate::buildservice::build_request::*;
 mod dummy;
 mod from_addr;
 mod grpc;
 
+#[cfg(target_os = "linux")]
+mod oci;
+
 pub use dummy::DummyBuildService;
 pub use from_addr::from_addr;
 
 #[async_trait]
 pub trait BuildService: Send + Sync {
     /// TODO: document
-    async fn do_build(&self, request: BuildRequest) -> std::io::Result<Build>;
+    async fn do_build(&self, request: BuildRequest) -> std::io::Result<proto::Build>;
 }
diff --git a/tvix/build/src/buildservice/oci.rs b/tvix/build/src/buildservice/oci.rs
new file mode 100644
index 000000000000..52efede6597b
--- /dev/null
+++ b/tvix/build/src/buildservice/oci.rs
@@ -0,0 +1,266 @@
+use anyhow::Context;
+use bstr::BStr;
+use oci_spec::runtime::{LinuxIdMapping, LinuxIdMappingBuilder};
+use tokio::process::{Child, Command};
+use tonic::async_trait;
+use tracing::{debug, instrument, warn, Span};
+use tvix_castore::{
+    blobservice::BlobService,
+    directoryservice::DirectoryService,
+    fs::fuse::FuseDaemon,
+    import::fs::ingest_path,
+    refscan::{ReferencePattern, ReferenceScanner},
+};
+use uuid::Uuid;
+
+use crate::buildservice::BuildRequest;
+use crate::{
+    oci::{get_host_output_paths, make_bundle, make_spec},
+    proto::{self, build::OutputNeedles},
+};
+use std::{ffi::OsStr, path::PathBuf, process::Stdio};
+
+use super::BuildService;
+
+const SANDBOX_SHELL: &str = env!("TVIX_BUILD_SANDBOX_SHELL");
+const MAX_CONCURRENT_BUILDS: usize = 2; // TODO: make configurable
+
+pub struct OCIBuildService<BS, DS> {
+    /// Root path in which all bundles are created in
+    bundle_root: PathBuf,
+
+    /// uid mappings to set up for the workloads
+    uid_mappings: Vec<LinuxIdMapping>,
+    /// uid mappings to set up for the workloads
+    gid_mappings: Vec<LinuxIdMapping>,
+
+    /// Handle to a [BlobService], used by filesystems spawned during builds.
+    blob_service: BS,
+    /// Handle to a [DirectoryService], used by filesystems spawned during builds.
+    directory_service: DS,
+
+    // semaphore to track number of concurrently running builds.
+    // this is necessary, as otherwise we very quickly run out of open file handles.
+    concurrent_builds: tokio::sync::Semaphore,
+}
+
+impl<BS, DS> OCIBuildService<BS, DS> {
+    pub fn new(bundle_root: PathBuf, blob_service: BS, directory_service: DS) -> Self {
+        // We map root inside the container to the uid/gid this is running at,
+        // and allocate one for uid 1000 into the container from the range we
+        // got in /etc/sub{u,g}id.
+        // TODO: actually read uid, and /etc/subuid. Maybe only when we try to build?
+        // FUTUREWORK: use different uids?
+        Self {
+            bundle_root,
+            blob_service,
+            directory_service,
+            uid_mappings: vec![
+                LinuxIdMappingBuilder::default()
+                    .host_id(1000_u32)
+                    .container_id(0_u32)
+                    .size(1_u32)
+                    .build()
+                    .unwrap(),
+                LinuxIdMappingBuilder::default()
+                    .host_id(100000_u32)
+                    .container_id(1000_u32)
+                    .size(1_u32)
+                    .build()
+                    .unwrap(),
+            ],
+            gid_mappings: vec![
+                LinuxIdMappingBuilder::default()
+                    .host_id(100_u32)
+                    .container_id(0_u32)
+                    .size(1_u32)
+                    .build()
+                    .unwrap(),
+                LinuxIdMappingBuilder::default()
+                    .host_id(100000_u32)
+                    .container_id(100_u32)
+                    .size(1_u32)
+                    .build()
+                    .unwrap(),
+            ],
+            concurrent_builds: tokio::sync::Semaphore::new(MAX_CONCURRENT_BUILDS),
+        }
+    }
+}
+
+#[async_trait]
+impl<BS, DS> BuildService for OCIBuildService<BS, DS>
+where
+    BS: BlobService + Clone + 'static,
+    DS: DirectoryService + Clone + 'static,
+{
+    #[instrument(skip_all, err)]
+    async fn do_build(&self, request: BuildRequest) -> std::io::Result<proto::Build> {
+        let _permit = self.concurrent_builds.acquire().await.unwrap();
+
+        let bundle_name = Uuid::new_v4();
+        let bundle_path = self.bundle_root.join(bundle_name.to_string());
+
+        let span = Span::current();
+        span.record("bundle_name", bundle_name.to_string());
+
+        let mut runtime_spec = make_spec(&request, true, SANDBOX_SHELL)
+            .context("failed to create spec")
+            .map_err(std::io::Error::other)?;
+
+        let mut linux = runtime_spec.linux().clone().unwrap();
+
+        // edit the spec, we need to setup uid/gid mappings.
+        linux.set_uid_mappings(Some(self.uid_mappings.clone()));
+        linux.set_gid_mappings(Some(self.gid_mappings.clone()));
+
+        runtime_spec.set_linux(Some(linux));
+
+        make_bundle(&request, &runtime_spec, &bundle_path)
+            .context("failed to produce bundle")
+            .map_err(std::io::Error::other)?;
+
+        // pre-calculate the locations we want to later ingest, in the order of
+        // the original outputs.
+        // If we can't find calculate that path, don't start the build in first place.
+        let host_output_paths = get_host_output_paths(&request, &bundle_path)
+            .context("failed to calculate host output paths")
+            .map_err(std::io::Error::other)?;
+
+        // assemble a BTreeMap of Nodes to pass into TvixStoreFs.
+        let patterns = ReferencePattern::new(request.refscan_needles.clone());
+        // NOTE: impl Drop for FuseDaemon unmounts, so if the call is cancelled, umount.
+        let _fuse_daemon = tokio::task::spawn_blocking({
+            let blob_service = self.blob_service.clone();
+            let directory_service = self.directory_service.clone();
+
+            let dest = bundle_path.join("inputs");
+
+            let root_nodes = Box::new(request.inputs.clone());
+            move || {
+                let fs = tvix_castore::fs::TvixStoreFs::new(
+                    blob_service,
+                    directory_service,
+                    root_nodes,
+                    true,
+                    false,
+                );
+                // mount the filesystem and wait for it to be unmounted.
+                // FUTUREWORK: make fuse daemon threads configurable?
+                FuseDaemon::new(fs, dest, 4, true).context("failed to start fuse daemon")
+            }
+        })
+        .await?
+        .context("mounting")
+        .map_err(std::io::Error::other)?;
+
+        debug!(bundle.path=?bundle_path, bundle.name=%bundle_name, "about to spawn bundle");
+
+        // start the bundle as another process.
+        let child = spawn_bundle(bundle_path, &bundle_name.to_string())?;
+
+        // wait for the process to exit
+        // FUTUREWORK: change the trait to allow reporting progress / logs…
+        let child_output = child
+            .wait_with_output()
+            .await
+            .context("failed to run process")
+            .map_err(std::io::Error::other)?;
+
+        // Check the exit code
+        if !child_output.status.success() {
+            let stdout = BStr::new(&child_output.stdout);
+            let stderr = BStr::new(&child_output.stderr);
+
+            warn!(stdout=%stdout, stderr=%stderr, exit_code=%child_output.status, "build failed");
+
+            return Err(std::io::Error::new(
+                std::io::ErrorKind::Other,
+                "nonzero exit code".to_string(),
+            ));
+        }
+
+        // Ingest build outputs into the castore.
+        // We use try_join_all here. No need to spawn new tasks, as this is
+        // mostly IO bound.
+        let (outputs, outputs_needles) = futures::future::try_join_all(
+            host_output_paths.into_iter().enumerate().map(|(i, p)| {
+                let output_path = request.outputs[i].clone();
+                let patterns = patterns.clone();
+                async move {
+                    debug!(host.path=?p, output.path=?output_path, "ingesting path");
+
+                    let scanner = ReferenceScanner::new(patterns);
+                    let output_node = ingest_path(
+                        self.blob_service.clone(),
+                        &self.directory_service,
+                        p,
+                        Some(&scanner),
+                    )
+                    .await
+                    .map_err(|e| {
+                        std::io::Error::new(
+                            std::io::ErrorKind::InvalidData,
+                            format!("Unable to ingest output: {}", e),
+                        )
+                    })?;
+
+                    let needles = OutputNeedles {
+                        needles: scanner
+                            .matches()
+                            .into_iter()
+                            .enumerate()
+                            .filter(|(_, val)| *val)
+                            .map(|(idx, _)| idx as u64)
+                            .collect(),
+                    };
+
+                    Ok::<_, std::io::Error>((
+                        tvix_castore::proto::Node::from_name_and_node(
+                            output_path
+                                .file_name()
+                                .and_then(|s| s.to_str())
+                                .map(|s| s.to_string())
+                                .unwrap_or("".into())
+                                .into(),
+                            output_node,
+                        ),
+                        needles,
+                    ))
+                }
+            }),
+        )
+        .await?
+        .into_iter()
+        .unzip();
+
+        Ok(proto::Build {
+            build_request: Some(request.into()),
+            outputs,
+            outputs_needles,
+        })
+    }
+}
+
+/// Spawns runc with the bundle at bundle_path.
+/// On success, returns the child.
+#[instrument(err)]
+fn spawn_bundle(
+    bundle_path: impl AsRef<OsStr> + std::fmt::Debug,
+    bundle_name: &str,
+) -> std::io::Result<Child> {
+    let mut command = Command::new("runc");
+
+    command
+        .args(&[
+            "run".into(),
+            "--bundle".into(),
+            bundle_path.as_ref().to_os_string(),
+            bundle_name.into(),
+        ])
+        .stderr(Stdio::piped())
+        .stdout(Stdio::piped())
+        .stdin(Stdio::null());
+
+    command.spawn()
+}
diff --git a/tvix/build/src/lib.rs b/tvix/build/src/lib.rs
index b173657e431c..86c1862d629e 100644
--- a/tvix/build/src/lib.rs
+++ b/tvix/build/src/lib.rs
@@ -1,2 +1,4 @@
 pub mod buildservice;
+#[cfg(target_os = "linux")]
+mod oci;
 pub mod proto;
diff --git a/tvix/build/src/oci/bundle.rs b/tvix/build/src/oci/bundle.rs
new file mode 100644
index 000000000000..52789362a58d
--- /dev/null
+++ b/tvix/build/src/oci/bundle.rs
@@ -0,0 +1,144 @@
+//! Module to create an OCI runtime bundle for a given [BuildRequest].
+use std::{
+    fs,
+    path::{Path, PathBuf},
+};
+
+use super::scratch_name;
+use crate::buildservice::BuildRequest;
+use anyhow::{bail, Context};
+use tracing::{debug, instrument};
+
+/// Produce an OCI bundle in a given path.
+/// Check [make_spec] for a description about the paths produced.
+#[instrument(err)]
+pub(crate) fn make_bundle<'a>(
+    request: &BuildRequest,
+    runtime_spec: &oci_spec::runtime::Spec,
+    path: &Path,
+) -> anyhow::Result<()> {
+    fs::create_dir_all(path).context("failed to create bundle path")?;
+
+    let spec_json = serde_json::to_string(runtime_spec).context("failed to render spec to json")?;
+    fs::write(path.join("config.json"), spec_json).context("failed to write config.json")?;
+
+    fs::create_dir_all(path.join("inputs")).context("failed to create inputs dir")?;
+
+    let root_path = path.join("root");
+
+    fs::create_dir_all(&root_path).context("failed to create root path dir")?;
+    fs::create_dir_all(root_path.join("etc")).context("failed to create root/etc dir")?;
+
+    // TODO: populate /etc/{group,passwd}. It's a mess?
+
+    let scratch_root = path.join("scratch");
+    fs::create_dir_all(&scratch_root).context("failed to create scratch/ dir")?;
+
+    // for each scratch path, calculate its name inside scratch, and ensure the
+    // directory exists.
+    for p in request.scratch_paths.iter() {
+        let scratch_path = scratch_root.join(scratch_name(p));
+        debug!(scratch_path=?scratch_path, path=?p, "about to create scratch dir");
+        fs::create_dir_all(scratch_path).context("Unable to create scratch dir")?;
+    }
+
+    Ok(())
+}
+
+/// Determine the path of all outputs specified in a [BuildRequest]
+/// as seen from the host, for post-build ingestion.
+/// This lookup needs to take scratch paths into consideration, as the build
+/// root is not writable on its own.
+/// If a path can't be determined, an error is returned.
+pub(crate) fn get_host_output_paths(
+    request: &BuildRequest,
+    bundle_path: &Path,
+) -> anyhow::Result<Vec<PathBuf>> {
+    let scratch_root = bundle_path.join("scratch");
+
+    let mut host_output_paths: Vec<PathBuf> = Vec::with_capacity(request.outputs.len());
+
+    for output_path in request.outputs.iter() {
+        // calculate the location of the path.
+        if let Some((mp, relpath)) = find_path_in_scratchs(output_path, &request.scratch_paths) {
+            host_output_paths.push(scratch_root.join(scratch_name(mp)).join(relpath));
+        } else {
+            bail!("unable to find path {output_path:?}");
+        }
+    }
+
+    Ok(host_output_paths)
+}
+
+/// For a given list of mountpoints (sorted) and a search_path, find the
+/// specific mountpoint parenting that search_path and return it, as well as the
+/// relative path from there to the search_path.
+/// mountpoints must be sorted, so we can iterate over the list from the back
+/// and match on the prefix.
+fn find_path_in_scratchs<'a, 'b, I>(
+    search_path: &'a Path,
+    mountpoints: I,
+) -> Option<(&'b Path, &'a Path)>
+where
+    I: IntoIterator<Item = &'b PathBuf>,
+    I::IntoIter: DoubleEndedIterator,
+{
+    mountpoints
+        .into_iter()
+        .rev()
+        .find_map(|mp| Some((mp.as_path(), search_path.strip_prefix(mp).ok()?)))
+}
+
+#[cfg(test)]
+mod tests {
+    use std::path::{Path, PathBuf};
+
+    use rstest::rstest;
+
+    use crate::{buildservice::BuildRequest, oci::scratch_name};
+
+    use super::{find_path_in_scratchs, get_host_output_paths};
+
+    #[rstest]
+    #[case::simple("nix/store/aaaa", &["nix/store".into()], Some(("nix/store", "aaaa")))]
+    #[case::prefix_no_sep("nix/store/aaaa", &["nix/sto".into()], None)]
+    #[case::not_found("nix/store/aaaa", &["build".into()], None)]
+    fn test_test_find_path_in_scratchs(
+        #[case] search_path: &str,
+        #[case] mountpoints: &[String],
+        #[case] expected: Option<(&str, &str)>,
+    ) {
+        let expected = expected.map(|e| (Path::new(e.0), Path::new(e.1)));
+        assert_eq!(
+            find_path_in_scratchs(
+                Path::new(search_path),
+                mountpoints
+                    .iter()
+                    .map(PathBuf::from)
+                    .collect::<Vec<_>>()
+                    .as_slice()
+            ),
+            expected
+        );
+    }
+
+    #[test]
+    fn test_get_host_output_paths_simple() {
+        let request = BuildRequest {
+            outputs: vec!["nix/store/fhaj6gmwns62s6ypkcldbaj2ybvkhx3p-foo".into()],
+            scratch_paths: vec!["build".into(), "nix/store".into()],
+            ..Default::default()
+        };
+
+        let paths =
+            get_host_output_paths(&request, Path::new("bundle-root")).expect("must succeed");
+
+        let mut expected_path = PathBuf::new();
+        expected_path.push("bundle-root");
+        expected_path.push("scratch");
+        expected_path.push(scratch_name(Path::new("nix/store")));
+        expected_path.push("fhaj6gmwns62s6ypkcldbaj2ybvkhx3p-foo");
+
+        assert_eq!(vec![expected_path], paths)
+    }
+}
diff --git a/tvix/build/src/oci/mod.rs b/tvix/build/src/oci/mod.rs
new file mode 100644
index 000000000000..a2400c4a6eba
--- /dev/null
+++ b/tvix/build/src/oci/mod.rs
@@ -0,0 +1,16 @@
+mod bundle;
+mod spec;
+
+pub(crate) use bundle::get_host_output_paths;
+pub(crate) use bundle::make_bundle;
+pub(crate) use spec::make_spec;
+
+use std::path::Path;
+
+/// For a given scratch path, return the scratch_name that's allocated.
+// We currently use use lower hex encoding of the b3 digest of the scratch
+// path, so we don't need to globally allocate and pass down some uuids.
+pub(crate) fn scratch_name(scratch_path: &Path) -> String {
+    data_encoding::BASE32
+        .encode(blake3::hash(scratch_path.as_os_str().as_encoded_bytes()).as_bytes())
+}
diff --git a/tvix/build/src/oci/spec.rs b/tvix/build/src/oci/spec.rs
new file mode 100644
index 000000000000..e3b6172def8a
--- /dev/null
+++ b/tvix/build/src/oci/spec.rs
@@ -0,0 +1,307 @@
+//! Module to create a OCI runtime spec for a given [BuildRequest].
+use crate::buildservice::{BuildConstraints, BuildRequest};
+use oci_spec::{
+    runtime::{Capability, LinuxNamespace, LinuxNamespaceBuilder, LinuxNamespaceType},
+    OciSpecError,
+};
+use std::{collections::HashSet, path::Path};
+
+use super::scratch_name;
+
+/// For a given [BuildRequest], return an OCI runtime spec.
+///
+/// While there's no IO occuring in this function, the generated spec contains
+/// path references relative to the "bundle location".
+/// Due to overlayfs requiring its layers to be absolute paths, we also need a
+/// [bundle_dir] parameter, pointing to the location of the bundle dir itself.
+///
+/// The paths used in the spec are the following (relative to a "bundle root"):
+///
+/// - `inputs`, a directory where the castore nodes specified the build request
+///   inputs are supposed to be populated.
+/// - `outputs`, a directory where all writes to the store_dir during the build
+///   are directed to.
+/// - `root`, a minimal skeleton of files that'll be present at /.
+/// - `scratch`, a directory containing other directories which will be
+///   bind-mounted read-write into the container and used as scratch space
+///   during the build.
+///   No assumptions should be made about what's inside this directory.
+///
+/// Generating these paths, and populating contents, like a skeleton root
+/// is up to another function, this function doesn't do filesystem IO.
+pub(crate) fn make_spec(
+    request: &BuildRequest,
+    rootless: bool,
+    sandbox_shell: &str,
+) -> Result<oci_spec::runtime::Spec, oci_spec::OciSpecError> {
+    let allow_network = request
+        .constraints
+        .contains(&BuildConstraints::NetworkAccess);
+
+    // Assemble ro_host_mounts. Start with constraints.available_ro_paths.
+    let mut ro_host_mounts: Vec<_> = request
+        .constraints
+        .iter()
+        .filter_map(|constraint| match constraint {
+            BuildConstraints::AvailableReadOnlyPath(path) => Some((path.as_path(), path.as_path())),
+            _ => None,
+        })
+        .collect();
+
+    // If provide_bin_sh is set, mount sandbox_shell to /bin/sh
+    if request
+        .constraints
+        .contains(&BuildConstraints::ProvideBinSh)
+    {
+        ro_host_mounts.push((Path::new(sandbox_shell), Path::new("/bin/sh")))
+    }
+
+    oci_spec::runtime::SpecBuilder::default()
+        .process(configure_process(
+            &request.command_args,
+            &request.working_dir,
+            request
+                .environment_vars
+                .iter()
+                .map(|e| {
+                    (
+                        e.key.as_str(),
+                        // TODO: decide what to do with non-bytes env values
+                        String::from_utf8(e.value.to_vec()).expect("invalid string in env"),
+                    )
+                })
+                .collect::<Vec<_>>(),
+            rootless,
+        )?)
+        .linux(configure_linux(allow_network, rootless)?)
+        .root(
+            oci_spec::runtime::RootBuilder::default()
+                .path("root")
+                .readonly(true)
+                .build()?,
+        )
+        .hostname("localhost")
+        .mounts(configure_mounts(
+            rootless,
+            allow_network,
+            request.scratch_paths.iter().map(|e| e.as_path()),
+            request.inputs.iter(),
+            &request.inputs_dir,
+            ro_host_mounts,
+        )?)
+        .build()
+}
+
+/// Return the Process part of the OCI Runtime spec.
+/// This configures the command, it's working dir, env and terminal setup.
+/// It also takes care of setting rlimits and capabilities.
+/// Capabilities are a bit more complicated in case rootless building is requested.
+fn configure_process<'a>(
+    command_args: &[String],
+    cwd: &Path,
+    env: impl IntoIterator<Item = (&'a str, String)>,
+    rootless: bool,
+) -> Result<oci_spec::runtime::Process, oci_spec::OciSpecError> {
+    let spec_builder = oci_spec::runtime::ProcessBuilder::default()
+        .args(command_args)
+        .env(
+            env.into_iter()
+                .map(|(k, v)| format!("{}={}", k, v))
+                .collect::<Vec<_>>(),
+        )
+        .terminal(true)
+        .user(
+            oci_spec::runtime::UserBuilder::default()
+                .uid(1000u32)
+                .gid(100u32)
+                .build()?,
+        )
+        .cwd(Path::new("/").join(cwd)) // relative to the bundle root, but at least runc wants it to also be absolute.
+        .capabilities({
+            let caps: HashSet<Capability> = if !rootless {
+                HashSet::from([Capability::AuditWrite, Capability::Kill])
+            } else {
+                HashSet::from([
+                    Capability::AuditWrite,
+                    Capability::Chown,
+                    Capability::DacOverride,
+                    Capability::Fowner,
+                    Capability::Fsetid,
+                    Capability::Kill,
+                    Capability::Mknod,
+                    Capability::NetBindService,
+                    Capability::NetRaw,
+                    Capability::Setfcap,
+                    Capability::Setgid,
+                    Capability::Setpcap,
+                    Capability::Setuid,
+                    Capability::SysChroot,
+                ])
+            };
+
+            oci_spec::runtime::LinuxCapabilitiesBuilder::default()
+                .bounding(caps.clone())
+                .effective(caps.clone())
+                .inheritable(caps.clone())
+                .permitted(caps.clone())
+                .ambient(caps)
+                .build()?
+        })
+        .rlimits([oci_spec::runtime::PosixRlimitBuilder::default()
+            .typ(oci_spec::runtime::PosixRlimitType::RlimitNofile)
+            .hard(1024_u64)
+            .soft(1024_u64)
+            .build()?])
+        .no_new_privileges(true);
+
+    spec_builder.build()
+}
+
+/// Return the Linux part of the OCI Runtime spec.
+/// This configures various namespaces, masked and read-only paths.
+fn configure_linux(
+    allow_network: bool,
+    rootless: bool,
+) -> Result<oci_spec::runtime::Linux, OciSpecError> {
+    let mut linux = oci_spec::runtime::Linux::default();
+
+    // explicitly set namespaces, depending on allow_network.
+    linux.set_namespaces(Some({
+        let mut namespace_types = vec![
+            LinuxNamespaceType::Pid,
+            LinuxNamespaceType::Ipc,
+            LinuxNamespaceType::Uts,
+            LinuxNamespaceType::Mount,
+            LinuxNamespaceType::Cgroup,
+        ];
+        if !allow_network {
+            namespace_types.push(LinuxNamespaceType::Network)
+        }
+        if rootless {
+            namespace_types.push(LinuxNamespaceType::User)
+        }
+
+        namespace_types
+            .into_iter()
+            .map(|e| LinuxNamespaceBuilder::default().typ(e).build())
+            .collect::<Result<Vec<LinuxNamespace>, _>>()?
+    }));
+
+    linux.set_masked_paths(Some(
+        [
+            "/proc/kcore",
+            "/proc/latency_stats",
+            "/proc/timer_list",
+            "/proc/timer_stats",
+            "/proc/sched_debug",
+            "/sys/firmware",
+        ]
+        .into_iter()
+        .map(|e| e.to_string())
+        .collect::<Vec<_>>(),
+    ));
+
+    linux.set_readonly_paths(Some(
+        [
+            "/proc/asound",
+            "/proc/bus",
+            "/proc/fs",
+            "/proc/irq",
+            "/proc/sys",
+            "/proc/sysrq-trigger",
+        ]
+        .into_iter()
+        .map(|e| e.to_string())
+        .collect::<Vec<_>>(),
+    ));
+
+    Ok(linux)
+}
+
+/// Return the Mounts part of the OCI Runtime spec.
+/// It first sets up the standard mounts, then scratch paths, bind mounts for
+/// all inputs, and finally read-only paths from the hosts.
+fn configure_mounts<'a>(
+    rootless: bool,
+    allow_network: bool,
+    scratch_paths: impl IntoIterator<Item = &'a Path>,
+    inputs: impl Iterator<Item = (&'a tvix_castore::PathComponent, &'a tvix_castore::Node)>,
+
+    inputs_dir: &Path,
+    ro_host_mounts: impl IntoIterator<Item = (&'a Path, &'a Path)>,
+) -> Result<Vec<oci_spec::runtime::Mount>, oci_spec::OciSpecError> {
+    let mut mounts: Vec<_> = if rootless {
+        oci_spec::runtime::get_rootless_mounts()
+    } else {
+        oci_spec::runtime::get_default_mounts()
+    };
+
+    mounts.push(configure_mount(
+        Path::new("tmpfs"),
+        Path::new("/tmp"),
+        "tmpfs",
+        &["nosuid", "noatime", "mode=700"],
+    )?);
+
+    // For each scratch path, create a bind mount entry.
+    let scratch_root = Path::new("scratch"); // relative path
+    for scratch_path in scratch_paths.into_iter() {
+        let src = scratch_root.join(scratch_name(scratch_path));
+        mounts.push(configure_mount(
+            &src,
+            &Path::new("/").join(scratch_path),
+            "none",
+            &["rbind", "rw"],
+        )?);
+    }
+
+    // For each input, create a bind mount from inputs/$name into $inputs_dir/$name.
+    for (input_name, _input) in inputs {
+        let input_name = std::str::from_utf8(input_name.as_ref()).expect("invalid input name");
+        mounts.push(configure_mount(
+            &Path::new("inputs").join(input_name),
+            &Path::new("/").join(inputs_dir).join(input_name),
+            "none",
+            &[
+                "rbind", "ro",
+                // "nosuid" is required, otherwise mounting will just fail with
+                // a generic permission error.
+                // See https://github.com/wllenyj/containerd/commit/42a386c8164bef16d59590c61ab00806f854d8fd
+                "nosuid", "nodev",
+            ],
+        )?);
+    }
+
+    // Process ro_host_mounts
+    for (src, dst) in ro_host_mounts.into_iter() {
+        mounts.push(configure_mount(src, dst, "none", &["rbind", "ro"])?);
+    }
+
+    // In case network is enabled, also mount in /etc/{resolv.conf,services,hosts}
+    if allow_network {
+        for p in [
+            Path::new("/etc/resolv.conf"),
+            Path::new("/etc/services"),
+            Path::new("/etc/hosts"),
+        ] {
+            mounts.push(configure_mount(p, p, "none", &["rbind", "ro"])?);
+        }
+    }
+
+    Ok(mounts)
+}
+
+/// Helper function to produce a mount.
+fn configure_mount(
+    source: &Path,
+    destination: &Path,
+    typ: &str,
+    options: &[&str],
+) -> Result<oci_spec::runtime::Mount, oci_spec::OciSpecError> {
+    oci_spec::runtime::MountBuilder::default()
+        .destination(destination)
+        .typ(typ.to_string())
+        .source(source)
+        .options(options.iter().map(|e| e.to_string()).collect::<Vec<_>>())
+        .build()
+}
diff --git a/tvix/build/src/proto/grpc_buildservice_wrapper.rs b/tvix/build/src/proto/grpc_buildservice_wrapper.rs
index 024f075de9ad..8a2d36ac569f 100644
--- a/tvix/build/src/proto/grpc_buildservice_wrapper.rs
+++ b/tvix/build/src/proto/grpc_buildservice_wrapper.rs
@@ -27,7 +27,9 @@ where
         &self,
         request: tonic::Request<BuildRequest>,
     ) -> Result<tonic::Response<Build>, tonic::Status> {
-        match self.inner.do_build(request.into_inner()).await {
+        let request = TryInto::<crate::buildservice::BuildRequest>::try_into(request.into_inner())
+            .map_err(|err| tonic::Status::new(tonic::Code::InvalidArgument, err.to_string()))?;
+        match self.inner.do_build(request).await {
             Ok(resp) => Ok(tonic::Response::new(resp)),
             Err(e) => Err(tonic::Status::internal(e.to_string())),
         }
diff --git a/tvix/build/src/proto/mod.rs b/tvix/build/src/proto/mod.rs
index e359b5b5b70e..be757bb56272 100644
--- a/tvix/build/src/proto/mod.rs
+++ b/tvix/build/src/proto/mod.rs
@@ -1,7 +1,8 @@
+use std::collections::{BTreeMap, HashSet};
 use std::path::{Path, PathBuf};
 
 use itertools::Itertools;
-use tvix_castore::proto::{NamedNode, ValidateNodeError};
+use tvix_castore::{DirectoryError, Node, PathComponent};
 
 mod grpc_buildservice_wrapper;
 
@@ -19,7 +20,7 @@ pub const FILE_DESCRIPTOR_SET: &[u8] = tonic::include_file_descriptor_set!("tvix
 #[derive(Debug, thiserror::Error)]
 pub enum ValidateBuildRequestError {
     #[error("invalid input node at position {0}: {1}")]
-    InvalidInputNode(usize, ValidateNodeError),
+    InvalidInputNode(usize, DirectoryError),
 
     #[error("input nodes are not sorted by name")]
     InputNodesNotSorted,
@@ -117,86 +118,149 @@ where
     data.tuple_windows().all(|(a, b)| a <= b)
 }
 
-impl BuildRequest {
-    /// Ensures the build request is valid.
-    /// This means, all input nodes need to be valid, paths in lists need to be sorted,
-    /// and all restrictions around paths themselves (relative, clean, …) need
-    // to be fulfilled.
-    pub fn validate(&self) -> Result<(), ValidateBuildRequestError> {
-        // validate all input nodes
-        for (i, n) in self.inputs.iter().enumerate() {
-            // ensure the input node itself is valid
-            n.validate()
-                .map_err(|e| ValidateBuildRequestError::InvalidInputNode(i, e))?;
-        }
+fn path_to_string(path: &Path) -> String {
+    path.to_str()
+        .expect("Tvix Bug: unable to convert Path to String")
+        .to_string()
+}
 
-        // now we can look at the names, and make sure they're sorted.
-        if !is_sorted(
-            self.inputs
+impl From<crate::buildservice::BuildRequest> for BuildRequest {
+    fn from(value: crate::buildservice::BuildRequest) -> Self {
+        let constraints = if value.constraints.is_empty() {
+            None
+        } else {
+            let mut constraints = build_request::BuildConstraints::default();
+            for constraint in value.constraints {
+                use crate::buildservice::BuildConstraints;
+                match constraint {
+                    BuildConstraints::System(system) => constraints.system = system,
+                    BuildConstraints::MinMemory(min_memory) => constraints.min_memory = min_memory,
+                    BuildConstraints::AvailableReadOnlyPath(path) => {
+                        constraints.available_ro_paths.push(path_to_string(&path))
+                    }
+                    BuildConstraints::ProvideBinSh => constraints.provide_bin_sh = true,
+                    BuildConstraints::NetworkAccess => constraints.network_access = true,
+                }
+            }
+            Some(constraints)
+        };
+        Self {
+            inputs: value
+                .inputs
+                .into_iter()
+                .map(|(name, node)| {
+                    tvix_castore::proto::Node::from_name_and_node(name.into(), node)
+                })
+                .collect(),
+            command_args: value.command_args,
+            working_dir: path_to_string(&value.working_dir),
+            scratch_paths: value
+                .scratch_paths
                 .iter()
-                .map(|e| e.node.as_ref().unwrap().get_name()),
-        ) {
-            Err(ValidateBuildRequestError::InputNodesNotSorted)?
+                .map(|p| path_to_string(p))
+                .collect(),
+            inputs_dir: path_to_string(&value.inputs_dir),
+            outputs: value.outputs.iter().map(|p| path_to_string(p)).collect(),
+            environment_vars: value.environment_vars.into_iter().map(Into::into).collect(),
+            constraints,
+            additional_files: value.additional_files.into_iter().map(Into::into).collect(),
+            refscan_needles: value.refscan_needles,
+        }
+    }
+}
+
+impl TryFrom<BuildRequest> for crate::buildservice::BuildRequest {
+    type Error = ValidateBuildRequestError;
+    fn try_from(value: BuildRequest) -> Result<Self, Self::Error> {
+        // validate input names. Make sure they're sorted
+
+        let mut last_name: bytes::Bytes = "".into();
+        let mut inputs: BTreeMap<PathComponent, Node> = BTreeMap::new();
+        for (i, node) in value.inputs.iter().enumerate() {
+            let (name, node) = node
+                .clone()
+                .try_into_name_and_node()
+                .map_err(|e| ValidateBuildRequestError::InvalidInputNode(i, e))?;
+
+            if name.as_ref() <= last_name.as_ref() {
+                return Err(ValidateBuildRequestError::InputNodesNotSorted);
+            } else {
+                inputs.insert(name.clone(), node);
+                last_name = name.into();
+            }
         }
 
         // validate working_dir
-        if !is_clean_relative_path(&self.working_dir) {
+        if !is_clean_relative_path(&value.working_dir) {
             Err(ValidateBuildRequestError::InvalidWorkingDir)?;
         }
 
         // validate scratch paths
-        for (i, p) in self.scratch_paths.iter().enumerate() {
+        for (i, p) in value.scratch_paths.iter().enumerate() {
             if !is_clean_relative_path(p) {
                 Err(ValidateBuildRequestError::InvalidScratchPath(i))?
             }
         }
-        if !is_sorted(self.scratch_paths.iter().map(|e| e.as_bytes())) {
+        if !is_sorted(value.scratch_paths.iter().map(|e| e.as_bytes())) {
             Err(ValidateBuildRequestError::ScratchPathsNotSorted)?;
         }
 
         // validate inputs_dir
-        if !is_clean_relative_path(&self.inputs_dir) {
+        if !is_clean_relative_path(&value.inputs_dir) {
             Err(ValidateBuildRequestError::InvalidInputsDir)?;
         }
 
         // validate outputs
-        for (i, p) in self.outputs.iter().enumerate() {
+        for (i, p) in value.outputs.iter().enumerate() {
             if !is_clean_relative_path(p) {
                 Err(ValidateBuildRequestError::InvalidOutputPath(i))?
             }
         }
-        if !is_sorted(self.outputs.iter().map(|e| e.as_bytes())) {
+        if !is_sorted(value.outputs.iter().map(|e| e.as_bytes())) {
             Err(ValidateBuildRequestError::OutputsNotSorted)?;
         }
 
         // validate environment_vars.
-        for (i, e) in self.environment_vars.iter().enumerate() {
+        for (i, e) in value.environment_vars.iter().enumerate() {
             if e.key.is_empty() || e.key.contains('=') {
                 Err(ValidateBuildRequestError::InvalidEnvVar(i))?
             }
         }
-        if !is_sorted(self.environment_vars.iter().map(|e| e.key.as_bytes())) {
+        if !is_sorted(value.environment_vars.iter().map(|e| e.key.as_bytes())) {
             Err(ValidateBuildRequestError::EnvVarNotSorted)?;
         }
 
         // validate build constraints
-        if let Some(constraints) = self.constraints.as_ref() {
-            constraints
-                .validate()
-                .map_err(ValidateBuildRequestError::InvalidBuildConstraints)?;
-        }
+        let constraints = value
+            .constraints
+            .map_or(Ok(HashSet::new()), |constraints| {
+                constraints
+                    .try_into()
+                    .map_err(ValidateBuildRequestError::InvalidBuildConstraints)
+            })?;
 
         // validate additional_files
-        for (i, additional_file) in self.additional_files.iter().enumerate() {
+        for (i, additional_file) in value.additional_files.iter().enumerate() {
             if !is_clean_relative_path(&additional_file.path) {
                 Err(ValidateBuildRequestError::InvalidAdditionalFilePath(i))?
             }
         }
-        if !is_sorted(self.additional_files.iter().map(|e| e.path.as_bytes())) {
+        if !is_sorted(value.additional_files.iter().map(|e| e.path.as_bytes())) {
             Err(ValidateBuildRequestError::AdditionalFilesNotSorted)?;
         }
 
-        Ok(())
+        Ok(Self {
+            inputs,
+            command_args: value.command_args,
+            working_dir: PathBuf::from(value.working_dir),
+            scratch_paths: value.scratch_paths.iter().map(PathBuf::from).collect(),
+            inputs_dir: PathBuf::from(value.inputs_dir),
+            outputs: value.outputs.iter().map(PathBuf::from).collect(),
+            environment_vars: value.environment_vars.into_iter().map(Into::into).collect(),
+            constraints,
+            additional_files: value.additional_files.into_iter().map(Into::into).collect(),
+            refscan_needles: value.refscan_needles,
+        })
     }
 }
 
@@ -214,27 +278,90 @@ pub enum ValidateBuildConstraintsError {
     AvailableRoPathsNotSorted,
 }
 
-impl build_request::BuildConstraints {
-    pub fn validate(&self) -> Result<(), ValidateBuildConstraintsError> {
+impl From<build_request::EnvVar> for crate::buildservice::EnvVar {
+    fn from(value: build_request::EnvVar) -> Self {
+        Self {
+            key: value.key,
+            value: value.value,
+        }
+    }
+}
+
+impl From<crate::buildservice::EnvVar> for build_request::EnvVar {
+    fn from(value: crate::buildservice::EnvVar) -> Self {
+        Self {
+            key: value.key,
+            value: value.value,
+        }
+    }
+}
+
+impl From<build_request::AdditionalFile> for crate::buildservice::AdditionalFile {
+    fn from(value: build_request::AdditionalFile) -> Self {
+        Self {
+            path: PathBuf::from(value.path),
+            contents: value.contents,
+        }
+    }
+}
+
+impl From<crate::buildservice::AdditionalFile> for build_request::AdditionalFile {
+    fn from(value: crate::buildservice::AdditionalFile) -> Self {
+        Self {
+            path: value
+                .path
+                .to_str()
+                .expect("Tvix bug: expected a valid path")
+                .to_string(),
+            contents: value.contents,
+        }
+    }
+}
+
+impl TryFrom<build_request::BuildConstraints> for HashSet<crate::buildservice::BuildConstraints> {
+    type Error = ValidateBuildConstraintsError;
+    fn try_from(value: build_request::BuildConstraints) -> Result<Self, Self::Error> {
+        use crate::buildservice::BuildConstraints;
+
         // validate system
-        if self.system.is_empty() {
+        if value.system.is_empty() {
             Err(ValidateBuildConstraintsError::InvalidSystem)?;
         }
+
+        let mut build_constraints = HashSet::from([
+            BuildConstraints::System(value.system),
+            BuildConstraints::MinMemory(value.min_memory),
+        ]);
+
         // validate available_ro_paths
-        for (i, p) in self.available_ro_paths.iter().enumerate() {
+        for (i, p) in value.available_ro_paths.iter().enumerate() {
             if !is_clean_absolute_path(p) {
                 Err(ValidateBuildConstraintsError::InvalidAvailableRoPaths(i))?
+            } else {
+                build_constraints.insert(BuildConstraints::AvailableReadOnlyPath(PathBuf::from(p)));
             }
         }
-        if !is_sorted(self.available_ro_paths.iter().map(|e| e.as_bytes())) {
+        if !is_sorted(value.available_ro_paths.iter().map(|e| e.as_bytes())) {
             Err(ValidateBuildConstraintsError::AvailableRoPathsNotSorted)?;
         }
 
-        Ok(())
+        if value.network_access {
+            build_constraints.insert(BuildConstraints::NetworkAccess);
+        }
+        if value.provide_bin_sh {
+            build_constraints.insert(BuildConstraints::ProvideBinSh);
+        }
+
+        Ok(build_constraints)
     }
 }
 
 #[cfg(test)]
+// TODO: add testcases for constraints special cases. The default cases in the protos
+// should result in the constraints not being added. For example min_memory 0 can be omitted.
+// Also interesting testcases are "merging semantics". MimMemory(1) and MinMemory(100) will
+// result in mim_memory 100, multiple AvailableReadOnlyPaths need to be merged. Contradicting
+// system constraints need to fail somewhere (maybe an assertion, as only buggy code can construct it)
 mod tests {
     use super::{is_clean_path, is_clean_relative_path};
     use rstest::rstest;