about summary refs log tree commit diff
path: root/tvix/castore/src
diff options
context:
space:
mode:
Diffstat (limited to 'tvix/castore/src')
-rw-r--r--tvix/castore/src/blobservice/combinator.rs96
-rw-r--r--tvix/castore/src/blobservice/from_addr.rs55
-rw-r--r--tvix/castore/src/blobservice/grpc.rs71
-rw-r--r--tvix/castore/src/blobservice/memory.rs30
-rw-r--r--tvix/castore/src/blobservice/mod.rs19
-rw-r--r--tvix/castore/src/blobservice/naive_seeker.rs265
-rw-r--r--tvix/castore/src/blobservice/object_store.rs165
-rw-r--r--tvix/castore/src/blobservice/tests/utils.rs3
-rw-r--r--tvix/castore/src/composition.rs541
-rw-r--r--tvix/castore/src/digests.rs13
-rw-r--r--tvix/castore/src/directoryservice/bigtable.rs131
-rw-r--r--tvix/castore/src/directoryservice/closure_validator.rs309
-rw-r--r--tvix/castore/src/directoryservice/combinators.rs180
-rw-r--r--tvix/castore/src/directoryservice/directory_graph.rs414
-rw-r--r--tvix/castore/src/directoryservice/from_addr.rs123
-rw-r--r--tvix/castore/src/directoryservice/grpc.rs135
-rw-r--r--tvix/castore/src/directoryservice/memory.rs62
-rw-r--r--tvix/castore/src/directoryservice/mod.rs67
-rw-r--r--tvix/castore/src/directoryservice/object_store.rs116
-rw-r--r--tvix/castore/src/directoryservice/order_validator.rs188
-rw-r--r--tvix/castore/src/directoryservice/redb.rs303
-rw-r--r--tvix/castore/src/directoryservice/simple_putter.rs17
-rw-r--r--tvix/castore/src/directoryservice/sled.rs132
-rw-r--r--tvix/castore/src/directoryservice/tests/mod.rs115
-rw-r--r--tvix/castore/src/directoryservice/tests/utils.rs3
-rw-r--r--tvix/castore/src/directoryservice/traverse.rs56
-rw-r--r--tvix/castore/src/directoryservice/utils.rs60
-rw-r--r--tvix/castore/src/errors.rs84
-rw-r--r--tvix/castore/src/fixtures.rs118
-rw-r--r--tvix/castore/src/fs/fuse/mod.rs (renamed from tvix/castore/src/fs/fuse.rs)59
-rw-r--r--tvix/castore/src/fs/fuse/tests.rs (renamed from tvix/castore/src/fs/tests.rs)164
-rw-r--r--tvix/castore/src/fs/inodes.rs33
-rw-r--r--tvix/castore/src/fs/mod.rs106
-rw-r--r--tvix/castore/src/fs/root_nodes.rs22
-rw-r--r--tvix/castore/src/import/archive.rs111
-rw-r--r--tvix/castore/src/import/blobs.rs190
-rw-r--r--tvix/castore/src/import/fs.rs57
-rw-r--r--tvix/castore/src/import/mod.rs92
-rw-r--r--tvix/castore/src/lib.rs8
-rw-r--r--tvix/castore/src/nodes/directory.rs287
-rw-r--r--tvix/castore/src/nodes/mod.rs48
-rw-r--r--tvix/castore/src/nodes/symlink_target.rs223
-rw-r--r--tvix/castore/src/path/component.rs268
-rw-r--r--tvix/castore/src/path/mod.rs (renamed from tvix/castore/src/path.rs)48
-rw-r--r--tvix/castore/src/proto/grpc_directoryservice_wrapper.rs32
-rw-r--r--tvix/castore/src/proto/mod.rs561
-rw-r--r--tvix/castore/src/proto/tests/directory.rs200
-rw-r--r--tvix/castore/src/proto/tests/directory_nodes_iterator.rs78
-rw-r--r--tvix/castore/src/proto/tests/mod.rs1
-rw-r--r--tvix/castore/src/tests/import.rs41
-rw-r--r--tvix/castore/src/tonic.rs6
51 files changed, 4323 insertions, 2183 deletions
diff --git a/tvix/castore/src/blobservice/combinator.rs b/tvix/castore/src/blobservice/combinator.rs
index 067eff96f488..6a964c8a8440 100644
--- a/tvix/castore/src/blobservice/combinator.rs
+++ b/tvix/castore/src/blobservice/combinator.rs
@@ -1,11 +1,12 @@
-use futures::{StreamExt, TryStreamExt};
-use tokio_util::io::{ReaderStream, StreamReader};
+use std::sync::Arc;
+
 use tonic::async_trait;
-use tracing::{instrument, warn};
+use tracing::instrument;
 
-use crate::B3Digest;
+use crate::composition::{CompositionContext, ServiceBuilder};
+use crate::{B3Digest, Error};
 
-use super::{naive_seeker::NaiveSeeker, BlobReader, BlobService, BlobWriter};
+use super::{BlobReader, BlobService, BlobWriter, ChunkedReader};
 
 /// Combinator for a BlobService, using a "local" and "remote" blobservice.
 /// Requests are tried in (and returned from) the local store first, only if
@@ -68,19 +69,16 @@ where
                     // otherwise, a chunked reader, which will always try the
                     // local backend first.
 
-                    // map Vec<ChunkMeta> to Vec<(B3Digest, u64)>
-                    let chunks: Vec<(B3Digest, u64)> = remote_chunks
-                        .into_iter()
-                        .map(|chunk_meta| {
+                    let chunked_reader = ChunkedReader::from_chunks(
+                        remote_chunks.into_iter().map(|chunk| {
                             (
-                                B3Digest::try_from(chunk_meta.digest)
-                                    .expect("invalid chunk digest"),
-                                chunk_meta.size,
+                                chunk.digest.try_into().expect("invalid b3 digest"),
+                                chunk.size,
                             )
-                        })
-                        .collect();
-
-                    Ok(Some(make_chunked_reader(self.clone(), chunks)))
+                        }),
+                        Arc::new(self.clone()) as Arc<dyn BlobService>,
+                    );
+                    Ok(Some(Box::new(chunked_reader)))
                 }
             }
         }
@@ -93,40 +91,38 @@ where
     }
 }
 
-fn make_chunked_reader<BS>(
-    // This must consume, as we can't retain references to blob_service,
-    // as it'd add a lifetime to BlobReader in general, which will get
-    // problematic in TvixStoreFs, which is using async move closures and cloning.
-    blob_service: BS,
-    // A list of b3 digests for individual chunks, and their sizes.
-    chunks: Vec<(B3Digest, u64)>,
-) -> Box<dyn BlobReader>
-where
-    BS: BlobService + Clone + 'static,
-{
-    // TODO: offset, verified streaming
-
-    // construct readers for each chunk
-    let blob_service = blob_service.clone();
-    let readers_stream = tokio_stream::iter(chunks).map(move |(digest, _)| {
-        let d = digest.to_owned();
-        let blob_service = blob_service.clone();
-        async move {
-            blob_service.open_read(&d.to_owned()).await?.ok_or_else(|| {
-                warn!(chunk.digest = %digest, "chunk not found");
-                std::io::Error::new(std::io::ErrorKind::NotFound, "chunk not found")
-            })
-        }
-    });
-
-    // convert the stream of readers to a stream of streams of byte chunks
-    let bytes_streams = readers_stream.then(|elem| async { elem.await.map(ReaderStream::new) });
-
-    // flatten into one stream of byte chunks
-    let bytes_stream = bytes_streams.try_flatten();
+#[derive(serde::Deserialize, Debug, Clone)]
+#[serde(deny_unknown_fields)]
+pub struct CombinedBlobServiceConfig {
+    local: String,
+    remote: String,
+}
 
-    // convert into AsyncRead
-    let blob_reader = StreamReader::new(bytes_stream);
+impl TryFrom<url::Url> for CombinedBlobServiceConfig {
+    type Error = Box<dyn std::error::Error + Send + Sync>;
+    fn try_from(_url: url::Url) -> Result<Self, Self::Error> {
+        Err(Error::StorageError(
+            "Instantiating a CombinedBlobService from a url is not supported".into(),
+        )
+        .into())
+    }
+}
 
-    Box::new(NaiveSeeker::new(Box::pin(blob_reader)))
+#[async_trait]
+impl ServiceBuilder for CombinedBlobServiceConfig {
+    type Output = dyn BlobService;
+    async fn build<'a>(
+        &'a self,
+        _instance_name: &str,
+        context: &CompositionContext,
+    ) -> Result<Arc<dyn BlobService>, Box<dyn std::error::Error + Send + Sync>> {
+        let (local, remote) = futures::join!(
+            context.resolve(self.local.clone()),
+            context.resolve(self.remote.clone())
+        );
+        Ok(Arc::new(CombinedBlobService {
+            local: local?,
+            remote: remote?,
+        }))
+    }
 }
diff --git a/tvix/castore/src/blobservice/from_addr.rs b/tvix/castore/src/blobservice/from_addr.rs
index 8898bbfb95ce..c5cabaa9d945 100644
--- a/tvix/castore/src/blobservice/from_addr.rs
+++ b/tvix/castore/src/blobservice/from_addr.rs
@@ -1,8 +1,12 @@
+use std::sync::Arc;
+
 use url::Url;
 
-use crate::{proto::blob_service_client::BlobServiceClient, Error};
+use crate::composition::{
+    with_registry, CompositionContext, DeserializeWithRegistry, ServiceBuilder, REG,
+};
 
-use super::{BlobService, GRPCBlobService, MemoryBlobService, ObjectStoreBlobService};
+use super::BlobService;
 
 /// Constructs a new instance of a [BlobService] from an URI.
 ///
@@ -12,46 +16,19 @@ use super::{BlobService, GRPCBlobService, MemoryBlobService, ObjectStoreBlobServ
 /// - `objectstore+*://` ([ObjectStoreBlobService])
 ///
 /// See their `from_url` methods for more details about their syntax.
-pub async fn from_addr(uri: &str) -> Result<Box<dyn BlobService>, crate::Error> {
+pub async fn from_addr(
+    uri: &str,
+) -> Result<Arc<dyn BlobService>, Box<dyn std::error::Error + Send + Sync>> {
     let url = Url::parse(uri)
         .map_err(|e| crate::Error::StorageError(format!("unable to parse url: {}", e)))?;
 
-    let blob_service: Box<dyn BlobService> = match url.scheme() {
-        "memory" => {
-            // memory doesn't support host or path in the URL.
-            if url.has_host() || !url.path().is_empty() {
-                return Err(Error::StorageError("invalid url".to_string()));
-            }
-            Box::<MemoryBlobService>::default()
-        }
-        scheme if scheme.starts_with("grpc+") => {
-            // schemes starting with grpc+ go to the GRPCPathInfoService.
-            //   That's normally grpc+unix for unix sockets, and grpc+http(s) for the HTTP counterparts.
-            // - In the case of unix sockets, there must be a path, but may not be a host.
-            // - In the case of non-unix sockets, there must be a host, but no path.
-            // Constructing the channel is handled by tvix_castore::channel::from_url.
-            let client = BlobServiceClient::new(crate::tonic::channel_from_url(&url).await?);
-            Box::new(GRPCBlobService::from_client(client))
-        }
-        scheme if scheme.starts_with("objectstore+") => {
-            // We need to convert the URL to string, strip the prefix there, and then
-            // parse it back as url, as Url::set_scheme() rejects some of the transitions we want to do.
-            let trimmed_url = {
-                let s = url.to_string();
-                Url::parse(s.strip_prefix("objectstore+").unwrap()).unwrap()
-            };
-            Box::new(
-                ObjectStoreBlobService::parse_url(&trimmed_url)
-                    .map_err(|e| Error::StorageError(e.to_string()))?,
-            )
-        }
-        scheme => {
-            return Err(crate::Error::StorageError(format!(
-                "unknown scheme: {}",
-                scheme
-            )))
-        }
-    };
+    let blob_service_config = with_registry(&REG, || {
+        <DeserializeWithRegistry<Box<dyn ServiceBuilder<Output = dyn BlobService>>>>::try_from(url)
+    })?
+    .0;
+    let blob_service = blob_service_config
+        .build("anonymous", &CompositionContext::blank())
+        .await?;
 
     Ok(blob_service)
 }
diff --git a/tvix/castore/src/blobservice/grpc.rs b/tvix/castore/src/blobservice/grpc.rs
index 5663cd3838ec..0db3dfea4ad8 100644
--- a/tvix/castore/src/blobservice/grpc.rs
+++ b/tvix/castore/src/blobservice/grpc.rs
@@ -1,4 +1,5 @@
 use super::{BlobReader, BlobService, BlobWriter, ChunkedReader};
+use crate::composition::{CompositionContext, ServiceBuilder};
 use crate::{
     proto::{self, stat_blob_response::ChunkMeta},
     B3Digest,
@@ -17,40 +18,43 @@ use tokio_util::{
     io::{CopyToBytes, SinkWriter},
     sync::PollSender,
 };
-use tonic::{async_trait, transport::Channel, Code, Status};
-use tracing::instrument;
+use tonic::{async_trait, Code, Status};
+use tracing::{instrument, Instrument as _};
 
 /// Connects to a (remote) tvix-store BlobService over gRPC.
 #[derive(Clone)]
-pub struct GRPCBlobService {
+pub struct GRPCBlobService<T> {
     /// The internal reference to a gRPC client.
     /// Cloning it is cheap, and it internally handles concurrent requests.
-    grpc_client: proto::blob_service_client::BlobServiceClient<Channel>,
+    grpc_client: proto::blob_service_client::BlobServiceClient<T>,
 }
 
-impl GRPCBlobService {
+impl<T> GRPCBlobService<T> {
     /// construct a [GRPCBlobService] from a [proto::blob_service_client::BlobServiceClient].
-    /// panics if called outside the context of a tokio runtime.
-    pub fn from_client(
-        grpc_client: proto::blob_service_client::BlobServiceClient<Channel>,
-    ) -> Self {
+    pub fn from_client(grpc_client: proto::blob_service_client::BlobServiceClient<T>) -> Self {
         Self { grpc_client }
     }
 }
 
 #[async_trait]
-impl BlobService for GRPCBlobService {
+impl<T> BlobService for GRPCBlobService<T>
+where
+    T: tonic::client::GrpcService<tonic::body::BoxBody> + Send + Sync + Clone + 'static,
+    T::ResponseBody: tonic::codegen::Body<Data = tonic::codegen::Bytes> + Send + 'static,
+    <T::ResponseBody as tonic::codegen::Body>::Error: Into<tonic::codegen::StdError> + Send,
+    T::Future: Send,
+{
     #[instrument(skip(self, digest), fields(blob.digest=%digest))]
     async fn has(&self, digest: &B3Digest) -> io::Result<bool> {
-        let mut grpc_client = self.grpc_client.clone();
-        let resp = grpc_client
+        match self
+            .grpc_client
+            .clone()
             .stat(proto::StatBlobRequest {
                 digest: digest.clone().into(),
                 ..Default::default()
             })
-            .await;
-
-        match resp {
+            .await
+        {
             Ok(_blob_meta) => Ok(true),
             Err(e) if e.code() == Code::NotFound => Ok(false),
             Err(e) => Err(io::Error::new(io::ErrorKind::Other, e)),
@@ -133,6 +137,8 @@ impl BlobService for GRPCBlobService {
         let task = tokio::spawn({
             let mut grpc_client = self.grpc_client.clone();
             async move { Ok::<_, Status>(grpc_client.put(blobchunk_stream).await?.into_inner()) }
+                // instrument the task with the current span, this is not done by default
+                .in_current_span()
         });
 
         // The tx part of the channel is converted to a sink of byte chunks.
@@ -175,6 +181,40 @@ impl BlobService for GRPCBlobService {
     }
 }
 
+#[derive(serde::Deserialize, Debug)]
+#[serde(deny_unknown_fields)]
+pub struct GRPCBlobServiceConfig {
+    url: String,
+}
+
+impl TryFrom<url::Url> for GRPCBlobServiceConfig {
+    type Error = Box<dyn std::error::Error + Send + Sync>;
+    fn try_from(url: url::Url) -> Result<Self, Self::Error> {
+        //   normally grpc+unix for unix sockets, and grpc+http(s) for the HTTP counterparts.
+        // - In the case of unix sockets, there must be a path, but may not be a host.
+        // - In the case of non-unix sockets, there must be a host, but no path.
+        // Constructing the channel is handled by tvix_castore::channel::from_url.
+        Ok(GRPCBlobServiceConfig {
+            url: url.to_string(),
+        })
+    }
+}
+
+#[async_trait]
+impl ServiceBuilder for GRPCBlobServiceConfig {
+    type Output = dyn BlobService;
+    async fn build<'a>(
+        &'a self,
+        _instance_name: &str,
+        _context: &CompositionContext,
+    ) -> Result<Arc<dyn BlobService>, Box<dyn std::error::Error + Send + Sync + 'static>> {
+        let client = proto::blob_service_client::BlobServiceClient::new(
+            crate::tonic::channel_from_url(&self.url.parse()?).await?,
+        );
+        Ok(Arc::new(GRPCBlobService::from_client(client)))
+    }
+}
+
 pub struct GRPCBlobWriter<W: tokio::io::AsyncWrite> {
     /// The task containing the put request, and the inner writer, if we're still writing.
     task_and_writer: Option<(JoinHandle<Result<proto::PutBlobResponse, Status>>, W)>,
@@ -335,7 +375,6 @@ mod tests {
                     .await
                     .expect("must succeed"),
             );
-
             GRPCBlobService::from_client(client)
         };
 
diff --git a/tvix/castore/src/blobservice/memory.rs b/tvix/castore/src/blobservice/memory.rs
index 873d06b461de..3d733f950470 100644
--- a/tvix/castore/src/blobservice/memory.rs
+++ b/tvix/castore/src/blobservice/memory.rs
@@ -6,7 +6,8 @@ use tonic::async_trait;
 use tracing::instrument;
 
 use super::{BlobReader, BlobService, BlobWriter};
-use crate::B3Digest;
+use crate::composition::{CompositionContext, ServiceBuilder};
+use crate::{B3Digest, Error};
 
 #[derive(Clone, Default)]
 pub struct MemoryBlobService {
@@ -37,6 +38,33 @@ impl BlobService for MemoryBlobService {
     }
 }
 
+#[derive(serde::Deserialize, Debug)]
+#[serde(deny_unknown_fields)]
+pub struct MemoryBlobServiceConfig {}
+
+impl TryFrom<url::Url> for MemoryBlobServiceConfig {
+    type Error = Box<dyn std::error::Error + Send + Sync>;
+    fn try_from(url: url::Url) -> Result<Self, Self::Error> {
+        // memory doesn't support host or path in the URL.
+        if url.has_host() || !url.path().is_empty() {
+            return Err(Error::StorageError("invalid url".to_string()).into());
+        }
+        Ok(MemoryBlobServiceConfig {})
+    }
+}
+
+#[async_trait]
+impl ServiceBuilder for MemoryBlobServiceConfig {
+    type Output = dyn BlobService;
+    async fn build<'a>(
+        &'a self,
+        _instance_name: &str,
+        _context: &CompositionContext,
+    ) -> Result<Arc<dyn BlobService>, Box<dyn std::error::Error + Send + Sync + 'static>> {
+        Ok(Arc::new(MemoryBlobService::default()))
+    }
+}
+
 pub struct MemoryBlobWriter {
     db: Arc<RwLock<HashMap<B3Digest, Vec<u8>>>>,
 
diff --git a/tvix/castore/src/blobservice/mod.rs b/tvix/castore/src/blobservice/mod.rs
index 50acd40bf769..85292722fa7e 100644
--- a/tvix/castore/src/blobservice/mod.rs
+++ b/tvix/castore/src/blobservice/mod.rs
@@ -1,6 +1,8 @@
 use std::io;
+
 use tonic::async_trait;
 
+use crate::composition::{Registry, ServiceBuilder};
 use crate::proto::stat_blob_response::ChunkMeta;
 use crate::B3Digest;
 
@@ -9,18 +11,17 @@ mod combinator;
 mod from_addr;
 mod grpc;
 mod memory;
-mod naive_seeker;
 mod object_store;
 
 #[cfg(test)]
 pub mod tests;
 
 pub use self::chunked_reader::ChunkedReader;
-pub use self::combinator::CombinedBlobService;
+pub use self::combinator::{CombinedBlobService, CombinedBlobServiceConfig};
 pub use self::from_addr::from_addr;
-pub use self::grpc::GRPCBlobService;
-pub use self::memory::MemoryBlobService;
-pub use self::object_store::ObjectStoreBlobService;
+pub use self::grpc::{GRPCBlobService, GRPCBlobServiceConfig};
+pub use self::memory::{MemoryBlobService, MemoryBlobServiceConfig};
+pub use self::object_store::{ObjectStoreBlobService, ObjectStoreBlobServiceConfig};
 
 /// The base trait all BlobService services need to implement.
 /// It provides functions to check whether a given blob exists,
@@ -101,3 +102,11 @@ impl BlobReader for io::Cursor<&'static [u8; 0]> {}
 impl BlobReader for io::Cursor<Vec<u8>> {}
 impl BlobReader for io::Cursor<bytes::Bytes> {}
 impl BlobReader for tokio::fs::File {}
+
+/// Registers the builtin BlobService implementations with the registry
+pub(crate) fn register_blob_services(reg: &mut Registry) {
+    reg.register::<Box<dyn ServiceBuilder<Output = dyn BlobService>>, super::blobservice::ObjectStoreBlobServiceConfig>("objectstore");
+    reg.register::<Box<dyn ServiceBuilder<Output = dyn BlobService>>, super::blobservice::MemoryBlobServiceConfig>("memory");
+    reg.register::<Box<dyn ServiceBuilder<Output = dyn BlobService>>, super::blobservice::CombinedBlobServiceConfig>("combined");
+    reg.register::<Box<dyn ServiceBuilder<Output = dyn BlobService>>, super::blobservice::GRPCBlobServiceConfig>("grpc");
+}
diff --git a/tvix/castore/src/blobservice/naive_seeker.rs b/tvix/castore/src/blobservice/naive_seeker.rs
deleted file mode 100644
index f5a530715093..000000000000
--- a/tvix/castore/src/blobservice/naive_seeker.rs
+++ /dev/null
@@ -1,265 +0,0 @@
-use super::BlobReader;
-use futures::ready;
-use pin_project_lite::pin_project;
-use std::io;
-use std::task::Poll;
-use tokio::io::AsyncRead;
-use tracing::{debug, instrument, trace, warn};
-
-pin_project! {
-    /// This implements [tokio::io::AsyncSeek] for and [tokio::io::AsyncRead] by
-    /// simply skipping over some bytes, keeping track of the position.
-    /// It fails whenever you try to seek backwards.
-    ///
-    /// ## Pinning concerns:
-    ///
-    /// [NaiveSeeker] is itself pinned by callers, and we do not need to concern
-    /// ourselves regarding that.
-    ///
-    /// Though, its fields as per
-    /// <https://doc.rust-lang.org/std/pin/#pinning-is-not-structural-for-field>
-    /// can be pinned or unpinned.
-    ///
-    /// So we need to go over each field and choose our policy carefully.
-    ///
-    /// The obvious cases are the bookkeeping integers we keep in the structure,
-    /// those are private and not shared to anyone, we never build a
-    /// `Pin<&mut X>` out of them at any point, therefore, we can safely never
-    /// mark them as pinned. Of course, it is expected that no developer here
-    /// attempt to `pin!(self.pos)` to pin them because it makes no sense. If
-    /// they have to become pinned, they should be marked `#[pin]` and we need
-    /// to discuss it.
-    ///
-    /// So the bookkeeping integers are in the right state with respect to their
-    /// pinning status. The projection should offer direct access.
-    ///
-    /// On the `r` field, i.e. a `BufReader<R>`, given that
-    /// <https://docs.rs/tokio/latest/tokio/io/struct.BufReader.html#impl-Unpin-for-BufReader%3CR%3E>
-    /// is available, even a `Pin<&mut BufReader<R>>` can be safely moved.
-    ///
-    /// The only care we should have regards the internal reader itself, i.e.
-    /// the `R` instance, see that Tokio decided to `#[pin]` it too:
-    /// <https://docs.rs/tokio/latest/src/tokio/io/util/buf_reader.rs.html#29>
-    ///
-    /// In general, there's no `Unpin` instance for `R: tokio::io::AsyncRead`
-    /// (see <https://docs.rs/tokio/latest/tokio/io/trait.AsyncRead.html>).
-    ///
-    /// Therefore, we could keep it unpinned and pin it in every call site
-    /// whenever we need to call `poll_*` which can be confusing to the non-
-    /// expert developer and we have a fair share amount of situations where the
-    /// [BufReader] instance is naked, i.e. in its `&mut BufReader<R>`
-    /// form, this is annoying because it could lead to expose the naked `R`
-    /// internal instance somehow and would produce a risk of making it move
-    /// unexpectedly.
-    ///
-    /// We choose the path of the least resistance as we have no reason to have
-    /// access to the raw `BufReader<R>` instance, we just `#[pin]` it too and
-    /// enjoy its `poll_*` safe APIs and push the unpinning concerns to the
-    /// internal implementations themselves, which studied the question longer
-    /// than us.
-    pub struct NaiveSeeker<R: tokio::io::AsyncRead> {
-        #[pin]
-        r: tokio::io::BufReader<R>,
-        pos: u64,
-        bytes_to_skip: u64,
-    }
-}
-
-/// The buffer size used to discard data.
-const DISCARD_BUF_SIZE: usize = 4096;
-
-impl<R: tokio::io::AsyncRead> NaiveSeeker<R> {
-    pub fn new(r: R) -> Self {
-        NaiveSeeker {
-            r: tokio::io::BufReader::new(r),
-            pos: 0,
-            bytes_to_skip: 0,
-        }
-    }
-}
-
-impl<R: tokio::io::AsyncRead> tokio::io::AsyncRead for NaiveSeeker<R> {
-    #[instrument(level = "trace", skip_all)]
-    fn poll_read(
-        self: std::pin::Pin<&mut Self>,
-        cx: &mut std::task::Context<'_>,
-        buf: &mut tokio::io::ReadBuf<'_>,
-    ) -> Poll<std::io::Result<()>> {
-        // The amount of data read can be determined by the increase
-        // in the length of the slice returned by `ReadBuf::filled`.
-        let filled_before = buf.filled().len();
-
-        let this = self.project();
-        ready!(this.r.poll_read(cx, buf))?;
-
-        let bytes_read = buf.filled().len() - filled_before;
-        *this.pos += bytes_read as u64;
-
-        trace!(bytes_read = bytes_read, new_pos = this.pos, "poll_read");
-
-        Ok(()).into()
-    }
-}
-
-impl<R: tokio::io::AsyncRead> tokio::io::AsyncBufRead for NaiveSeeker<R> {
-    fn poll_fill_buf(
-        self: std::pin::Pin<&mut Self>,
-        cx: &mut std::task::Context<'_>,
-    ) -> Poll<io::Result<&[u8]>> {
-        self.project().r.poll_fill_buf(cx)
-    }
-
-    #[instrument(level = "trace", skip(self))]
-    fn consume(self: std::pin::Pin<&mut Self>, amt: usize) {
-        let this = self.project();
-        this.r.consume(amt);
-        *this.pos += amt as u64;
-
-        trace!(new_pos = this.pos, "consume");
-    }
-}
-
-impl<R: tokio::io::AsyncRead> tokio::io::AsyncSeek for NaiveSeeker<R> {
-    #[instrument(level="trace", skip(self), fields(inner_pos=%self.pos), err(Debug))]
-    fn start_seek(
-        self: std::pin::Pin<&mut Self>,
-        position: std::io::SeekFrom,
-    ) -> std::io::Result<()> {
-        let absolute_offset: u64 = match position {
-            io::SeekFrom::Start(start_offset) => {
-                if start_offset < self.pos {
-                    return Err(io::Error::new(
-                        io::ErrorKind::Unsupported,
-                        format!("can't seek backwards ({} -> {})", self.pos, start_offset),
-                    ));
-                } else {
-                    start_offset
-                }
-            }
-            // we don't know the total size, can't support this.
-            io::SeekFrom::End(_end_offset) => {
-                return Err(io::Error::new(
-                    io::ErrorKind::Unsupported,
-                    "can't seek from end",
-                ));
-            }
-            io::SeekFrom::Current(relative_offset) => {
-                if relative_offset < 0 {
-                    return Err(io::Error::new(
-                        io::ErrorKind::Unsupported,
-                        "can't seek backwards relative to current position",
-                    ));
-                } else {
-                    self.pos + relative_offset as u64
-                }
-            }
-        };
-
-        // we already know absolute_offset is >= self.pos
-        debug_assert!(
-            absolute_offset >= self.pos,
-            "absolute_offset {} must be >= self.pos {}",
-            absolute_offset,
-            self.pos
-        );
-
-        // calculate bytes to skip
-        let this = self.project();
-        *this.bytes_to_skip = absolute_offset - *this.pos;
-
-        debug!(bytes_to_skip = *this.bytes_to_skip, "seek");
-
-        Ok(())
-    }
-
-    #[instrument(skip_all)]
-    fn poll_complete(
-        mut self: std::pin::Pin<&mut Self>,
-        cx: &mut std::task::Context<'_>,
-    ) -> Poll<std::io::Result<u64>> {
-        if self.bytes_to_skip == 0 {
-            // return the new position (from the start of the stream)
-            return Poll::Ready(Ok(self.pos));
-        }
-
-        // discard some bytes, until pos is where we want it to be.
-        // We create a buffer that we'll discard later on.
-        let mut discard_buf = [0; DISCARD_BUF_SIZE];
-
-        // Loop until we've reached the desired seek position. This is done by issuing repeated
-        // `poll_read` calls.
-        // If the data is not available yet, we will yield back to the executor
-        // and wait to be polled again.
-        loop {
-            if self.bytes_to_skip == 0 {
-                return Poll::Ready(Ok(self.pos));
-            }
-
-            // calculate the length we want to skip at most, which is either a max
-            // buffer size, or the number of remaining bytes to read, whatever is
-            // smaller.
-            let bytes_to_skip_now = std::cmp::min(self.bytes_to_skip as usize, discard_buf.len());
-            let mut discard_buf = tokio::io::ReadBuf::new(&mut discard_buf[..bytes_to_skip_now]);
-
-            ready!(self.as_mut().poll_read(cx, &mut discard_buf))?;
-            let bytes_skipped = discard_buf.filled().len();
-
-            if bytes_skipped == 0 {
-                return Poll::Ready(Err(io::Error::new(
-                    io::ErrorKind::UnexpectedEof,
-                    "got EOF while trying to skip bytes",
-                )));
-            }
-            // decrement bytes to skip. The poll_read call already updated self.pos.
-            *self.as_mut().project().bytes_to_skip -= bytes_skipped as u64;
-        }
-    }
-}
-
-impl<R: tokio::io::AsyncRead + Send + Unpin + 'static> BlobReader for NaiveSeeker<R> {}
-
-#[cfg(test)]
-mod tests {
-    use super::{NaiveSeeker, DISCARD_BUF_SIZE};
-    use std::io::{Cursor, SeekFrom};
-    use tokio::io::{AsyncReadExt, AsyncSeekExt};
-
-    /// This seek requires multiple `poll_read` as we use a multiples of
-    /// DISCARD_BUF_SIZE when doing the seek.
-    /// This ensures we don't hang indefinitely.
-    #[tokio::test]
-    async fn seek() {
-        let buf = vec![0u8; DISCARD_BUF_SIZE * 4];
-        let reader = Cursor::new(&buf);
-        let mut seeker = NaiveSeeker::new(reader);
-        seeker.seek(SeekFrom::Start(4000)).await.unwrap();
-    }
-
-    #[tokio::test]
-    async fn seek_read() {
-        let mut buf = vec![0u8; DISCARD_BUF_SIZE * 2];
-        buf.extend_from_slice(&[1u8; DISCARD_BUF_SIZE * 2]);
-        buf.extend_from_slice(&[2u8; DISCARD_BUF_SIZE * 2]);
-
-        let reader = Cursor::new(&buf);
-        let mut seeker = NaiveSeeker::new(reader);
-
-        let mut read_buf = vec![0u8; DISCARD_BUF_SIZE];
-        seeker.read_exact(&mut read_buf).await.expect("must read");
-        assert_eq!(read_buf.as_slice(), &[0u8; DISCARD_BUF_SIZE]);
-
-        seeker
-            .seek(SeekFrom::Current(DISCARD_BUF_SIZE as i64))
-            .await
-            .expect("must seek");
-        seeker.read_exact(&mut read_buf).await.expect("must read");
-        assert_eq!(read_buf.as_slice(), &[1u8; DISCARD_BUF_SIZE]);
-
-        seeker
-            .seek(SeekFrom::Start(2 * 2 * DISCARD_BUF_SIZE as u64))
-            .await
-            .expect("must seek");
-        seeker.read_exact(&mut read_buf).await.expect("must read");
-        assert_eq!(read_buf.as_slice(), &[2u8; DISCARD_BUF_SIZE]);
-    }
-}
diff --git a/tvix/castore/src/blobservice/object_store.rs b/tvix/castore/src/blobservice/object_store.rs
index d2d0a288a557..5bb05cf26123 100644
--- a/tvix/castore/src/blobservice/object_store.rs
+++ b/tvix/castore/src/blobservice/object_store.rs
@@ -1,4 +1,5 @@
 use std::{
+    collections::HashMap,
     io::{self, Cursor},
     pin::pin,
     sync::Arc,
@@ -18,22 +19,13 @@ use tracing::{debug, instrument, trace, Level};
 use url::Url;
 
 use crate::{
+    composition::{CompositionContext, ServiceBuilder},
     proto::{stat_blob_response::ChunkMeta, StatBlobResponse},
-    B3Digest, B3HashingReader,
+    B3Digest, B3HashingReader, Error,
 };
 
 use super::{BlobReader, BlobService, BlobWriter, ChunkedReader};
 
-#[derive(Clone)]
-pub struct ObjectStoreBlobService {
-    object_store: Arc<dyn ObjectStore>,
-    base_path: Path,
-
-    /// Average chunk size for FastCDC, in bytes.
-    /// min value is half, max value double of that number.
-    avg_chunk_size: u32,
-}
-
 /// Uses any object storage supported by the [object_store] crate to provide a
 /// tvix-castore [BlobService].
 ///
@@ -70,31 +62,14 @@ pub struct ObjectStoreBlobService {
 /// It also allows signalling any compression of chunks in the content-type.
 /// Migration *should* be possible by simply adding the right content-types to
 /// all keys stored so far, but no promises ;-)
-impl ObjectStoreBlobService {
-    /// Constructs a new [ObjectStoreBlobService] from a [Url] supported by
-    /// [object_store].
-    /// Any path suffix becomes the base path of the object store.
-    /// additional options, the same as in [object_store::parse_url_opts] can
-    /// be passed.
-    pub fn parse_url_opts<I, K, V>(url: &Url, options: I) -> Result<Self, object_store::Error>
-    where
-        I: IntoIterator<Item = (K, V)>,
-        K: AsRef<str>,
-        V: Into<String>,
-    {
-        let (object_store, path) = object_store::parse_url_opts(url, options)?;
-
-        Ok(Self {
-            object_store: Arc::new(object_store),
-            base_path: path,
-            avg_chunk_size: 256 * 1024,
-        })
-    }
+#[derive(Clone)]
+pub struct ObjectStoreBlobService {
+    object_store: Arc<dyn ObjectStore>,
+    base_path: Path,
 
-    /// Like [Self::parse_url_opts], except without the options.
-    pub fn parse_url(url: &Url) -> Result<Self, object_store::Error> {
-        Self::parse_url_opts(url, Vec::<(String, String)>::new())
-    }
+    /// Average chunk size for FastCDC, in bytes.
+    /// min value is half, max value double of that number.
+    avg_chunk_size: u32,
 }
 
 #[instrument(level=Level::TRACE, skip_all,fields(base_path=%base_path,blob.digest=%digest),ret(Display))]
@@ -269,6 +244,71 @@ impl BlobService for ObjectStoreBlobService {
     }
 }
 
+fn default_avg_chunk_size() -> u32 {
+    256 * 1024
+}
+
+#[derive(serde::Deserialize)]
+#[serde(deny_unknown_fields)]
+pub struct ObjectStoreBlobServiceConfig {
+    object_store_url: String,
+    #[serde(default = "default_avg_chunk_size")]
+    avg_chunk_size: u32,
+    object_store_options: HashMap<String, String>,
+}
+
+impl TryFrom<url::Url> for ObjectStoreBlobServiceConfig {
+    type Error = Box<dyn std::error::Error + Send + Sync>;
+    /// Constructs a new [ObjectStoreBlobService] from a [Url] supported by
+    /// [object_store].
+    /// Any path suffix becomes the base path of the object store.
+    /// additional options, the same as in [object_store::parse_url_opts] can
+    /// be passed.
+    fn try_from(url: url::Url) -> Result<Self, Self::Error> {
+        // We need to convert the URL to string, strip the prefix there, and then
+        // parse it back as url, as Url::set_scheme() rejects some of the transitions we want to do.
+        let trimmed_url = {
+            let s = url.to_string();
+            let mut url = Url::parse(
+                s.strip_prefix("objectstore+")
+                    .ok_or(Error::StorageError("Missing objectstore uri".into()))?,
+            )?;
+            // trim the query pairs, they might contain credentials or local settings we don't want to send as-is.
+            url.set_query(None);
+            url
+        };
+        Ok(ObjectStoreBlobServiceConfig {
+            object_store_url: trimmed_url.into(),
+            object_store_options: url
+                .query_pairs()
+                .into_iter()
+                .map(|(k, v)| (k.to_string(), v.to_string()))
+                .collect(),
+            avg_chunk_size: 256 * 1024,
+        })
+    }
+}
+
+#[async_trait]
+impl ServiceBuilder for ObjectStoreBlobServiceConfig {
+    type Output = dyn BlobService;
+    async fn build<'a>(
+        &'a self,
+        _instance_name: &str,
+        _context: &CompositionContext,
+    ) -> Result<Arc<dyn BlobService>, Box<dyn std::error::Error + Send + Sync + 'static>> {
+        let (object_store, path) = object_store::parse_url_opts(
+            &self.object_store_url.parse()?,
+            &self.object_store_options,
+        )?;
+        Ok(Arc::new(ObjectStoreBlobService {
+            object_store: Arc::new(object_store),
+            base_path: path,
+            avg_chunk_size: self.avg_chunk_size,
+        }))
+    }
+}
+
 /// Reads blob contents from a AsyncRead, chunks and uploads them.
 /// On success, returns a [StatBlobResponse] pointing to the individual chunks.
 #[instrument(skip_all, fields(base_path=%base_path, min_chunk_size, avg_chunk_size, max_chunk_size), err)]
@@ -309,6 +349,15 @@ async fn chunk_and_upload<R: AsyncRead + Unpin>(
         .collect::<io::Result<Vec<ChunkMeta>>>()
         .await?;
 
+    let chunks = if chunks.len() < 2 {
+        // The chunker returned only one chunk, which is the entire blob.
+        // According to the protocol, we must return an empty list of chunks
+        // when the blob is not split up further.
+        vec![]
+    } else {
+        chunks
+    };
+
     let stat_blob_response = StatBlobResponse {
         chunks,
         bao: "".into(), // still todo
@@ -512,24 +561,35 @@ where
 
 #[cfg(test)]
 mod test {
-    use super::chunk_and_upload;
+    use super::{chunk_and_upload, default_avg_chunk_size};
     use crate::{
         blobservice::{BlobService, ObjectStoreBlobService},
-        fixtures::{BLOB_A, BLOB_A_DIGEST},
+        fixtures::{BLOB_A, BLOB_A_DIGEST, BLOB_B, BLOB_B_DIGEST},
     };
     use std::{io::Cursor, sync::Arc};
     use url::Url;
 
     /// Tests chunk_and_upload directly, bypassing the BlobWriter at open_write().
+    #[rstest::rstest]
+    #[case::a(&BLOB_A, &BLOB_A_DIGEST)]
+    #[case::b(&BLOB_B, &BLOB_B_DIGEST)]
     #[tokio::test]
-    async fn test_chunk_and_upload() {
-        let blobsvc = Arc::new(
-            ObjectStoreBlobService::parse_url(&Url::parse("memory:///").unwrap()).unwrap(),
-        );
-
-        let blob_digest = chunk_and_upload(
-            &mut Cursor::new(BLOB_A.to_vec()),
-            blobsvc.object_store.clone(),
+    async fn test_chunk_and_upload(
+        #[case] blob: &bytes::Bytes,
+        #[case] blob_digest: &crate::B3Digest,
+    ) {
+        let (object_store, base_path) =
+            object_store::parse_url(&Url::parse("memory:///").unwrap()).unwrap();
+        let object_store: Arc<dyn object_store::ObjectStore> = Arc::from(object_store);
+        let blobsvc = Arc::new(ObjectStoreBlobService {
+            object_store: object_store.clone(),
+            avg_chunk_size: default_avg_chunk_size(),
+            base_path,
+        });
+
+        let inserted_blob_digest = chunk_and_upload(
+            &mut Cursor::new(blob.to_vec()),
+            object_store,
             object_store::path::Path::from("/"),
             1024 / 2,
             1024,
@@ -538,9 +598,20 @@ mod test {
         .await
         .expect("chunk_and_upload succeeds");
 
-        assert_eq!(BLOB_A_DIGEST.clone(), blob_digest);
+        assert_eq!(blob_digest.clone(), inserted_blob_digest);
 
         // Now we should have the blob
-        assert!(blobsvc.has(&BLOB_A_DIGEST).await.unwrap());
+        assert!(blobsvc.has(blob_digest).await.unwrap());
+
+        // Check if it was chunked correctly
+        let chunks = blobsvc.chunks(blob_digest).await.unwrap().unwrap();
+        if blob.len() < 1024 / 2 {
+            // The blob is smaller than the min chunk size, it should have been inserted as a whole
+            assert!(chunks.is_empty());
+        } else if blob.len() > 1024 * 2 {
+            // The blob is larger than the max chunk size, make sure it was split up into at least
+            // two chunks
+            assert!(chunks.len() >= 2);
+        }
     }
 }
diff --git a/tvix/castore/src/blobservice/tests/utils.rs b/tvix/castore/src/blobservice/tests/utils.rs
index 706c4b5e4319..7df4f00d3a09 100644
--- a/tvix/castore/src/blobservice/tests/utils.rs
+++ b/tvix/castore/src/blobservice/tests/utils.rs
@@ -2,6 +2,7 @@ use crate::blobservice::{BlobService, MemoryBlobService};
 use crate::proto::blob_service_client::BlobServiceClient;
 use crate::proto::GRPCBlobServiceWrapper;
 use crate::{blobservice::GRPCBlobService, proto::blob_service_server::BlobServiceServer};
+use hyper_util::rt::TokioIo;
 use tonic::transport::{Endpoint, Server, Uri};
 
 /// Constructs and returns a gRPC BlobService.
@@ -33,7 +34,7 @@ pub async fn make_grpc_blob_service_client() -> Box<dyn BlobService> {
             .unwrap()
             .connect_with_connector(tower::service_fn(move |_: Uri| {
                 let right = maybe_right.take().unwrap();
-                async move { Ok::<_, std::io::Error>(right) }
+                async move { Ok::<_, std::io::Error>(TokioIo::new(right)) }
             }))
             .await
             .unwrap(),
diff --git a/tvix/castore/src/composition.rs b/tvix/castore/src/composition.rs
new file mode 100644
index 000000000000..c76daafc523d
--- /dev/null
+++ b/tvix/castore/src/composition.rs
@@ -0,0 +1,541 @@
+//! The composition module allows composing different kinds of services based on a set of service
+//! configurations _at runtime_.
+//!
+//! Store configs are deserialized with serde. The registry provides a stateful mapping from the
+//! `type` tag of an internally tagged enum on the serde side to a Config struct which is
+//! deserialized and then returned as a `Box<dyn ServiceBuilder<Output = dyn BlobService>>`
+//! (the same for DirectoryService instead of BlobService etc).
+//!
+//! ### Example 1.: Implementing a new BlobService
+//!
+//! You need a Config struct which implements `DeserializeOwned` and
+//! `ServiceBuilder<Output = dyn BlobService>`.
+//! Provide the user with a function to call with
+//! their registry. You register your new type as:
+//!
+//! ```
+//! use std::sync::Arc;
+//!
+//! use tvix_castore::composition::*;
+//! use tvix_castore::blobservice::BlobService;
+//!
+//! #[derive(serde::Deserialize)]
+//! struct MyBlobServiceConfig {
+//! }
+//!
+//! #[tonic::async_trait]
+//! impl ServiceBuilder for MyBlobServiceConfig {
+//!     type Output = dyn BlobService;
+//!     async fn build(&self, _: &str, _: &CompositionContext) -> Result<Arc<Self::Output>, Box<dyn std::error::Error + Send + Sync + 'static>> {
+//!         todo!()
+//!     }
+//! }
+//!
+//! impl TryFrom<url::Url> for MyBlobServiceConfig {
+//!     type Error = Box<dyn std::error::Error + Send + Sync>;
+//!     fn try_from(url: url::Url) -> Result<Self, Self::Error> {
+//!         todo!()
+//!     }
+//! }
+//!
+//! pub fn add_my_service(reg: &mut Registry) {
+//!     reg.register::<Box<dyn ServiceBuilder<Output = dyn BlobService>>, MyBlobServiceConfig>("myblobservicetype");
+//! }
+//! ```
+//!
+//! Now, when a user deserializes a store config with the type tag "myblobservicetype" into a
+//! `Box<dyn ServiceBuilder<Output = Arc<dyn BlobService>>>`, it will be done via `MyBlobServiceConfig`.
+//!
+//! ### Example 2.: Composing stores to get one store
+//!
+//! ```
+//! use std::sync::Arc;
+//! use tvix_castore::composition::*;
+//! use tvix_castore::blobservice::BlobService;
+//!
+//! # fn main() -> Result<(), Box<dyn std::error::Error>> {
+//! # tokio::runtime::Builder::new_current_thread().enable_all().build().unwrap().block_on(async move {
+//! let blob_services_configs_json = serde_json::json!({
+//!   "blobstore1": {
+//!     "type": "memory"
+//!   },
+//!   "blobstore2": {
+//!     "type": "memory"
+//!   },
+//!   "default": {
+//!     "type": "combined",
+//!     "local": "blobstore1",
+//!     "remote": "blobstore2"
+//!   }
+//! });
+//!
+//! let blob_services_configs = with_registry(&REG, || serde_json::from_value(blob_services_configs_json))?;
+//! let mut blob_service_composition = Composition::default();
+//! blob_service_composition.extend_with_configs::<dyn BlobService>(blob_services_configs);
+//! let blob_service: Arc<dyn BlobService> = blob_service_composition.build("default").await?;
+//! # Ok(())
+//! # })
+//! # }
+//! ```
+//!
+//! ### Example 3.: Creating another registry extending the default registry with third-party types
+//!
+//! ```
+//! # pub fn add_my_service(reg: &mut tvix_castore::composition::Registry) {}
+//! let mut my_registry = tvix_castore::composition::Registry::default();
+//! tvix_castore::composition::add_default_services(&mut my_registry);
+//! add_my_service(&mut my_registry);
+//! ```
+//!
+//! Continue with Example 2, with my_registry instead of REG
+
+use erased_serde::deserialize;
+use futures::future::BoxFuture;
+use futures::FutureExt;
+use lazy_static::lazy_static;
+use serde::de::DeserializeOwned;
+use serde_tagged::de::{BoxFnSeed, SeedFactory};
+use serde_tagged::util::TagString;
+use std::any::{Any, TypeId};
+use std::cell::Cell;
+use std::collections::BTreeMap;
+use std::collections::HashMap;
+use std::marker::PhantomData;
+use std::sync::Arc;
+use tonic::async_trait;
+
+/// Resolves tag names to the corresponding Config type.
+// Registry implementation details:
+// This is really ugly. Really we would want to store this as a generic static field:
+//
+// ```
+// struct Registry<T>(BTreeMap<(&'static str), RegistryEntry<T>);
+// static REG<T>: Registry<T>;
+// ```
+//
+// so that one version of the static is generated for each Type that the registry is accessed for.
+// However, this is not possible, because generics are only a thing in functions, and even there
+// they will not interact with static items:
+// https://doc.rust-lang.org/reference/items/static-items.html#statics--generics
+//
+// So instead, we make this lookup at runtime by putting the TypeId into the key.
+// But now we can no longer store the `BoxFnSeed<T>` because we are lacking the generic parameter
+// T, so instead store it as `Box<dyn Any>` and downcast to `&BoxFnSeed<T>` when performing the
+// lookup.
+// I said it was ugly...
+#[derive(Default)]
+pub struct Registry(BTreeMap<(TypeId, &'static str), Box<dyn Any + Sync>>);
+pub type FromUrlSeed<T> =
+    Box<dyn Fn(url::Url) -> Result<T, Box<dyn std::error::Error + Send + Sync>> + Sync>;
+pub struct RegistryEntry<T> {
+    serde_deserialize_seed: BoxFnSeed<DeserializeWithRegistry<T>>,
+    from_url_seed: FromUrlSeed<DeserializeWithRegistry<T>>,
+}
+
+struct RegistryWithFakeType<'r, T>(&'r Registry, PhantomData<T>);
+
+impl<'r, 'de: 'r, T: 'static> SeedFactory<'de, TagString<'de>> for RegistryWithFakeType<'r, T> {
+    type Value = DeserializeWithRegistry<T>;
+    type Seed = &'r BoxFnSeed<Self::Value>;
+
+    // Required method
+    fn seed<E>(self, tag: TagString<'de>) -> Result<Self::Seed, E>
+    where
+        E: serde::de::Error,
+    {
+        // using find() and not get() because of https://github.com/rust-lang/rust/issues/80389
+        let seed: &Box<dyn Any + Sync> = self
+            .0
+             .0
+            .iter()
+            .find(|(k, _)| *k == &(TypeId::of::<T>(), tag.as_ref()))
+            .ok_or_else(|| serde::de::Error::custom(format!("Unknown type: {}", tag)))?
+            .1;
+
+        let entry: &RegistryEntry<T> = <dyn Any>::downcast_ref(&**seed).unwrap();
+
+        Ok(&entry.serde_deserialize_seed)
+    }
+}
+
+/// Wrapper type which implements Deserialize using the registry
+///
+/// Wrap your type in this in order to deserialize it using a registry, e.g.
+/// `RegistryWithFakeType<Box<dyn MyTrait>>`, then the types registered for `Box<dyn MyTrait>`
+/// will be used.
+pub struct DeserializeWithRegistry<T>(pub T);
+
+impl Registry {
+    /// Registers a mapping from type tag to a concrete type into the registry.
+    ///
+    /// The type parameters are very important:
+    /// After calling `register::<Box<dyn FooTrait>, FooStruct>("footype")`, when a user
+    /// deserializes into an input with the type tag "myblobservicetype" into a
+    /// `Box<dyn FooTrait>`, it will first call the Deserialize imple of `FooStruct` and
+    /// then convert it into a `Box<dyn FooTrait>` using From::from.
+    pub fn register<
+        T: 'static,
+        C: DeserializeOwned
+            + TryFrom<url::Url, Error = Box<dyn std::error::Error + Send + Sync>>
+            + Into<T>,
+    >(
+        &mut self,
+        type_name: &'static str,
+    ) {
+        self.0.insert(
+            (TypeId::of::<T>(), type_name),
+            Box::new(RegistryEntry {
+                serde_deserialize_seed: BoxFnSeed::new(|x| {
+                    deserialize::<C>(x)
+                        .map(Into::into)
+                        .map(DeserializeWithRegistry)
+                }),
+                from_url_seed: Box::new(|url| {
+                    C::try_from(url)
+                        .map(Into::into)
+                        .map(DeserializeWithRegistry)
+                }),
+            }),
+        );
+    }
+}
+
+impl<'de, T: 'static> serde::Deserialize<'de> for DeserializeWithRegistry<T> {
+    fn deserialize<D>(de: D) -> std::result::Result<Self, D::Error>
+    where
+        D: serde::Deserializer<'de>,
+    {
+        serde_tagged::de::internal::deserialize(
+            de,
+            "type",
+            RegistryWithFakeType(ACTIVE_REG.get().unwrap(), PhantomData::<T>),
+        )
+    }
+}
+
+#[derive(Debug, thiserror::Error)]
+enum TryFromUrlError {
+    #[error("Unknown type: {0}")]
+    UnknownTag(String),
+}
+
+impl<T: 'static> TryFrom<url::Url> for DeserializeWithRegistry<T> {
+    type Error = Box<dyn std::error::Error + Send + Sync>;
+    fn try_from(url: url::Url) -> Result<Self, Self::Error> {
+        let tag = url.scheme().split('+').next().unwrap();
+        // same as in the SeedFactory impl: using find() and not get() because of https://github.com/rust-lang/rust/issues/80389
+        let seed = ACTIVE_REG
+            .get()
+            .unwrap()
+            .0
+            .iter()
+            .find(|(k, _)| *k == &(TypeId::of::<T>(), tag))
+            .ok_or_else(|| Box::new(TryFromUrlError::UnknownTag(tag.into())))?
+            .1;
+        let entry: &RegistryEntry<T> = <dyn Any>::downcast_ref(&**seed).unwrap();
+        (entry.from_url_seed)(url)
+    }
+}
+
+thread_local! {
+    /// The active Registry is global state, because there is no convenient and universal way to pass state
+    /// into the functions usually used for deserialization, e.g. `serde_json::from_str`, `toml::from_str`,
+    /// `serde_qs::from_str`.
+    static ACTIVE_REG: Cell<Option<&'static Registry>> = panic!("reg was accessed before initialization");
+}
+
+/// Run the provided closure with a registry context.
+/// Any serde deserialize calls within the closure will use the registry to resolve tag names to
+/// the corresponding Config type.
+pub fn with_registry<R>(reg: &'static Registry, f: impl FnOnce() -> R) -> R {
+    ACTIVE_REG.set(Some(reg));
+    let result = f();
+    ACTIVE_REG.set(None);
+    result
+}
+
+lazy_static! {
+    /// The provided registry of tvix_castore, with all builtin BlobStore/DirectoryStore implementations
+    pub static ref REG: Registry = {
+        let mut reg = Default::default();
+        add_default_services(&mut reg);
+        reg
+    };
+}
+
+// ---------- End of generic registry code --------- //
+
+/// Register the builtin services of tvix_castore with the given registry.
+/// This is useful for creating your own registry with the builtin types _and_
+/// extra third party types.
+pub fn add_default_services(reg: &mut Registry) {
+    crate::blobservice::register_blob_services(reg);
+    crate::directoryservice::register_directory_services(reg);
+}
+
+pub struct CompositionContext<'a> {
+    // The stack used to detect recursive instantiations and prevent deadlocks
+    // The TypeId of the trait object is included to distinguish e.g. the
+    // BlobService "default" and the DirectoryService "default".
+    stack: Vec<(TypeId, String)>,
+    composition: Option<&'a Composition>,
+}
+
+impl<'a> CompositionContext<'a> {
+    pub fn blank() -> Self {
+        Self {
+            stack: Default::default(),
+            composition: None,
+        }
+    }
+
+    pub async fn resolve<T: ?Sized + Send + Sync + 'static>(
+        &self,
+        entrypoint: String,
+    ) -> Result<Arc<T>, Box<dyn std::error::Error + Send + Sync + 'static>> {
+        // disallow recursion
+        if self
+            .stack
+            .contains(&(TypeId::of::<T>(), entrypoint.clone()))
+        {
+            return Err(CompositionError::Recursion(
+                self.stack.iter().map(|(_, n)| n.clone()).collect(),
+            )
+            .into());
+        }
+        match self.composition {
+            Some(comp) => Ok(comp.build_internal(self.stack.clone(), entrypoint).await?),
+            None => Err(CompositionError::NotFound(entrypoint).into()),
+        }
+    }
+}
+
+#[async_trait]
+/// This is the trait usually implemented on a per-store-type Config struct and
+/// used to instantiate it.
+pub trait ServiceBuilder: Send + Sync {
+    type Output: ?Sized;
+    async fn build(
+        &self,
+        instance_name: &str,
+        context: &CompositionContext,
+    ) -> Result<Arc<Self::Output>, Box<dyn std::error::Error + Send + Sync + 'static>>;
+}
+
+impl<T: ?Sized, S: ServiceBuilder<Output = T> + 'static> From<S>
+    for Box<dyn ServiceBuilder<Output = T>>
+{
+    fn from(t: S) -> Self {
+        Box::new(t)
+    }
+}
+
+enum InstantiationState<T: ?Sized> {
+    Config(Box<dyn ServiceBuilder<Output = T>>),
+    InProgress(tokio::sync::watch::Receiver<Option<Result<Arc<T>, CompositionError>>>),
+    Done(Result<Arc<T>, CompositionError>),
+}
+
+#[derive(Default)]
+pub struct Composition {
+    stores: std::sync::Mutex<HashMap<(TypeId, String), Box<dyn Any + Send + Sync>>>,
+}
+
+#[derive(thiserror::Error, Clone, Debug)]
+pub enum CompositionError {
+    #[error("store not found: {0}")]
+    NotFound(String),
+    #[error("recursion not allowed {0:?}")]
+    Recursion(Vec<String>),
+    #[error("store construction panicked {0}")]
+    Poisoned(String),
+    #[error("instantiation of service {0} failed: {1}")]
+    Failed(String, Arc<dyn std::error::Error + Send + Sync>),
+}
+
+impl<T: ?Sized + Send + Sync + 'static>
+    Extend<(
+        String,
+        DeserializeWithRegistry<Box<dyn ServiceBuilder<Output = T>>>,
+    )> for Composition
+{
+    fn extend<I>(&mut self, configs: I)
+    where
+        I: IntoIterator<
+            Item = (
+                String,
+                DeserializeWithRegistry<Box<dyn ServiceBuilder<Output = T>>>,
+            ),
+        >,
+    {
+        self.stores
+            .lock()
+            .unwrap()
+            .extend(configs.into_iter().map(|(k, v)| {
+                (
+                    (TypeId::of::<T>(), k),
+                    Box::new(InstantiationState::Config(v.0)) as Box<dyn Any + Send + Sync>,
+                )
+            }))
+    }
+}
+
+impl Composition {
+    pub fn extend_with_configs<T: ?Sized + Send + Sync + 'static>(
+        &mut self,
+        // Keep the concrete `HashMap` type here since it allows for type
+        // inference of what type is previously being deserialized.
+        configs: HashMap<String, DeserializeWithRegistry<Box<dyn ServiceBuilder<Output = T>>>>,
+    ) {
+        self.extend(configs);
+    }
+
+    pub async fn build<T: ?Sized + Send + Sync + 'static>(
+        &self,
+        entrypoint: &str,
+    ) -> Result<Arc<T>, CompositionError> {
+        self.build_internal(vec![], entrypoint.to_string()).await
+    }
+
+    fn build_internal<T: ?Sized + Send + Sync + 'static>(
+        &self,
+        stack: Vec<(TypeId, String)>,
+        entrypoint: String,
+    ) -> BoxFuture<'_, Result<Arc<T>, CompositionError>> {
+        let mut stores = self.stores.lock().unwrap();
+        let entry = match stores.get_mut(&(TypeId::of::<T>(), entrypoint.clone())) {
+            Some(v) => v,
+            None => return Box::pin(futures::future::err(CompositionError::NotFound(entrypoint))),
+        };
+        // for lifetime reasons, we put a placeholder value in the hashmap while we figure out what
+        // the new value should be. the Mutex stays locked the entire time, so nobody will ever see
+        // this temporary value.
+        let prev_val = std::mem::replace(
+            entry,
+            Box::new(InstantiationState::<T>::Done(Err(
+                CompositionError::Poisoned(entrypoint.clone()),
+            ))),
+        );
+        let (new_val, ret) = match *prev_val.downcast::<InstantiationState<T>>().unwrap() {
+            InstantiationState::Done(service) => (
+                InstantiationState::Done(service.clone()),
+                futures::future::ready(service).boxed(),
+            ),
+            // the construction of the store has not started yet.
+            InstantiationState::Config(config) => {
+                let (tx, rx) = tokio::sync::watch::channel(None);
+                (
+                    InstantiationState::InProgress(rx),
+                    (async move {
+                        let mut new_context = CompositionContext {
+                            stack: stack.clone(),
+                            composition: Some(self),
+                        };
+                        new_context
+                            .stack
+                            .push((TypeId::of::<T>(), entrypoint.clone()));
+                        let res =
+                            config.build(&entrypoint, &new_context).await.map_err(|e| {
+                                match e.downcast() {
+                                    Ok(e) => *e,
+                                    Err(e) => CompositionError::Failed(entrypoint, e.into()),
+                                }
+                            });
+                        tx.send(Some(res.clone())).unwrap();
+                        res
+                    })
+                    .boxed(),
+                )
+            }
+            // there is already a task driving forward the construction of this store, wait for it
+            // to notify us via the provided channel
+            InstantiationState::InProgress(mut recv) => {
+                (InstantiationState::InProgress(recv.clone()), {
+                    (async move {
+                        loop {
+                            if let Some(v) =
+                                recv.borrow_and_update().as_ref().map(|res| res.clone())
+                            {
+                                break v;
+                            }
+                            recv.changed().await.unwrap();
+                        }
+                    })
+                    .boxed()
+                })
+            }
+        };
+        *entry = Box::new(new_val);
+        ret
+    }
+
+    pub fn context(&self) -> CompositionContext {
+        CompositionContext {
+            stack: vec![],
+            composition: Some(self),
+        }
+    }
+}
+
+#[cfg(test)]
+mod test {
+    use super::*;
+    use crate::blobservice::BlobService;
+    use std::sync::Arc;
+
+    /// Test that we return a reference to the same instance of MemoryBlobService (via ptr_eq)
+    /// when instantiating the same entrypoint twice. By instantiating concurrently, we also
+    /// test the channels notifying the second consumer when the store has been instantiated.
+    #[tokio::test]
+    async fn concurrent() {
+        let blob_services_configs_json = serde_json::json!({
+            "default": {
+                "type": "memory",
+            }
+        });
+
+        let blob_services_configs =
+            with_registry(&REG, || serde_json::from_value(blob_services_configs_json)).unwrap();
+        let mut blob_service_composition = Composition::default();
+        blob_service_composition.extend_with_configs::<dyn BlobService>(blob_services_configs);
+        let (blob_service1, blob_service2) = tokio::join!(
+            blob_service_composition.build::<dyn BlobService>("default"),
+            blob_service_composition.build::<dyn BlobService>("default")
+        );
+        assert!(Arc::ptr_eq(
+            &blob_service1.unwrap(),
+            &blob_service2.unwrap()
+        ));
+    }
+
+    /// Test that we throw the correct error when an instantiation would recurse (deadlock)
+    #[tokio::test]
+    async fn reject_recursion() {
+        let blob_services_configs_json = serde_json::json!({
+            "default": {
+                "type": "combined",
+                "local": "other",
+                "remote": "other"
+            },
+            "other": {
+                "type": "combined",
+                "local": "default",
+                "remote": "default"
+            }
+        });
+
+        let blob_services_configs =
+            with_registry(&REG, || serde_json::from_value(blob_services_configs_json)).unwrap();
+        let mut blob_service_composition = Composition::default();
+        blob_service_composition.extend_with_configs::<dyn BlobService>(blob_services_configs);
+        match blob_service_composition
+            .build::<dyn BlobService>("default")
+            .await
+        {
+            Err(CompositionError::Recursion(stack)) => {
+                assert_eq!(stack, vec!["default".to_string(), "other".to_string()])
+            }
+            other => panic!("should have returned an error, returned: {:?}", other.err()),
+        }
+    }
+}
diff --git a/tvix/castore/src/digests.rs b/tvix/castore/src/digests.rs
index 2311c95c4ddc..4d919ff0d873 100644
--- a/tvix/castore/src/digests.rs
+++ b/tvix/castore/src/digests.rs
@@ -6,7 +6,7 @@ use thiserror::Error;
 pub struct B3Digest(Bytes);
 
 // TODO: allow converting these errors to crate::Error
-#[derive(Error, Debug)]
+#[derive(Error, Debug, PartialEq)]
 pub enum Error {
     #[error("invalid digest length: {0}")]
     InvalidDigestLen(usize),
@@ -26,6 +26,11 @@ impl From<B3Digest> for bytes::Bytes {
     }
 }
 
+impl From<blake3::Hash> for B3Digest {
+    fn from(value: blake3::Hash) -> Self {
+        Self(Bytes::copy_from_slice(value.as_bytes()))
+    }
+}
 impl From<digest::Output<blake3::Hasher>> for B3Digest {
     fn from(value: digest::Output<blake3::Hasher>) -> Self {
         let v = Into::<[u8; B3_LEN]>::into(value);
@@ -67,6 +72,12 @@ impl From<&[u8; B3_LEN]> for B3Digest {
     }
 }
 
+impl From<B3Digest> for [u8; B3_LEN] {
+    fn from(value: B3Digest) -> Self {
+        value.0.to_vec().try_into().unwrap()
+    }
+}
+
 impl Clone for B3Digest {
     fn clone(&self) -> Self {
         Self(self.0.to_owned())
diff --git a/tvix/castore/src/directoryservice/bigtable.rs b/tvix/castore/src/directoryservice/bigtable.rs
index 1194c6ddc999..73ab4342d832 100644
--- a/tvix/castore/src/directoryservice/bigtable.rs
+++ b/tvix/castore/src/directoryservice/bigtable.rs
@@ -5,10 +5,14 @@ use futures::stream::BoxStream;
 use prost::Message;
 use serde::{Deserialize, Serialize};
 use serde_with::{serde_as, DurationSeconds};
+use std::sync::Arc;
 use tonic::async_trait;
 use tracing::{instrument, trace, warn};
 
-use super::{utils::traverse_directory, DirectoryPutter, DirectoryService, SimplePutter};
+use super::{
+    utils::traverse_directory, Directory, DirectoryPutter, DirectoryService, SimplePutter,
+};
+use crate::composition::{CompositionContext, ServiceBuilder};
 use crate::{proto, B3Digest, Error};
 
 /// There should not be more than 10 MiB in a single cell.
@@ -43,41 +47,6 @@ pub struct BigtableDirectoryService {
     emulator: std::sync::Arc<(tempfile::TempDir, async_process::Child)>,
 }
 
-/// Represents configuration of [BigtableDirectoryService].
-/// This currently conflates both connect parameters and data model/client
-/// behaviour parameters.
-#[serde_as]
-#[derive(Clone, Debug, PartialEq, Deserialize, Serialize)]
-pub struct BigtableParameters {
-    project_id: String,
-    instance_name: String,
-    #[serde(default)]
-    is_read_only: bool,
-    #[serde(default = "default_channel_size")]
-    channel_size: usize,
-
-    #[serde_as(as = "Option<DurationSeconds<String>>")]
-    #[serde(default = "default_timeout")]
-    timeout: Option<std::time::Duration>,
-    table_name: String,
-    family_name: String,
-
-    #[serde(default = "default_app_profile_id")]
-    app_profile_id: String,
-}
-
-fn default_app_profile_id() -> String {
-    "default".to_owned()
-}
-
-fn default_channel_size() -> usize {
-    4
-}
-
-fn default_timeout() -> Option<std::time::Duration> {
-    Some(std::time::Duration::from_secs(4))
-}
-
 impl BigtableDirectoryService {
     #[cfg(not(test))]
     pub async fn connect(params: BigtableParameters) -> Result<Self, bigtable::Error> {
@@ -182,7 +151,7 @@ fn derive_directory_key(digest: &B3Digest) -> String {
 #[async_trait]
 impl DirectoryService for BigtableDirectoryService {
     #[instrument(skip(self, digest), err, fields(directory.digest = %digest))]
-    async fn get(&self, digest: &B3Digest) -> Result<Option<proto::Directory>, Error> {
+    async fn get(&self, digest: &B3Digest) -> Result<Option<Directory>, Error> {
         let mut client = self.client.clone();
         let directory_key = derive_directory_key(digest);
 
@@ -274,28 +243,20 @@ impl DirectoryService for BigtableDirectoryService {
 
         // Try to parse the value into a Directory message.
         let directory = proto::Directory::decode(Bytes::from(row_cell.value))
-            .map_err(|e| Error::StorageError(format!("unable to decode directory proto: {}", e)))?;
-
-        // validate the Directory.
-        directory
-            .validate()
+            .map_err(|e| Error::StorageError(format!("unable to decode directory proto: {}", e)))?
+            .try_into()
             .map_err(|e| Error::StorageError(format!("invalid Directory message: {}", e)))?;
 
         Ok(Some(directory))
     }
 
     #[instrument(skip(self, directory), err, fields(directory.digest = %directory.digest()))]
-    async fn put(&self, directory: proto::Directory) -> Result<B3Digest, Error> {
+    async fn put(&self, directory: Directory) -> Result<B3Digest, Error> {
         let directory_digest = directory.digest();
         let mut client = self.client.clone();
         let directory_key = derive_directory_key(&directory_digest);
 
-        // Ensure the directory we're trying to upload passes validation
-        directory
-            .validate()
-            .map_err(|e| Error::InvalidRequest(format!("directory is invalid: {}", e)))?;
-
-        let data = directory.encode_to_vec();
+        let data = proto::Directory::from(directory).encode_to_vec();
         if data.len() as u64 > CELL_SIZE_LIMIT {
             return Err(Error::StorageError(
                 "Directory exceeds cell limit on Bigtable".into(),
@@ -343,7 +304,7 @@ impl DirectoryService for BigtableDirectoryService {
     fn get_recursive(
         &self,
         root_directory_digest: &B3Digest,
-    ) -> BoxStream<'static, Result<proto::Directory, Error>> {
+    ) -> BoxStream<'static, Result<Directory, Error>> {
         traverse_directory(self.clone(), root_directory_digest)
     }
 
@@ -355,3 +316,73 @@ impl DirectoryService for BigtableDirectoryService {
         Box::new(SimplePutter::new(self.clone()))
     }
 }
+
+/// Represents configuration of [BigtableDirectoryService].
+/// This currently conflates both connect parameters and data model/client
+/// behaviour parameters.
+#[serde_as]
+#[derive(Clone, Debug, PartialEq, Deserialize, Serialize)]
+#[serde(deny_unknown_fields)]
+pub struct BigtableParameters {
+    project_id: String,
+    instance_name: String,
+    #[serde(default)]
+    is_read_only: bool,
+    #[serde(default = "default_channel_size")]
+    channel_size: usize,
+
+    #[serde_as(as = "Option<DurationSeconds<String>>")]
+    #[serde(default = "default_timeout")]
+    timeout: Option<std::time::Duration>,
+    table_name: String,
+    family_name: String,
+
+    #[serde(default = "default_app_profile_id")]
+    app_profile_id: String,
+}
+
+#[async_trait]
+impl ServiceBuilder for BigtableParameters {
+    type Output = dyn DirectoryService;
+    async fn build<'a>(
+        &'a self,
+        _instance_name: &str,
+        _context: &CompositionContext,
+    ) -> Result<Arc<dyn DirectoryService>, Box<dyn std::error::Error + Send + Sync>> {
+        Ok(Arc::new(
+            BigtableDirectoryService::connect(self.clone()).await?,
+        ))
+    }
+}
+
+impl TryFrom<url::Url> for BigtableParameters {
+    type Error = Box<dyn std::error::Error + Send + Sync>;
+    fn try_from(mut url: url::Url) -> Result<Self, Self::Error> {
+        // parse the instance name from the hostname.
+        let instance_name = url
+            .host_str()
+            .ok_or_else(|| Error::StorageError("instance name missing".into()))?
+            .to_string();
+
+        // โ€ฆ but add it to the query string now, so we just need to parse that.
+        url.query_pairs_mut()
+            .append_pair("instance_name", &instance_name);
+
+        let params: BigtableParameters = serde_qs::from_str(url.query().unwrap_or_default())
+            .map_err(|e| Error::InvalidRequest(format!("failed to parse parameters: {}", e)))?;
+
+        Ok(params)
+    }
+}
+
+fn default_app_profile_id() -> String {
+    "default".to_owned()
+}
+
+fn default_channel_size() -> usize {
+    4
+}
+
+fn default_timeout() -> Option<std::time::Duration> {
+    Some(std::time::Duration::from_secs(4))
+}
diff --git a/tvix/castore/src/directoryservice/closure_validator.rs b/tvix/castore/src/directoryservice/closure_validator.rs
deleted file mode 100644
index b9746a5a0501..000000000000
--- a/tvix/castore/src/directoryservice/closure_validator.rs
+++ /dev/null
@@ -1,309 +0,0 @@
-use std::collections::{HashMap, HashSet};
-
-use bstr::ByteSlice;
-
-use petgraph::{
-    graph::{DiGraph, NodeIndex},
-    visit::{Bfs, Walker},
-};
-use tracing::instrument;
-
-use crate::{
-    proto::{self, Directory},
-    B3Digest, Error,
-};
-
-type DirectoryGraph = DiGraph<Directory, ()>;
-
-/// This can be used to validate a Directory closure (DAG of connected
-/// Directories), and their insertion order.
-///
-/// Directories need to be inserted (via `add`), in an order from the leaves to
-/// the root (DFS Post-Order).
-/// During insertion, We validate as much as we can at that time:
-///
-///  - individual validation of Directory messages
-///  - validation of insertion order (no upload of not-yet-known Directories)
-///  - validation of size fields of referred Directories
-///
-/// Internally it keeps all received Directories in a directed graph,
-/// with node weights being the Directories and edges pointing to child
-/// directories.
-///
-/// Once all Directories have been inserted, a finalize function can be
-/// called to get a (deduplicated and) validated list of directories, in
-/// insertion order.
-/// During finalize, a check for graph connectivity is performed too, to ensure
-/// there's no disconnected components, and only one root.
-#[derive(Default)]
-pub struct ClosureValidator {
-    // A directed graph, using Directory as node weight, without edge weights.
-    // Edges point from parents to children.
-    graph: DirectoryGraph,
-
-    // A lookup table from directory digest to node index.
-    digest_to_node_ix: HashMap<B3Digest, NodeIndex>,
-
-    /// Keeps track of the last-inserted directory graph node index.
-    /// On a correct insert, this will be the root node, from which the DFS post
-    /// order traversal will start from.
-    last_directory_ix: Option<NodeIndex>,
-}
-
-impl ClosureValidator {
-    /// Insert a new Directory into the closure.
-    /// Perform individual Directory validation, validation of insertion order
-    /// and size fields.
-    #[instrument(level = "trace", skip_all, fields(directory.digest=%directory.digest(), directory.size=%directory.size()), err)]
-    pub fn add(&mut self, directory: proto::Directory) -> Result<(), Error> {
-        let digest = directory.digest();
-
-        // If we already saw this node previously, it's already validated and in the graph.
-        if self.digest_to_node_ix.contains_key(&digest) {
-            return Ok(());
-        }
-
-        // Do some general validation
-        directory
-            .validate()
-            .map_err(|e| Error::InvalidRequest(e.to_string()))?;
-
-        // Ensure the directory only refers to directories which we already accepted.
-        // We lookup their node indices and add them to a HashSet.
-        let mut child_ixs = HashSet::new();
-        for dir in &directory.directories {
-            let child_digest = B3Digest::try_from(dir.digest.to_owned()).unwrap(); // validated
-
-            // Ensure the digest has already been seen
-            let child_ix = *self.digest_to_node_ix.get(&child_digest).ok_or_else(|| {
-                Error::InvalidRequest(format!(
-                    "'{}' refers to unseen child dir: {}",
-                    dir.name.as_bstr(),
-                    &child_digest
-                ))
-            })?;
-
-            // Ensure the size specified in the child node matches the directory size itself.
-            let recorded_child_size = self
-                .graph
-                .node_weight(child_ix)
-                .expect("node not found")
-                .size();
-
-            // Ensure the size specified in the child node matches our records.
-            if dir.size != recorded_child_size {
-                return Err(Error::InvalidRequest(format!(
-                    "'{}' has wrong size, specified {}, recorded {}",
-                    dir.name.as_bstr(),
-                    dir.size,
-                    recorded_child_size
-                )));
-            }
-
-            child_ixs.insert(child_ix);
-        }
-
-        // Insert node into the graph, and add edges to all children.
-        let node_ix = self.graph.add_node(directory);
-        for child_ix in child_ixs {
-            self.graph.add_edge(node_ix, child_ix, ());
-        }
-
-        // Record the mapping from digest to node_ix in our lookup table.
-        self.digest_to_node_ix.insert(digest, node_ix);
-
-        // Update last_directory_ix.
-        self.last_directory_ix = Some(node_ix);
-
-        Ok(())
-    }
-
-    /// Ensure that all inserted Directories are connected, then return a
-    /// (deduplicated) and validated list of directories, in from-leaves-to-root
-    /// order.
-    /// In case no elements have been inserted, returns an empty list.
-    #[instrument(level = "trace", skip_all, err)]
-    pub(crate) fn finalize(self) -> Result<Vec<Directory>, Error> {
-        let (graph, _) = match self.finalize_raw()? {
-            None => return Ok(vec![]),
-            Some(v) => v,
-        };
-        // Dissolve the graph, returning the nodes as a Vec.
-        // As the graph was populated in a valid DFS PostOrder, we can return
-        // nodes in that same order.
-        let (nodes, _edges) = graph.into_nodes_edges();
-        Ok(nodes.into_iter().map(|x| x.weight).collect())
-    }
-
-    /// Ensure that all inserted Directories are connected, then return a
-    /// (deduplicated) and validated list of directories, in from-root-to-leaves
-    /// order.
-    /// In case no elements have been inserted, returns an empty list.
-    #[instrument(level = "trace", skip_all, err)]
-    pub(crate) fn finalize_root_to_leaves(self) -> Result<Vec<Directory>, Error> {
-        let (graph, root) = match self.finalize_raw()? {
-            None => return Ok(vec![]),
-            Some(v) => v,
-        };
-
-        // do a BFS traversal of the graph, starting with the root node to get
-        // all nodes reachable from there.
-        let traversal = Bfs::new(&graph, root);
-
-        let order = traversal.iter(&graph).collect::<Vec<_>>();
-
-        let (nodes, _edges) = graph.into_nodes_edges();
-
-        // Convert to option, so that we can take individual nodes out without messing up the
-        // indices
-        let mut nodes = nodes.into_iter().map(Some).collect::<Vec<_>>();
-
-        Ok(order
-            .iter()
-            .map(|i| nodes[i.index()].take().unwrap().weight)
-            .collect())
-    }
-
-    /// Internal implementation of closure validation
-    #[instrument(level = "trace", skip_all, err)]
-    fn finalize_raw(self) -> Result<Option<(DirectoryGraph, NodeIndex)>, Error> {
-        // If no nodes were inserted, an empty list is returned.
-        let last_directory_ix = if let Some(x) = self.last_directory_ix {
-            x
-        } else {
-            return Ok(None);
-        };
-
-        // do a BFS traversal of the graph, starting with the root node to get
-        // (the count of) all nodes reachable from there.
-        let mut traversal = Bfs::new(&self.graph, last_directory_ix);
-
-        let mut visited_directory_count = 0;
-        #[cfg(debug_assertions)]
-        let mut visited_directory_ixs = HashSet::new();
-        #[cfg_attr(not(debug_assertions), allow(unused))]
-        while let Some(directory_ix) = traversal.next(&self.graph) {
-            #[cfg(debug_assertions)]
-            visited_directory_ixs.insert(directory_ix);
-
-            visited_directory_count += 1;
-        }
-
-        // If the number of nodes collected equals the total number of nodes in
-        // the graph, we know all nodes are connected.
-        if visited_directory_count != self.graph.node_count() {
-            // more or less exhaustive error reporting.
-            #[cfg(debug_assertions)]
-            {
-                let all_directory_ixs: HashSet<_> = self.graph.node_indices().collect();
-
-                let unvisited_directories: HashSet<_> = all_directory_ixs
-                    .difference(&visited_directory_ixs)
-                    .map(|ix| self.graph.node_weight(*ix).expect("node not found"))
-                    .collect();
-
-                return Err(Error::InvalidRequest(format!(
-                    "found {} disconnected directories: {:?}",
-                    self.graph.node_count() - visited_directory_ixs.len(),
-                    unvisited_directories
-                )));
-            }
-            #[cfg(not(debug_assertions))]
-            {
-                return Err(Error::InvalidRequest(format!(
-                    "found {} disconnected directories",
-                    self.graph.node_count() - visited_directory_count
-                )));
-            }
-        }
-
-        Ok(Some((self.graph, last_directory_ix)))
-    }
-}
-
-#[cfg(test)]
-mod tests {
-    use crate::{
-        fixtures::{DIRECTORY_A, DIRECTORY_B, DIRECTORY_C},
-        proto::{self, Directory},
-    };
-    use lazy_static::lazy_static;
-    use rstest::rstest;
-
-    lazy_static! {
-        pub static ref BROKEN_DIRECTORY : Directory = Directory {
-            symlinks: vec![proto::SymlinkNode {
-                name: "".into(), // invalid name!
-                target: "doesntmatter".into(),
-            }],
-            ..Default::default()
-        };
-
-        pub static ref BROKEN_PARENT_DIRECTORY: Directory = Directory {
-            directories: vec![proto::DirectoryNode {
-                name: "foo".into(),
-                digest: DIRECTORY_A.digest().into(),
-                size: DIRECTORY_A.size() + 42, // wrong!
-            }],
-            ..Default::default()
-        };
-    }
-
-    use super::ClosureValidator;
-
-    #[rstest]
-    /// Uploading an empty directory should succeed.
-    #[case::empty_directory(&[&*DIRECTORY_A], false, Some(vec![&*DIRECTORY_A]))]
-    /// Uploading A, then B (referring to A) should succeed.
-    #[case::simple_closure(&[&*DIRECTORY_A, &*DIRECTORY_B], false, Some(vec![&*DIRECTORY_A, &*DIRECTORY_B]))]
-    /// Uploading A, then A, then C (referring to A twice) should succeed.
-    /// We pretend to be a dumb client not deduping directories.
-    #[case::same_child(&[&*DIRECTORY_A, &*DIRECTORY_A, &*DIRECTORY_C], false, Some(vec![&*DIRECTORY_A, &*DIRECTORY_C]))]
-    /// Uploading A, then C (referring to A twice) should succeed.
-    #[case::same_child_dedup(&[&*DIRECTORY_A, &*DIRECTORY_C], false, Some(vec![&*DIRECTORY_A, &*DIRECTORY_C]))]
-    /// Uploading A, then C (referring to A twice), then B (itself referring to A) should fail during close,
-    /// as B itself would be left unconnected.
-    #[case::unconnected_node(&[&*DIRECTORY_A, &*DIRECTORY_C, &*DIRECTORY_B], false, None)]
-    /// Uploading B (referring to A) should fail immediately, because A was never uploaded.
-    #[case::dangling_pointer(&[&*DIRECTORY_B], true, None)]
-    /// Uploading a directory failing validation should fail immediately.
-    #[case::failing_validation(&[&*BROKEN_DIRECTORY], true, None)]
-    /// Uploading a directory which refers to another Directory with a wrong size should fail.
-    #[case::wrong_size_in_parent(&[&*DIRECTORY_A, &*BROKEN_PARENT_DIRECTORY], true, None)]
-    fn test_uploads(
-        #[case] directories_to_upload: &[&Directory],
-        #[case] exp_fail_upload_last: bool,
-        #[case] exp_finalize: Option<Vec<&Directory>>, // Some(_) if finalize successful, None if not.
-    ) {
-        let mut dcv = ClosureValidator::default();
-        let len_directories_to_upload = directories_to_upload.len();
-
-        for (i, d) in directories_to_upload.iter().enumerate() {
-            let resp = dcv.add((*d).clone());
-            if i == len_directories_to_upload - 1 && exp_fail_upload_last {
-                assert!(resp.is_err(), "expect last put to fail");
-
-                // We don't really care anymore what finalize() would return, as
-                // the add() failed.
-                return;
-            } else {
-                assert!(resp.is_ok(), "expect put to succeed");
-            }
-        }
-
-        // everything was uploaded successfully. Test finalize().
-        let resp = dcv.finalize();
-
-        match exp_finalize {
-            Some(directories) => {
-                assert_eq!(
-                    Vec::from_iter(directories.iter().map(|e| (*e).to_owned())),
-                    resp.expect("drain should succeed")
-                );
-            }
-            None => {
-                resp.expect_err("drain should fail");
-            }
-        }
-    }
-}
diff --git a/tvix/castore/src/directoryservice/combinators.rs b/tvix/castore/src/directoryservice/combinators.rs
new file mode 100644
index 000000000000..4283142231f9
--- /dev/null
+++ b/tvix/castore/src/directoryservice/combinators.rs
@@ -0,0 +1,180 @@
+use std::sync::Arc;
+
+use futures::stream::BoxStream;
+use futures::StreamExt;
+use futures::TryFutureExt;
+use futures::TryStreamExt;
+use tonic::async_trait;
+use tracing::{instrument, trace};
+
+use super::{Directory, DirectoryGraph, DirectoryService, RootToLeavesValidator, SimplePutter};
+use crate::composition::{CompositionContext, ServiceBuilder};
+use crate::directoryservice::DirectoryPutter;
+use crate::B3Digest;
+use crate::Error;
+
+/// Asks near first, if not found, asks far.
+/// If found in there, returns it, and *inserts* it into
+/// near.
+/// Specifically, it always obtains the entire directory closure from far and inserts it into near,
+/// which is useful when far does not support accessing intermediate directories (but near does).
+/// There is no negative cache.
+/// Inserts and listings are not implemented for now.
+#[derive(Clone)]
+pub struct Cache<DS1, DS2> {
+    near: DS1,
+    far: DS2,
+}
+
+impl<DS1, DS2> Cache<DS1, DS2> {
+    pub fn new(near: DS1, far: DS2) -> Self {
+        Self { near, far }
+    }
+}
+
+#[async_trait]
+impl<DS1, DS2> DirectoryService for Cache<DS1, DS2>
+where
+    DS1: DirectoryService + Clone + 'static,
+    DS2: DirectoryService + Clone + 'static,
+{
+    #[instrument(skip(self, digest), fields(directory.digest = %digest))]
+    async fn get(&self, digest: &B3Digest) -> Result<Option<Directory>, Error> {
+        match self.near.get(digest).await? {
+            Some(directory) => {
+                trace!("serving from cache");
+                Ok(Some(directory))
+            }
+            None => {
+                trace!("not found in near, asking remoteโ€ฆ");
+
+                let mut copy = DirectoryGraph::with_order(
+                    RootToLeavesValidator::new_with_root_digest(digest.clone()),
+                );
+
+                let mut stream = self.far.get_recursive(digest);
+                let root = stream.try_next().await?;
+
+                if let Some(root) = root.clone() {
+                    copy.add(root)
+                        .map_err(|e| Error::StorageError(e.to_string()))?;
+                }
+
+                while let Some(dir) = stream.try_next().await? {
+                    copy.add(dir)
+                        .map_err(|e| Error::StorageError(e.to_string()))?;
+                }
+
+                let copy = copy
+                    .validate()
+                    .map_err(|e| Error::StorageError(e.to_string()))?;
+
+                let mut put = self.near.put_multiple_start();
+                for dir in copy.drain_leaves_to_root() {
+                    put.put(dir).await?;
+                }
+                put.close().await?;
+
+                Ok(root)
+            }
+        }
+    }
+
+    #[instrument(skip_all)]
+    async fn put(&self, _directory: Directory) -> Result<B3Digest, Error> {
+        Err(Error::StorageError("unimplemented".to_string()))
+    }
+
+    #[instrument(skip_all, fields(directory.digest = %root_directory_digest))]
+    fn get_recursive(
+        &self,
+        root_directory_digest: &B3Digest,
+    ) -> BoxStream<'static, Result<Directory, Error>> {
+        let near = self.near.clone();
+        let far = self.far.clone();
+        let digest = root_directory_digest.clone();
+        Box::pin(
+            (async move {
+                let mut stream = near.get_recursive(&digest);
+                match stream.try_next().await? {
+                    Some(first) => {
+                        trace!("serving from cache");
+                        Ok(futures::stream::once(async { Ok(first) })
+                            .chain(stream)
+                            .left_stream())
+                    }
+                    None => {
+                        trace!("not found in near, asking remoteโ€ฆ");
+
+                        let mut copy_for_near = DirectoryGraph::with_order(
+                            RootToLeavesValidator::new_with_root_digest(digest.clone()),
+                        );
+                        let mut copy_for_client = vec![];
+
+                        let mut stream = far.get_recursive(&digest);
+                        while let Some(dir) = stream.try_next().await? {
+                            copy_for_near
+                                .add(dir.clone())
+                                .map_err(|e| Error::StorageError(e.to_string()))?;
+                            copy_for_client.push(dir);
+                        }
+
+                        let copy_for_near = copy_for_near
+                            .validate()
+                            .map_err(|e| Error::StorageError(e.to_string()))?;
+                        let mut put = near.put_multiple_start();
+                        for dir in copy_for_near.drain_leaves_to_root() {
+                            put.put(dir).await?;
+                        }
+                        put.close().await?;
+
+                        Ok(futures::stream::iter(copy_for_client.into_iter().map(Ok))
+                            .right_stream())
+                    }
+                }
+            })
+            .try_flatten_stream(),
+        )
+    }
+
+    #[instrument(skip_all)]
+    fn put_multiple_start(&self) -> Box<(dyn DirectoryPutter + 'static)> {
+        Box::new(SimplePutter::new((*self).clone()))
+    }
+}
+
+#[derive(serde::Deserialize, Debug)]
+#[serde(deny_unknown_fields)]
+pub struct CacheConfig {
+    near: String,
+    far: String,
+}
+
+impl TryFrom<url::Url> for CacheConfig {
+    type Error = Box<dyn std::error::Error + Send + Sync>;
+    fn try_from(_url: url::Url) -> Result<Self, Self::Error> {
+        Err(Error::StorageError(
+            "Instantiating a CombinedDirectoryService from a url is not supported".into(),
+        )
+        .into())
+    }
+}
+
+#[async_trait]
+impl ServiceBuilder for CacheConfig {
+    type Output = dyn DirectoryService;
+    async fn build<'a>(
+        &'a self,
+        _instance_name: &str,
+        context: &CompositionContext,
+    ) -> Result<Arc<dyn DirectoryService>, Box<dyn std::error::Error + Send + Sync + 'static>> {
+        let (near, far) = futures::join!(
+            context.resolve(self.near.clone()),
+            context.resolve(self.far.clone())
+        );
+        Ok(Arc::new(Cache {
+            near: near?,
+            far: far?,
+        }))
+    }
+}
diff --git a/tvix/castore/src/directoryservice/directory_graph.rs b/tvix/castore/src/directoryservice/directory_graph.rs
new file mode 100644
index 000000000000..017cef024059
--- /dev/null
+++ b/tvix/castore/src/directoryservice/directory_graph.rs
@@ -0,0 +1,414 @@
+use std::collections::HashMap;
+
+use petgraph::{
+    graph::{DiGraph, NodeIndex},
+    visit::{Bfs, DfsPostOrder, EdgeRef, IntoNodeIdentifiers, Walker},
+    Direction, Incoming,
+};
+use tracing::instrument;
+
+use super::order_validator::{LeavesToRootValidator, OrderValidator, RootToLeavesValidator};
+use crate::{path::PathComponent, B3Digest, Directory, Node};
+
+#[derive(thiserror::Error, Debug)]
+pub enum Error {
+    #[error("{0}")]
+    ValidationError(String),
+}
+
+struct EdgeWeight {
+    name: PathComponent,
+    size: u64,
+}
+
+/// This can be used to validate and/or re-order a Directory closure (DAG of
+/// connected Directories), and their insertion order.
+///
+/// The DirectoryGraph is parametrized on the insertion order, and can be
+/// constructed using the Default trait, or using `with_order` if the
+/// OrderValidator needs to be customized.
+///
+/// If the user is receiving directories from canonical protobuf encoding in
+/// root-to-leaves order, and parsing them, she can call `digest_allowed`
+/// _before_ parsing the protobuf record and then add it with `add_unchecked`.
+/// All other users insert the directories via `add`, in their specified order.
+/// During insertion, we validate as much as we can at that time:
+///
+///  - individual validation of Directory messages
+///  - validation of insertion order
+///  - validation of size fields of referred Directories
+///
+/// Internally it keeps all received Directories in a directed graph,
+/// with node weights being the Directories and edges pointing to child/parent
+/// directories.
+///
+/// Once all Directories have been inserted, a validate function can be
+/// called to perform a check for graph connectivity and ensure there's no
+/// disconnected components or missing nodes.
+/// Finally, the `drain_leaves_to_root` or `drain_root_to_leaves` can be
+/// _chained_ on validate to get an iterator over the (deduplicated and)
+/// validated list of directories in either order.
+#[derive(Default)]
+pub struct DirectoryGraph<O> {
+    // A directed graph, using Directory as node weight.
+    // Edges point from parents to children.
+    //
+    // Nodes with None weigths might exist when a digest has been referred to but the directory
+    // with this digest has not yet been sent.
+    //
+    // The option in the edge weight tracks the pending validation state of the respective edge, for example if
+    // the child has not been added yet.
+    graph: DiGraph<Option<Directory>, Option<EdgeWeight>>,
+
+    // A lookup table from directory digest to node index.
+    digest_to_node_ix: HashMap<B3Digest, NodeIndex>,
+
+    order_validator: O,
+}
+
+pub struct ValidatedDirectoryGraph {
+    graph: DiGraph<Option<Directory>, Option<EdgeWeight>>,
+
+    root: Option<NodeIndex>,
+}
+
+fn check_edge(edge: &EdgeWeight, child: &Directory) -> Result<(), Error> {
+    // Ensure the size specified in the child node matches our records.
+    if edge.size != child.size() {
+        return Err(Error::ValidationError(format!(
+            "'{}' has wrong size, specified {}, recorded {}",
+            edge.name,
+            edge.size,
+            child.size(),
+        )));
+    }
+    Ok(())
+}
+
+impl DirectoryGraph<LeavesToRootValidator> {
+    /// Insert a new Directory into the closure
+    #[instrument(level = "trace", skip_all, fields(directory.digest=%directory.digest(), directory.size=%directory.size()), err)]
+    pub fn add(&mut self, directory: Directory) -> Result<(), Error> {
+        if !self.order_validator.add_directory(&directory) {
+            return Err(Error::ValidationError(
+                "unknown directory was referenced".into(),
+            ));
+        }
+        self.add_order_unchecked(directory)
+    }
+}
+
+impl DirectoryGraph<RootToLeavesValidator> {
+    /// If the user is parsing directories from canonical protobuf encoding, she can
+    /// call `digest_allowed` _before_ parsing the protobuf record and then add it
+    /// with `add_unchecked`.
+    pub fn digest_allowed(&self, digest: B3Digest) -> bool {
+        self.order_validator.digest_allowed(&digest)
+    }
+
+    /// Insert a new Directory into the closure
+    #[instrument(level = "trace", skip_all, fields(directory.digest=%directory.digest(), directory.size=%directory.size()), err)]
+    pub fn add(&mut self, directory: Directory) -> Result<(), Error> {
+        let digest = directory.digest();
+        if !self.order_validator.digest_allowed(&digest) {
+            return Err(Error::ValidationError("unexpected digest".into()));
+        }
+        self.order_validator.add_directory_unchecked(&directory);
+        self.add_order_unchecked(directory)
+    }
+}
+
+impl<O: OrderValidator> DirectoryGraph<O> {
+    /// Customize the ordering, i.e. for pre-setting the root of the RootToLeavesValidator
+    pub fn with_order(order_validator: O) -> Self {
+        Self {
+            graph: Default::default(),
+            digest_to_node_ix: Default::default(),
+            order_validator,
+        }
+    }
+
+    /// Adds a directory which has already been confirmed to be in-order to the graph
+    pub fn add_order_unchecked(&mut self, directory: Directory) -> Result<(), Error> {
+        let digest = directory.digest();
+
+        // Teach the graph about the existence of a node with this digest
+        let ix = *self
+            .digest_to_node_ix
+            .entry(digest)
+            .or_insert_with(|| self.graph.add_node(None));
+
+        if self.graph[ix].is_some() {
+            // The node is already in the graph, there is nothing to do here.
+            return Ok(());
+        }
+
+        // set up edges to all child directories
+        for (name, node) in directory.nodes() {
+            if let Node::Directory { digest, size } = node {
+                let child_ix = *self
+                    .digest_to_node_ix
+                    .entry(digest.clone())
+                    .or_insert_with(|| self.graph.add_node(None));
+
+                let pending_edge_check = match &self.graph[child_ix] {
+                    Some(child) => {
+                        // child is already available, validate the edge now
+                        check_edge(
+                            &EdgeWeight {
+                                name: name.clone(),
+                                size: *size,
+                            },
+                            child,
+                        )?;
+                        None
+                    }
+                    None => Some(EdgeWeight {
+                        name: name.clone(),
+                        size: *size,
+                    }), // pending validation
+                };
+                self.graph.add_edge(ix, child_ix, pending_edge_check);
+            }
+        }
+
+        // validate the edges from parents to this node
+        // this collects edge ids in a Vec because there is no edges_directed_mut :'c
+        for edge_id in self
+            .graph
+            .edges_directed(ix, Direction::Incoming)
+            .map(|edge_ref| edge_ref.id())
+            .collect::<Vec<_>>()
+            .into_iter()
+        {
+            let edge_weight = self
+                .graph
+                .edge_weight_mut(edge_id)
+                .expect("edge not found")
+                .take()
+                .expect("edge is already validated");
+
+            check_edge(&edge_weight, &directory)?;
+        }
+
+        // finally, store the directory information in the node weight
+        self.graph[ix] = Some(directory);
+
+        Ok(())
+    }
+
+    #[instrument(level = "trace", skip_all, err)]
+    pub fn validate(self) -> Result<ValidatedDirectoryGraph, Error> {
+        // find all initial nodes (nodes without incoming edges)
+        let mut roots = self
+            .graph
+            .node_identifiers()
+            .filter(|&a| self.graph.neighbors_directed(a, Incoming).next().is_none());
+
+        let root = roots.next();
+        if roots.next().is_some() {
+            return Err(Error::ValidationError(
+                "graph has disconnected roots".into(),
+            ));
+        }
+
+        // test that the graph is complete
+        if self.graph.raw_nodes().iter().any(|n| n.weight.is_none()) {
+            return Err(Error::ValidationError("graph is incomplete".into()));
+        }
+
+        Ok(ValidatedDirectoryGraph {
+            graph: self.graph,
+            root,
+        })
+    }
+}
+
+impl ValidatedDirectoryGraph {
+    /// Return the list of directories in from-root-to-leaves order.
+    /// In case no elements have been inserted, returns an empty list.
+    ///
+    /// panics if the specified root is not in the graph
+    #[instrument(level = "trace", skip_all)]
+    pub fn drain_root_to_leaves(self) -> impl Iterator<Item = Directory> {
+        let order = match self.root {
+            Some(root) => {
+                // do a BFS traversal of the graph, starting with the root node
+                Bfs::new(&self.graph, root)
+                    .iter(&self.graph)
+                    .collect::<Vec<_>>()
+            }
+            None => vec![], // No nodes have been inserted, do not traverse
+        };
+
+        let (mut nodes, _edges) = self.graph.into_nodes_edges();
+
+        order
+            .into_iter()
+            .filter_map(move |i| nodes[i.index()].weight.take())
+    }
+
+    /// Return the list of directories in from-leaves-to-root order.
+    /// In case no elements have been inserted, returns an empty list.
+    ///
+    /// panics when the specified root is not in the graph
+    #[instrument(level = "trace", skip_all)]
+    pub fn drain_leaves_to_root(self) -> impl Iterator<Item = Directory> {
+        let order = match self.root {
+            Some(root) => {
+                // do a DFS Post-Order traversal of the graph, starting with the root node
+                DfsPostOrder::new(&self.graph, root)
+                    .iter(&self.graph)
+                    .collect::<Vec<_>>()
+            }
+            None => vec![], // No nodes have been inserted, do not traverse
+        };
+
+        let (mut nodes, _edges) = self.graph.into_nodes_edges();
+
+        order
+            .into_iter()
+            .filter_map(move |i| nodes[i.index()].weight.take())
+    }
+}
+/*
+        pub static ref BROKEN_DIRECTORY : Directory = Directory {
+            symlinks: vec![SymlinkNode {
+                name: "".into(), // invalid name!
+                target: "doesntmatter".into(),
+            }],
+            ..Default::default()
+        };
+*/
+#[cfg(test)]
+mod tests {
+    use crate::fixtures::{DIRECTORY_A, DIRECTORY_B, DIRECTORY_C};
+    use crate::{Directory, Node};
+    use lazy_static::lazy_static;
+    use rstest::rstest;
+
+    use super::{DirectoryGraph, LeavesToRootValidator, RootToLeavesValidator};
+
+    lazy_static! {
+        pub static ref BROKEN_PARENT_DIRECTORY: Directory =
+            Directory::try_from_iter([
+                (
+                    "foo".try_into().unwrap(),
+                    Node::Directory{
+                        digest: DIRECTORY_A.digest(),
+                        size: DIRECTORY_A.size() + 42, // wrong!
+                    }
+                )
+            ]).unwrap();
+    }
+
+    #[rstest]
+    /// Uploading an empty directory should succeed.
+    #[case::empty_directory(&[&*DIRECTORY_A], false, Some(vec![&*DIRECTORY_A]))]
+    /// Uploading A, then B (referring to A) should succeed.
+    #[case::simple_closure(&[&*DIRECTORY_A, &*DIRECTORY_B], false, Some(vec![&*DIRECTORY_A, &*DIRECTORY_B]))]
+    /// Uploading A, then A, then C (referring to A twice) should succeed.
+    /// We pretend to be a dumb client not deduping directories.
+    #[case::same_child(&[&*DIRECTORY_A, &*DIRECTORY_A, &*DIRECTORY_C], false, Some(vec![&*DIRECTORY_A, &*DIRECTORY_C]))]
+    /// Uploading A, then C (referring to A twice) should succeed.
+    #[case::same_child_dedup(&[&*DIRECTORY_A, &*DIRECTORY_C], false, Some(vec![&*DIRECTORY_A, &*DIRECTORY_C]))]
+    /// Uploading A, then C (referring to A twice), then B (itself referring to A) should fail during close,
+    /// as B itself would be left unconnected.
+    #[case::unconnected_node(&[&*DIRECTORY_A, &*DIRECTORY_C, &*DIRECTORY_B], false, None)]
+    /// Uploading B (referring to A) should fail immediately, because A was never uploaded.
+    #[case::dangling_pointer(&[&*DIRECTORY_B], true, None)]
+    /// Uploading a directory which refers to another Directory with a wrong size should fail.
+    #[case::wrong_size_in_parent(&[&*DIRECTORY_A, &*BROKEN_PARENT_DIRECTORY], true, None)]
+    fn test_uploads(
+        #[case] directories_to_upload: &[&Directory],
+        #[case] exp_fail_upload_last: bool,
+        #[case] exp_finalize: Option<Vec<&Directory>>, // Some(_) if finalize successful, None if not.
+    ) {
+        let mut dcv = DirectoryGraph::<LeavesToRootValidator>::default();
+        let len_directories_to_upload = directories_to_upload.len();
+
+        for (i, d) in directories_to_upload.iter().enumerate() {
+            let resp = dcv.add((*d).clone());
+            if i == len_directories_to_upload - 1 && exp_fail_upload_last {
+                assert!(resp.is_err(), "expect last put to fail");
+
+                // We don't really care anymore what finalize() would return, as
+                // the add() failed.
+                return;
+            } else {
+                assert!(resp.is_ok(), "expect put to succeed");
+            }
+        }
+
+        // everything was uploaded successfully. Test finalize().
+        let resp = dcv
+            .validate()
+            .map(|validated| validated.drain_leaves_to_root().collect::<Vec<_>>());
+
+        match exp_finalize {
+            Some(directories) => {
+                assert_eq!(
+                    Vec::from_iter(directories.iter().map(|e| (*e).to_owned())),
+                    resp.expect("drain should succeed")
+                );
+            }
+            None => {
+                resp.expect_err("drain should fail");
+            }
+        }
+    }
+
+    #[rstest]
+    /// Downloading an empty directory should succeed.
+    #[case::empty_directory(&*DIRECTORY_A, &[&*DIRECTORY_A], false, Some(vec![&*DIRECTORY_A]))]
+    /// Downlading B, then A (referenced by B) should succeed.
+    #[case::simple_closure(&*DIRECTORY_B, &[&*DIRECTORY_B, &*DIRECTORY_A], false, Some(vec![&*DIRECTORY_A, &*DIRECTORY_B]))]
+    /// Downloading C (referring to A twice), then A should succeed.
+    #[case::same_child_dedup(&*DIRECTORY_C, &[&*DIRECTORY_C, &*DIRECTORY_A], false, Some(vec![&*DIRECTORY_A, &*DIRECTORY_C]))]
+    /// Downloading C, then B (both referring to A but not referring to each other) should fail immediately as B has no connection to C (the root)
+    #[case::unconnected_node(&*DIRECTORY_C, &[&*DIRECTORY_C, &*DIRECTORY_B], true, None)]
+    /// Downloading B (specified as the root) but receiving A instead should fail immediately, because A has no connection to B (the root).
+    #[case::dangling_pointer(&*DIRECTORY_B, &[&*DIRECTORY_A], true, None)]
+    /// Downloading a directory which refers to another Directory with a wrong size should fail.
+    #[case::wrong_size_in_parent(&*BROKEN_PARENT_DIRECTORY, &[&*BROKEN_PARENT_DIRECTORY, &*DIRECTORY_A], true, None)]
+    fn test_downloads(
+        #[case] root: &Directory,
+        #[case] directories_to_upload: &[&Directory],
+        #[case] exp_fail_upload_last: bool,
+        #[case] exp_finalize: Option<Vec<&Directory>>, // Some(_) if finalize successful, None if not.
+    ) {
+        let mut dcv =
+            DirectoryGraph::with_order(RootToLeavesValidator::new_with_root_digest(root.digest()));
+        let len_directories_to_upload = directories_to_upload.len();
+
+        for (i, d) in directories_to_upload.iter().enumerate() {
+            let resp = dcv.add((*d).clone());
+            if i == len_directories_to_upload - 1 && exp_fail_upload_last {
+                assert!(resp.is_err(), "expect last put to fail");
+
+                // We don't really care anymore what finalize() would return, as
+                // the add() failed.
+                return;
+            } else {
+                assert!(resp.is_ok(), "expect put to succeed");
+            }
+        }
+
+        // everything was uploaded successfully. Test finalize().
+        let resp = dcv
+            .validate()
+            .map(|validated| validated.drain_leaves_to_root().collect::<Vec<_>>());
+
+        match exp_finalize {
+            Some(directories) => {
+                assert_eq!(
+                    Vec::from_iter(directories.iter().map(|e| (*e).to_owned())),
+                    resp.expect("drain should succeed")
+                );
+            }
+            None => {
+                resp.expect_err("drain should fail");
+            }
+        }
+    }
+}
diff --git a/tvix/castore/src/directoryservice/from_addr.rs b/tvix/castore/src/directoryservice/from_addr.rs
index ee675ca68a9f..3feb8f3509fe 100644
--- a/tvix/castore/src/directoryservice/from_addr.rs
+++ b/tvix/castore/src/directoryservice/from_addr.rs
@@ -1,12 +1,13 @@
-use url::Url;
+use std::sync::Arc;
 
-use crate::{proto::directory_service_client::DirectoryServiceClient, Error};
+use url::Url;
 
-use super::{
-    DirectoryService, GRPCDirectoryService, MemoryDirectoryService, ObjectStoreDirectoryService,
-    SledDirectoryService,
+use crate::composition::{
+    with_registry, CompositionContext, DeserializeWithRegistry, ServiceBuilder, REG,
 };
 
+use super::DirectoryService;
+
 /// Constructs a new instance of a [DirectoryService] from an URI.
 ///
 /// The following URIs are supported:
@@ -17,98 +18,32 @@ use super::{
 /// - `sled:///absolute/path/to/somewhere`
 ///   Uses sled, using a path on the disk for persistency. Can be only opened
 ///   from one process at the same time.
+/// - `redb:`
+///   Uses a in-memory redb implementation.
+/// - `redb:///absolute/path/to/somewhere`
+///   Uses redb, using a path on the disk for persistency. Can be only opened
+///   from one process at the same time.
 /// - `grpc+unix:///absolute/path/to/somewhere`
 ///   Connects to a local tvix-store gRPC service via Unix socket.
 /// - `grpc+http://host:port`, `grpc+https://host:port`
 ///    Connects to a (remote) tvix-store gRPC service.
-pub async fn from_addr(uri: &str) -> Result<Box<dyn DirectoryService>, crate::Error> {
+pub async fn from_addr(
+    uri: &str,
+) -> Result<Arc<dyn DirectoryService>, Box<dyn std::error::Error + Send + Sync>> {
     #[allow(unused_mut)]
     let mut url = Url::parse(uri)
         .map_err(|e| crate::Error::StorageError(format!("unable to parse url: {}", e)))?;
 
-    let directory_service: Box<dyn DirectoryService> = match url.scheme() {
-        "memory" => {
-            // memory doesn't support host or path in the URL.
-            if url.has_host() || !url.path().is_empty() {
-                return Err(Error::StorageError("invalid url".to_string()));
-            }
-            Box::<MemoryDirectoryService>::default()
-        }
-        "sled" => {
-            // sled doesn't support host, and a path can be provided (otherwise
-            // it'll live in memory only).
-            if url.has_host() {
-                return Err(Error::StorageError("no host allowed".to_string()));
-            }
-
-            if url.path() == "/" {
-                return Err(Error::StorageError(
-                    "cowardly refusing to open / with sled".to_string(),
-                ));
-            }
-
-            // TODO: expose compression and other parameters as URL parameters?
-
-            Box::new(if url.path().is_empty() {
-                SledDirectoryService::new_temporary()
-                    .map_err(|e| Error::StorageError(e.to_string()))?
-            } else {
-                SledDirectoryService::new(url.path())
-                    .map_err(|e| Error::StorageError(e.to_string()))?
-            })
-        }
-        scheme if scheme.starts_with("grpc+") => {
-            // schemes starting with grpc+ go to the GRPCPathInfoService.
-            //   That's normally grpc+unix for unix sockets, and grpc+http(s) for the HTTP counterparts.
-            // - In the case of unix sockets, there must be a path, but may not be a host.
-            // - In the case of non-unix sockets, there must be a host, but no path.
-            // Constructing the channel is handled by tvix_castore::channel::from_url.
-            let client = DirectoryServiceClient::new(crate::tonic::channel_from_url(&url).await?);
-            Box::new(GRPCDirectoryService::from_client(client))
-        }
-        scheme if scheme.starts_with("objectstore+") => {
-            // We need to convert the URL to string, strip the prefix there, and then
-            // parse it back as url, as Url::set_scheme() rejects some of the transitions we want to do.
-            let trimmed_url = {
-                let s = url.to_string();
-                Url::parse(s.strip_prefix("objectstore+").unwrap()).unwrap()
-            };
-            Box::new(
-                ObjectStoreDirectoryService::parse_url(&trimmed_url)
-                    .map_err(|e| Error::StorageError(e.to_string()))?,
-            )
-        }
-        #[cfg(feature = "cloud")]
-        "bigtable" => {
-            use super::bigtable::BigtableParameters;
-            use super::BigtableDirectoryService;
-
-            // parse the instance name from the hostname.
-            let instance_name = url
-                .host_str()
-                .ok_or_else(|| Error::StorageError("instance name missing".into()))?
-                .to_string();
-
-            // โ€ฆ but add it to the query string now, so we just need to parse that.
-            url.query_pairs_mut()
-                .append_pair("instance_name", &instance_name);
-
-            let params: BigtableParameters = serde_qs::from_str(url.query().unwrap_or_default())
-                .map_err(|e| Error::InvalidRequest(format!("failed to parse parameters: {}", e)))?;
+    let directory_service_config = with_registry(&REG, || {
+        <DeserializeWithRegistry<Box<dyn ServiceBuilder<Output = dyn DirectoryService>>>>::try_from(
+            url,
+        )
+    })?
+    .0;
+    let directory_service = directory_service_config
+        .build("anonymous", &CompositionContext::blank())
+        .await?;
 
-            Box::new(
-                BigtableDirectoryService::connect(params)
-                    .await
-                    .map_err(|e| Error::StorageError(e.to_string()))?,
-            )
-        }
-        _ => {
-            return Err(crate::Error::StorageError(format!(
-                "unknown scheme: {}",
-                url.scheme()
-            )))
-        }
-    };
     Ok(directory_service)
 }
 
@@ -122,6 +57,8 @@ mod tests {
     lazy_static! {
         static ref TMPDIR_SLED_1: TempDir = TempDir::new().unwrap();
         static ref TMPDIR_SLED_2: TempDir = TempDir::new().unwrap();
+        static ref TMPDIR_REDB_1: TempDir = TempDir::new().unwrap();
+        static ref TMPDIR_REDB_2: TempDir = TempDir::new().unwrap();
     }
 
     #[rstest]
@@ -145,6 +82,16 @@ mod tests {
     #[case::memory_invalid_root_path("memory:///", false)]
     /// This sets a memory url path to "/foo", which is invalid.
     #[case::memory_invalid_root_path_foo("memory:///foo", false)]
+    /// This configures redb in temporary mode.
+    #[case::redb_valid_temporary("redb://", true)]
+    /// This configures redb with /, which should fail.
+    #[case::redb_invalid_root("redb:///", false)]
+    /// This configures redb with a host, not path, which should fail.
+    #[case::redb_invalid_host("redb://foo.example", false)]
+    /// This configures redb with a valid path, which should succeed.
+    #[case::redb_valid_path(&format!("redb://{}", &TMPDIR_REDB_1.path().join("foo").to_str().unwrap()), true)]
+    /// This configures redb with a host, and a valid path path, which should fail.
+    #[case::redb_invalid_host_with_valid_path(&format!("redb://foo.example{}", &TMPDIR_REDB_2.path().join("bar").to_str().unwrap()), false)]
     /// Correct scheme to connect to a unix socket.
     #[case::grpc_valid_unix_socket("grpc+unix:///path/to/somewhere", true)]
     /// Correct scheme for unix socket, but setting a host too, which is invalid.
diff --git a/tvix/castore/src/directoryservice/grpc.rs b/tvix/castore/src/directoryservice/grpc.rs
index fe935629bfcb..9696c5631949 100644
--- a/tvix/castore/src/directoryservice/grpc.rs
+++ b/tvix/castore/src/directoryservice/grpc.rs
@@ -1,44 +1,47 @@
 use std::collections::HashSet;
 
-use super::{DirectoryPutter, DirectoryService};
+use super::{Directory, DirectoryPutter, DirectoryService};
+use crate::composition::{CompositionContext, ServiceBuilder};
 use crate::proto::{self, get_directory_request::ByWhat};
-use crate::{B3Digest, Error};
+use crate::{B3Digest, DirectoryError, Error};
 use async_stream::try_stream;
 use futures::stream::BoxStream;
+use std::sync::Arc;
 use tokio::spawn;
 use tokio::sync::mpsc::UnboundedSender;
 use tokio::task::JoinHandle;
 use tokio_stream::wrappers::UnboundedReceiverStream;
-use tonic::async_trait;
-use tonic::Code;
-use tonic::{transport::Channel, Status};
-use tracing::{instrument, warn};
+use tonic::{async_trait, Code, Status};
+use tracing::{instrument, warn, Instrument as _};
 
 /// Connects to a (remote) tvix-store DirectoryService over gRPC.
 #[derive(Clone)]
-pub struct GRPCDirectoryService {
+pub struct GRPCDirectoryService<T> {
     /// The internal reference to a gRPC client.
     /// Cloning it is cheap, and it internally handles concurrent requests.
-    grpc_client: proto::directory_service_client::DirectoryServiceClient<Channel>,
+    grpc_client: proto::directory_service_client::DirectoryServiceClient<T>,
 }
 
-impl GRPCDirectoryService {
+impl<T> GRPCDirectoryService<T> {
     /// construct a [GRPCDirectoryService] from a [proto::directory_service_client::DirectoryServiceClient].
     /// panics if called outside the context of a tokio runtime.
     pub fn from_client(
-        grpc_client: proto::directory_service_client::DirectoryServiceClient<Channel>,
+        grpc_client: proto::directory_service_client::DirectoryServiceClient<T>,
     ) -> Self {
         Self { grpc_client }
     }
 }
 
 #[async_trait]
-impl DirectoryService for GRPCDirectoryService {
+impl<T> DirectoryService for GRPCDirectoryService<T>
+where
+    T: tonic::client::GrpcService<tonic::body::BoxBody> + Send + Sync + Clone + 'static,
+    T::ResponseBody: tonic::codegen::Body<Data = tonic::codegen::Bytes> + Send + 'static,
+    <T::ResponseBody as tonic::codegen::Body>::Error: Into<tonic::codegen::StdError> + Send,
+    T::Future: Send,
+{
     #[instrument(level = "trace", skip_all, fields(directory.digest = %digest))]
-    async fn get(
-        &self,
-        digest: &B3Digest,
-    ) -> Result<Option<crate::proto::Directory>, crate::Error> {
+    async fn get(&self, digest: &B3Digest) -> Result<Option<Directory>, crate::Error> {
         // Get a new handle to the gRPC client, and copy the digest.
         let mut grpc_client = self.grpc_client.clone();
         let digest_cpy = digest.clone();
@@ -66,15 +69,10 @@ impl DirectoryService for GRPCDirectoryService {
                         "requested directory with digest {}, but got {}",
                         digest, actual_digest
                     )))
-                } else if let Err(e) = directory.validate() {
-                    // Validate the Directory itself is valid.
-                    warn!("directory failed validation: {}", e.to_string());
-                    Err(crate::Error::StorageError(format!(
-                        "directory {} failed validation: {}",
-                        digest, e,
-                    )))
                 } else {
-                    Ok(Some(directory))
+                    Ok(Some(directory.try_into().map_err(|_| {
+                        Error::StorageError("invalid root digest length in response".to_string())
+                    })?))
                 }
             }
             Ok(None) => Ok(None),
@@ -84,11 +82,11 @@ impl DirectoryService for GRPCDirectoryService {
     }
 
     #[instrument(level = "trace", skip_all, fields(directory.digest = %directory.digest()))]
-    async fn put(&self, directory: crate::proto::Directory) -> Result<B3Digest, crate::Error> {
+    async fn put(&self, directory: Directory) -> Result<B3Digest, crate::Error> {
         let resp = self
             .grpc_client
             .clone()
-            .put(tokio_stream::once(directory))
+            .put(tokio_stream::once(proto::Directory::from(directory)))
             .await;
 
         match resp {
@@ -107,7 +105,7 @@ impl DirectoryService for GRPCDirectoryService {
     fn get_recursive(
         &self,
         root_directory_digest: &B3Digest,
-    ) -> BoxStream<'static, Result<proto::Directory, Error>> {
+    ) -> BoxStream<'static, Result<Directory, Error>> {
         let mut grpc_client = self.grpc_client.clone();
         let root_directory_digest = root_directory_digest.clone();
 
@@ -124,19 +122,11 @@ impl DirectoryService for GRPCDirectoryService {
             // The Directory digests we received so far
             let mut received_directory_digests: HashSet<B3Digest> = HashSet::new();
             // The Directory digests we're still expecting to get sent.
-            let mut expected_directory_digests: HashSet<B3Digest> = HashSet::from([root_directory_digest]);
+            let mut expected_directory_digests: HashSet<B3Digest> = HashSet::from([root_directory_digest.clone()]);
 
             loop {
                 match stream.message().await {
                     Ok(Some(directory)) => {
-                        // validate the directory itself.
-                        if let Err(e) = directory.validate() {
-                            Err(crate::Error::StorageError(format!(
-                                "directory {} failed validation: {}",
-                                directory.digest(),
-                                e,
-                            )))?;
-                        }
                         // validate we actually expected that directory, and move it from expected to received.
                         let directory_digest = directory.digest();
                         let was_expected = expected_directory_digests.remove(&directory_digest);
@@ -162,14 +152,28 @@ impl DirectoryService for GRPCDirectoryService {
                                 .insert(child_directory_digest);
                         }
 
+                        let directory = directory.try_into()
+                            .map_err(|e: DirectoryError| Error::StorageError(e.to_string()))?;
+
                         yield directory;
                     },
+                    Ok(None) if expected_directory_digests.len() == 1 && expected_directory_digests.contains(&root_directory_digest) => {
+                        // The root directory of the requested closure was not found, return an
+                        // empty stream
+                        return
+                    }
                     Ok(None) => {
-                        // If we were still expecting something, that's an error.
-                        if !expected_directory_digests.is_empty() {
+                        // The stream has ended
+                        let diff_len = expected_directory_digests
+                            // Account for directories which have been referenced more than once,
+                            // but only received once since they were deduplicated
+                            .difference(&received_directory_digests)
+                            .count();
+                        // If this is not empty, then the closure is incomplete
+                        if diff_len != 0 {
                             Err(crate::Error::StorageError(format!(
                                 "still expected {} directories, but got premature end of stream",
-                                expected_directory_digests.len(),
+                                diff_len
                             )))?
                         } else {
                             return
@@ -194,14 +198,17 @@ impl DirectoryService for GRPCDirectoryService {
 
         let (tx, rx) = tokio::sync::mpsc::unbounded_channel();
 
-        let task: JoinHandle<Result<proto::PutDirectoryResponse, Status>> = spawn(async move {
-            let s = grpc_client
-                .put(UnboundedReceiverStream::new(rx))
-                .await?
-                .into_inner();
+        let task: JoinHandle<Result<proto::PutDirectoryResponse, Status>> = spawn(
+            async move {
+                let s = grpc_client
+                    .put(UnboundedReceiverStream::new(rx))
+                    .await?
+                    .into_inner();
 
-            Ok(s)
-        });
+                Ok(s)
+            } // instrument the task with the current span, this is not done by default
+            .in_current_span(),
+        );
 
         Box::new(GRPCPutter {
             rq: Some((task, tx)),
@@ -209,6 +216,40 @@ impl DirectoryService for GRPCDirectoryService {
     }
 }
 
+#[derive(serde::Deserialize, Debug)]
+#[serde(deny_unknown_fields)]
+pub struct GRPCDirectoryServiceConfig {
+    url: String,
+}
+
+impl TryFrom<url::Url> for GRPCDirectoryServiceConfig {
+    type Error = Box<dyn std::error::Error + Send + Sync>;
+    fn try_from(url: url::Url) -> Result<Self, Self::Error> {
+        //   This is normally grpc+unix for unix sockets, and grpc+http(s) for the HTTP counterparts.
+        // - In the case of unix sockets, there must be a path, but may not be a host.
+        // - In the case of non-unix sockets, there must be a host, but no path.
+        // Constructing the channel is handled by tvix_castore::channel::from_url.
+        Ok(GRPCDirectoryServiceConfig {
+            url: url.to_string(),
+        })
+    }
+}
+
+#[async_trait]
+impl ServiceBuilder for GRPCDirectoryServiceConfig {
+    type Output = dyn DirectoryService;
+    async fn build<'a>(
+        &'a self,
+        _instance_name: &str,
+        _context: &CompositionContext,
+    ) -> Result<Arc<dyn DirectoryService>, Box<dyn std::error::Error + Send + Sync + 'static>> {
+        let client = proto::directory_service_client::DirectoryServiceClient::new(
+            crate::tonic::channel_from_url(&self.url.parse()?).await?,
+        );
+        Ok(Arc::new(GRPCDirectoryService::from_client(client)))
+    }
+}
+
 /// Allows uploading multiple Directory messages in the same gRPC stream.
 pub struct GRPCPutter {
     /// Data about the current request - a handle to the task, and the tx part
@@ -225,11 +266,11 @@ pub struct GRPCPutter {
 #[async_trait]
 impl DirectoryPutter for GRPCPutter {
     #[instrument(level = "trace", skip_all, fields(directory.digest=%directory.digest()), err)]
-    async fn put(&mut self, directory: proto::Directory) -> Result<(), crate::Error> {
+    async fn put(&mut self, directory: Directory) -> Result<(), crate::Error> {
         match self.rq {
             // If we're not already closed, send the directory to directory_sender.
             Some((_, ref directory_sender)) => {
-                if directory_sender.send(directory).is_err() {
+                if directory_sender.send(directory.into()).is_err() {
                     // If the channel has been prematurely closed, invoke close (so we can peek at the error code)
                     // That error code is much more helpful, because it
                     // contains the error message from the server.
diff --git a/tvix/castore/src/directoryservice/memory.rs b/tvix/castore/src/directoryservice/memory.rs
index 3b2795c3968c..b039d9bc7d84 100644
--- a/tvix/castore/src/directoryservice/memory.rs
+++ b/tvix/castore/src/directoryservice/memory.rs
@@ -1,4 +1,4 @@
-use crate::{proto, B3Digest, Error};
+use crate::{B3Digest, Error};
 use futures::stream::BoxStream;
 use std::collections::HashMap;
 use std::sync::Arc;
@@ -7,7 +7,9 @@ use tonic::async_trait;
 use tracing::{instrument, warn};
 
 use super::utils::traverse_directory;
-use super::{DirectoryPutter, DirectoryService, SimplePutter};
+use super::{Directory, DirectoryPutter, DirectoryService, SimplePutter};
+use crate::composition::{CompositionContext, ServiceBuilder};
+use crate::proto;
 
 #[derive(Clone, Default)]
 pub struct MemoryDirectoryService {
@@ -17,7 +19,7 @@ pub struct MemoryDirectoryService {
 #[async_trait]
 impl DirectoryService for MemoryDirectoryService {
     #[instrument(skip(self, digest), fields(directory.digest = %digest))]
-    async fn get(&self, digest: &B3Digest) -> Result<Option<proto::Directory>, Error> {
+    async fn get(&self, digest: &B3Digest) -> Result<Option<Directory>, Error> {
         let db = self.db.read().await;
 
         match db.get(digest) {
@@ -36,35 +38,20 @@ impl DirectoryService for MemoryDirectoryService {
                     )));
                 }
 
-                // Validate the Directory itself is valid.
-                if let Err(e) = directory.validate() {
-                    warn!("directory failed validation: {}", e.to_string());
-                    return Err(Error::StorageError(format!(
-                        "directory {} failed validation: {}",
-                        actual_digest, e,
-                    )));
-                }
-
-                Ok(Some(directory.clone()))
+                Ok(Some(directory.clone().try_into().map_err(|e| {
+                    crate::Error::StorageError(format!("corrupted directory: {}", e))
+                })?))
             }
         }
     }
 
     #[instrument(skip(self, directory), fields(directory.digest = %directory.digest()))]
-    async fn put(&self, directory: proto::Directory) -> Result<B3Digest, Error> {
+    async fn put(&self, directory: Directory) -> Result<B3Digest, Error> {
         let digest = directory.digest();
 
-        // validate the directory itself.
-        if let Err(e) = directory.validate() {
-            return Err(Error::InvalidRequest(format!(
-                "directory {} failed validation: {}",
-                digest, e,
-            )));
-        }
-
         // store it
         let mut db = self.db.write().await;
-        db.insert(digest.clone(), directory);
+        db.insert(digest.clone(), directory.into());
 
         Ok(digest)
     }
@@ -73,7 +60,7 @@ impl DirectoryService for MemoryDirectoryService {
     fn get_recursive(
         &self,
         root_directory_digest: &B3Digest,
-    ) -> BoxStream<'static, Result<proto::Directory, Error>> {
+    ) -> BoxStream<'static, Result<Directory, Error>> {
         traverse_directory(self.clone(), root_directory_digest)
     }
 
@@ -85,3 +72,30 @@ impl DirectoryService for MemoryDirectoryService {
         Box::new(SimplePutter::new(self.clone()))
     }
 }
+
+#[derive(serde::Deserialize, Debug)]
+#[serde(deny_unknown_fields)]
+pub struct MemoryDirectoryServiceConfig {}
+
+impl TryFrom<url::Url> for MemoryDirectoryServiceConfig {
+    type Error = Box<dyn std::error::Error + Send + Sync>;
+    fn try_from(url: url::Url) -> Result<Self, Self::Error> {
+        // memory doesn't support host or path in the URL.
+        if url.has_host() || !url.path().is_empty() {
+            return Err(Error::StorageError("invalid url".to_string()).into());
+        }
+        Ok(MemoryDirectoryServiceConfig {})
+    }
+}
+
+#[async_trait]
+impl ServiceBuilder for MemoryDirectoryServiceConfig {
+    type Output = dyn DirectoryService;
+    async fn build<'a>(
+        &'a self,
+        _instance_name: &str,
+        _context: &CompositionContext,
+    ) -> Result<Arc<dyn DirectoryService>, Box<dyn std::error::Error + Send + Sync + 'static>> {
+        Ok(Arc::new(MemoryDirectoryService::default()))
+    }
+}
diff --git a/tvix/castore/src/directoryservice/mod.rs b/tvix/castore/src/directoryservice/mod.rs
index 3f180ef162d8..25162e4de853 100644
--- a/tvix/castore/src/directoryservice/mod.rs
+++ b/tvix/castore/src/directoryservice/mod.rs
@@ -1,12 +1,16 @@
-use crate::{proto, B3Digest, Error};
+use crate::composition::{Registry, ServiceBuilder};
+use crate::{B3Digest, Directory, Error};
+
 use futures::stream::BoxStream;
 use tonic::async_trait;
-
-mod closure_validator;
+mod combinators;
+mod directory_graph;
 mod from_addr;
 mod grpc;
 mod memory;
 mod object_store;
+mod order_validator;
+mod redb;
 mod simple_putter;
 mod sled;
 #[cfg(test)]
@@ -14,13 +18,16 @@ pub mod tests;
 mod traverse;
 mod utils;
 
-pub use self::closure_validator::ClosureValidator;
+pub use self::combinators::{Cache, CacheConfig};
+pub use self::directory_graph::{DirectoryGraph, ValidatedDirectoryGraph};
 pub use self::from_addr::from_addr;
-pub use self::grpc::GRPCDirectoryService;
-pub use self::memory::MemoryDirectoryService;
-pub use self::object_store::ObjectStoreDirectoryService;
+pub use self::grpc::{GRPCDirectoryService, GRPCDirectoryServiceConfig};
+pub use self::memory::{MemoryDirectoryService, MemoryDirectoryServiceConfig};
+pub use self::object_store::{ObjectStoreDirectoryService, ObjectStoreDirectoryServiceConfig};
+pub use self::order_validator::{LeavesToRootValidator, OrderValidator, RootToLeavesValidator};
+pub use self::redb::{RedbDirectoryService, RedbDirectoryServiceConfig};
 pub use self::simple_putter::SimplePutter;
-pub use self::sled::SledDirectoryService;
+pub use self::sled::{SledDirectoryService, SledDirectoryServiceConfig};
 pub use self::traverse::descend_to;
 pub use self::utils::traverse_directory;
 
@@ -28,10 +35,10 @@ pub use self::utils::traverse_directory;
 mod bigtable;
 
 #[cfg(feature = "cloud")]
-pub use self::bigtable::BigtableDirectoryService;
+pub use self::bigtable::{BigtableDirectoryService, BigtableParameters};
 
 /// The base trait all Directory services need to implement.
-/// This is a simple get and put of [crate::proto::Directory], returning their
+/// This is a simple get and put of [Directory], returning their
 /// digest.
 #[async_trait]
 pub trait DirectoryService: Send + Sync {
@@ -43,14 +50,14 @@ pub trait DirectoryService: Send + Sync {
     /// Directory digests that are at the "root", aka the last element that's
     /// sent to a DirectoryPutter. This makes sense for implementations bundling
     /// closures of directories together in batches.
-    async fn get(&self, digest: &B3Digest) -> Result<Option<proto::Directory>, Error>;
+    async fn get(&self, digest: &B3Digest) -> Result<Option<Directory>, Error>;
     /// Uploads a single Directory message, and returns the calculated
     /// digest, or an error. An error *must* also be returned if the message is
     /// not valid.
-    async fn put(&self, directory: proto::Directory) -> Result<B3Digest, Error>;
+    async fn put(&self, directory: Directory) -> Result<B3Digest, Error>;
 
-    /// Looks up a closure of [proto::Directory].
-    /// Ideally this would be a `impl Stream<Item = Result<proto::Directory, Error>>`,
+    /// Looks up a closure of [Directory].
+    /// Ideally this would be a `impl Stream<Item = Result<Directory, Error>>`,
     /// and we'd be able to add a default implementation for it here, but
     /// we can't have that yet.
     ///
@@ -63,12 +70,14 @@ pub trait DirectoryService: Send + Sync {
     /// Directories are sent in an order from the root to the leaves, so that
     /// the receiving side can validate each message to be a connected to the root
     /// that has initially been requested.
+    ///
+    /// In case the directory can not be found, this should return an empty stream.
     fn get_recursive(
         &self,
         root_directory_digest: &B3Digest,
-    ) -> BoxStream<'static, Result<proto::Directory, Error>>;
+    ) -> BoxStream<'static, Result<Directory, Error>>;
 
-    /// Allows persisting a closure of [proto::Directory], which is a graph of
+    /// Allows persisting a closure of [Directory], which is a graph of
     /// connected Directory messages.
     fn put_multiple_start(&self) -> Box<dyn DirectoryPutter>;
 }
@@ -78,18 +87,18 @@ impl<A> DirectoryService for A
 where
     A: AsRef<dyn DirectoryService> + Send + Sync,
 {
-    async fn get(&self, digest: &B3Digest) -> Result<Option<proto::Directory>, Error> {
+    async fn get(&self, digest: &B3Digest) -> Result<Option<Directory>, Error> {
         self.as_ref().get(digest).await
     }
 
-    async fn put(&self, directory: proto::Directory) -> Result<B3Digest, Error> {
+    async fn put(&self, directory: Directory) -> Result<B3Digest, Error> {
         self.as_ref().put(directory).await
     }
 
     fn get_recursive(
         &self,
         root_directory_digest: &B3Digest,
-    ) -> BoxStream<'static, Result<proto::Directory, Error>> {
+    ) -> BoxStream<'static, Result<Directory, Error>> {
         self.as_ref().get_recursive(root_directory_digest)
     }
 
@@ -98,7 +107,7 @@ where
     }
 }
 
-/// Provides a handle to put a closure of connected [proto::Directory] elements.
+/// Provides a handle to put a closure of connected [Directory] elements.
 ///
 /// The consumer can periodically call [DirectoryPutter::put], starting from the
 /// leaves. Once the root is reached, [DirectoryPutter::close] can be called to
@@ -110,15 +119,29 @@ where
 /// but a single file or symlink.
 #[async_trait]
 pub trait DirectoryPutter: Send {
-    /// Put a individual [proto::Directory] into the store.
+    /// Put a individual [Directory] into the store.
     /// Error semantics and behaviour is up to the specific implementation of
     /// this trait.
     /// Due to bursting, the returned error might refer to an object previously
     /// sent via `put`.
-    async fn put(&mut self, directory: proto::Directory) -> Result<(), Error>;
+    async fn put(&mut self, directory: Directory) -> Result<(), Error>;
 
     /// Close the stream, and wait for any errors.
     /// If there's been any invalid Directory message uploaded, and error *must*
     /// be returned.
     async fn close(&mut self) -> Result<B3Digest, Error>;
 }
+
+/// Registers the builtin DirectoryService implementations with the registry
+pub(crate) fn register_directory_services(reg: &mut Registry) {
+    reg.register::<Box<dyn ServiceBuilder<Output = dyn DirectoryService>>, super::directoryservice::ObjectStoreDirectoryServiceConfig>("objectstore");
+    reg.register::<Box<dyn ServiceBuilder<Output = dyn DirectoryService>>, super::directoryservice::MemoryDirectoryServiceConfig>("memory");
+    reg.register::<Box<dyn ServiceBuilder<Output = dyn DirectoryService>>, super::directoryservice::CacheConfig>("cache");
+    reg.register::<Box<dyn ServiceBuilder<Output = dyn DirectoryService>>, super::directoryservice::GRPCDirectoryServiceConfig>("grpc");
+    reg.register::<Box<dyn ServiceBuilder<Output = dyn DirectoryService>>, super::directoryservice::SledDirectoryServiceConfig>("sled");
+    reg.register::<Box<dyn ServiceBuilder<Output = dyn DirectoryService>>, super::directoryservice::RedbDirectoryServiceConfig>("redb");
+    #[cfg(feature = "cloud")]
+    {
+        reg.register::<Box<dyn ServiceBuilder<Output = dyn DirectoryService>>, super::directoryservice::BigtableParameters>("bigtable");
+    }
+}
diff --git a/tvix/castore/src/directoryservice/object_store.rs b/tvix/castore/src/directoryservice/object_store.rs
index 64ce335edb86..5b5281abcd2f 100644
--- a/tvix/castore/src/directoryservice/object_store.rs
+++ b/tvix/castore/src/directoryservice/object_store.rs
@@ -1,4 +1,4 @@
-use std::collections::HashSet;
+use std::collections::HashMap;
 use std::sync::Arc;
 
 use data_encoding::HEXLOWER;
@@ -16,8 +16,12 @@ use tonic::async_trait;
 use tracing::{instrument, trace, warn, Level};
 use url::Url;
 
-use super::{ClosureValidator, DirectoryPutter, DirectoryService};
-use crate::{proto, B3Digest, Error};
+use super::{
+    Directory, DirectoryGraph, DirectoryPutter, DirectoryService, LeavesToRootValidator,
+    RootToLeavesValidator,
+};
+use crate::composition::{CompositionContext, ServiceBuilder};
+use crate::{proto, B3Digest, Error, Node};
 
 /// Stores directory closures in an object store.
 /// Notably, this makes use of the option to disallow accessing child directories except when
@@ -45,7 +49,7 @@ fn derive_dirs_path(base_path: &Path, digest: &B3Digest) -> Path {
 const MAX_FRAME_LENGTH: usize = 1 * 1024 * 1024 * 1000; // 1 MiB
                                                         //
 impl ObjectStoreDirectoryService {
-    /// Constructs a new [ObjectStoreBlobService] from a [Url] supported by
+    /// Constructs a new [ObjectStoreDirectoryService] from a [Url] supported by
     /// [object_store].
     /// Any path suffix becomes the base path of the object store.
     /// additional options, the same as in [object_store::parse_url_opts] can
@@ -75,13 +79,17 @@ impl DirectoryService for ObjectStoreDirectoryService {
     /// This is the same steps as for get_recursive anyways, so we just call get_recursive and
     /// return the first element of the stream and drop the request.
     #[instrument(skip(self, digest), fields(directory.digest = %digest))]
-    async fn get(&self, digest: &B3Digest) -> Result<Option<proto::Directory>, Error> {
+    async fn get(&self, digest: &B3Digest) -> Result<Option<Directory>, Error> {
         self.get_recursive(digest).take(1).next().await.transpose()
     }
 
     #[instrument(skip(self, directory), fields(directory.digest = %directory.digest()))]
-    async fn put(&self, directory: proto::Directory) -> Result<B3Digest, Error> {
-        if !directory.directories.is_empty() {
+    async fn put(&self, directory: Directory) -> Result<B3Digest, Error> {
+        // Ensure the directory doesn't contain other directory children
+        if directory
+            .nodes()
+            .any(|(_, e)| matches!(e, Node::Directory { .. }))
+        {
             return Err(Error::InvalidRequest(
                     "only put_multiple_start is supported by the ObjectStoreDirectoryService for directories with children".into(),
             ));
@@ -96,10 +104,11 @@ impl DirectoryService for ObjectStoreDirectoryService {
     fn get_recursive(
         &self,
         root_directory_digest: &B3Digest,
-    ) -> BoxStream<'static, Result<proto::Directory, Error>> {
-        // The Directory digests we're expecting to receive.
-        let mut expected_directory_digests: HashSet<B3Digest> =
-            HashSet::from([root_directory_digest.clone()]);
+    ) -> BoxStream<'static, Result<Directory, Error>> {
+        // Check that we are not passing on bogus from the object store to the client, and that the
+        // trust chain from the root digest to the leaves is intact
+        let mut order_validator =
+            RootToLeavesValidator::new_with_root_digest(root_directory_digest.clone());
 
         let dir_path = derive_dirs_path(&self.base_path, root_directory_digest);
         let object_store = self.object_store.clone();
@@ -130,8 +139,7 @@ impl DirectoryService for ObjectStoreDirectoryService {
                         let digest: B3Digest = hasher.update(&buf).finalize().as_bytes().into();
 
                         // Ensure to only decode the directory objects whose digests we trust
-                        let was_expected = expected_directory_digests.remove(&digest);
-                        if !was_expected {
+                        if !order_validator.digest_allowed(&digest) {
                             return Err(crate::Error::StorageError(format!(
                                 "received unexpected directory {}",
                                 digest
@@ -142,14 +150,13 @@ impl DirectoryService for ObjectStoreDirectoryService {
                             warn!("unable to parse directory {}: {}", digest, e);
                             Error::StorageError(e.to_string())
                         })?;
+                        let directory = Directory::try_from(directory).map_err(|e| {
+                            warn!("unable to convert directory {}: {}", digest, e);
+                            Error::StorageError(e.to_string())
+                        })?;
 
-                        for directory in &directory.directories {
-                            // Allow the children to appear next
-                            expected_directory_digests.insert(
-                                B3Digest::try_from(directory.digest.clone())
-                                    .map_err(|e| Error::StorageError(e.to_string()))?,
-                            );
-                        }
+                        // Allow the children to appear next
+                        order_validator.add_directory_unchecked(&directory);
 
                         Ok(directory)
                     })())
@@ -173,11 +180,64 @@ impl DirectoryService for ObjectStoreDirectoryService {
     }
 }
 
+#[derive(serde::Deserialize)]
+#[serde(deny_unknown_fields)]
+pub struct ObjectStoreDirectoryServiceConfig {
+    object_store_url: String,
+    #[serde(default)]
+    object_store_options: HashMap<String, String>,
+}
+
+impl TryFrom<url::Url> for ObjectStoreDirectoryServiceConfig {
+    type Error = Box<dyn std::error::Error + Send + Sync>;
+    fn try_from(url: url::Url) -> Result<Self, Self::Error> {
+        // We need to convert the URL to string, strip the prefix there, and then
+        // parse it back as url, as Url::set_scheme() rejects some of the transitions we want to do.
+        let trimmed_url = {
+            let s = url.to_string();
+            let mut url = Url::parse(
+                s.strip_prefix("objectstore+")
+                    .ok_or(Error::StorageError("Missing objectstore uri".into()))?,
+            )?;
+            // trim the query pairs, they might contain credentials or local settings we don't want to send as-is.
+            url.set_query(None);
+            url
+        };
+        Ok(ObjectStoreDirectoryServiceConfig {
+            object_store_url: trimmed_url.into(),
+            object_store_options: url
+                .query_pairs()
+                .into_iter()
+                .map(|(k, v)| (k.to_string(), v.to_string()))
+                .collect(),
+        })
+    }
+}
+
+#[async_trait]
+impl ServiceBuilder for ObjectStoreDirectoryServiceConfig {
+    type Output = dyn DirectoryService;
+    async fn build<'a>(
+        &'a self,
+        _instance_name: &str,
+        _context: &CompositionContext,
+    ) -> Result<Arc<dyn DirectoryService>, Box<dyn std::error::Error + Send + Sync + 'static>> {
+        let (object_store, path) = object_store::parse_url_opts(
+            &self.object_store_url.parse()?,
+            &self.object_store_options,
+        )?;
+        Ok(Arc::new(ObjectStoreDirectoryService {
+            object_store: Arc::new(object_store),
+            base_path: path,
+        }))
+    }
+}
+
 struct ObjectStoreDirectoryPutter {
     object_store: Arc<dyn ObjectStore>,
     base_path: Path,
 
-    directory_validator: Option<ClosureValidator>,
+    directory_validator: Option<DirectoryGraph<LeavesToRootValidator>>,
 }
 
 impl ObjectStoreDirectoryPutter {
@@ -193,11 +253,13 @@ impl ObjectStoreDirectoryPutter {
 #[async_trait]
 impl DirectoryPutter for ObjectStoreDirectoryPutter {
     #[instrument(level = "trace", skip_all, fields(directory.digest=%directory.digest()), err)]
-    async fn put(&mut self, directory: proto::Directory) -> Result<(), Error> {
+    async fn put(&mut self, directory: Directory) -> Result<(), Error> {
         match self.directory_validator {
             None => return Err(Error::StorageError("already closed".to_string())),
             Some(ref mut validator) => {
-                validator.add(directory)?;
+                validator
+                    .add(directory)
+                    .map_err(|e| Error::StorageError(e.to_string()))?;
             }
         }
 
@@ -214,7 +276,11 @@ impl DirectoryPutter for ObjectStoreDirectoryPutter {
         // retrieve the validated directories.
         // It is important that they are in topological order (root first),
         // since that's how we want to retrieve them from the object store in the end.
-        let directories = validator.finalize_root_to_leaves()?;
+        let directories = validator
+            .validate()
+            .map_err(|e| Error::StorageError(e.to_string()))?
+            .drain_root_to_leaves()
+            .collect::<Vec<_>>();
 
         // Get the root digest
         let root_digest = directories
@@ -245,7 +311,7 @@ impl DirectoryPutter for ObjectStoreDirectoryPutter {
 
                 for directory in directories {
                     directories_sink
-                        .send(directory.encode_to_vec().into())
+                        .send(proto::Directory::from(directory).encode_to_vec().into())
                         .await?;
                 }
 
diff --git a/tvix/castore/src/directoryservice/order_validator.rs b/tvix/castore/src/directoryservice/order_validator.rs
new file mode 100644
index 000000000000..973af92e1294
--- /dev/null
+++ b/tvix/castore/src/directoryservice/order_validator.rs
@@ -0,0 +1,188 @@
+use std::collections::HashSet;
+use tracing::warn;
+
+use super::Directory;
+use crate::{B3Digest, Node};
+
+pub trait OrderValidator {
+    /// Update the order validator's state with the directory
+    /// Returns whether the directory was accepted
+    fn add_directory(&mut self, directory: &Directory) -> bool;
+}
+
+#[derive(Default)]
+/// Validates that newly introduced directories are already referenced from
+/// the root via existing directories.
+/// Commonly used when _receiving_ a directory closure _from_ a store.
+pub struct RootToLeavesValidator {
+    /// Only used to remember the root node, not for validation
+    expected_digests: HashSet<B3Digest>,
+}
+
+impl RootToLeavesValidator {
+    /// Use to validate the root digest of the closure upon receiving the first
+    /// directory.
+    pub fn new_with_root_digest(root_digest: B3Digest) -> Self {
+        let mut this = Self::default();
+        this.expected_digests.insert(root_digest);
+        this
+    }
+
+    /// Checks if a directory is in-order based on its digest.
+    ///
+    /// Particularly useful when receiving directories in canonical protobuf
+    /// encoding, so that directories not connected to the root can be rejected
+    /// without parsing.
+    ///
+    /// After parsing, the directory must be passed to `add_directory_unchecked`
+    /// to add its children to the list of expected digests.
+    pub fn digest_allowed(&self, digest: &B3Digest) -> bool {
+        self.expected_digests.is_empty() // we don't know the root node; allow any
+            || self.expected_digests.contains(digest)
+    }
+
+    /// Update the order validator's state with the directory
+    pub fn add_directory_unchecked(&mut self, directory: &Directory) {
+        // No initial root was specified and this is the first directory
+        if self.expected_digests.is_empty() {
+            self.expected_digests.insert(directory.digest());
+        }
+
+        // Allow the children to appear next
+        for (_, node) in directory.nodes() {
+            if let Node::Directory { digest, .. } = node {
+                self.expected_digests.insert(digest.clone());
+            }
+        }
+    }
+}
+
+impl OrderValidator for RootToLeavesValidator {
+    fn add_directory(&mut self, directory: &Directory) -> bool {
+        if !self.digest_allowed(&directory.digest()) {
+            return false;
+        }
+        self.add_directory_unchecked(directory);
+        true
+    }
+}
+
+#[derive(Default)]
+/// Validates that newly uploaded directories only reference directories which
+/// have already been introduced.
+/// Commonly used when _uploading_ a directory closure _to_ a store.
+pub struct LeavesToRootValidator {
+    /// This is empty in the beginning, and gets filled as leaves and intermediates are
+    /// inserted
+    allowed_references: HashSet<B3Digest>,
+}
+
+impl OrderValidator for LeavesToRootValidator {
+    fn add_directory(&mut self, directory: &Directory) -> bool {
+        let digest = directory.digest();
+
+        for (_, node) in directory.nodes() {
+            if let Node::Directory {
+                digest: subdir_node_digest,
+                ..
+            } = node
+            {
+                if !self.allowed_references.contains(subdir_node_digest) {
+                    warn!(
+                        directory.digest = %digest,
+                        subdirectory.digest = %subdir_node_digest,
+                        "unexpected directory reference"
+                    );
+                    return false;
+                }
+            }
+        }
+
+        self.allowed_references.insert(digest.clone());
+
+        true
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::{LeavesToRootValidator, RootToLeavesValidator};
+    use crate::directoryservice::order_validator::OrderValidator;
+    use crate::directoryservice::Directory;
+    use crate::fixtures::{DIRECTORY_A, DIRECTORY_B, DIRECTORY_C};
+    use rstest::rstest;
+
+    #[rstest]
+    /// Uploading an empty directory should succeed.
+    #[case::empty_directory(&[&*DIRECTORY_A], false)]
+    /// Uploading A, then B (referring to A) should succeed.
+    #[case::simple_closure(&[&*DIRECTORY_A, &*DIRECTORY_B], false)]
+    /// Uploading A, then A, then C (referring to A twice) should succeed.
+    /// We pretend to be a dumb client not deduping directories.
+    #[case::same_child(&[&*DIRECTORY_A, &*DIRECTORY_A, &*DIRECTORY_C], false)]
+    /// Uploading A, then C (referring to A twice) should succeed.
+    #[case::same_child_dedup(&[&*DIRECTORY_A, &*DIRECTORY_C], false)]
+    /// Uploading A, then C (referring to A twice), then B (itself referring to A) should fail during close,
+    /// as B itself would be left unconnected.
+    #[case::unconnected_node(&[&*DIRECTORY_A, &*DIRECTORY_C, &*DIRECTORY_B], false)]
+    /// Uploading B (referring to A) should fail immediately, because A was never uploaded.
+    #[case::dangling_pointer(&[&*DIRECTORY_B], true)]
+    fn leaves_to_root(
+        #[case] directories_to_upload: &[&Directory],
+        #[case] exp_fail_upload_last: bool,
+    ) {
+        let mut validator = LeavesToRootValidator::default();
+        let len_directories_to_upload = directories_to_upload.len();
+
+        for (i, d) in directories_to_upload.iter().enumerate() {
+            let resp = validator.add_directory(d);
+            if i == len_directories_to_upload - 1 && exp_fail_upload_last {
+                assert!(!resp, "expect last put to fail");
+
+                // We don't really care anymore what finalize() would return, as
+                // the add() failed.
+                return;
+            } else {
+                assert!(resp, "expect put to succeed");
+            }
+        }
+    }
+
+    #[rstest]
+    /// Downloading an empty directory should succeed.
+    #[case::empty_directory(&*DIRECTORY_A, &[&*DIRECTORY_A], false)]
+    /// Downlading B, then A (referenced by B) should succeed.
+    #[case::simple_closure(&*DIRECTORY_B, &[&*DIRECTORY_B, &*DIRECTORY_A], false)]
+    /// Downloading C (referring to A twice), then A should succeed.
+    #[case::same_child_dedup(&*DIRECTORY_C, &[&*DIRECTORY_C, &*DIRECTORY_A], false)]
+    /// Downloading C, then B (both referring to A but not referring to each other) should fail immediately as B has no connection to C (the root)
+    #[case::unconnected_node(&*DIRECTORY_C, &[&*DIRECTORY_C, &*DIRECTORY_B], true)]
+    /// Downloading B (specified as the root) but receiving A instead should fail immediately, because A has no connection to B (the root).
+    #[case::dangling_pointer(&*DIRECTORY_B, &[&*DIRECTORY_A], true)]
+    fn root_to_leaves(
+        #[case] root: &Directory,
+        #[case] directories_to_upload: &[&Directory],
+        #[case] exp_fail_upload_last: bool,
+    ) {
+        let mut validator = RootToLeavesValidator::new_with_root_digest(root.digest());
+        let len_directories_to_upload = directories_to_upload.len();
+
+        for (i, d) in directories_to_upload.iter().enumerate() {
+            let resp1 = validator.digest_allowed(&d.digest());
+            let resp = validator.add_directory(d);
+            assert_eq!(
+                resp1, resp,
+                "digest_allowed should return the same value as add_directory"
+            );
+            if i == len_directories_to_upload - 1 && exp_fail_upload_last {
+                assert!(!resp, "expect last put to fail");
+
+                // We don't really care anymore what finalize() would return, as
+                // the add() failed.
+                return;
+            } else {
+                assert!(resp, "expect put to succeed");
+            }
+        }
+    }
+}
diff --git a/tvix/castore/src/directoryservice/redb.rs b/tvix/castore/src/directoryservice/redb.rs
new file mode 100644
index 000000000000..d253df503bb3
--- /dev/null
+++ b/tvix/castore/src/directoryservice/redb.rs
@@ -0,0 +1,303 @@
+use futures::stream::BoxStream;
+use prost::Message;
+use redb::{Database, TableDefinition};
+use std::{path::PathBuf, sync::Arc};
+use tonic::async_trait;
+use tracing::{instrument, warn};
+
+use super::{
+    traverse_directory, Directory, DirectoryGraph, DirectoryPutter, DirectoryService,
+    LeavesToRootValidator,
+};
+use crate::{
+    composition::{CompositionContext, ServiceBuilder},
+    digests, proto, B3Digest, Error,
+};
+
+const DIRECTORY_TABLE: TableDefinition<[u8; digests::B3_LEN], Vec<u8>> =
+    TableDefinition::new("directory");
+
+#[derive(Clone)]
+pub struct RedbDirectoryService {
+    // We wrap the db in an Arc to be able to move it into spawn_blocking,
+    // as discussed in https://github.com/cberner/redb/issues/789
+    db: Arc<Database>,
+}
+
+impl RedbDirectoryService {
+    /// Constructs a new instance using the specified filesystem path for
+    /// storage.
+    pub async fn new(path: PathBuf) -> Result<Self, Error> {
+        if path == PathBuf::from("/") {
+            return Err(Error::StorageError(
+                "cowardly refusing to open / with redb".to_string(),
+            ));
+        }
+
+        let db = tokio::task::spawn_blocking(|| -> Result<_, redb::Error> {
+            let db = redb::Database::create(path)?;
+            create_schema(&db)?;
+            Ok(db)
+        })
+        .await??;
+
+        Ok(Self { db: Arc::new(db) })
+    }
+
+    /// Constructs a new instance using the in-memory backend.
+    pub fn new_temporary() -> Result<Self, Error> {
+        let db =
+            redb::Database::builder().create_with_backend(redb::backends::InMemoryBackend::new())?;
+
+        create_schema(&db)?;
+
+        Ok(Self { db: Arc::new(db) })
+    }
+}
+
+/// Ensures all tables are present.
+/// Opens a write transaction and calls open_table on DIRECTORY_TABLE, which will
+/// create it if not present.
+fn create_schema(db: &redb::Database) -> Result<(), redb::Error> {
+    let txn = db.begin_write()?;
+    txn.open_table(DIRECTORY_TABLE)?;
+    txn.commit()?;
+
+    Ok(())
+}
+
+#[async_trait]
+impl DirectoryService for RedbDirectoryService {
+    #[instrument(skip(self, digest), fields(directory.digest = %digest))]
+    async fn get(&self, digest: &B3Digest) -> Result<Option<Directory>, Error> {
+        let db = self.db.clone();
+
+        // Retrieves the protobuf-encoded Directory for the corresponding digest.
+        let db_get_resp = tokio::task::spawn_blocking({
+            let digest_as_array: [u8; digests::B3_LEN] = digest.to_owned().into();
+            move || -> Result<_, redb::Error> {
+                let txn = db.begin_read()?;
+                let table = txn.open_table(DIRECTORY_TABLE)?;
+                Ok(table.get(digest_as_array)?)
+            }
+        })
+        .await?
+        .map_err(|e| {
+            warn!(err=%e, "failed to retrieve Directory");
+            Error::StorageError("failed to retrieve Directory".to_string())
+        })?;
+
+        // The Directory was not found, return None.
+        let directory_data = match db_get_resp {
+            None => return Ok(None),
+            Some(d) => d,
+        };
+
+        // We check that the digest of the retrieved Directory matches the expected digest.
+        let actual_digest = blake3::hash(directory_data.value().as_slice());
+        if actual_digest.as_bytes() != digest.as_slice() {
+            warn!(directory.actual_digest=%actual_digest, "requested Directory got the wrong digest");
+            return Err(Error::StorageError(
+                "requested Directory got the wrong digest".to_string(),
+            ));
+        }
+
+        // Attempt to decode the retrieved protobuf-encoded Directory, returning a parsing error if
+        // the decoding failed.
+        let directory = match proto::Directory::decode(&*directory_data.value()) {
+            Ok(dir) => {
+                // The returned Directory must be valid.
+                dir.try_into().map_err(|e| {
+                    warn!(err=%e, "Directory failed validation");
+                    Error::StorageError("Directory failed validation".to_string())
+                })?
+            }
+            Err(e) => {
+                warn!(err=%e, "failed to parse Directory");
+                return Err(Error::StorageError("failed to parse Directory".to_string()));
+            }
+        };
+
+        Ok(Some(directory))
+    }
+
+    #[instrument(skip(self, directory), fields(directory.digest = %directory.digest()))]
+    async fn put(&self, directory: Directory) -> Result<B3Digest, Error> {
+        tokio::task::spawn_blocking({
+            let db = self.db.clone();
+            move || {
+                let digest = directory.digest();
+
+                // Store the directory in the table.
+                let txn = db.begin_write()?;
+                {
+                    let mut table = txn.open_table(DIRECTORY_TABLE)?;
+                    let digest_as_array: [u8; digests::B3_LEN] = digest.clone().into();
+                    table.insert(
+                        digest_as_array,
+                        proto::Directory::from(directory).encode_to_vec(),
+                    )?;
+                }
+                txn.commit()?;
+
+                Ok(digest)
+            }
+        })
+        .await?
+    }
+
+    #[instrument(skip_all, fields(directory.digest = %root_directory_digest))]
+    fn get_recursive(
+        &self,
+        root_directory_digest: &B3Digest,
+    ) -> BoxStream<'static, Result<Directory, Error>> {
+        // FUTUREWORK: Ideally we should have all of the directory traversing happen in a single
+        // redb transaction to avoid constantly closing and opening new transactions for the
+        // database.
+        traverse_directory(self.clone(), root_directory_digest)
+    }
+
+    #[instrument(skip_all)]
+    fn put_multiple_start(&self) -> Box<dyn DirectoryPutter> {
+        Box::new(RedbDirectoryPutter {
+            db: self.db.clone(),
+            directory_validator: Some(Default::default()),
+        })
+    }
+}
+
+pub struct RedbDirectoryPutter {
+    db: Arc<Database>,
+
+    /// The directories (inside the directory validator) that we insert later,
+    /// or None, if they were already inserted.
+    directory_validator: Option<DirectoryGraph<LeavesToRootValidator>>,
+}
+
+#[async_trait]
+impl DirectoryPutter for RedbDirectoryPutter {
+    #[instrument(level = "trace", skip_all, fields(directory.digest=%directory.digest()), err)]
+    async fn put(&mut self, directory: Directory) -> Result<(), Error> {
+        match self.directory_validator {
+            None => return Err(Error::StorageError("already closed".to_string())),
+            Some(ref mut validator) => {
+                validator
+                    .add(directory)
+                    .map_err(|e| Error::StorageError(e.to_string()))?;
+            }
+        }
+
+        Ok(())
+    }
+
+    #[instrument(level = "trace", skip_all, ret, err)]
+    async fn close(&mut self) -> Result<B3Digest, Error> {
+        match self.directory_validator.take() {
+            None => Err(Error::StorageError("already closed".to_string())),
+            Some(validator) => {
+                // Insert all directories as a batch.
+                tokio::task::spawn_blocking({
+                    let txn = self.db.begin_write()?;
+                    move || {
+                        // Retrieve the validated directories.
+                        let directories = validator
+                            .validate()
+                            .map_err(|e| Error::StorageError(e.to_string()))?
+                            .drain_leaves_to_root()
+                            .collect::<Vec<_>>();
+
+                        // Get the root digest, which is at the end (cf. insertion order)
+                        let root_digest = directories
+                            .last()
+                            .ok_or_else(|| Error::StorageError("got no directories".to_string()))?
+                            .digest();
+
+                        {
+                            let mut table = txn.open_table(DIRECTORY_TABLE)?;
+
+                            // Looping over all the verified directories, queuing them up for a
+                            // batch insertion.
+                            for directory in directories {
+                                let digest_as_array: [u8; digests::B3_LEN] =
+                                    directory.digest().into();
+                                table.insert(
+                                    digest_as_array,
+                                    proto::Directory::from(directory).encode_to_vec(),
+                                )?;
+                            }
+                        }
+
+                        txn.commit()?;
+
+                        Ok(root_digest)
+                    }
+                })
+                .await?
+            }
+        }
+    }
+}
+
+#[derive(serde::Deserialize)]
+#[serde(deny_unknown_fields)]
+pub struct RedbDirectoryServiceConfig {
+    is_temporary: bool,
+    #[serde(default)]
+    /// required when is_temporary = false
+    path: Option<PathBuf>,
+}
+
+impl TryFrom<url::Url> for RedbDirectoryServiceConfig {
+    type Error = Box<dyn std::error::Error + Send + Sync>;
+    fn try_from(url: url::Url) -> Result<Self, Self::Error> {
+        // redb doesn't support host, and a path can be provided (otherwise
+        // it'll live in memory only).
+        if url.has_host() {
+            return Err(Error::StorageError("no host allowed".to_string()).into());
+        }
+
+        Ok(if url.path().is_empty() {
+            RedbDirectoryServiceConfig {
+                is_temporary: true,
+                path: None,
+            }
+        } else {
+            RedbDirectoryServiceConfig {
+                is_temporary: false,
+                path: Some(url.path().into()),
+            }
+        })
+    }
+}
+
+#[async_trait]
+impl ServiceBuilder for RedbDirectoryServiceConfig {
+    type Output = dyn DirectoryService;
+    async fn build<'a>(
+        &'a self,
+        _instance_name: &str,
+        _context: &CompositionContext,
+    ) -> Result<Arc<dyn DirectoryService>, Box<dyn std::error::Error + Send + Sync + 'static>> {
+        match self {
+            RedbDirectoryServiceConfig {
+                is_temporary: true,
+                path: None,
+            } => Ok(Arc::new(RedbDirectoryService::new_temporary()?)),
+            RedbDirectoryServiceConfig {
+                is_temporary: true,
+                path: Some(_),
+            } => Err(Error::StorageError(
+                "Temporary RedbDirectoryService can not have path".into(),
+            )
+            .into()),
+            RedbDirectoryServiceConfig {
+                is_temporary: false,
+                path: None,
+            } => Err(Error::StorageError("RedbDirectoryService is missing path".into()).into()),
+            RedbDirectoryServiceConfig {
+                is_temporary: false,
+                path: Some(path),
+            } => Ok(Arc::new(RedbDirectoryService::new(path.into()).await?)),
+        }
+    }
+}
diff --git a/tvix/castore/src/directoryservice/simple_putter.rs b/tvix/castore/src/directoryservice/simple_putter.rs
index 25617ebcac82..b4daaee61b22 100644
--- a/tvix/castore/src/directoryservice/simple_putter.rs
+++ b/tvix/castore/src/directoryservice/simple_putter.rs
@@ -1,7 +1,6 @@
-use super::ClosureValidator;
 use super::DirectoryPutter;
 use super::DirectoryService;
-use crate::proto;
+use super::{Directory, DirectoryGraph, LeavesToRootValidator};
 use crate::B3Digest;
 use crate::Error;
 use tonic::async_trait;
@@ -14,7 +13,7 @@ use tracing::warn;
 pub struct SimplePutter<DS: DirectoryService> {
     directory_service: DS,
 
-    directory_validator: Option<ClosureValidator>,
+    directory_validator: Option<DirectoryGraph<LeavesToRootValidator>>,
 }
 
 impl<DS: DirectoryService> SimplePutter<DS> {
@@ -29,11 +28,13 @@ impl<DS: DirectoryService> SimplePutter<DS> {
 #[async_trait]
 impl<DS: DirectoryService + 'static> DirectoryPutter for SimplePutter<DS> {
     #[instrument(level = "trace", skip_all, fields(directory.digest=%directory.digest()), err)]
-    async fn put(&mut self, directory: proto::Directory) -> Result<(), Error> {
+    async fn put(&mut self, directory: Directory) -> Result<(), Error> {
         match self.directory_validator {
             None => return Err(Error::StorageError("already closed".to_string())),
             Some(ref mut validator) => {
-                validator.add(directory)?;
+                validator
+                    .add(directory)
+                    .map_err(|e| Error::StorageError(e.to_string()))?;
             }
         }
 
@@ -46,7 +47,11 @@ impl<DS: DirectoryService + 'static> DirectoryPutter for SimplePutter<DS> {
             None => Err(Error::InvalidRequest("already closed".to_string())),
             Some(validator) => {
                 // retrieve the validated directories.
-                let directories = validator.finalize()?;
+                let directories = validator
+                    .validate()
+                    .map_err(|e| Error::StorageError(e.to_string()))?
+                    .drain_leaves_to_root()
+                    .collect::<Vec<_>>();
 
                 // Get the root digest, which is at the end (cf. insertion order)
                 let root_digest = directories
diff --git a/tvix/castore/src/directoryservice/sled.rs b/tvix/castore/src/directoryservice/sled.rs
index 9490a49c00ae..4f3a860d14e4 100644
--- a/tvix/castore/src/directoryservice/sled.rs
+++ b/tvix/castore/src/directoryservice/sled.rs
@@ -1,14 +1,15 @@
-use crate::proto::Directory;
-use crate::{proto, B3Digest, Error};
 use futures::stream::BoxStream;
 use prost::Message;
 use std::ops::Deref;
 use std::path::Path;
+use std::sync::Arc;
 use tonic::async_trait;
 use tracing::{instrument, warn};
 
 use super::utils::traverse_directory;
-use super::{ClosureValidator, DirectoryPutter, DirectoryService};
+use super::{Directory, DirectoryGraph, DirectoryPutter, DirectoryService, LeavesToRootValidator};
+use crate::composition::{CompositionContext, ServiceBuilder};
+use crate::{proto, B3Digest, Error};
 
 #[derive(Clone)]
 pub struct SledDirectoryService {
@@ -17,6 +18,12 @@ pub struct SledDirectoryService {
 
 impl SledDirectoryService {
     pub fn new<P: AsRef<Path>>(p: P) -> Result<Self, sled::Error> {
+        if p.as_ref() == Path::new("/") {
+            return Err(sled::Error::Unsupported(
+                "cowardly refusing to open / with sled".to_string(),
+            ));
+        }
+
         let config = sled::Config::default()
             .use_compression(false) // is a required parameter
             .path(p);
@@ -36,7 +43,7 @@ impl SledDirectoryService {
 #[async_trait]
 impl DirectoryService for SledDirectoryService {
     #[instrument(skip(self, digest), fields(directory.digest = %digest))]
-    async fn get(&self, digest: &B3Digest) -> Result<Option<proto::Directory>, Error> {
+    async fn get(&self, digest: &B3Digest) -> Result<Option<Directory>, Error> {
         let resp = tokio::task::spawn_blocking({
             let db = self.db.clone();
             let digest = digest.clone();
@@ -53,7 +60,7 @@ impl DirectoryService for SledDirectoryService {
             None => Ok(None),
 
             // The directory was found, try to parse the data as Directory message
-            Some(data) => match Directory::decode(&*data) {
+            Some(data) => match proto::Directory::decode(&*data) {
                 Ok(directory) => {
                     // Validate the retrieved Directory indeed has the
                     // digest we expect it to have, to detect corruptions.
@@ -65,14 +72,10 @@ impl DirectoryService for SledDirectoryService {
                         )));
                     }
 
-                    // Validate the Directory itself is valid.
-                    if let Err(e) = directory.validate() {
-                        warn!("directory failed validation: {}", e.to_string());
-                        return Err(Error::StorageError(format!(
-                            "directory {} failed validation: {}",
-                            actual_digest, e,
-                        )));
-                    }
+                    let directory = directory.try_into().map_err(|e| {
+                        warn!("failed to retrieve directory: {}", e);
+                        Error::StorageError(format!("failed to retrieve directory: {}", e))
+                    })?;
 
                     Ok(Some(directory))
                 }
@@ -85,22 +88,18 @@ impl DirectoryService for SledDirectoryService {
     }
 
     #[instrument(skip(self, directory), fields(directory.digest = %directory.digest()))]
-    async fn put(&self, directory: proto::Directory) -> Result<B3Digest, Error> {
+    async fn put(&self, directory: Directory) -> Result<B3Digest, Error> {
         tokio::task::spawn_blocking({
             let db = self.db.clone();
             move || {
                 let digest = directory.digest();
 
-                // validate the directory itself.
-                if let Err(e) = directory.validate() {
-                    return Err(Error::InvalidRequest(format!(
-                        "directory {} failed validation: {}",
-                        digest, e,
-                    )));
-                }
                 // store it
-                db.insert(digest.as_slice(), directory.encode_to_vec())
-                    .map_err(|e| Error::StorageError(e.to_string()))?;
+                db.insert(
+                    digest.as_slice(),
+                    proto::Directory::from(directory).encode_to_vec(),
+                )
+                .map_err(|e| Error::StorageError(e.to_string()))?;
 
                 Ok(digest)
             }
@@ -112,7 +111,7 @@ impl DirectoryService for SledDirectoryService {
     fn get_recursive(
         &self,
         root_directory_digest: &B3Digest,
-    ) -> BoxStream<'static, Result<proto::Directory, Error>> {
+    ) -> BoxStream<'static, Result<Directory, Error>> {
         traverse_directory(self.clone(), root_directory_digest)
     }
 
@@ -128,6 +127,72 @@ impl DirectoryService for SledDirectoryService {
     }
 }
 
+#[derive(serde::Deserialize)]
+#[serde(deny_unknown_fields)]
+pub struct SledDirectoryServiceConfig {
+    is_temporary: bool,
+    #[serde(default)]
+    /// required when is_temporary = false
+    path: Option<String>,
+}
+
+impl TryFrom<url::Url> for SledDirectoryServiceConfig {
+    type Error = Box<dyn std::error::Error + Send + Sync>;
+    fn try_from(url: url::Url) -> Result<Self, Self::Error> {
+        // sled doesn't support host, and a path can be provided (otherwise
+        // it'll live in memory only).
+        if url.has_host() {
+            return Err(Error::StorageError("no host allowed".to_string()).into());
+        }
+
+        // TODO: expose compression and other parameters as URL parameters?
+
+        Ok(if url.path().is_empty() {
+            SledDirectoryServiceConfig {
+                is_temporary: true,
+                path: None,
+            }
+        } else {
+            SledDirectoryServiceConfig {
+                is_temporary: false,
+                path: Some(url.path().to_string()),
+            }
+        })
+    }
+}
+
+#[async_trait]
+impl ServiceBuilder for SledDirectoryServiceConfig {
+    type Output = dyn DirectoryService;
+    async fn build<'a>(
+        &'a self,
+        _instance_name: &str,
+        _context: &CompositionContext,
+    ) -> Result<Arc<dyn DirectoryService>, Box<dyn std::error::Error + Send + Sync + 'static>> {
+        match self {
+            SledDirectoryServiceConfig {
+                is_temporary: true,
+                path: None,
+            } => Ok(Arc::new(SledDirectoryService::new_temporary()?)),
+            SledDirectoryServiceConfig {
+                is_temporary: true,
+                path: Some(_),
+            } => Err(Error::StorageError(
+                "Temporary SledDirectoryService can not have path".into(),
+            )
+            .into()),
+            SledDirectoryServiceConfig {
+                is_temporary: false,
+                path: None,
+            } => Err(Error::StorageError("SledDirectoryService is missing path".into()).into()),
+            SledDirectoryServiceConfig {
+                is_temporary: false,
+                path: Some(path),
+            } => Ok(Arc::new(SledDirectoryService::new(path)?)),
+        }
+    }
+}
+
 /// Buffers Directory messages to be uploaded and inserts them in a batch
 /// transaction on close.
 pub struct SledDirectoryPutter {
@@ -135,17 +200,19 @@ pub struct SledDirectoryPutter {
 
     /// The directories (inside the directory validator) that we insert later,
     /// or None, if they were already inserted.
-    directory_validator: Option<ClosureValidator>,
+    directory_validator: Option<DirectoryGraph<LeavesToRootValidator>>,
 }
 
 #[async_trait]
 impl DirectoryPutter for SledDirectoryPutter {
     #[instrument(level = "trace", skip_all, fields(directory.digest=%directory.digest()), err)]
-    async fn put(&mut self, directory: proto::Directory) -> Result<(), Error> {
+    async fn put(&mut self, directory: Directory) -> Result<(), Error> {
         match self.directory_validator {
             None => return Err(Error::StorageError("already closed".to_string())),
             Some(ref mut validator) => {
-                validator.add(directory)?;
+                validator
+                    .add(directory)
+                    .map_err(|e| Error::StorageError(e.to_string()))?;
             }
         }
 
@@ -162,7 +229,11 @@ impl DirectoryPutter for SledDirectoryPutter {
                     let tree = self.tree.clone();
                     move || {
                         // retrieve the validated directories.
-                        let directories = validator.finalize()?;
+                        let directories = validator
+                            .validate()
+                            .map_err(|e| Error::StorageError(e.to_string()))?
+                            .drain_leaves_to_root()
+                            .collect::<Vec<_>>();
 
                         // Get the root digest, which is at the end (cf. insertion order)
                         let root_digest = directories
@@ -172,7 +243,10 @@ impl DirectoryPutter for SledDirectoryPutter {
 
                         let mut batch = sled::Batch::default();
                         for directory in directories {
-                            batch.insert(directory.digest().as_slice(), directory.encode_to_vec());
+                            batch.insert(
+                                directory.digest().as_slice(),
+                                proto::Directory::from(directory).encode_to_vec(),
+                            );
                         }
 
                         tree.apply_batch(batch).map_err(|e| {
diff --git a/tvix/castore/src/directoryservice/tests/mod.rs b/tvix/castore/src/directoryservice/tests/mod.rs
index cc3c5b788a2c..ad189564bfe7 100644
--- a/tvix/castore/src/directoryservice/tests/mod.rs
+++ b/tvix/castore/src/directoryservice/tests/mod.rs
@@ -8,10 +8,8 @@ use rstest_reuse::{self, *};
 
 use super::DirectoryService;
 use crate::directoryservice;
-use crate::{
-    fixtures::{DIRECTORY_A, DIRECTORY_B, DIRECTORY_C},
-    proto::{self, Directory},
-};
+use crate::fixtures::{DIRECTORY_A, DIRECTORY_B, DIRECTORY_C, DIRECTORY_D};
+use crate::{Directory, Node};
 
 mod utils;
 use self::utils::make_grpc_directory_service_client;
@@ -26,16 +24,27 @@ use self::utils::make_grpc_directory_service_client;
 #[case::grpc(make_grpc_directory_service_client().await)]
 #[case::memory(directoryservice::from_addr("memory://").await.unwrap())]
 #[case::sled(directoryservice::from_addr("sled://").await.unwrap())]
+#[case::redb(directoryservice::from_addr("redb://").await.unwrap())]
 #[case::objectstore(directoryservice::from_addr("objectstore+memory://").await.unwrap())]
 #[cfg_attr(all(feature = "cloud", feature = "integration"), case::bigtable(directoryservice::from_addr("bigtable://instance-1?project_id=project-1&table_name=table-1&family_name=cf1").await.unwrap()))]
 pub fn directory_services(#[case] directory_service: impl DirectoryService) {}
 
-/// Ensures asking for a directory that doesn't exist returns a Ok(None).
+/// Ensures asking for a directory that doesn't exist returns a Ok(None), and a get_recursive
+/// returns an empty stream.
 #[apply(directory_services)]
 #[tokio::test]
 async fn test_non_exist(directory_service: impl DirectoryService) {
-    let resp = directory_service.get(&DIRECTORY_A.digest()).await;
-    assert!(resp.unwrap().is_none())
+    // single get
+    assert_eq!(Ok(None), directory_service.get(&DIRECTORY_A.digest()).await);
+
+    // recursive get
+    assert_eq!(
+        Vec::<Result<Directory, crate::Error>>::new(),
+        directory_service
+            .get_recursive(&DIRECTORY_A.digest())
+            .collect::<Vec<Result<Directory, crate::Error>>>()
+            .await
+    );
 }
 
 /// Putting a single directory into the store, and then getting it out both via
@@ -123,6 +132,46 @@ async fn put_get_multiple_dedup(directory_service: impl DirectoryService) {
     )
 }
 
+/// This tests the insertion and retrieval of a closure which contains a duplicated directory
+/// (DIRECTORY_A, which is an empty directory), once in the root, and once in a subdir.
+#[apply(directory_services)]
+#[tokio::test]
+async fn put_get_foo(directory_service: impl DirectoryService) {
+    let mut handle = directory_service.put_multiple_start();
+    handle.put(DIRECTORY_A.clone()).await.unwrap();
+    handle.put(DIRECTORY_B.clone()).await.unwrap();
+    handle.put(DIRECTORY_D.clone()).await.unwrap();
+    let root_digest = handle.close().await.unwrap();
+    assert_eq!(
+        DIRECTORY_D.digest(),
+        root_digest,
+        "root digest should match"
+    );
+
+    // Ensure we can get the closure back out of the service, and it is returned in a valid order
+    // (there are multiple valid possibilities)
+    let retrieved_closure = directory_service
+        .get_recursive(&DIRECTORY_D.digest())
+        .collect::<Vec<_>>()
+        .await;
+
+    let valid_closures = [
+        vec![
+            Ok(DIRECTORY_D.clone()),
+            Ok(DIRECTORY_B.clone()),
+            Ok(DIRECTORY_A.clone()),
+        ],
+        vec![
+            Ok(DIRECTORY_D.clone()),
+            Ok(DIRECTORY_A.clone()),
+            Ok(DIRECTORY_B.clone()),
+        ],
+    ];
+    if !valid_closures.contains(&retrieved_closure) {
+        panic!("invalid closure returned: {:?}", retrieved_closure);
+    }
+}
+
 /// Uploading A, then C (referring to A twice), then B (itself referring to A) should fail during close,
 /// as B itself would be left unconnected.
 #[apply(directory_services)]
@@ -161,58 +210,20 @@ async fn upload_reject_dangling_pointer(directory_service: impl DirectoryService
     }
 }
 
-/// Try uploading a Directory failing its internal validation, ensure it gets
-/// rejected.
-#[apply(directory_services)]
-#[tokio::test]
-async fn upload_reject_failing_validation(directory_service: impl DirectoryService) {
-    let broken_directory = Directory {
-        symlinks: vec![proto::SymlinkNode {
-            name: "".into(), // wrong!
-            target: "doesntmatter".into(),
-        }],
-        ..Default::default()
-    };
-    assert!(broken_directory.validate().is_err());
-
-    // Try to upload via single upload.
-    assert!(
-        directory_service
-            .put(broken_directory.clone())
-            .await
-            .is_err(),
-        "single upload must fail"
-    );
-
-    // Try to upload via put_multiple. We're a bit more permissive here, the
-    // intermediate .put() might succeed, due to client-side bursting (in the
-    // case of gRPC), but then the close MUST fail.
-    let mut handle = directory_service.put_multiple_start();
-    if handle.put(broken_directory).await.is_ok() {
-        assert!(
-            handle.close().await.is_err(),
-            "when succeeding put, close must fail"
-        )
-    }
-}
-
 /// Try uploading a Directory that refers to a previously-uploaded directory.
 /// Both pass their isolated validation, but the size field in the parent is wrong.
 /// This should be rejected.
 #[apply(directory_services)]
 #[tokio::test]
 async fn upload_reject_wrong_size(directory_service: impl DirectoryService) {
-    let wrong_parent_directory = Directory {
-        directories: vec![proto::DirectoryNode {
-            name: "foo".into(),
-            digest: DIRECTORY_A.digest().into(),
+    let wrong_parent_directory = Directory::try_from_iter([(
+        "foo".try_into().unwrap(),
+        Node::Directory {
+            digest: DIRECTORY_A.digest(),
             size: DIRECTORY_A.size() + 42, // wrong!
-        }],
-        ..Default::default()
-    };
-
-    // Make sure isolated validation itself is ok
-    assert!(wrong_parent_directory.validate().is_ok());
+        },
+    )])
+    .unwrap();
 
     // Now upload both. Ensure it either fails during the second put, or during
     // the close.
diff --git a/tvix/castore/src/directoryservice/tests/utils.rs b/tvix/castore/src/directoryservice/tests/utils.rs
index 0f706695eec8..3d245ea412d5 100644
--- a/tvix/castore/src/directoryservice/tests/utils.rs
+++ b/tvix/castore/src/directoryservice/tests/utils.rs
@@ -6,6 +6,7 @@ use crate::{
     proto::directory_service_server::DirectoryServiceServer,
 };
 
+use hyper_util::rt::TokioIo;
 use tonic::transport::{Endpoint, Server, Uri};
 
 /// Constructs and returns a gRPC DirectoryService.
@@ -37,7 +38,7 @@ pub async fn make_grpc_directory_service_client() -> Box<dyn DirectoryService> {
                 .unwrap()
                 .connect_with_connector(tower::service_fn(move |_: Uri| {
                     let right = maybe_right.take().unwrap();
-                    async move { Ok::<_, std::io::Error>(right) }
+                    async move { Ok::<_, std::io::Error>(TokioIo::new(right)) }
                 }))
                 .await
                 .unwrap(),
diff --git a/tvix/castore/src/directoryservice/traverse.rs b/tvix/castore/src/directoryservice/traverse.rs
index 17a51ae2bbff..0bd67e9bcf1f 100644
--- a/tvix/castore/src/directoryservice/traverse.rs
+++ b/tvix/castore/src/directoryservice/traverse.rs
@@ -1,8 +1,4 @@
-use super::DirectoryService;
-use crate::{
-    proto::{node::Node, NamedNode},
-    B3Digest, Error, Path,
-};
+use crate::{directoryservice::DirectoryService, Error, Node, Path};
 use tracing::{instrument, warn};
 
 /// This descends from a (root) node to the given (sub)path, returning the Node
@@ -17,19 +13,14 @@ where
     DS: AsRef<dyn DirectoryService>,
 {
     let mut parent_node = root_node;
-    for component in path.as_ref().components() {
+    for component in path.as_ref().components_bytes() {
         match parent_node {
-            Node::File(_) | Node::Symlink(_) => {
+            Node::File { .. } | Node::Symlink { .. } => {
                 // There's still some path left, but the parent node is no directory.
                 // This means the path doesn't exist, as we can't reach it.
                 return Ok(None);
             }
-            Node::Directory(directory_node) => {
-                let digest: B3Digest = directory_node
-                    .digest
-                    .try_into()
-                    .map_err(|_e| Error::StorageError("invalid digest length".to_string()))?;
-
+            Node::Directory { digest, .. } => {
                 // fetch the linked node from the directory_service.
                 let directory =
                     directory_service
@@ -44,15 +35,16 @@ where
                         })?;
 
                 // look for the component in the [Directory].
-                // FUTUREWORK: as the nodes() iterator returns in a sorted fashion, we
-                // could stop as soon as e.name is larger than the search string.
-                if let Some(child_node) = directory.nodes().find(|n| n.get_name() == component) {
+                if let Some((_child_name, child_node)) = directory
+                    .into_nodes()
+                    .find(|(name, _node)| name.as_ref() == component)
+                {
                     // child node found, update prev_node to that and continue.
-                    parent_node = child_node;
+                    parent_node = child_node.clone();
                 } else {
                     // child node not found means there's no such element inside the directory.
                     return Ok(None);
-                }
+                };
             }
         }
     }
@@ -65,8 +57,8 @@ where
 mod tests {
     use crate::{
         directoryservice,
-        fixtures::{DIRECTORY_COMPLICATED, DIRECTORY_WITH_KEEP},
-        PathBuf,
+        fixtures::{DIRECTORY_COMPLICATED, DIRECTORY_WITH_KEEP, EMPTY_BLOB_DIGEST},
+        Node, PathBuf,
     };
 
     use super::descend_to;
@@ -88,21 +80,23 @@ mod tests {
         handle.close().await.expect("must upload");
 
         // construct the node for DIRECTORY_COMPLICATED
-        let node_directory_complicated =
-            crate::proto::node::Node::Directory(crate::proto::DirectoryNode {
-                name: "doesntmatter".into(),
-                digest: DIRECTORY_COMPLICATED.digest().into(),
-                size: DIRECTORY_COMPLICATED.size(),
-            });
+        let node_directory_complicated = Node::Directory {
+            digest: DIRECTORY_COMPLICATED.digest(),
+            size: DIRECTORY_COMPLICATED.size(),
+        };
 
         // construct the node for DIRECTORY_COMPLICATED
-        let node_directory_with_keep = crate::proto::node::Node::Directory(
-            DIRECTORY_COMPLICATED.directories.first().unwrap().clone(),
-        );
+        let node_directory_with_keep = Node::Directory {
+            digest: DIRECTORY_WITH_KEEP.digest(),
+            size: DIRECTORY_WITH_KEEP.size(),
+        };
 
         // construct the node for the .keep file
-        let node_file_keep =
-            crate::proto::node::Node::File(DIRECTORY_WITH_KEEP.files.first().unwrap().clone());
+        let node_file_keep = Node::File {
+            digest: EMPTY_BLOB_DIGEST.clone(),
+            size: 0,
+            executable: false,
+        };
 
         // traversal to an empty subpath should return the root node.
         {
diff --git a/tvix/castore/src/directoryservice/utils.rs b/tvix/castore/src/directoryservice/utils.rs
index a0ba395ecda8..d073c2c3c8ec 100644
--- a/tvix/castore/src/directoryservice/utils.rs
+++ b/tvix/castore/src/directoryservice/utils.rs
@@ -1,21 +1,22 @@
+use super::Directory;
 use super::DirectoryService;
-use crate::proto;
 use crate::B3Digest;
 use crate::Error;
+use crate::Node;
 use async_stream::try_stream;
 use futures::stream::BoxStream;
 use std::collections::{HashSet, VecDeque};
 use tracing::instrument;
 use tracing::warn;
 
-/// Traverses a [proto::Directory] from the root to the children.
+/// Traverses a [Directory] from the root to the children.
 ///
 /// This is mostly BFS, but directories are only returned once.
 #[instrument(skip(directory_service))]
 pub fn traverse_directory<'a, DS: DirectoryService + 'static>(
     directory_service: DS,
     root_directory_digest: &B3Digest,
-) -> BoxStream<'a, Result<proto::Directory, Error>> {
+) -> BoxStream<'a, Result<Directory, Error>> {
     // The list of all directories that still need to be traversed. The next
     // element is picked from the front, new elements are enqueued at the
     // back.
@@ -25,32 +26,30 @@ pub fn traverse_directory<'a, DS: DirectoryService + 'static>(
     // We omit sending the same directories multiple times.
     let mut sent_directory_digests: HashSet<B3Digest> = HashSet::new();
 
+    let root_directory_digest = root_directory_digest.clone();
+
     Box::pin(try_stream! {
         while let Some(current_directory_digest) = worklist_directory_digests.pop_front() {
-            let current_directory = directory_service.get(&current_directory_digest).await.map_err(|e| {
+            let current_directory = match directory_service.get(&current_directory_digest).await.map_err(|e| {
                 warn!("failed to look up directory");
                 Error::StorageError(format!(
                     "unable to look up directory {}: {}",
                     current_directory_digest, e
                 ))
-            })?.ok_or_else(|| {
-                // if it's not there, we have an inconsistent store!
-                warn!("directory {} does not exist", current_directory_digest);
-                Error::StorageError(format!(
-                    "directory {} does not exist",
-                    current_directory_digest
-                ))
-
-            })?;
-
-            // validate, we don't want to send invalid directories.
-            current_directory.validate().map_err(|e| {
-               warn!("directory failed validation: {}", e.to_string());
-               Error::StorageError(format!(
-                   "invalid directory: {}",
-                   current_directory_digest
-               ))
-            })?;
+            })? {
+                // the root node of the requested closure was not found, return an empty list
+                None if current_directory_digest == root_directory_digest => break,
+                // if a child directory of the closure is not there, we have an inconsistent store!
+                None => {
+                    warn!("directory {} does not exist", current_directory_digest);
+                    Err(Error::StorageError(format!(
+                        "directory {} does not exist",
+                        current_directory_digest
+                    )))?;
+                    break;
+                }
+                Some(dir) => dir,
+            };
 
             // We're about to send this directory, so let's avoid sending it again if a
             // descendant has it.
@@ -59,16 +58,15 @@ pub fn traverse_directory<'a, DS: DirectoryService + 'static>(
             // enqueue all child directory digests to the work queue, as
             // long as they're not part of the worklist or already sent.
             // This panics if the digest looks invalid, it's supposed to be checked first.
-            for child_directory_node in &current_directory.directories {
-                // TODO: propagate error
-                let child_digest: B3Digest = child_directory_node.digest.clone().try_into().unwrap();
-
-                if worklist_directory_digests.contains(&child_digest)
-                    || sent_directory_digests.contains(&child_digest)
-                {
-                    continue;
+            for (_, child_directory_node) in current_directory.nodes() {
+                if let Node::Directory{digest: child_digest, ..} = child_directory_node {
+                    if worklist_directory_digests.contains(child_digest)
+                        || sent_directory_digests.contains(child_digest)
+                    {
+                        continue;
+                    }
+                    worklist_directory_digests.push_back(child_digest.clone());
                 }
-                worklist_directory_digests.push_back(child_digest);
             }
 
             yield current_directory;
diff --git a/tvix/castore/src/errors.rs b/tvix/castore/src/errors.rs
index 8343d0774aec..7b5d1a422c99 100644
--- a/tvix/castore/src/errors.rs
+++ b/tvix/castore/src/errors.rs
@@ -1,7 +1,13 @@
+use bstr::ByteSlice;
 use thiserror::Error;
 use tokio::task::JoinError;
 use tonic::Status;
 
+use crate::{
+    path::{PathComponent, PathComponentError},
+    SymlinkTargetError,
+};
+
 /// Errors related to communication with the store.
 #[derive(Debug, Error, PartialEq)]
 pub enum Error {
@@ -12,6 +18,48 @@ pub enum Error {
     StorageError(String),
 }
 
+/// Errors that occur during construction of [crate::Node]
+#[derive(Debug, thiserror::Error, PartialEq)]
+pub enum ValidateNodeError {
+    /// Invalid digest length encountered
+    #[error("invalid digest length: {0}")]
+    InvalidDigestLen(usize),
+    /// Invalid symlink target
+    #[error("Invalid symlink target: {0}")]
+    InvalidSymlinkTarget(SymlinkTargetError),
+}
+
+impl From<crate::digests::Error> for ValidateNodeError {
+    fn from(e: crate::digests::Error) -> Self {
+        match e {
+            crate::digests::Error::InvalidDigestLen(n) => ValidateNodeError::InvalidDigestLen(n),
+        }
+    }
+}
+
+/// Errors that can occur when populating [crate::Directory] messages,
+/// or parsing [crate::proto::Directory]
+#[derive(Debug, thiserror::Error, PartialEq)]
+pub enum DirectoryError {
+    /// Multiple elements with the same name encountered
+    #[error("{:?} is a duplicate name", .0)]
+    DuplicateName(PathComponent),
+    /// Node failed validation
+    #[error("invalid node with name {}: {:?}", .0, .1.to_string())]
+    InvalidNode(PathComponent, ValidateNodeError),
+    #[error("Total size exceeds u64::MAX")]
+    SizeOverflow,
+    /// Invalid name encountered
+    #[error("Invalid name: {0}")]
+    InvalidName(PathComponentError),
+    /// Elements are not in sorted order. Can only happen on protos
+    #[error("{:?} is not sorted", .0.as_bstr())]
+    WrongSorting(bytes::Bytes),
+    /// This can only happen if there's an unknown node type (on protos)
+    #[error("No node set")]
+    NoNodeSet,
+}
+
 impl From<JoinError> for Error {
     fn from(value: JoinError) -> Self {
         Error::StorageError(value.to_string())
@@ -33,6 +81,42 @@ impl From<crate::tonic::Error> for Error {
     }
 }
 
+impl From<redb::Error> for Error {
+    fn from(value: redb::Error) -> Self {
+        Error::StorageError(value.to_string())
+    }
+}
+
+impl From<redb::DatabaseError> for Error {
+    fn from(value: redb::DatabaseError) -> Self {
+        Error::StorageError(value.to_string())
+    }
+}
+
+impl From<redb::TableError> for Error {
+    fn from(value: redb::TableError) -> Self {
+        Error::StorageError(value.to_string())
+    }
+}
+
+impl From<redb::TransactionError> for Error {
+    fn from(value: redb::TransactionError) -> Self {
+        Error::StorageError(value.to_string())
+    }
+}
+
+impl From<redb::StorageError> for Error {
+    fn from(value: redb::StorageError) -> Self {
+        Error::StorageError(value.to_string())
+    }
+}
+
+impl From<redb::CommitError> for Error {
+    fn from(value: redb::CommitError) -> Self {
+        Error::StorageError(value.to_string())
+    }
+}
+
 impl From<std::io::Error> for Error {
     fn from(value: std::io::Error) -> Self {
         if value.kind() == std::io::ErrorKind::InvalidInput {
diff --git a/tvix/castore/src/fixtures.rs b/tvix/castore/src/fixtures.rs
index a206d9b7ddc6..05bad916d55f 100644
--- a/tvix/castore/src/fixtures.rs
+++ b/tvix/castore/src/fixtures.rs
@@ -1,7 +1,4 @@
-use crate::{
-    proto::{self, Directory, DirectoryNode, FileNode, SymlinkNode},
-    B3Digest,
-};
+use crate::{B3Digest, Directory, Node};
 use lazy_static::lazy_static;
 
 pub const HELLOWORLD_BLOB_CONTENTS: &[u8] = b"Hello World!";
@@ -34,55 +31,74 @@ lazy_static! {
     pub static ref BLOB_B_DIGEST: B3Digest = blake3::hash(&BLOB_B).as_bytes().into();
 
     // Directories
-    pub static ref DIRECTORY_WITH_KEEP: proto::Directory = proto::Directory {
-        directories: vec![],
-        files: vec![FileNode {
-            name: b".keep".to_vec().into(),
-            digest: EMPTY_BLOB_DIGEST.clone().into(),
+    pub static ref DIRECTORY_WITH_KEEP: Directory = Directory::try_from_iter([(
+        ".keep".try_into().unwrap(),
+        Node::File{
+            digest: EMPTY_BLOB_DIGEST.clone(),
             size: 0,
-            executable: false,
-        }],
-        symlinks: vec![],
-    };
-    pub static ref DIRECTORY_COMPLICATED: proto::Directory = proto::Directory {
-        directories: vec![DirectoryNode {
-            name: b"keep".to_vec().into(),
-            digest: DIRECTORY_WITH_KEEP.digest().into(),
-            size: DIRECTORY_WITH_KEEP.size(),
-        }],
-        files: vec![FileNode {
-            name: b".keep".to_vec().into(),
-            digest: EMPTY_BLOB_DIGEST.clone().into(),
-            size: 0,
-            executable: false,
-        }],
-        symlinks: vec![SymlinkNode {
-            name: b"aa".to_vec().into(),
-            target: b"/nix/store/somewhereelse".to_vec().into(),
-        }],
-    };
-    pub static ref DIRECTORY_A: Directory = Directory::default();
-    pub static ref DIRECTORY_B: Directory = Directory {
-        directories: vec![DirectoryNode {
-            name: b"a".to_vec().into(),
-            digest: DIRECTORY_A.digest().into(),
-            size: DIRECTORY_A.size(),
-        }],
-        ..Default::default()
-    };
-    pub static ref DIRECTORY_C: Directory = Directory {
-        directories: vec![
-            DirectoryNode {
-                name: b"a".to_vec().into(),
-                digest: DIRECTORY_A.digest().into(),
+            executable: false
+    })]).unwrap();
+    pub static ref DIRECTORY_COMPLICATED: Directory = Directory::try_from_iter([
+        (
+            "keep".try_into().unwrap(),
+            Node::Directory{
+                digest: DIRECTORY_WITH_KEEP.digest(),
+                size: DIRECTORY_WITH_KEEP.size()
+            }
+        ),
+        (
+            ".keep".try_into().unwrap(),
+            Node::File{
+                digest: EMPTY_BLOB_DIGEST.clone(),
+                size: 0,
+                executable: false
+            }
+        ),
+        (
+            "aa".try_into().unwrap(),
+            Node::Symlink{
+                target: "/nix/store/somewhereelse".try_into().unwrap()
+            }
+        )
+    ]).unwrap();
+    pub static ref DIRECTORY_A: Directory = Directory::new();
+    pub static ref DIRECTORY_B: Directory = Directory::try_from_iter([(
+            "a".try_into().unwrap(),
+            Node::Directory{
+                digest: DIRECTORY_A.digest(),
                 size: DIRECTORY_A.size(),
-            },
-            DirectoryNode {
-                name: b"a'".to_vec().into(),
-                digest: DIRECTORY_A.digest().into(),
+            }
+    )]).unwrap();
+    pub static ref DIRECTORY_C: Directory = Directory::try_from_iter([
+        (
+            "a".try_into().unwrap(),
+            Node::Directory{
+                digest: DIRECTORY_A.digest(),
                 size: DIRECTORY_A.size(),
             }
-        ],
-        ..Default::default()
-    };
+        ),
+        (
+            "a'".try_into().unwrap(),
+            Node::Directory{
+                digest: DIRECTORY_A.digest(),
+                size: DIRECTORY_A.size(),
+            }
+        )
+    ]).unwrap();
+    pub static ref DIRECTORY_D: Directory = Directory::try_from_iter([
+        (
+            "a".try_into().unwrap(),
+            Node::Directory{
+                digest: DIRECTORY_A.digest(),
+                size: DIRECTORY_A.size(),
+            }
+        ),
+        (
+            "b".try_into().unwrap(),
+            Node::Directory{
+                digest: DIRECTORY_B.digest(),
+                size: DIRECTORY_B.size(),
+            }
+        )
+    ]).unwrap();
 }
diff --git a/tvix/castore/src/fs/fuse.rs b/tvix/castore/src/fs/fuse/mod.rs
index cd50618ff5bc..64ef29ed2aa1 100644
--- a/tvix/castore/src/fs/fuse.rs
+++ b/tvix/castore/src/fs/fuse/mod.rs
@@ -1,8 +1,13 @@
-use std::{io, path::Path, sync::Arc, thread};
+use std::{io, path::Path, sync::Arc};
 
 use fuse_backend_rs::{api::filesystem::FileSystem, transport::FuseSession};
+use parking_lot::Mutex;
+use threadpool::ThreadPool;
 use tracing::{error, instrument};
 
+#[cfg(test)]
+mod tests;
+
 struct FuseServer<FS>
 where
     FS: FileSystem + Sync + Send,
@@ -46,9 +51,12 @@ where
     }
 }
 
+/// Starts a [Filesystem] with the specified number of threads, and provides
+/// functions to unmount, and wait for it to have completed.
+#[derive(Clone)]
 pub struct FuseDaemon {
-    session: FuseSession,
-    threads: Vec<thread::JoinHandle<()>>,
+    session: Arc<Mutex<FuseSession>>,
+    threads: Arc<ThreadPool>,
 }
 
 impl FuseDaemon {
@@ -56,7 +64,7 @@ impl FuseDaemon {
     pub fn new<FS, P>(
         fs: FS,
         mountpoint: P,
-        threads: usize,
+        num_threads: usize,
         allow_other: bool,
     ) -> Result<Self, io::Error>
     where
@@ -73,40 +81,49 @@ impl FuseDaemon {
         session
             .mount()
             .map_err(|e| io::Error::new(io::ErrorKind::Other, e.to_string()))?;
-        let mut join_handles = Vec::with_capacity(threads);
-        for _ in 0..threads {
+
+        // construct a thread pool
+        let threads = threadpool::Builder::new()
+            .num_threads(num_threads)
+            .thread_name("fuse_server".to_string())
+            .build();
+
+        for _ in 0..num_threads {
+            // for each thread requested, create and start a FuseServer accepting requests.
             let mut server = FuseServer {
                 server: server.clone(),
                 channel: session
                     .new_channel()
                     .map_err(|e| io::Error::new(io::ErrorKind::Other, e.to_string()))?,
             };
-            let join_handle = thread::Builder::new()
-                .name("fuse_server".to_string())
-                .spawn(move || {
-                    let _ = server.start();
-                })?;
-            join_handles.push(join_handle);
+
+            threads.execute(move || {
+                let _ = server.start();
+            });
         }
 
         Ok(FuseDaemon {
-            session,
-            threads: join_handles,
+            session: Arc::new(Mutex::new(session)),
+            threads: Arc::new(threads),
         })
     }
 
+    /// Waits for all threads to finish.
+    #[instrument(skip_all)]
+    pub fn wait(&self) {
+        self.threads.join()
+    }
+
+    /// Send the unmount command, and waits for all threads to finish.
     #[instrument(skip_all, err)]
-    pub fn unmount(&mut self) -> Result<(), io::Error> {
+    pub fn unmount(&self) -> Result<(), io::Error> {
+        // Send the unmount command.
         self.session
+            .lock()
             .umount()
             .map_err(|e| io::Error::new(io::ErrorKind::Other, e.to_string()))?;
 
-        for thread in self.threads.drain(..) {
-            thread.join().map_err(|_| {
-                io::Error::new(io::ErrorKind::Other, "failed to join fuse server thread")
-            })?;
-        }
-
+        self.wait();
         Ok(())
     }
 }
diff --git a/tvix/castore/src/fs/tests.rs b/tvix/castore/src/fs/fuse/tests.rs
index d6eeb8a4113d..9e01204d5da7 100644
--- a/tvix/castore/src/fs/tests.rs
+++ b/tvix/castore/src/fs/fuse/tests.rs
@@ -1,5 +1,4 @@
 use bstr::ByteSlice;
-use bytes::Bytes;
 use std::{
     collections::BTreeMap,
     ffi::{OsStr, OsString},
@@ -11,13 +10,15 @@ use std::{
 use tempfile::TempDir;
 use tokio_stream::{wrappers::ReadDirStream, StreamExt};
 
-use super::{fuse::FuseDaemon, TvixStoreFs};
-use crate::proto as castorepb;
-use crate::proto::node::Node;
+use super::FuseDaemon;
 use crate::{
     blobservice::{BlobService, MemoryBlobService},
     directoryservice::{DirectoryService, MemoryDirectoryService},
-    fixtures,
+    fixtures, Node,
+};
+use crate::{
+    fs::{TvixStoreFs, XATTR_NAME_BLOB_DIGEST, XATTR_NAME_DIRECTORY_DIGEST},
+    PathComponent,
 };
 
 const BLOB_A_NAME: &str = "00000000000000000000000000000000-test";
@@ -38,7 +39,7 @@ fn gen_svcs() -> (Arc<dyn BlobService>, Arc<dyn DirectoryService>) {
 fn do_mount<P: AsRef<Path>, BS, DS>(
     blob_service: BS,
     directory_service: DS,
-    root_nodes: BTreeMap<bytes::Bytes, Node>,
+    root_nodes: BTreeMap<PathComponent, Node>,
     mountpoint: P,
     list_root: bool,
     show_xattr: bool,
@@ -59,7 +60,7 @@ where
 
 async fn populate_blob_a(
     blob_service: &Arc<dyn BlobService>,
-    root_nodes: &mut BTreeMap<Bytes, Node>,
+    root_nodes: &mut BTreeMap<PathComponent, Node>,
 ) {
     let mut bw = blob_service.open_write().await;
     tokio::io::copy(&mut Cursor::new(fixtures::BLOB_A.to_vec()), &mut bw)
@@ -68,19 +69,18 @@ async fn populate_blob_a(
     bw.close().await.expect("must succeed closing");
 
     root_nodes.insert(
-        BLOB_A_NAME.into(),
-        Node::File(castorepb::FileNode {
-            name: BLOB_A_NAME.into(),
-            digest: fixtures::BLOB_A_DIGEST.clone().into(),
+        BLOB_A_NAME.try_into().unwrap(),
+        Node::File {
+            digest: fixtures::BLOB_A_DIGEST.clone(),
             size: fixtures::BLOB_A.len() as u64,
             executable: false,
-        }),
+        },
     );
 }
 
 async fn populate_blob_b(
     blob_service: &Arc<dyn BlobService>,
-    root_nodes: &mut BTreeMap<Bytes, Node>,
+    root_nodes: &mut BTreeMap<PathComponent, Node>,
 ) {
     let mut bw = blob_service.open_write().await;
     tokio::io::copy(&mut Cursor::new(fixtures::BLOB_B.to_vec()), &mut bw)
@@ -89,20 +89,19 @@ async fn populate_blob_b(
     bw.close().await.expect("must succeed closing");
 
     root_nodes.insert(
-        BLOB_B_NAME.into(),
-        Node::File(castorepb::FileNode {
-            name: BLOB_B_NAME.into(),
-            digest: fixtures::BLOB_B_DIGEST.clone().into(),
+        BLOB_B_NAME.try_into().unwrap(),
+        Node::File {
+            digest: fixtures::BLOB_B_DIGEST.clone(),
             size: fixtures::BLOB_B.len() as u64,
             executable: false,
-        }),
+        },
     );
 }
 
 /// adds a blob containing helloworld and marks it as executable
 async fn populate_blob_helloworld(
     blob_service: &Arc<dyn BlobService>,
-    root_nodes: &mut BTreeMap<Bytes, Node>,
+    root_nodes: &mut BTreeMap<PathComponent, Node>,
 ) {
     let mut bw = blob_service.open_write().await;
     tokio::io::copy(
@@ -114,42 +113,39 @@ async fn populate_blob_helloworld(
     bw.close().await.expect("must succeed closing");
 
     root_nodes.insert(
-        HELLOWORLD_BLOB_NAME.into(),
-        Node::File(castorepb::FileNode {
-            name: HELLOWORLD_BLOB_NAME.into(),
-            digest: fixtures::HELLOWORLD_BLOB_DIGEST.clone().into(),
+        HELLOWORLD_BLOB_NAME.try_into().unwrap(),
+        Node::File {
+            digest: fixtures::HELLOWORLD_BLOB_DIGEST.clone(),
             size: fixtures::HELLOWORLD_BLOB_CONTENTS.len() as u64,
             executable: true,
-        }),
+        },
     );
 }
 
-async fn populate_symlink(root_nodes: &mut BTreeMap<Bytes, Node>) {
+async fn populate_symlink(root_nodes: &mut BTreeMap<PathComponent, Node>) {
     root_nodes.insert(
-        SYMLINK_NAME.into(),
-        Node::Symlink(castorepb::SymlinkNode {
-            name: SYMLINK_NAME.into(),
-            target: BLOB_A_NAME.into(),
-        }),
+        SYMLINK_NAME.try_into().unwrap(),
+        Node::Symlink {
+            target: BLOB_A_NAME.try_into().unwrap(),
+        },
     );
 }
 
 /// This writes a symlink pointing to /nix/store/somewhereelse,
 /// which is the same symlink target as "aa" inside DIRECTORY_COMPLICATED.
-async fn populate_symlink2(root_nodes: &mut BTreeMap<Bytes, Node>) {
+async fn populate_symlink2(root_nodes: &mut BTreeMap<PathComponent, Node>) {
     root_nodes.insert(
-        SYMLINK_NAME2.into(),
-        Node::Symlink(castorepb::SymlinkNode {
-            name: SYMLINK_NAME2.into(),
-            target: "/nix/store/somewhereelse".into(),
-        }),
+        SYMLINK_NAME2.try_into().unwrap(),
+        Node::Symlink {
+            target: "/nix/store/somewhereelse".try_into().unwrap(),
+        },
     );
 }
 
 async fn populate_directory_with_keep(
     blob_service: &Arc<dyn BlobService>,
     directory_service: &Arc<dyn DirectoryService>,
-    root_nodes: &mut BTreeMap<Bytes, Node>,
+    root_nodes: &mut BTreeMap<PathComponent, Node>,
 ) {
     // upload empty blob
     let mut bw = blob_service.open_write().await;
@@ -165,45 +161,42 @@ async fn populate_directory_with_keep(
         .expect("must succeed uploading");
 
     root_nodes.insert(
-        DIRECTORY_WITH_KEEP_NAME.into(),
-        castorepb::node::Node::Directory(castorepb::DirectoryNode {
-            name: DIRECTORY_WITH_KEEP_NAME.into(),
-            digest: fixtures::DIRECTORY_WITH_KEEP.digest().into(),
+        DIRECTORY_WITH_KEEP_NAME.try_into().unwrap(),
+        Node::Directory {
+            digest: fixtures::DIRECTORY_WITH_KEEP.digest(),
             size: fixtures::DIRECTORY_WITH_KEEP.size(),
-        }),
+        },
     );
 }
 
 /// Create a root node for DIRECTORY_WITH_KEEP, but don't upload the Directory
 /// itself.
-async fn populate_directorynode_without_directory(root_nodes: &mut BTreeMap<Bytes, Node>) {
+async fn populate_directorynode_without_directory(root_nodes: &mut BTreeMap<PathComponent, Node>) {
     root_nodes.insert(
-        DIRECTORY_WITH_KEEP_NAME.into(),
-        castorepb::node::Node::Directory(castorepb::DirectoryNode {
-            name: DIRECTORY_WITH_KEEP_NAME.into(),
-            digest: fixtures::DIRECTORY_WITH_KEEP.digest().into(),
+        DIRECTORY_WITH_KEEP_NAME.try_into().unwrap(),
+        Node::Directory {
+            digest: fixtures::DIRECTORY_WITH_KEEP.digest(),
             size: fixtures::DIRECTORY_WITH_KEEP.size(),
-        }),
+        },
     );
 }
 
 /// Insert BLOB_A, but don't provide the blob .keep is pointing to.
-async fn populate_filenode_without_blob(root_nodes: &mut BTreeMap<Bytes, Node>) {
+async fn populate_filenode_without_blob(root_nodes: &mut BTreeMap<PathComponent, Node>) {
     root_nodes.insert(
-        BLOB_A_NAME.into(),
-        Node::File(castorepb::FileNode {
-            name: BLOB_A_NAME.into(),
-            digest: fixtures::BLOB_A_DIGEST.clone().into(),
+        BLOB_A_NAME.try_into().unwrap(),
+        Node::File {
+            digest: fixtures::BLOB_A_DIGEST.clone(),
             size: fixtures::BLOB_A.len() as u64,
             executable: false,
-        }),
+        },
     );
 }
 
 async fn populate_directory_complicated(
     blob_service: &Arc<dyn BlobService>,
     directory_service: &Arc<dyn DirectoryService>,
-    root_nodes: &mut BTreeMap<Bytes, Node>,
+    root_nodes: &mut BTreeMap<PathComponent, Node>,
 ) {
     // upload empty blob
     let mut bw = blob_service.open_write().await;
@@ -225,12 +218,11 @@ async fn populate_directory_complicated(
         .expect("must succeed uploading");
 
     root_nodes.insert(
-        DIRECTORY_COMPLICATED_NAME.into(),
-        Node::Directory(castorepb::DirectoryNode {
-            name: DIRECTORY_COMPLICATED_NAME.into(),
-            digest: fixtures::DIRECTORY_COMPLICATED.digest().into(),
+        DIRECTORY_COMPLICATED_NAME.try_into().unwrap(),
+        Node::Directory {
+            digest: fixtures::DIRECTORY_COMPLICATED.digest(),
             size: fixtures::DIRECTORY_COMPLICATED.size(),
-        }),
+        },
     );
 }
 
@@ -247,7 +239,7 @@ async fn mount() {
 
     let (blob_service, directory_service) = gen_svcs();
 
-    let mut fuse_daemon = do_mount(
+    let fuse_daemon = do_mount(
         blob_service,
         directory_service,
         BTreeMap::default(),
@@ -270,7 +262,7 @@ async fn root() {
     let tmpdir = TempDir::new().unwrap();
 
     let (blob_service, directory_service) = gen_svcs();
-    let mut fuse_daemon = do_mount(
+    let fuse_daemon = do_mount(
         blob_service,
         directory_service,
         BTreeMap::default(),
@@ -304,7 +296,7 @@ async fn root_with_listing() {
 
     populate_blob_a(&blob_service, &mut root_nodes).await;
 
-    let mut fuse_daemon = do_mount(
+    let fuse_daemon = do_mount(
         blob_service,
         directory_service,
         root_nodes,
@@ -348,7 +340,7 @@ async fn stat_file_at_root() {
 
     populate_blob_a(&blob_service, &mut root_nodes).await;
 
-    let mut fuse_daemon = do_mount(
+    let fuse_daemon = do_mount(
         blob_service,
         directory_service,
         root_nodes,
@@ -385,7 +377,7 @@ async fn read_file_at_root() {
 
     populate_blob_a(&blob_service, &mut root_nodes).await;
 
-    let mut fuse_daemon = do_mount(
+    let fuse_daemon = do_mount(
         blob_service,
         directory_service,
         root_nodes,
@@ -422,7 +414,7 @@ async fn read_large_file_at_root() {
 
     populate_blob_b(&blob_service, &mut root_nodes).await;
 
-    let mut fuse_daemon = do_mount(
+    let fuse_daemon = do_mount(
         blob_service,
         directory_service,
         root_nodes,
@@ -467,7 +459,7 @@ async fn symlink_readlink() {
 
     populate_symlink(&mut root_nodes).await;
 
-    let mut fuse_daemon = do_mount(
+    let fuse_daemon = do_mount(
         blob_service,
         directory_service,
         root_nodes,
@@ -514,7 +506,7 @@ async fn read_stat_through_symlink() {
     populate_blob_a(&blob_service, &mut root_nodes).await;
     populate_symlink(&mut root_nodes).await;
 
-    let mut fuse_daemon = do_mount(
+    let fuse_daemon = do_mount(
         blob_service,
         directory_service,
         root_nodes,
@@ -559,7 +551,7 @@ async fn read_stat_directory() {
 
     populate_directory_with_keep(&blob_service, &directory_service, &mut root_nodes).await;
 
-    let mut fuse_daemon = do_mount(
+    let fuse_daemon = do_mount(
         blob_service,
         directory_service,
         root_nodes,
@@ -596,7 +588,7 @@ async fn xattr() {
     populate_directory_with_keep(&blob_service, &directory_service, &mut root_nodes).await;
     populate_blob_a(&blob_service, &mut root_nodes).await;
 
-    let mut fuse_daemon = do_mount(
+    let fuse_daemon = do_mount(
         blob_service,
         directory_service,
         root_nodes,
@@ -614,12 +606,12 @@ async fn xattr() {
         // There should be 1 key, XATTR_NAME_DIRECTORY_DIGEST.
         assert_eq!(1, xattr_names.len(), "there should be 1 xattr name");
         assert_eq!(
-            super::XATTR_NAME_DIRECTORY_DIGEST,
+            XATTR_NAME_DIRECTORY_DIGEST,
             xattr_names.first().unwrap().as_encoded_bytes()
         );
 
         // The key should equal to the string-formatted b3 digest.
-        let val = xattr::get(&p, OsStr::from_bytes(super::XATTR_NAME_DIRECTORY_DIGEST))
+        let val = xattr::get(&p, OsStr::from_bytes(XATTR_NAME_DIRECTORY_DIGEST))
             .expect("must succeed")
             .expect("must be some");
         assert_eq!(
@@ -643,12 +635,12 @@ async fn xattr() {
         // There should be 1 key, XATTR_NAME_BLOB_DIGEST.
         assert_eq!(1, xattr_names.len(), "there should be 1 xattr name");
         assert_eq!(
-            super::XATTR_NAME_BLOB_DIGEST,
+            XATTR_NAME_BLOB_DIGEST,
             xattr_names.first().unwrap().as_encoded_bytes()
         );
 
         // The key should equal to the string-formatted b3 digest.
-        let val = xattr::get(&p, OsStr::from_bytes(super::XATTR_NAME_BLOB_DIGEST))
+        let val = xattr::get(&p, OsStr::from_bytes(XATTR_NAME_BLOB_DIGEST))
             .expect("must succeed")
             .expect("must be some");
         assert_eq!(
@@ -679,7 +671,7 @@ async fn read_blob_inside_dir() {
 
     populate_directory_with_keep(&blob_service, &directory_service, &mut root_nodes).await;
 
-    let mut fuse_daemon = do_mount(
+    let fuse_daemon = do_mount(
         blob_service,
         directory_service,
         root_nodes,
@@ -719,7 +711,7 @@ async fn read_blob_deep_inside_dir() {
 
     populate_directory_complicated(&blob_service, &directory_service, &mut root_nodes).await;
 
-    let mut fuse_daemon = do_mount(
+    let fuse_daemon = do_mount(
         blob_service,
         directory_service,
         root_nodes,
@@ -762,7 +754,7 @@ async fn readdir() {
 
     populate_directory_complicated(&blob_service, &directory_service, &mut root_nodes).await;
 
-    let mut fuse_daemon = do_mount(
+    let fuse_daemon = do_mount(
         blob_service,
         directory_service,
         root_nodes,
@@ -822,7 +814,7 @@ async fn readdir_deep() {
 
     populate_directory_complicated(&blob_service, &directory_service, &mut root_nodes).await;
 
-    let mut fuse_daemon = do_mount(
+    let fuse_daemon = do_mount(
         blob_service,
         directory_service,
         root_nodes,
@@ -872,7 +864,7 @@ async fn check_attributes() {
     populate_symlink(&mut root_nodes).await;
     populate_blob_helloworld(&blob_service, &mut root_nodes).await;
 
-    let mut fuse_daemon = do_mount(
+    let fuse_daemon = do_mount(
         blob_service,
         directory_service,
         root_nodes,
@@ -947,7 +939,7 @@ async fn compare_inodes_directories() {
     populate_directory_with_keep(&blob_service, &directory_service, &mut root_nodes).await;
     populate_directory_complicated(&blob_service, &directory_service, &mut root_nodes).await;
 
-    let mut fuse_daemon = do_mount(
+    let fuse_daemon = do_mount(
         blob_service,
         directory_service,
         root_nodes,
@@ -991,7 +983,7 @@ async fn compare_inodes_files() {
 
     populate_directory_complicated(&blob_service, &directory_service, &mut root_nodes).await;
 
-    let mut fuse_daemon = do_mount(
+    let fuse_daemon = do_mount(
         blob_service,
         directory_service,
         root_nodes,
@@ -1040,7 +1032,7 @@ async fn compare_inodes_symlinks() {
     populate_directory_complicated(&blob_service, &directory_service, &mut root_nodes).await;
     populate_symlink2(&mut root_nodes).await;
 
-    let mut fuse_daemon = do_mount(
+    let fuse_daemon = do_mount(
         blob_service,
         directory_service,
         root_nodes,
@@ -1083,7 +1075,7 @@ async fn read_wrong_paths_in_root() {
 
     populate_blob_a(&blob_service, &mut root_nodes).await;
 
-    let mut fuse_daemon = do_mount(
+    let fuse_daemon = do_mount(
         blob_service,
         directory_service,
         root_nodes,
@@ -1138,7 +1130,7 @@ async fn disallow_writes() {
     let (blob_service, directory_service) = gen_svcs();
     let root_nodes = BTreeMap::default();
 
-    let mut fuse_daemon = do_mount(
+    let fuse_daemon = do_mount(
         blob_service,
         directory_service,
         root_nodes,
@@ -1170,7 +1162,7 @@ async fn missing_directory() {
 
     populate_directorynode_without_directory(&mut root_nodes).await;
 
-    let mut fuse_daemon = do_mount(
+    let fuse_daemon = do_mount(
         blob_service,
         directory_service,
         root_nodes,
@@ -1218,7 +1210,7 @@ async fn missing_blob() {
 
     populate_filenode_without_blob(&mut root_nodes).await;
 
-    let mut fuse_daemon = do_mount(
+    let fuse_daemon = do_mount(
         blob_service,
         directory_service,
         root_nodes,
diff --git a/tvix/castore/src/fs/inodes.rs b/tvix/castore/src/fs/inodes.rs
index bdd459543470..2696fdede378 100644
--- a/tvix/castore/src/fs/inodes.rs
+++ b/tvix/castore/src/fs/inodes.rs
@@ -2,10 +2,7 @@
 //! about inodes, which present tvix-castore nodes in a filesystem.
 use std::time::Duration;
 
-use bytes::Bytes;
-
-use crate::proto as castorepb;
-use crate::B3Digest;
+use crate::{path::PathComponent, B3Digest, Node};
 
 #[derive(Clone, Debug)]
 pub enum InodeData {
@@ -20,27 +17,23 @@ pub enum InodeData {
 /// lookup and did fetch the data.
 #[derive(Clone, Debug)]
 pub enum DirectoryInodeData {
-    Sparse(B3Digest, u64),                                  // digest, size
-    Populated(B3Digest, Vec<(u64, castorepb::node::Node)>), // [(child_inode, node)]
+    Sparse(B3Digest, u64),                                // digest, size
+    Populated(B3Digest, Vec<(u64, PathComponent, Node)>), // [(child_inode, name, node)]
 }
 
 impl InodeData {
     /// Constructs a new InodeData by consuming a [Node].
-    /// It splits off the orginal name, so it can be used later.
-    pub fn from_node(node: castorepb::node::Node) -> (Self, Bytes) {
+    pub fn from_node(node: &Node) -> Self {
         match node {
-            castorepb::node::Node::Directory(n) => (
-                Self::Directory(DirectoryInodeData::Sparse(
-                    n.digest.try_into().unwrap(),
-                    n.size,
-                )),
-                n.name,
-            ),
-            castorepb::node::Node::File(n) => (
-                Self::Regular(n.digest.try_into().unwrap(), n.size, n.executable),
-                n.name,
-            ),
-            castorepb::node::Node::Symlink(n) => (Self::Symlink(n.target), n.name),
+            Node::Directory { digest, size } => {
+                Self::Directory(DirectoryInodeData::Sparse(digest.clone(), *size))
+            }
+            Node::File {
+                digest,
+                size,
+                executable,
+            } => Self::Regular(digest.clone(), *size, *executable),
+            Node::Symlink { target } => Self::Symlink(target.clone().into()),
         }
     }
 
diff --git a/tvix/castore/src/fs/mod.rs b/tvix/castore/src/fs/mod.rs
index 826523131fbd..e700a25d3966 100644
--- a/tvix/castore/src/fs/mod.rs
+++ b/tvix/castore/src/fs/mod.rs
@@ -9,24 +9,19 @@ pub mod fuse;
 #[cfg(feature = "virtiofs")]
 pub mod virtiofs;
 
-#[cfg(test)]
-mod tests;
-
 pub use self::root_nodes::RootNodes;
 use self::{
     file_attr::ROOT_FILE_ATTR,
     inode_tracker::InodeTracker,
     inodes::{DirectoryInodeData, InodeData},
 };
-use crate::proto as castorepb;
 use crate::{
     blobservice::{BlobReader, BlobService},
     directoryservice::DirectoryService,
-    proto::{node::Node, NamedNode},
-    B3Digest,
+    path::PathComponent,
+    B3Digest, Node,
 };
 use bstr::ByteVec;
-use bytes::Bytes;
 use fuse_backend_rs::abi::fuse_abi::{stat64, OpenOptions};
 use fuse_backend_rs::api::filesystem::{
     Context, FileSystem, FsOptions, GetxattrReply, ListxattrReply, ROOT_ID,
@@ -46,7 +41,7 @@ use tokio::{
     io::{AsyncReadExt, AsyncSeekExt},
     sync::mpsc,
 };
-use tracing::{debug, error, instrument, warn, Span};
+use tracing::{debug, error, instrument, warn, Instrument as _, Span};
 
 /// This implements a read-only FUSE filesystem for a tvix-store
 /// with the passed [BlobService], [DirectoryService] and [RootNodes].
@@ -92,7 +87,7 @@ pub struct TvixStoreFs<BS, DS, RN> {
     show_xattr: bool,
 
     /// This maps a given basename in the root to the inode we allocated for the node.
-    root_nodes: RwLock<HashMap<Bytes, u64>>,
+    root_nodes: RwLock<HashMap<PathComponent, u64>>,
 
     /// This keeps track of inodes and data alongside them.
     inode_tracker: RwLock<InodeTracker>,
@@ -108,7 +103,7 @@ pub struct TvixStoreFs<BS, DS, RN> {
             u64,
             (
                 Span,
-                Arc<Mutex<mpsc::Receiver<(usize, Result<Node, crate::Error>)>>>,
+                Arc<Mutex<mpsc::Receiver<(usize, Result<(PathComponent, Node), crate::Error>)>>>,
             ),
         >,
     >,
@@ -159,7 +154,7 @@ where
 
     /// Retrieves the inode for a given root node basename, if present.
     /// This obtains a read lock on self.root_nodes.
-    fn get_inode_for_root_name(&self, name: &[u8]) -> Option<u64> {
+    fn get_inode_for_root_name(&self, name: &PathComponent) -> Option<u64> {
         self.root_nodes.read().get(name).cloned()
     }
 
@@ -170,8 +165,12 @@ where
     /// It is ok if it's a [DirectoryInodeData::Sparse] - in that case, a lookup
     /// in self.directory_service is performed, and self.inode_tracker is updated with the
     /// [DirectoryInodeData::Populated].
+    #[allow(clippy::type_complexity)]
     #[instrument(skip(self), err)]
-    fn get_directory_children(&self, ino: u64) -> io::Result<(B3Digest, Vec<(u64, Node)>)> {
+    fn get_directory_children(
+        &self,
+        ino: u64,
+    ) -> io::Result<(B3Digest, Vec<(u64, PathComponent, Node)>)> {
         let data = self.inode_tracker.read().get(ino).unwrap();
         match *data {
             // if it's populated already, return children.
@@ -201,13 +200,13 @@ where
                 let children = {
                     let mut inode_tracker = self.inode_tracker.write();
 
-                    let children: Vec<(u64, castorepb::node::Node)> = directory
-                        .nodes()
-                        .map(|child_node| {
-                            let (inode_data, _) = InodeData::from_node(child_node.clone());
+                    let children: Vec<(u64, PathComponent, Node)> = directory
+                        .into_nodes()
+                        .map(|(child_name, child_node)| {
+                            let inode_data = InodeData::from_node(&child_node);
 
                             let child_ino = inode_tracker.put(inode_data);
-                            (child_ino, child_node)
+                            (child_ino, child_name, child_node)
                         })
                         .collect();
 
@@ -241,12 +240,12 @@ where
     /// In the case the name can't be found, a libc::ENOENT is returned.
     fn name_in_root_to_ino_and_data(
         &self,
-        name: &std::ffi::CStr,
+        name: &PathComponent,
     ) -> io::Result<(u64, Arc<InodeData>)> {
         // Look up the inode for that root node.
         // If there's one, [self.inode_tracker] MUST also contain the data,
         // which we can then return.
-        if let Some(inode) = self.get_inode_for_root_name(name.to_bytes()) {
+        if let Some(inode) = self.get_inode_for_root_name(name) {
             return Ok((
                 inode,
                 self.inode_tracker
@@ -260,7 +259,8 @@ where
         // We don't have it yet, look it up in [self.root_nodes].
         match self.tokio_handle.block_on({
             let root_nodes_provider = self.root_nodes_provider.clone();
-            async move { root_nodes_provider.get_by_basename(name.to_bytes()).await }
+            let name = name.clone();
+            async move { root_nodes_provider.get_by_basename(&name).await }
         }) {
             // if there was an error looking up the root node, propagate up an IO error.
             Err(_e) => Err(io::Error::from_raw_os_error(libc::EIO)),
@@ -268,15 +268,9 @@ where
             Ok(None) => Err(io::Error::from_raw_os_error(libc::ENOENT)),
             // The root node does exist
             Ok(Some(root_node)) => {
-                // The name must match what's passed in the lookup, otherwise this is also a ENOENT.
-                if root_node.get_name() != name.to_bytes() {
-                    debug!(root_node.name=?root_node.get_name(), found_node.name=%name.to_string_lossy(), "node name mismatch");
-                    return Err(io::Error::from_raw_os_error(libc::ENOENT));
-                }
-
                 // Let's check if someone else beat us to updating the inode tracker and
                 // root_nodes map. This avoids locking inode_tracker for writing.
-                if let Some(ino) = self.root_nodes.read().get(name.to_bytes()) {
+                if let Some(ino) = self.root_nodes.read().get(name) {
                     return Ok((
                         *ino,
                         self.inode_tracker.read().get(*ino).expect("must exist"),
@@ -290,9 +284,9 @@ where
 
                 // insert the (sparse) inode data and register in
                 // self.root_nodes.
-                let (inode_data, name) = InodeData::from_node(root_node);
+                let inode_data = InodeData::from_node(&root_node);
                 let ino = inode_tracker.put(inode_data.clone());
-                root_nodes.insert(name, ino);
+                root_nodes.insert(name.to_owned(), ino);
 
                 Ok((ino, Arc::new(inode_data)))
             }
@@ -348,13 +342,17 @@ where
     ) -> io::Result<fuse_backend_rs::api::filesystem::Entry> {
         debug!("lookup");
 
+        // convert the CStr to a PathComponent
+        // If it can't be converted, we definitely don't have anything here.
+        let name: PathComponent = name.try_into().map_err(|_| std::io::ErrorKind::NotFound)?;
+
         // This goes from a parent inode to a node.
         // - If the parent is [ROOT_ID], we need to check
         //   [self.root_nodes] (fetching from a [RootNode] provider if needed)
         // - Otherwise, lookup the parent in [self.inode_tracker] (which must be
         //   a [InodeData::Directory]), and find the child with that name.
         if parent == ROOT_ID {
-            let (ino, inode_data) = self.name_in_root_to_ino_and_data(name)?;
+            let (ino, inode_data) = self.name_in_root_to_ino_and_data(&name)?;
 
             debug!(inode_data=?&inode_data, ino=ino, "Some");
             return Ok(inode_data.as_fuse_entry(ino));
@@ -367,7 +365,7 @@ where
         // Search for that name in the list of children and return the FileAttrs.
 
         // in the children, find the one with the desired name.
-        if let Some((child_ino, _)) = children.iter().find(|e| e.1.get_name() == name.to_bytes()) {
+        if let Some((child_ino, _, _)) = children.iter().find(|(_, n, _)| n == &name) {
             // lookup the child [InodeData] in [self.inode_tracker].
             // We know the inodes for children have already been allocated.
             let child_inode_data = self.inode_tracker.read().get(*child_ino).unwrap();
@@ -400,16 +398,20 @@ where
 
             // This task will run in the background immediately and will exit
             // after the stream ends or if we no longer want any more entries.
-            self.tokio_handle.spawn(async move {
-                let mut stream = root_nodes_provider.list().enumerate();
-                while let Some(node) = stream.next().await {
-                    if tx.send(node).await.is_err() {
-                        // If we get a send error, it means the sync code
-                        // doesn't want any more entries.
-                        break;
+            self.tokio_handle.spawn(
+                async move {
+                    let mut stream = root_nodes_provider.list().enumerate();
+                    while let Some(e) = stream.next().await {
+                        if tx.send(e).await.is_err() {
+                            // If we get a send error, it means the sync code
+                            // doesn't want any more entries.
+                            break;
+                        }
                     }
                 }
-            });
+                // instrument the task with the current span, this is not done by default
+                .in_current_span(),
+            );
 
             // Put the rx part into [self.dir_handles].
             // TODO: this will overflow after 2**64 operations,
@@ -462,12 +464,12 @@ where
                 .map_err(|_| crate::Error::StorageError("mutex poisoned".into()))?;
 
             while let Some((i, n)) = rx.blocking_recv() {
-                let root_node = n.map_err(|e| {
+                let (name, node) = n.map_err(|e| {
                     warn!("failed to retrieve root node: {}", e);
                     io::Error::from_raw_os_error(libc::EIO)
                 })?;
 
-                let (inode_data, name) = InodeData::from_node(root_node);
+                let inode_data = InodeData::from_node(&node);
 
                 // obtain the inode, or allocate a new one.
                 let ino = self.get_inode_for_root_name(&name).unwrap_or_else(|| {
@@ -482,7 +484,7 @@ where
                     ino,
                     offset: offset + (i as u64) + 1,
                     type_: inode_data.as_fuse_type(),
-                    name: &name,
+                    name: name.as_ref(),
                 })?;
                 // If the buffer is full, add_entry will return `Ok(0)`.
                 if written == 0 {
@@ -496,15 +498,17 @@ where
         let (parent_digest, children) = self.get_directory_children(inode)?;
         Span::current().record("directory.digest", parent_digest.to_string());
 
-        for (i, (ino, child_node)) in children.into_iter().skip(offset as usize).enumerate() {
-            let (inode_data, name) = InodeData::from_node(child_node);
+        for (i, (ino, child_name, child_node)) in
+            children.into_iter().skip(offset as usize).enumerate()
+        {
+            let inode_data = InodeData::from_node(&child_node);
 
             // the second parameter will become the "offset" parameter on the next call.
             let written = add_entry(fuse_backend_rs::api::filesystem::DirEntry {
                 ino,
                 offset: offset + (i as u64) + 1,
                 type_: inode_data.as_fuse_type(),
-                name: &name,
+                name: child_name.as_ref(),
             })?;
             // If the buffer is full, add_entry will return `Ok(0)`.
             if written == 0 {
@@ -549,12 +553,12 @@ where
                 .map_err(|_| crate::Error::StorageError("mutex poisoned".into()))?;
 
             while let Some((i, n)) = rx.blocking_recv() {
-                let root_node = n.map_err(|e| {
+                let (name, node) = n.map_err(|e| {
                     warn!("failed to retrieve root node: {}", e);
                     io::Error::from_raw_os_error(libc::EPERM)
                 })?;
 
-                let (inode_data, name) = InodeData::from_node(root_node);
+                let inode_data = InodeData::from_node(&node);
 
                 // obtain the inode, or allocate a new one.
                 let ino = self.get_inode_for_root_name(&name).unwrap_or_else(|| {
@@ -570,7 +574,7 @@ where
                         ino,
                         offset: offset + (i as u64) + 1,
                         type_: inode_data.as_fuse_type(),
-                        name: &name,
+                        name: name.as_ref(),
                     },
                     inode_data.as_fuse_entry(ino),
                 )?;
@@ -586,8 +590,8 @@ where
         let (parent_digest, children) = self.get_directory_children(inode)?;
         Span::current().record("directory.digest", parent_digest.to_string());
 
-        for (i, (ino, child_node)) in children.into_iter().skip(offset as usize).enumerate() {
-            let (inode_data, name) = InodeData::from_node(child_node);
+        for (i, (ino, name, child_node)) in children.into_iter().skip(offset as usize).enumerate() {
+            let inode_data = InodeData::from_node(&child_node);
 
             // the second parameter will become the "offset" parameter on the next call.
             let written = add_entry(
@@ -595,7 +599,7 @@ where
                     ino,
                     offset: offset + (i as u64) + 1,
                     type_: inode_data.as_fuse_type(),
-                    name: &name,
+                    name: name.as_ref(),
                 },
                 inode_data.as_fuse_entry(ino),
             )?;
diff --git a/tvix/castore/src/fs/root_nodes.rs b/tvix/castore/src/fs/root_nodes.rs
index 6609e049a1fc..5ed1a4d8d6c0 100644
--- a/tvix/castore/src/fs/root_nodes.rs
+++ b/tvix/castore/src/fs/root_nodes.rs
@@ -1,7 +1,6 @@
 use std::collections::BTreeMap;
 
-use crate::{proto::node::Node, Error};
-use bytes::Bytes;
+use crate::{path::PathComponent, Error, Node};
 use futures::stream::BoxStream;
 use tonic::async_trait;
 
@@ -11,11 +10,12 @@ use tonic::async_trait;
 pub trait RootNodes: Send + Sync {
     /// Looks up a root CA node based on the basename of the node in the root
     /// directory of the filesystem.
-    async fn get_by_basename(&self, name: &[u8]) -> Result<Option<Node>, Error>;
+    async fn get_by_basename(&self, name: &PathComponent) -> Result<Option<Node>, Error>;
 
-    /// Lists all root CA nodes in the filesystem. An error can be returned
-    /// in case listing is not allowed
-    fn list(&self) -> BoxStream<Result<Node, Error>>;
+    /// Lists all root CA nodes in the filesystem, as a tuple of (base)name
+    /// and Node.
+    /// An error can be returned in case listing is not allowed.
+    fn list(&self) -> BoxStream<Result<(PathComponent, Node), Error>>;
 }
 
 #[async_trait]
@@ -23,15 +23,17 @@ pub trait RootNodes: Send + Sync {
 /// the key is the node name.
 impl<T> RootNodes for T
 where
-    T: AsRef<BTreeMap<Bytes, Node>> + Send + Sync,
+    T: AsRef<BTreeMap<PathComponent, Node>> + Send + Sync,
 {
-    async fn get_by_basename(&self, name: &[u8]) -> Result<Option<Node>, Error> {
+    async fn get_by_basename(&self, name: &PathComponent) -> Result<Option<Node>, Error> {
         Ok(self.as_ref().get(name).cloned())
     }
 
-    fn list(&self) -> BoxStream<Result<Node, Error>> {
+    fn list(&self) -> BoxStream<Result<(PathComponent, Node), Error>> {
         Box::pin(tokio_stream::iter(
-            self.as_ref().iter().map(|(_, v)| Ok(v.clone())),
+            self.as_ref()
+                .iter()
+                .map(|(name, node)| Ok((name.to_owned(), node.to_owned()))),
         ))
     }
 }
diff --git a/tvix/castore/src/import/archive.rs b/tvix/castore/src/import/archive.rs
index 0ebb4a236117..167f799efa0f 100644
--- a/tvix/castore/src/import/archive.rs
+++ b/tvix/castore/src/import/archive.rs
@@ -1,38 +1,23 @@
 //! Imports from an archive (tarballs)
 
 use std::collections::HashMap;
-use std::io::{Cursor, Write};
-use std::sync::Arc;
 
 use petgraph::graph::{DiGraph, NodeIndex};
 use petgraph::visit::{DfsPostOrder, EdgeRef};
 use petgraph::Direction;
 use tokio::io::AsyncRead;
-use tokio::sync::Semaphore;
-use tokio::task::JoinSet;
 use tokio_stream::StreamExt;
 use tokio_tar::Archive;
-use tokio_util::io::InspectReader;
 use tracing::{instrument, warn, Level};
 
 use crate::blobservice::BlobService;
 use crate::directoryservice::DirectoryService;
 use crate::import::{ingest_entries, IngestionEntry, IngestionError};
-use crate::proto::node::Node;
-use crate::B3Digest;
+use crate::Node;
 
-type TarPathBuf = std::path::PathBuf;
-
-/// Files smaller than this threshold, in bytes, are uploaded to the [BlobService] in the
-/// background.
-///
-/// This is a u32 since we acquire a weighted semaphore using the size of the blob.
-/// [Semaphore::acquire_many_owned] takes a u32, so we need to ensure the size of
-/// the blob can be represented using a u32 and will not cause an overflow.
-const CONCURRENT_BLOB_UPLOAD_THRESHOLD: u32 = 1024 * 1024;
+use super::blobs::{self, ConcurrentBlobUploader};
 
-/// The maximum amount of bytes allowed to be buffered in memory to perform async blob uploads.
-const MAX_TARBALL_BUFFER_SIZE: usize = 128 * 1024 * 1024;
+type TarPathBuf = std::path::PathBuf;
 
 #[derive(Debug, thiserror::Error)]
 pub enum Error {
@@ -57,13 +42,6 @@ pub enum Error {
     #[error("unable to read link name field for {0}: {1}")]
     LinkName(TarPathBuf, std::io::Error),
 
-    #[error("unable to read blob contents for {0}: {1}")]
-    BlobRead(TarPathBuf, std::io::Error),
-
-    // FUTUREWORK: proper error for blob finalize
-    #[error("unable to finalize blob {0}: {1}")]
-    BlobFinalize(TarPathBuf, std::io::Error),
-
     #[error("unsupported tar entry {0} type: {1:?}")]
     EntryType(TarPathBuf, tokio_tar::EntryType),
 
@@ -72,6 +50,9 @@ pub enum Error {
 
     #[error("unexpected number of top level directory entries")]
     UnexpectedNumberOfTopLevelEntries,
+
+    #[error(transparent)]
+    BlobUploadError(#[from] blobs::Error),
 }
 
 /// Ingests elements from the given tar [`Archive`] into a the passed [`BlobService`] and
@@ -94,8 +75,7 @@ where
     // In the first phase, collect up all the regular files and symlinks.
     let mut nodes = IngestionEntryGraph::new();
 
-    let semaphore = Arc::new(Semaphore::new(MAX_TARBALL_BUFFER_SIZE));
-    let mut async_blob_uploads: JoinSet<Result<(), Error>> = JoinSet::new();
+    let mut blob_uploader = ConcurrentBlobUploader::new(blob_service);
 
     let mut entries_iter = archive.entries().map_err(Error::Entries)?;
     while let Some(mut entry) = entries_iter.try_next().await.map_err(Error::NextEntry)? {
@@ -110,77 +90,14 @@ where
             tokio_tar::EntryType::Regular
             | tokio_tar::EntryType::GNUSparse
             | tokio_tar::EntryType::Continuous => {
-                let header_size = header
+                let size = header
                     .size()
                     .map_err(|e| Error::Size(tar_path.clone(), e))?;
 
-                // If the blob is small enough, read it off the wire, compute the digest,
-                // and upload it to the [BlobService] in the background.
-                let (size, digest) = if header_size <= CONCURRENT_BLOB_UPLOAD_THRESHOLD as u64 {
-                    let mut buffer = Vec::with_capacity(header_size as usize);
-                    let mut hasher = blake3::Hasher::new();
-                    let mut reader = InspectReader::new(&mut entry, |bytes| {
-                        hasher.write_all(bytes).unwrap();
-                    });
-
-                    // Ensure that we don't buffer into memory until we've acquired a permit.
-                    // This prevents consuming too much memory when performing concurrent
-                    // blob uploads.
-                    let permit = semaphore
-                        .clone()
-                        // This cast is safe because ensure the header_size is less than
-                        // CONCURRENT_BLOB_UPLOAD_THRESHOLD which is a u32.
-                        .acquire_many_owned(header_size as u32)
-                        .await
-                        .unwrap();
-                    let size = tokio::io::copy(&mut reader, &mut buffer)
-                        .await
-                        .map_err(|e| Error::Size(tar_path.clone(), e))?;
-
-                    let digest: B3Digest = hasher.finalize().as_bytes().into();
-
-                    {
-                        let blob_service = blob_service.clone();
-                        let digest = digest.clone();
-                        async_blob_uploads.spawn({
-                            let tar_path = tar_path.clone();
-                            async move {
-                                let mut writer = blob_service.open_write().await;
-
-                                tokio::io::copy(&mut Cursor::new(buffer), &mut writer)
-                                    .await
-                                    .map_err(|e| Error::BlobRead(tar_path.clone(), e))?;
-
-                                let blob_digest = writer
-                                    .close()
-                                    .await
-                                    .map_err(|e| Error::BlobFinalize(tar_path, e))?;
-
-                                assert_eq!(digest, blob_digest, "Tvix bug: blob digest mismatch");
-
-                                // Make sure we hold the permit until we finish writing the blob
-                                // to the [BlobService].
-                                drop(permit);
-                                Ok(())
-                            }
-                        });
-                    }
-
-                    (size, digest)
-                } else {
-                    let mut writer = blob_service.open_write().await;
-
-                    let size = tokio::io::copy(&mut entry, &mut writer)
-                        .await
-                        .map_err(|e| Error::BlobRead(tar_path.clone(), e))?;
-
-                    let digest = writer
-                        .close()
-                        .await
-                        .map_err(|e| Error::BlobFinalize(tar_path.clone(), e))?;
-
-                    (size, digest)
-                };
+                let digest = blob_uploader
+                    .upload(&path, size, &mut entry)
+                    .await
+                    .map_err(Error::BlobUploadError)?;
 
                 let executable = entry
                     .header()
@@ -219,9 +136,7 @@ where
         nodes.add(entry)?;
     }
 
-    while let Some(result) = async_blob_uploads.join_next().await {
-        result.expect("task panicked")?;
-    }
+    blob_uploader.join().await.map_err(Error::BlobUploadError)?;
 
     let root_node = ingest_entries(
         directory_service,
diff --git a/tvix/castore/src/import/blobs.rs b/tvix/castore/src/import/blobs.rs
new file mode 100644
index 000000000000..f71ee1e63768
--- /dev/null
+++ b/tvix/castore/src/import/blobs.rs
@@ -0,0 +1,190 @@
+use std::{
+    io::{Cursor, Write},
+    sync::Arc,
+};
+
+use tokio::{
+    io::AsyncRead,
+    sync::Semaphore,
+    task::{JoinError, JoinSet},
+};
+use tokio_util::io::InspectReader;
+
+use crate::{blobservice::BlobService, B3Digest, Path, PathBuf};
+
+/// Files smaller than this threshold, in bytes, are uploaded to the [BlobService] in the
+/// background.
+///
+/// This is a u32 since we acquire a weighted semaphore using the size of the blob.
+/// [Semaphore::acquire_many_owned] takes a u32, so we need to ensure the size of
+/// the blob can be represented using a u32 and will not cause an overflow.
+const CONCURRENT_BLOB_UPLOAD_THRESHOLD: u32 = 1024 * 1024;
+
+/// The maximum amount of bytes allowed to be buffered in memory to perform async blob uploads.
+const MAX_BUFFER_SIZE: usize = 128 * 1024 * 1024;
+
+#[derive(Debug, thiserror::Error)]
+pub enum Error {
+    #[error("unable to read blob contents for {0}: {1}")]
+    BlobRead(PathBuf, std::io::Error),
+
+    #[error("unable to check whether blob at {0} already exists: {1}")]
+    BlobCheck(PathBuf, std::io::Error),
+
+    // FUTUREWORK: proper error for blob finalize
+    #[error("unable to finalize blob {0}: {1}")]
+    BlobFinalize(PathBuf, std::io::Error),
+
+    #[error("unexpected size for {path} wanted: {wanted} got: {got}")]
+    UnexpectedSize {
+        path: PathBuf,
+        wanted: u64,
+        got: u64,
+    },
+
+    #[error("blob upload join error: {0}")]
+    JoinError(#[from] JoinError),
+}
+
+/// The concurrent blob uploader provides a mechanism for concurrently uploading small blobs.
+/// This is useful when ingesting from sources like tarballs and archives which each blob entry
+/// must be read sequentially. Ingesting many small blobs sequentially becomes slow due to
+/// round trip time with the blob service. The concurrent blob uploader will buffer small
+/// blobs in memory and upload them to the blob service in the background.
+///
+/// Once all blobs have been uploaded, make sure to call [ConcurrentBlobUploader::join] to wait
+/// for all background jobs to complete and check for any errors.
+pub struct ConcurrentBlobUploader<BS> {
+    blob_service: BS,
+    upload_tasks: JoinSet<Result<(), Error>>,
+    upload_semaphore: Arc<Semaphore>,
+}
+
+impl<BS> ConcurrentBlobUploader<BS>
+where
+    BS: BlobService + Clone + 'static,
+{
+    /// Creates a new concurrent blob uploader which uploads blobs to the provided
+    /// blob service.
+    pub fn new(blob_service: BS) -> Self {
+        Self {
+            blob_service,
+            upload_tasks: JoinSet::new(),
+            upload_semaphore: Arc::new(Semaphore::new(MAX_BUFFER_SIZE)),
+        }
+    }
+
+    /// Uploads a blob to the blob service. If the blob is small enough it will be read to a buffer
+    /// and uploaded in the background.
+    /// This will read the entirety of the provided reader unless an error occurs, even if blobs
+    /// are uploaded in the background..
+    pub async fn upload<R>(
+        &mut self,
+        path: &Path,
+        expected_size: u64,
+        mut r: R,
+    ) -> Result<B3Digest, Error>
+    where
+        R: AsyncRead + Unpin,
+    {
+        if expected_size < CONCURRENT_BLOB_UPLOAD_THRESHOLD as u64 {
+            let mut buffer = Vec::with_capacity(expected_size as usize);
+            let mut hasher = blake3::Hasher::new();
+            let mut reader = InspectReader::new(&mut r, |bytes| {
+                hasher.write_all(bytes).unwrap();
+            });
+
+            let permit = self
+                .upload_semaphore
+                .clone()
+                // This cast is safe because ensure the header_size is less than
+                // CONCURRENT_BLOB_UPLOAD_THRESHOLD which is a u32.
+                .acquire_many_owned(expected_size as u32)
+                .await
+                .unwrap();
+            let size = tokio::io::copy(&mut reader, &mut buffer)
+                .await
+                .map_err(|e| Error::BlobRead(path.into(), e))?;
+            let digest: B3Digest = hasher.finalize().as_bytes().into();
+
+            if size != expected_size {
+                return Err(Error::UnexpectedSize {
+                    path: path.into(),
+                    wanted: expected_size,
+                    got: size,
+                });
+            }
+
+            self.upload_tasks.spawn({
+                let blob_service = self.blob_service.clone();
+                let expected_digest = digest.clone();
+                let path = path.to_owned();
+                let r = Cursor::new(buffer);
+                async move {
+                    // We know the blob digest already, check it exists before sending it.
+                    if blob_service
+                        .has(&expected_digest)
+                        .await
+                        .map_err(|e| Error::BlobCheck(path.clone(), e))?
+                    {
+                        drop(permit);
+                        return Ok(());
+                    }
+
+                    let digest = upload_blob(&blob_service, &path, expected_size, r).await?;
+
+                    assert_eq!(digest, expected_digest, "Tvix bug: blob digest mismatch");
+
+                    // Make sure we hold the permit until we finish writing the blob
+                    // to the [BlobService].
+                    drop(permit);
+                    Ok(())
+                }
+            });
+
+            return Ok(digest);
+        }
+
+        upload_blob(&self.blob_service, path, expected_size, r).await
+    }
+
+    /// Waits for all background upload jobs to complete, returning any upload errors.
+    pub async fn join(mut self) -> Result<(), Error> {
+        while let Some(result) = self.upload_tasks.join_next().await {
+            result??;
+        }
+        Ok(())
+    }
+}
+
+async fn upload_blob<BS, R>(
+    blob_service: &BS,
+    path: &Path,
+    expected_size: u64,
+    mut r: R,
+) -> Result<B3Digest, Error>
+where
+    BS: BlobService,
+    R: AsyncRead + Unpin,
+{
+    let mut writer = blob_service.open_write().await;
+
+    let size = tokio::io::copy(&mut r, &mut writer)
+        .await
+        .map_err(|e| Error::BlobRead(path.into(), e))?;
+
+    let digest = writer
+        .close()
+        .await
+        .map_err(|e| Error::BlobFinalize(path.into(), e))?;
+
+    if size != expected_size {
+        return Err(Error::UnexpectedSize {
+            path: path.into(),
+            wanted: expected_size,
+            got: size,
+        });
+    }
+
+    Ok(digest)
+}
diff --git a/tvix/castore/src/import/fs.rs b/tvix/castore/src/import/fs.rs
index 9d3ecfe6ab7a..1332fdfe57b5 100644
--- a/tvix/castore/src/import/fs.rs
+++ b/tvix/castore/src/import/fs.rs
@@ -6,14 +6,17 @@ use std::fs::FileType;
 use std::os::unix::ffi::OsStringExt;
 use std::os::unix::fs::MetadataExt;
 use std::os::unix::fs::PermissionsExt;
+use tokio::io::BufReader;
+use tokio_util::io::InspectReader;
 use tracing::instrument;
+use tracing::Span;
+use tracing_indicatif::span_ext::IndicatifSpanExt;
 use walkdir::DirEntry;
 use walkdir::WalkDir;
 
 use crate::blobservice::BlobService;
 use crate::directoryservice::DirectoryService;
-use crate::proto::node::Node;
-use crate::B3Digest;
+use crate::{B3Digest, Node};
 
 use super::ingest_entries;
 use super::IngestionEntry;
@@ -26,7 +29,7 @@ use super::IngestionError;
 ///
 /// This function will walk the filesystem using `walkdir` and will consume
 /// `O(#number of entries)` space.
-#[instrument(skip(blob_service, directory_service), fields(path), err)]
+#[instrument(skip(blob_service, directory_service), fields(path, indicatif.pb_show=1), err)]
 pub async fn ingest_path<BS, DS, P>(
     blob_service: BS,
     directory_service: DS,
@@ -37,6 +40,10 @@ where
     BS: BlobService + Clone,
     DS: DirectoryService,
 {
+    let span = Span::current();
+    span.pb_set_message(&format!("Ingesting {:?}", path));
+    span.pb_start();
+
     let iter = WalkDir::new(path.as_ref())
         .follow_links(false)
         .follow_root_links(false)
@@ -44,7 +51,18 @@ where
         .into_iter();
 
     let entries = dir_entries_to_ingestion_stream(blob_service, iter, path.as_ref());
-    ingest_entries(directory_service, entries).await
+    ingest_entries(
+        directory_service,
+        entries.inspect({
+            let span = span.clone();
+            move |e| {
+                if e.is_ok() {
+                    span.pb_inc(1)
+                }
+            }
+        }),
+    )
+    .await
 }
 
 /// Converts an iterator of [walkdir::DirEntry]s into a stream of ingestion entries.
@@ -138,7 +156,7 @@ where
 }
 
 /// Uploads the file at the provided [Path] the the [BlobService].
-#[instrument(skip(blob_service), fields(path), err)]
+#[instrument(skip(blob_service), fields(path, indicatif.pb_show=1), err)]
 async fn upload_blob<BS>(
     blob_service: BS,
     path: impl AsRef<std::path::Path>,
@@ -146,16 +164,29 @@ async fn upload_blob<BS>(
 where
     BS: BlobService,
 {
-    let mut file = match tokio::fs::File::open(path.as_ref()).await {
-        Ok(file) => file,
-        Err(e) => return Err(Error::BlobRead(path.as_ref().to_path_buf(), e)),
-    };
+    let span = Span::current();
+    span.pb_set_style(&tvix_tracing::PB_TRANSFER_STYLE);
+    span.pb_set_message(&format!("Uploading blob for {:?}", path.as_ref()));
+    span.pb_start();
 
-    let mut writer = blob_service.open_write().await;
+    let file = tokio::fs::File::open(path.as_ref())
+        .await
+        .map_err(|e| Error::BlobRead(path.as_ref().to_path_buf(), e))?;
 
-    if let Err(e) = tokio::io::copy(&mut file, &mut writer).await {
-        return Err(Error::BlobRead(path.as_ref().to_path_buf(), e));
-    };
+    let metadata = file
+        .metadata()
+        .await
+        .map_err(|e| Error::Stat(path.as_ref().to_path_buf(), e))?;
+
+    span.pb_set_length(metadata.len());
+    let reader = InspectReader::new(file, |d| {
+        span.pb_inc(d.len() as u64);
+    });
+
+    let mut writer = blob_service.open_write().await;
+    tokio::io::copy(&mut BufReader::new(reader), &mut writer)
+        .await
+        .map_err(|e| Error::BlobRead(path.as_ref().to_path_buf(), e))?;
 
     let digest = writer
         .close()
diff --git a/tvix/castore/src/import/mod.rs b/tvix/castore/src/import/mod.rs
index e8b27e469c3d..6e10a64939a4 100644
--- a/tvix/castore/src/import/mod.rs
+++ b/tvix/castore/src/import/mod.rs
@@ -4,17 +4,10 @@
 //! Specific implementations, such as ingesting from the filesystem, live in
 //! child modules.
 
-use crate::directoryservice::DirectoryPutter;
-use crate::directoryservice::DirectoryService;
+use crate::directoryservice::{DirectoryPutter, DirectoryService};
 use crate::path::{Path, PathBuf};
-use crate::proto::node::Node;
-use crate::proto::Directory;
-use crate::proto::DirectoryNode;
-use crate::proto::FileNode;
-use crate::proto::SymlinkNode;
-use crate::B3Digest;
+use crate::{B3Digest, Directory, Node, SymlinkTargetError};
 use futures::{Stream, StreamExt};
-
 use tracing::Level;
 
 use std::collections::HashMap;
@@ -24,6 +17,7 @@ mod error;
 pub use error::IngestionError;
 
 pub mod archive;
+pub mod blobs;
 pub mod fs;
 
 /// Ingests [IngestionEntry] from the given stream into a the passed [DirectoryService].
@@ -65,14 +59,6 @@ where
             // we break the loop manually.
             .expect("Tvix bug: unexpected end of stream")?;
 
-        let name = entry
-            .path()
-            .file_name()
-            // If this is the root node, it will have an empty name.
-            .unwrap_or_default()
-            .to_owned()
-            .into();
-
         let node = match &mut entry {
             IngestionEntry::Dir { .. } => {
                 // If the entry is a directory, we traversed all its children (and
@@ -98,27 +84,31 @@ where
                         IngestionError::UploadDirectoryError(entry.path().to_owned(), e)
                     })?;
 
-                Node::Directory(DirectoryNode {
-                    name,
-                    digest: directory_digest.into(),
+                Node::Directory {
+                    digest: directory_digest,
                     size: directory_size,
-                })
+                }
             }
-            IngestionEntry::Symlink { ref target, .. } => Node::Symlink(SymlinkNode {
-                name,
-                target: target.to_owned().into(),
-            }),
+            IngestionEntry::Symlink { ref target, .. } => Node::Symlink {
+                target: bytes::Bytes::copy_from_slice(target).try_into().map_err(
+                    |e: SymlinkTargetError| {
+                        IngestionError::UploadDirectoryError(
+                            entry.path().to_owned(),
+                            crate::Error::StorageError(format!("invalid symlink target: {}", e)),
+                        )
+                    },
+                )?,
+            },
             IngestionEntry::Regular {
                 size,
                 executable,
                 digest,
                 ..
-            } => Node::File(FileNode {
-                name,
-                digest: digest.to_owned().into(),
+            } => Node::File {
+                digest: digest.clone(),
                 size: *size,
                 executable: *executable,
-            }),
+            },
         };
 
         let parent = entry
@@ -129,8 +119,24 @@ where
         if parent == crate::Path::ROOT {
             break node;
         } else {
+            let name = entry
+                .path()
+                .file_name()
+                // If this is the root node, it will have an empty name.
+                .unwrap_or_else(|| "".try_into().unwrap())
+                .to_owned();
+
             // record node in parent directory, creating a new [Directory] if not there yet.
-            directories.entry(parent.to_owned()).or_default().add(node);
+            directories
+                .entry(parent.to_owned())
+                .or_default()
+                .add(name, node)
+                .map_err(|e| {
+                    IngestionError::UploadDirectoryError(
+                        entry.path().to_owned(),
+                        crate::Error::StorageError(e.to_string()),
+                    )
+                })?;
         }
     };
 
@@ -155,15 +161,8 @@ where
 
         #[cfg(debug_assertions)]
         {
-            if let Node::Directory(directory_node) = &root_node {
-                debug_assert_eq!(
-                    root_directory_digest,
-                    directory_node
-                        .digest
-                        .to_vec()
-                        .try_into()
-                        .expect("invalid digest len")
-                )
+            if let Node::Directory { digest, .. } = &root_node {
+                debug_assert_eq!(&root_directory_digest, digest);
             } else {
                 unreachable!("Tvix bug: directory putter initialized but no root directory node");
             }
@@ -209,9 +208,8 @@ mod test {
     use rstest::rstest;
 
     use crate::fixtures::{DIRECTORY_COMPLICATED, DIRECTORY_WITH_KEEP, EMPTY_BLOB_DIGEST};
-    use crate::proto::node::Node;
-    use crate::proto::{Directory, DirectoryNode, FileNode, SymlinkNode};
     use crate::{directoryservice::MemoryDirectoryService, fixtures::DUMMY_DIGEST};
+    use crate::{Directory, Node};
 
     use super::ingest_entries;
     use super::IngestionEntry;
@@ -223,18 +221,18 @@ mod test {
         executable: true,
         digest: DUMMY_DIGEST.clone(),
     }],
-        Node::File(FileNode { name: "foo".into(), digest: DUMMY_DIGEST.clone().into(), size: 42, executable: true }
-    ))]
+        Node::File{digest: DUMMY_DIGEST.clone(), size: 42, executable: true}
+    )]
     #[case::single_symlink(vec![IngestionEntry::Symlink {
         path: "foo".parse().unwrap(),
         target: b"blub".into(),
     }],
-        Node::Symlink(SymlinkNode { name: "foo".into(), target: "blub".into()})
+        Node::Symlink{target: "blub".try_into().unwrap()}
     )]
     #[case::single_dir(vec![IngestionEntry::Dir {
         path: "foo".parse().unwrap(),
     }],
-        Node::Directory(DirectoryNode { name: "foo".into(), digest: Directory::default().digest().into(), size: Directory::default().size()})
+        Node::Directory{digest: Directory::default().digest(), size: Directory::default().size()}
     )]
     #[case::dir_with_keep(vec![
         IngestionEntry::Regular {
@@ -247,7 +245,7 @@ mod test {
             path: "foo".parse().unwrap(),
         },
     ],
-        Node::Directory(DirectoryNode { name: "foo".into(), digest: DIRECTORY_WITH_KEEP.digest().into(), size: DIRECTORY_WITH_KEEP.size() })
+        Node::Directory{ digest: DIRECTORY_WITH_KEEP.digest(), size: DIRECTORY_WITH_KEEP.size()}
     )]
     /// This is intentionally a bit unsorted, though it still satisfies all
     /// requirements we have on the order of elements in the stream.
@@ -275,7 +273,7 @@ mod test {
             path: "blub".parse().unwrap(),
         },
     ],
-        Node::Directory(DirectoryNode { name: "blub".into(), digest: DIRECTORY_COMPLICATED.digest().into(), size:DIRECTORY_COMPLICATED.size() })
+    Node::Directory{ digest: DIRECTORY_COMPLICATED.digest(), size: DIRECTORY_COMPLICATED.size() }
     )]
     #[tokio::test]
     async fn test_ingestion(#[case] entries: Vec<IngestionEntry>, #[case] exp_root_node: Node) {
diff --git a/tvix/castore/src/lib.rs b/tvix/castore/src/lib.rs
index bdc533a8c5e6..8ac6ca3dd66a 100644
--- a/tvix/castore/src/lib.rs
+++ b/tvix/castore/src/lib.rs
@@ -3,21 +3,25 @@ mod errors;
 mod hashing_reader;
 
 pub mod blobservice;
+pub mod composition;
 pub mod directoryservice;
 pub mod fixtures;
 
 #[cfg(feature = "fs")]
 pub mod fs;
 
+mod nodes;
+pub use nodes::*;
+
 mod path;
-pub use path::{Path, PathBuf};
+pub use path::{Path, PathBuf, PathComponent, PathComponentError};
 
 pub mod import;
 pub mod proto;
 pub mod tonic;
 
 pub use digests::{B3Digest, B3_LEN};
-pub use errors::Error;
+pub use errors::{DirectoryError, Error, ValidateNodeError};
 pub use hashing_reader::{B3HashingReader, HashingReader};
 
 #[cfg(test)]
diff --git a/tvix/castore/src/nodes/directory.rs b/tvix/castore/src/nodes/directory.rs
new file mode 100644
index 000000000000..f80e055dde80
--- /dev/null
+++ b/tvix/castore/src/nodes/directory.rs
@@ -0,0 +1,287 @@
+use std::collections::btree_map::{self, BTreeMap};
+
+use crate::{errors::DirectoryError, path::PathComponent, proto, B3Digest, Node};
+
+/// A Directory contains nodes, which can be Directory, File or Symlink nodes.
+/// It attaches names to these nodes, which is the basename in that directory.
+/// These names:
+///  - MUST not contain slashes or null bytes
+///  - MUST not be '.' or '..'
+///  - MUST be unique across all three lists
+#[derive(Default, Debug, Clone, PartialEq, Eq)]
+pub struct Directory {
+    nodes: BTreeMap<PathComponent, Node>,
+}
+
+impl Directory {
+    /// Constructs a new, empty Directory.
+    pub fn new() -> Self {
+        Directory {
+            nodes: BTreeMap::new(),
+        }
+    }
+
+    /// Construct a [Directory] from tuples of name and [Node].
+    ///
+    /// Inserting multiple elements with the same name will yield an error, as
+    /// well as exceeding the maximum size.
+    pub fn try_from_iter<T: IntoIterator<Item = (PathComponent, Node)>>(
+        iter: T,
+    ) -> Result<Directory, DirectoryError> {
+        let mut nodes = BTreeMap::new();
+
+        iter.into_iter().try_fold(0u64, |size, (name, node)| {
+            check_insert_node(size, &mut nodes, name, node)
+        })?;
+
+        Ok(Self { nodes })
+    }
+
+    /// The size of a directory is the number of all regular and symlink elements,
+    /// the number of directory elements, and their size fields.
+    pub fn size(&self) -> u64 {
+        // It's impossible to create a Directory where the size overflows, because we
+        // check before every add() that the size won't overflow.
+        (self.nodes.len() as u64)
+            + self
+                .nodes()
+                .map(|(_name, n)| match n {
+                    Node::Directory { size, .. } => 1 + size,
+                    Node::File { .. } | Node::Symlink { .. } => 1,
+                })
+                .sum::<u64>()
+    }
+
+    /// Calculates the digest of a Directory, which is the blake3 hash of a
+    /// Directory protobuf message, serialized in protobuf canonical form.
+    pub fn digest(&self) -> B3Digest {
+        proto::Directory::from(self.clone()).digest()
+    }
+
+    /// Allows iterating over all nodes (directories, files and symlinks)
+    /// For each, it returns a tuple of its name and node.
+    /// The elements are sorted by their names.
+    pub fn nodes(&self) -> impl Iterator<Item = (&PathComponent, &Node)> + Send + Sync + '_ {
+        self.nodes.iter()
+    }
+
+    /// Dissolves a Directory into its individual names and nodes.
+    /// The elements are sorted by their names.
+    pub fn into_nodes(self) -> impl Iterator<Item = (PathComponent, Node)> + Send + Sync {
+        self.nodes.into_iter()
+    }
+
+    /// Adds the specified [Node] to the [Directory] with a given name.
+    ///
+    /// Inserting a node that already exists with the same name in the directory
+    /// will yield an error, as well as exceeding the maximum size.
+    ///
+    /// In case you want to construct a [Directory] from multiple elements, use
+    /// [from_iter] instead.
+    pub fn add(&mut self, name: PathComponent, node: Node) -> Result<(), DirectoryError> {
+        check_insert_node(self.size(), &mut self.nodes, name, node)?;
+        Ok(())
+    }
+}
+
+fn checked_sum(iter: impl IntoIterator<Item = u64>) -> Option<u64> {
+    iter.into_iter().try_fold(0u64, |acc, i| acc.checked_add(i))
+}
+
+/// Helper function dealing with inserting nodes into the nodes [BTreeMap],
+/// after ensuring the new size doesn't overlow and the key doesn't exist already.
+///
+/// Returns the new total size, or an error.
+fn check_insert_node(
+    current_size: u64,
+    nodes: &mut BTreeMap<PathComponent, Node>,
+    name: PathComponent,
+    node: Node,
+) -> Result<u64, DirectoryError> {
+    // Check that the even after adding this new directory entry, the size calculation will not
+    // overflow
+    let new_size = checked_sum([
+        current_size,
+        1,
+        match node {
+            Node::Directory { size, .. } => size,
+            _ => 0,
+        },
+    ])
+    .ok_or(DirectoryError::SizeOverflow)?;
+
+    match nodes.entry(name) {
+        btree_map::Entry::Vacant(e) => {
+            e.insert(node);
+        }
+        btree_map::Entry::Occupied(occupied) => {
+            return Err(DirectoryError::DuplicateName(occupied.key().to_owned()))
+        }
+    }
+
+    Ok(new_size)
+}
+
+#[cfg(test)]
+mod test {
+    use super::{Directory, Node};
+    use crate::fixtures::DUMMY_DIGEST;
+    use crate::{DirectoryError, PathComponent};
+
+    #[test]
+    fn from_iter_single() {
+        Directory::try_from_iter([(
+            PathComponent::try_from("b").unwrap(),
+            Node::Directory {
+                digest: DUMMY_DIGEST.clone(),
+                size: 1,
+            },
+        )])
+        .unwrap();
+    }
+
+    #[test]
+    fn from_iter_multiple() {
+        let d = Directory::try_from_iter([
+            (
+                "b".try_into().unwrap(),
+                Node::Directory {
+                    digest: DUMMY_DIGEST.clone(),
+                    size: 1,
+                },
+            ),
+            (
+                "a".try_into().unwrap(),
+                Node::Directory {
+                    digest: DUMMY_DIGEST.clone(),
+                    size: 1,
+                },
+            ),
+            (
+                "z".try_into().unwrap(),
+                Node::Directory {
+                    digest: DUMMY_DIGEST.clone(),
+                    size: 1,
+                },
+            ),
+            (
+                "f".try_into().unwrap(),
+                Node::File {
+                    digest: DUMMY_DIGEST.clone(),
+                    size: 1,
+                    executable: true,
+                },
+            ),
+            (
+                "c".try_into().unwrap(),
+                Node::File {
+                    digest: DUMMY_DIGEST.clone(),
+                    size: 1,
+                    executable: true,
+                },
+            ),
+            (
+                "g".try_into().unwrap(),
+                Node::File {
+                    digest: DUMMY_DIGEST.clone(),
+                    size: 1,
+                    executable: true,
+                },
+            ),
+            (
+                "t".try_into().unwrap(),
+                Node::Symlink {
+                    target: "a".try_into().unwrap(),
+                },
+            ),
+            (
+                "o".try_into().unwrap(),
+                Node::Symlink {
+                    target: "a".try_into().unwrap(),
+                },
+            ),
+            (
+                "e".try_into().unwrap(),
+                Node::Symlink {
+                    target: "a".try_into().unwrap(),
+                },
+            ),
+        ])
+        .unwrap();
+
+        // Convert to proto struct and back to ensure we are not generating any invalid structures
+        crate::Directory::try_from(crate::proto::Directory::from(d))
+            .expect("directory should be valid");
+    }
+
+    #[test]
+    fn add_nodes_to_directory() {
+        let mut d = Directory::new();
+
+        d.add(
+            "b".try_into().unwrap(),
+            Node::Directory {
+                digest: DUMMY_DIGEST.clone(),
+                size: 1,
+            },
+        )
+        .unwrap();
+        d.add(
+            "a".try_into().unwrap(),
+            Node::Directory {
+                digest: DUMMY_DIGEST.clone(),
+                size: 1,
+            },
+        )
+        .unwrap();
+
+        // Convert to proto struct and back to ensure we are not generating any invalid structures
+        crate::Directory::try_from(crate::proto::Directory::from(d))
+            .expect("directory should be valid");
+    }
+
+    #[test]
+    fn validate_overflow() {
+        let mut d = Directory::new();
+
+        assert_eq!(
+            d.add(
+                "foo".try_into().unwrap(),
+                Node::Directory {
+                    digest: DUMMY_DIGEST.clone(),
+                    size: u64::MAX
+                }
+            ),
+            Err(DirectoryError::SizeOverflow)
+        );
+    }
+
+    #[test]
+    fn add_duplicate_node_to_directory() {
+        let mut d = Directory::new();
+
+        d.add(
+            "a".try_into().unwrap(),
+            Node::Directory {
+                digest: DUMMY_DIGEST.clone(),
+                size: 1,
+            },
+        )
+        .unwrap();
+        assert_eq!(
+            format!(
+                "{}",
+                d.add(
+                    "a".try_into().unwrap(),
+                    Node::File {
+                        digest: DUMMY_DIGEST.clone(),
+                        size: 1,
+                        executable: true
+                    }
+                )
+                .expect_err("adding duplicate dir entry must fail")
+            ),
+            "\"a\" is a duplicate name"
+        );
+    }
+}
diff --git a/tvix/castore/src/nodes/mod.rs b/tvix/castore/src/nodes/mod.rs
new file mode 100644
index 000000000000..ac7aa1e666df
--- /dev/null
+++ b/tvix/castore/src/nodes/mod.rs
@@ -0,0 +1,48 @@
+//! This holds types describing nodes in the tvix-castore model.
+mod directory;
+mod symlink_target;
+
+use crate::B3Digest;
+pub use directory::Directory;
+pub use symlink_target::{SymlinkTarget, SymlinkTargetError};
+
+/// A Node is either a [DirectoryNode], [FileNode] or [SymlinkNode].
+/// Nodes themselves don't have names, what gives them names is either them
+/// being inside a [Directory], or a root node with its own name attached to it.
+#[derive(Debug, Clone, PartialEq, Eq)]
+pub enum Node {
+    /// A DirectoryNode is a pointer to a [Directory], by its [Directory::digest].
+    /// It also records a`size`.
+    /// Such a node is either an element in the [Directory] it itself is contained in,
+    /// or a standalone root node.
+    Directory {
+        /// The blake3 hash of a Directory message, serialized in protobuf canonical form.
+        digest: B3Digest,
+        /// Number of child elements in the Directory referred to by `digest`.
+        /// Calculated by summing up the numbers of nodes, and for each directory,
+        /// its size field. Can be used for inode allocation.
+        /// This field is precisely as verifiable as any other Merkle tree edge.
+        /// Resolve `digest`, and you can compute it incrementally. Resolve the entire
+        /// tree, and you can fully compute it from scratch.
+        /// A credulous implementation won't reject an excessive size, but this is
+        /// harmless: you'll have some ordinals without nodes. Undersizing is obvious
+        /// and easy to reject: you won't have an ordinal for some nodes.
+        size: u64,
+    },
+    /// A FileNode represents a regular or executable file in a Directory or at the root.
+    File {
+        /// The blake3 digest of the file contents
+        digest: B3Digest,
+
+        /// The file content size
+        size: u64,
+
+        /// Whether the file is executable
+        executable: bool,
+    },
+    /// A SymlinkNode represents a symbolic link in a Directory or at the root.
+    Symlink {
+        /// The target of the symlink.
+        target: SymlinkTarget,
+    },
+}
diff --git a/tvix/castore/src/nodes/symlink_target.rs b/tvix/castore/src/nodes/symlink_target.rs
new file mode 100644
index 000000000000..e9a1a0bd05c2
--- /dev/null
+++ b/tvix/castore/src/nodes/symlink_target.rs
@@ -0,0 +1,223 @@
+use bstr::ByteSlice;
+use std::fmt::{self, Debug, Display};
+
+/// A wrapper type for symlink targets.
+/// Internally uses a [bytes::Bytes], but disallows empty targets and those
+/// containing null bytes.
+#[repr(transparent)]
+#[derive(Clone, PartialEq, Eq)]
+pub struct SymlinkTarget {
+    inner: bytes::Bytes,
+}
+
+/// The maximum length a symlink target can have.
+/// Linux allows 4095 bytes here.
+pub const MAX_TARGET_LEN: usize = 4095;
+
+impl AsRef<[u8]> for SymlinkTarget {
+    fn as_ref(&self) -> &[u8] {
+        self.inner.as_ref()
+    }
+}
+
+impl From<SymlinkTarget> for bytes::Bytes {
+    fn from(value: SymlinkTarget) -> Self {
+        value.inner
+    }
+}
+
+fn validate_symlink_target<B: AsRef<[u8]>>(symlink_target: B) -> Result<B, SymlinkTargetError> {
+    let v = symlink_target.as_ref();
+
+    if v.is_empty() {
+        return Err(SymlinkTargetError::Empty);
+    }
+    if v.len() > MAX_TARGET_LEN {
+        return Err(SymlinkTargetError::TooLong);
+    }
+    if v.contains(&0x00) {
+        return Err(SymlinkTargetError::Null);
+    }
+
+    Ok(symlink_target)
+}
+
+impl TryFrom<bytes::Bytes> for SymlinkTarget {
+    type Error = SymlinkTargetError;
+
+    fn try_from(value: bytes::Bytes) -> Result<Self, Self::Error> {
+        if let Err(e) = validate_symlink_target(&value) {
+            return Err(SymlinkTargetError::Convert(value, Box::new(e)));
+        }
+
+        Ok(Self { inner: value })
+    }
+}
+
+impl TryFrom<&'static [u8]> for SymlinkTarget {
+    type Error = SymlinkTargetError;
+
+    fn try_from(value: &'static [u8]) -> Result<Self, Self::Error> {
+        if let Err(e) = validate_symlink_target(&value) {
+            return Err(SymlinkTargetError::Convert(value.into(), Box::new(e)));
+        }
+
+        Ok(Self {
+            inner: bytes::Bytes::from_static(value),
+        })
+    }
+}
+
+impl TryFrom<&str> for SymlinkTarget {
+    type Error = SymlinkTargetError;
+
+    fn try_from(value: &str) -> Result<Self, Self::Error> {
+        if let Err(e) = validate_symlink_target(value) {
+            return Err(SymlinkTargetError::Convert(
+                value.to_owned().into(),
+                Box::new(e),
+            ));
+        }
+
+        Ok(Self {
+            inner: bytes::Bytes::copy_from_slice(value.as_bytes()),
+        })
+    }
+}
+
+impl Debug for SymlinkTarget {
+    fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
+        Debug::fmt(self.inner.as_bstr(), f)
+    }
+}
+
+impl Display for SymlinkTarget {
+    fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
+        Display::fmt(self.inner.as_bstr(), f)
+    }
+}
+
+/// Errors created when constructing / converting to [SymlinkTarget].
+#[derive(Debug, PartialEq, Eq, thiserror::Error)]
+#[cfg_attr(test, derive(Clone))]
+pub enum SymlinkTargetError {
+    #[error("cannot be empty")]
+    Empty,
+    #[error("cannot contain null bytes")]
+    Null,
+    #[error("cannot be over {} bytes long", MAX_TARGET_LEN)]
+    TooLong,
+    #[error("unable to convert '{:?}", .0.as_bstr())]
+    Convert(bytes::Bytes, Box<Self>),
+}
+
+#[cfg(test)]
+mod tests {
+    use bytes::Bytes;
+    use rstest::rstest;
+
+    use super::validate_symlink_target;
+    use super::{SymlinkTarget, SymlinkTargetError};
+
+    #[rstest]
+    #[case::empty(b"", SymlinkTargetError::Empty)]
+    #[case::null(b"foo\0", SymlinkTargetError::Null)]
+    fn errors(#[case] v: &'static [u8], #[case] err: SymlinkTargetError) {
+        {
+            assert_eq!(
+                Err(err.clone()),
+                validate_symlink_target(v),
+                "validate_symlink_target must fail as expected"
+            );
+        }
+
+        let exp_err_v = Bytes::from_static(v);
+
+        // Bytes
+        {
+            let v = Bytes::from_static(v);
+            assert_eq!(
+                Err(SymlinkTargetError::Convert(
+                    exp_err_v.clone(),
+                    Box::new(err.clone())
+                )),
+                SymlinkTarget::try_from(v),
+                "conversion must fail as expected"
+            );
+        }
+        // &[u8]
+        {
+            assert_eq!(
+                Err(SymlinkTargetError::Convert(
+                    exp_err_v.clone(),
+                    Box::new(err.clone())
+                )),
+                SymlinkTarget::try_from(v),
+                "conversion must fail as expected"
+            );
+        }
+        // &str, if this is valid UTF-8
+        {
+            if let Ok(v) = std::str::from_utf8(v) {
+                assert_eq!(
+                    Err(SymlinkTargetError::Convert(
+                        exp_err_v.clone(),
+                        Box::new(err.clone())
+                    )),
+                    SymlinkTarget::try_from(v),
+                    "conversion must fail as expected"
+                );
+            }
+        }
+    }
+
+    #[test]
+    fn error_toolong() {
+        assert_eq!(
+            Err(SymlinkTargetError::TooLong),
+            validate_symlink_target("X".repeat(5000).into_bytes().as_slice())
+        )
+    }
+
+    #[rstest]
+    #[case::boring(b"aa")]
+    #[case::dot(b".")]
+    #[case::dotsandslashes(b"./..")]
+    #[case::dotdot(b"..")]
+    #[case::slashes(b"a/b")]
+    #[case::slashes_and_absolute(b"/a/b")]
+    #[case::invalid_utf8(b"\xc5\xc4\xd6")]
+    fn success(#[case] v: &'static [u8]) {
+        let exp = SymlinkTarget { inner: v.into() };
+
+        // Bytes
+        {
+            let v: Bytes = v.into();
+            assert_eq!(
+                Ok(exp.clone()),
+                SymlinkTarget::try_from(v),
+                "conversion must succeed"
+            )
+        }
+
+        // &[u8]
+        {
+            assert_eq!(
+                Ok(exp.clone()),
+                SymlinkTarget::try_from(v),
+                "conversion must succeed"
+            )
+        }
+
+        // &str, if this is valid UTF-8
+        {
+            if let Ok(v) = std::str::from_utf8(v) {
+                assert_eq!(
+                    Ok(exp.clone()),
+                    SymlinkTarget::try_from(v),
+                    "conversion must succeed"
+                )
+            }
+        }
+    }
+}
diff --git a/tvix/castore/src/path/component.rs b/tvix/castore/src/path/component.rs
new file mode 100644
index 000000000000..78aca03c50fe
--- /dev/null
+++ b/tvix/castore/src/path/component.rs
@@ -0,0 +1,268 @@
+use bstr::ByteSlice;
+use std::fmt::{self, Debug, Display};
+
+/// A wrapper type for validated path components in the castore model.
+/// Internally uses a [bytes::Bytes], but disallows
+/// slashes, and null bytes to be present, as well as
+/// '.', '..' and the empty string.
+/// It also rejects components that are too long (> 255 bytes).
+#[repr(transparent)]
+#[derive(Clone, Hash, PartialEq, Eq, PartialOrd, Ord)]
+pub struct PathComponent {
+    pub(super) inner: bytes::Bytes,
+}
+
+/// The maximum length an individual path component can have.
+/// Linux allows 255 bytes of actual name, so we pick that.
+pub const MAX_NAME_LEN: usize = 255;
+
+impl AsRef<[u8]> for PathComponent {
+    fn as_ref(&self) -> &[u8] {
+        self.inner.as_ref()
+    }
+}
+
+impl From<PathComponent> for bytes::Bytes {
+    fn from(value: PathComponent) -> Self {
+        value.inner
+    }
+}
+
+pub(super) fn validate_name<B: AsRef<[u8]>>(name: B) -> Result<(), PathComponentError> {
+    match name.as_ref() {
+        b"" => Err(PathComponentError::Empty),
+        b".." => Err(PathComponentError::Parent),
+        b"." => Err(PathComponentError::CurDir),
+        v if v.len() > MAX_NAME_LEN => Err(PathComponentError::TooLong),
+        v if v.contains(&0x00) => Err(PathComponentError::Null),
+        v if v.contains(&b'/') => Err(PathComponentError::Slashes),
+        _ => Ok(()),
+    }
+}
+
+impl TryFrom<bytes::Bytes> for PathComponent {
+    type Error = PathComponentError;
+
+    fn try_from(value: bytes::Bytes) -> Result<Self, Self::Error> {
+        if let Err(e) = validate_name(&value) {
+            return Err(PathComponentError::Convert(value, Box::new(e)));
+        }
+
+        Ok(Self { inner: value })
+    }
+}
+
+impl TryFrom<&'static [u8]> for PathComponent {
+    type Error = PathComponentError;
+
+    fn try_from(value: &'static [u8]) -> Result<Self, Self::Error> {
+        if let Err(e) = validate_name(value) {
+            return Err(PathComponentError::Convert(value.into(), Box::new(e)));
+        }
+
+        Ok(Self {
+            inner: bytes::Bytes::from_static(value),
+        })
+    }
+}
+
+impl TryFrom<&str> for PathComponent {
+    type Error = PathComponentError;
+
+    fn try_from(value: &str) -> Result<Self, Self::Error> {
+        if let Err(e) = validate_name(value) {
+            return Err(PathComponentError::Convert(
+                value.to_owned().into(),
+                Box::new(e),
+            ));
+        }
+
+        Ok(Self {
+            inner: bytes::Bytes::copy_from_slice(value.as_bytes()),
+        })
+    }
+}
+
+impl TryFrom<&std::ffi::CStr> for PathComponent {
+    type Error = PathComponentError;
+
+    fn try_from(value: &std::ffi::CStr) -> Result<Self, Self::Error> {
+        let value = value.to_bytes();
+        if let Err(e) = validate_name(value) {
+            return Err(PathComponentError::Convert(
+                value.to_owned().into(),
+                Box::new(e),
+            ));
+        }
+
+        Ok(Self {
+            inner: bytes::Bytes::copy_from_slice(value),
+        })
+    }
+}
+
+impl Debug for PathComponent {
+    fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
+        Debug::fmt(self.inner.as_bstr(), f)
+    }
+}
+
+impl Display for PathComponent {
+    fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
+        Display::fmt(self.inner.as_bstr(), f)
+    }
+}
+
+/// Errors created when parsing / validating [PathComponent].
+#[derive(Debug, PartialEq, thiserror::Error)]
+#[cfg_attr(test, derive(Clone))]
+pub enum PathComponentError {
+    #[error("cannot be empty")]
+    Empty,
+    #[error("cannot contain null bytes")]
+    Null,
+    #[error("cannot be '.'")]
+    CurDir,
+    #[error("cannot be '..'")]
+    Parent,
+    #[error("cannot contain slashes")]
+    Slashes,
+    #[error("cannot be over {} bytes long", MAX_NAME_LEN)]
+    TooLong,
+    #[error("unable to convert '{:?}'", .0.as_bstr())]
+    Convert(bytes::Bytes, #[source] Box<Self>),
+}
+
+#[cfg(test)]
+mod tests {
+    use std::ffi::CString;
+
+    use bytes::Bytes;
+    use rstest::rstest;
+
+    use super::{validate_name, PathComponent, PathComponentError};
+
+    #[rstest]
+    #[case::empty(b"", PathComponentError::Empty)]
+    #[case::null(b"foo\0", PathComponentError::Null)]
+    #[case::curdir(b".", PathComponentError::CurDir)]
+    #[case::parent(b"..", PathComponentError::Parent)]
+    #[case::slashes1(b"a/b", PathComponentError::Slashes)]
+    #[case::slashes2(b"/", PathComponentError::Slashes)]
+    fn errors(#[case] v: &'static [u8], #[case] err: PathComponentError) {
+        {
+            assert_eq!(
+                Err(err.clone()),
+                validate_name(v),
+                "validate_name must fail as expected"
+            );
+        }
+
+        let exp_err_v = Bytes::from_static(v);
+
+        // Bytes
+        {
+            let v = Bytes::from_static(v);
+            assert_eq!(
+                Err(PathComponentError::Convert(
+                    exp_err_v.clone(),
+                    Box::new(err.clone())
+                )),
+                PathComponent::try_from(v),
+                "conversion must fail as expected"
+            );
+        }
+        // &[u8]
+        {
+            assert_eq!(
+                Err(PathComponentError::Convert(
+                    exp_err_v.clone(),
+                    Box::new(err.clone())
+                )),
+                PathComponent::try_from(v),
+                "conversion must fail as expected"
+            );
+        }
+        // &str, if it is valid UTF-8
+        {
+            if let Ok(v) = std::str::from_utf8(v) {
+                assert_eq!(
+                    Err(PathComponentError::Convert(
+                        exp_err_v.clone(),
+                        Box::new(err.clone())
+                    )),
+                    PathComponent::try_from(v),
+                    "conversion must fail as expected"
+                );
+            }
+        }
+        // &CStr, if it can be constructed (fails if the payload contains null bytes)
+        {
+            if let Ok(v) = CString::new(v) {
+                let v = v.as_ref();
+                assert_eq!(
+                    Err(PathComponentError::Convert(
+                        exp_err_v.clone(),
+                        Box::new(err.clone())
+                    )),
+                    PathComponent::try_from(v),
+                    "conversion must fail as expected"
+                );
+            }
+        }
+    }
+
+    #[test]
+    fn error_toolong() {
+        assert_eq!(
+            Err(PathComponentError::TooLong),
+            validate_name("X".repeat(500).into_bytes().as_slice())
+        )
+    }
+
+    #[test]
+    fn success() {
+        let exp = PathComponent { inner: "aa".into() };
+
+        // Bytes
+        {
+            let v: Bytes = "aa".into();
+            assert_eq!(
+                Ok(exp.clone()),
+                PathComponent::try_from(v),
+                "conversion must succeed"
+            );
+        }
+
+        // &[u8]
+        {
+            let v: &[u8] = b"aa";
+            assert_eq!(
+                Ok(exp.clone()),
+                PathComponent::try_from(v),
+                "conversion must succeed"
+            );
+        }
+
+        // &str
+        {
+            let v: &str = "aa";
+            assert_eq!(
+                Ok(exp.clone()),
+                PathComponent::try_from(v),
+                "conversion must succeed"
+            );
+        }
+
+        // &CStr
+        {
+            let v = CString::new("aa").expect("CString must construct");
+            let v = v.as_c_str();
+            assert_eq!(
+                Ok(exp.clone()),
+                PathComponent::try_from(v),
+                "conversion must succeed"
+            );
+        }
+    }
+}
diff --git a/tvix/castore/src/path.rs b/tvix/castore/src/path/mod.rs
index fcc2bd01fbd6..15f31a570da9 100644
--- a/tvix/castore/src/path.rs
+++ b/tvix/castore/src/path/mod.rs
@@ -1,5 +1,5 @@
 //! Contains data structures to deal with Paths in the tvix-castore model.
-
+use bstr::ByteSlice;
 use std::{
     borrow::Borrow,
     fmt::{self, Debug, Display},
@@ -8,9 +8,8 @@ use std::{
     str::FromStr,
 };
 
-use bstr::ByteSlice;
-
-use crate::proto::validate_node_name;
+mod component;
+pub use component::{PathComponent, PathComponentError};
 
 /// Represents a Path in the castore model.
 /// These are always relative, and platform-independent, which distinguishes
@@ -38,7 +37,9 @@ impl Path {
         if !bytes.is_empty() {
             // Ensure all components are valid castore node names.
             for component in bytes.split_str(b"/") {
-                validate_node_name(component).ok()?;
+                if component::validate_name(component).is_err() {
+                    return None;
+                }
             }
         }
 
@@ -81,10 +82,26 @@ impl Path {
         Ok(v)
     }
 
+    /// Provides an iterator over the components of the path,
+    /// which are invividual [PathComponent].
+    /// In case the path is empty, an empty iterator is returned.
+    pub fn components(&self) -> impl Iterator<Item = PathComponent> + '_ {
+        let mut iter = self.inner.split_str(&b"/");
+
+        // We don't want to return an empty element, consume it if it's the only one.
+        if self.inner.is_empty() {
+            let _ = iter.next();
+        }
+
+        iter.map(|b| PathComponent {
+            inner: bytes::Bytes::copy_from_slice(b),
+        })
+    }
+
     /// Produces an iterator over the components of the path, which are
     /// individual byte slices.
     /// In case the path is empty, an empty iterator is returned.
-    pub fn components(&self) -> impl Iterator<Item = &[u8]> {
+    pub fn components_bytes(&self) -> impl Iterator<Item = &[u8]> {
         let mut iter = self.inner.split_str(&b"/");
 
         // We don't want to return an empty element, consume it if it's the only one.
@@ -95,11 +112,16 @@ impl Path {
         iter
     }
 
-    /// Returns the final component of the Path, if there is one.
-    pub fn file_name(&self) -> Option<&[u8]> {
+    /// Returns the final component of the Path, if there is one, in bytes.
+    pub fn file_name(&self) -> Option<PathComponent> {
         self.components().last()
     }
 
+    /// Returns the final component of the Path, if there is one, in bytes.
+    pub fn file_name_bytes(&self) -> Option<&[u8]> {
+        self.components_bytes().last()
+    }
+
     pub fn as_bytes(&self) -> &[u8] {
         &self.inner
     }
@@ -211,7 +233,9 @@ impl PathBuf {
 
     /// Adjoins `name` to self.
     pub fn try_push(&mut self, name: &[u8]) -> Result<(), std::io::Error> {
-        validate_node_name(name).map_err(|_| std::io::ErrorKind::InvalidData)?;
+        if component::validate_name(name).is_err() {
+            return Err(std::io::ErrorKind::InvalidData.into());
+        }
 
         if !self.inner.is_empty() {
             self.inner.push(b'/');
@@ -329,7 +353,7 @@ mod test {
         assert_eq!(s.as_bytes(), p.as_bytes(), "inner bytes mismatch");
         assert_eq!(
             num_components,
-            p.components().count(),
+            p.components_bytes().count(),
             "number of components mismatch"
         );
     }
@@ -396,10 +420,10 @@ mod test {
     #[case("a", vec!["a"])]
     #[case("a/b", vec!["a", "b"])]
     #[case("a/b/c", vec!["a","b", "c"])]
-    pub fn components(#[case] p: PathBuf, #[case] exp_components: Vec<&str>) {
+    pub fn components_bytes(#[case] p: PathBuf, #[case] exp_components: Vec<&str>) {
         assert_eq!(
             exp_components,
-            p.components()
+            p.components_bytes()
                 .map(|x| x.to_str().unwrap())
                 .collect::<Vec<_>>()
         );
diff --git a/tvix/castore/src/proto/grpc_directoryservice_wrapper.rs b/tvix/castore/src/proto/grpc_directoryservice_wrapper.rs
index 5c1428690cb4..62fdb34a25a0 100644
--- a/tvix/castore/src/proto/grpc_directoryservice_wrapper.rs
+++ b/tvix/castore/src/proto/grpc_directoryservice_wrapper.rs
@@ -1,6 +1,5 @@
-use crate::directoryservice::ClosureValidator;
-use crate::proto;
-use crate::{directoryservice::DirectoryService, B3Digest};
+use crate::directoryservice::{DirectoryGraph, DirectoryService, LeavesToRootValidator};
+use crate::{proto, B3Digest, DirectoryError};
 use futures::stream::BoxStream;
 use futures::TryStreamExt;
 use std::ops::Deref;
@@ -57,13 +56,16 @@ where
                                 Status::not_found(format!("directory {} not found", digest))
                             })?;
 
-                        Box::pin(once(Ok(directory)))
+                        Box::pin(once(Ok(directory.into())))
                     } else {
                         // If recursive was requested, traverse via get_recursive.
                         Box::pin(
-                            self.directory_service.get_recursive(&digest).map_err(|e| {
-                                tonic::Status::new(tonic::Code::Internal, e.to_string())
-                            }),
+                            self.directory_service
+                                .get_recursive(&digest)
+                                .map_ok(proto::Directory::from)
+                                .map_err(|e| {
+                                    tonic::Status::new(tonic::Code::Internal, e.to_string())
+                                }),
                         )
                     }
                 }))
@@ -78,14 +80,22 @@ where
     ) -> Result<Response<proto::PutDirectoryResponse>, Status> {
         let mut req_inner = request.into_inner();
 
-        // We put all Directory messages we receive into ClosureValidator first.
-        let mut validator = ClosureValidator::default();
+        // We put all Directory messages we receive into DirectoryGraph.
+        let mut validator = DirectoryGraph::<LeavesToRootValidator>::default();
         while let Some(directory) = req_inner.message().await? {
-            validator.add(directory)?;
+            validator
+                .add(directory.try_into().map_err(|e: DirectoryError| {
+                    tonic::Status::new(tonic::Code::Internal, e.to_string())
+                })?)
+                .map_err(|e| tonic::Status::new(tonic::Code::Internal, e.to_string()))?;
         }
 
         // drain, which validates connectivity too.
-        let directories = validator.finalize()?;
+        let directories = validator
+            .validate()
+            .map_err(|e| tonic::Status::new(tonic::Code::Internal, e.to_string()))?
+            .drain_leaves_to_root()
+            .collect::<Vec<_>>();
 
         let mut directory_putter = self.directory_service.put_multiple_start();
         for directory in directories {
diff --git a/tvix/castore/src/proto/mod.rs b/tvix/castore/src/proto/mod.rs
index 5374e3ae5a80..8bc74b412676 100644
--- a/tvix/castore/src/proto/mod.rs
+++ b/tvix/castore/src/proto/mod.rs
@@ -1,18 +1,13 @@
-#![allow(non_snake_case)]
-// https://github.com/hyperium/tonic/issues/1056
-use bstr::ByteSlice;
-use std::{collections::HashSet, iter::Peekable, str};
-
 use prost::Message;
+use std::cmp::Ordering;
 
 mod grpc_blobservice_wrapper;
 mod grpc_directoryservice_wrapper;
 
+use crate::{path::PathComponent, B3Digest, DirectoryError};
 pub use grpc_blobservice_wrapper::GRPCBlobServiceWrapper;
 pub use grpc_directoryservice_wrapper::GRPCDirectoryServiceWrapper;
 
-use crate::{B3Digest, B3_LEN};
-
 tonic::include_proto!("tvix.castore.v1");
 
 #[cfg(feature = "tonic-reflection")]
@@ -24,38 +19,6 @@ pub const FILE_DESCRIPTOR_SET: &[u8] = tonic::include_file_descriptor_set!("tvix
 #[cfg(test)]
 mod tests;
 
-/// Errors that can occur during the validation of [Directory] messages.
-#[derive(Debug, PartialEq, Eq, thiserror::Error)]
-pub enum ValidateDirectoryError {
-    /// Elements are not in sorted order
-    #[error("{:?} is not sorted", .0.as_bstr())]
-    WrongSorting(Vec<u8>),
-    /// Multiple elements with the same name encountered
-    #[error("{:?} is a duplicate name", .0.as_bstr())]
-    DuplicateName(Vec<u8>),
-    /// Invalid node
-    #[error("invalid node with name {:?}: {:?}", .0.as_bstr(), .1.to_string())]
-    InvalidNode(Vec<u8>, ValidateNodeError),
-    #[error("Total size exceeds u32::MAX")]
-    SizeOverflow,
-}
-
-/// Errors that occur during Node validation
-#[derive(Debug, PartialEq, Eq, thiserror::Error)]
-pub enum ValidateNodeError {
-    #[error("No node set")]
-    NoNodeSet,
-    /// Invalid digest length encountered
-    #[error("Invalid Digest length: {0}")]
-    InvalidDigestLen(usize),
-    /// Invalid name encountered
-    #[error("Invalid name: {}", .0.as_bstr())]
-    InvalidName(Vec<u8>),
-    /// Invalid symlink target
-    #[error("Invalid symlink target: {}", .0.as_bstr())]
-    InvalidSymlinkTarget(Vec<u8>),
-}
-
 /// Errors that occur during StatBlobResponse validation
 #[derive(Debug, PartialEq, Eq, thiserror::Error)]
 pub enum ValidateStatBlobResponseError {
@@ -64,184 +27,6 @@ pub enum ValidateStatBlobResponseError {
     InvalidDigestLen(usize, usize),
 }
 
-/// Checks a Node name for validity as an intermediate node.
-/// We disallow slashes, null bytes, '.', '..' and the empty string.
-pub(crate) fn validate_node_name(name: &[u8]) -> Result<(), ValidateNodeError> {
-    if name.is_empty()
-        || name == b".."
-        || name == b"."
-        || name.contains(&0x00)
-        || name.contains(&b'/')
-    {
-        Err(ValidateNodeError::InvalidName(name.to_owned()))
-    } else {
-        Ok(())
-    }
-}
-
-/// NamedNode is implemented for [FileNode], [DirectoryNode] and [SymlinkNode]
-/// and [node::Node], so we can ask all of them for the name easily.
-pub trait NamedNode {
-    fn get_name(&self) -> &[u8];
-}
-
-impl NamedNode for &FileNode {
-    fn get_name(&self) -> &[u8] {
-        &self.name
-    }
-}
-
-impl NamedNode for &DirectoryNode {
-    fn get_name(&self) -> &[u8] {
-        &self.name
-    }
-}
-
-impl NamedNode for &SymlinkNode {
-    fn get_name(&self) -> &[u8] {
-        &self.name
-    }
-}
-
-impl NamedNode for node::Node {
-    fn get_name(&self) -> &[u8] {
-        match self {
-            node::Node::File(node_file) => &node_file.name,
-            node::Node::Directory(node_directory) => &node_directory.name,
-            node::Node::Symlink(node_symlink) => &node_symlink.name,
-        }
-    }
-}
-
-impl Node {
-    /// Ensures the node has a valid enum kind (is Some), and passes its
-    // per-enum validation.
-    pub fn validate(&self) -> Result<(), ValidateNodeError> {
-        if let Some(node) = self.node.as_ref() {
-            node.validate()
-        } else {
-            Err(ValidateNodeError::NoNodeSet)
-        }
-    }
-}
-
-impl node::Node {
-    /// Returns the node with a new name.
-    pub fn rename(self, name: bytes::Bytes) -> Self {
-        match self {
-            node::Node::Directory(n) => node::Node::Directory(DirectoryNode { name, ..n }),
-            node::Node::File(n) => node::Node::File(FileNode { name, ..n }),
-            node::Node::Symlink(n) => node::Node::Symlink(SymlinkNode { name, ..n }),
-        }
-    }
-
-    /// Ensures the node has a valid name, and checks the type-specific fields too.
-    pub fn validate(&self) -> Result<(), ValidateNodeError> {
-        match self {
-            // for a directory root node, ensure the digest has the appropriate size.
-            node::Node::Directory(directory_node) => {
-                if directory_node.digest.len() != B3_LEN {
-                    Err(ValidateNodeError::InvalidDigestLen(
-                        directory_node.digest.len(),
-                    ))?;
-                }
-                validate_node_name(&directory_node.name)
-            }
-            // for a file root node, ensure the digest has the appropriate size.
-            node::Node::File(file_node) => {
-                if file_node.digest.len() != B3_LEN {
-                    Err(ValidateNodeError::InvalidDigestLen(file_node.digest.len()))?;
-                }
-                validate_node_name(&file_node.name)
-            }
-            // ensure the symlink target is not empty and doesn't contain null bytes.
-            node::Node::Symlink(symlink_node) => {
-                if symlink_node.target.is_empty() || symlink_node.target.contains(&b'\0') {
-                    Err(ValidateNodeError::InvalidSymlinkTarget(
-                        symlink_node.target.to_vec(),
-                    ))?;
-                }
-                validate_node_name(&symlink_node.name)
-            }
-        }
-    }
-}
-
-impl PartialOrd for node::Node {
-    fn partial_cmp(&self, other: &Self) -> Option<std::cmp::Ordering> {
-        Some(self.cmp(other))
-    }
-}
-
-impl Ord for node::Node {
-    fn cmp(&self, other: &Self) -> std::cmp::Ordering {
-        self.get_name().cmp(other.get_name())
-    }
-}
-
-impl PartialOrd for FileNode {
-    fn partial_cmp(&self, other: &Self) -> Option<std::cmp::Ordering> {
-        Some(self.cmp(other))
-    }
-}
-
-impl Ord for FileNode {
-    fn cmp(&self, other: &Self) -> std::cmp::Ordering {
-        self.get_name().cmp(other.get_name())
-    }
-}
-
-impl PartialOrd for SymlinkNode {
-    fn partial_cmp(&self, other: &Self) -> Option<std::cmp::Ordering> {
-        Some(self.cmp(other))
-    }
-}
-
-impl Ord for SymlinkNode {
-    fn cmp(&self, other: &Self) -> std::cmp::Ordering {
-        self.get_name().cmp(other.get_name())
-    }
-}
-
-impl PartialOrd for DirectoryNode {
-    fn partial_cmp(&self, other: &Self) -> Option<std::cmp::Ordering> {
-        Some(self.cmp(other))
-    }
-}
-
-impl Ord for DirectoryNode {
-    fn cmp(&self, other: &Self) -> std::cmp::Ordering {
-        self.get_name().cmp(other.get_name())
-    }
-}
-
-/// Accepts a name, and a mutable reference to the previous name.
-/// If the passed name is larger than the previous one, the reference is updated.
-/// If it's not, an error is returned.
-fn update_if_lt_prev<'n>(
-    prev_name: &mut &'n [u8],
-    name: &'n [u8],
-) -> Result<(), ValidateDirectoryError> {
-    if *name < **prev_name {
-        return Err(ValidateDirectoryError::WrongSorting(name.to_vec()));
-    }
-    *prev_name = name;
-    Ok(())
-}
-
-/// Inserts the given name into a HashSet if it's not already in there.
-/// If it is, an error is returned.
-fn insert_once<'n>(
-    seen_names: &mut HashSet<&'n [u8]>,
-    name: &'n [u8],
-) -> Result<(), ValidateDirectoryError> {
-    if seen_names.get(name).is_some() {
-        return Err(ValidateDirectoryError::DuplicateName(name.to_vec()));
-    }
-    seen_names.insert(name);
-    Ok(())
-}
-
 fn checked_sum(iter: impl IntoIterator<Item = u64>) -> Option<u64> {
     iter.into_iter().try_fold(0u64, |acc, i| acc.checked_add(i))
 }
@@ -278,117 +63,211 @@ impl Directory {
             .as_bytes()
             .into()
     }
+}
 
-    /// validate checks the directory for invalid data, such as:
-    /// - violations of name restrictions
-    /// - invalid digest lengths
-    /// - not properly sorted lists
-    /// - duplicate names in the three lists
-    pub fn validate(&self) -> Result<(), ValidateDirectoryError> {
-        let mut seen_names: HashSet<&[u8]> = HashSet::new();
-
-        let mut last_directory_name: &[u8] = b"";
-        let mut last_file_name: &[u8] = b"";
-        let mut last_symlink_name: &[u8] = b"";
+impl TryFrom<Directory> for crate::Directory {
+    type Error = DirectoryError;
+
+    fn try_from(value: Directory) -> Result<Self, Self::Error> {
+        // Check directories, files and symlinks are sorted
+        // We'll notice duplicates across all three fields when constructing the Directory.
+        // FUTUREWORK: use is_sorted() once stable, and/or implement the producer for
+        // [crate::Directory::try_from_iter] iterating over all three and doing all checks inline.
+        value
+            .directories
+            .iter()
+            .try_fold(&b""[..], |prev_name, e| {
+                match e.name.as_ref().cmp(prev_name) {
+                    Ordering::Less => Err(DirectoryError::WrongSorting(e.name.to_owned())),
+                    Ordering::Equal => Err(DirectoryError::DuplicateName(
+                        e.name
+                            .to_owned()
+                            .try_into()
+                            .map_err(DirectoryError::InvalidName)?,
+                    )),
+                    Ordering::Greater => Ok(e.name.as_ref()),
+                }
+            })?;
+        value.files.iter().try_fold(&b""[..], |prev_name, e| {
+            match e.name.as_ref().cmp(prev_name) {
+                Ordering::Less => Err(DirectoryError::WrongSorting(e.name.to_owned())),
+                Ordering::Equal => Err(DirectoryError::DuplicateName(
+                    e.name
+                        .to_owned()
+                        .try_into()
+                        .map_err(DirectoryError::InvalidName)?,
+                )),
+                Ordering::Greater => Ok(e.name.as_ref()),
+            }
+        })?;
+        value.symlinks.iter().try_fold(&b""[..], |prev_name, e| {
+            match e.name.as_ref().cmp(prev_name) {
+                Ordering::Less => Err(DirectoryError::WrongSorting(e.name.to_owned())),
+                Ordering::Equal => Err(DirectoryError::DuplicateName(
+                    e.name
+                        .to_owned()
+                        .try_into()
+                        .map_err(DirectoryError::InvalidName)?,
+                )),
+                Ordering::Greater => Ok(e.name.as_ref()),
+            }
+        })?;
 
-        // check directories
-        for directory_node in &self.directories {
-            node::Node::Directory(directory_node.clone())
-                .validate()
-                .map_err(|e| {
-                    ValidateDirectoryError::InvalidNode(directory_node.name.to_vec(), e)
-                })?;
+        // FUTUREWORK: use is_sorted() once stable, and/or implement the producer for
+        // [crate::Directory::try_from_iter] iterating over all three and doing all checks inline.
+        let mut elems: Vec<(PathComponent, crate::Node)> =
+            Vec::with_capacity(value.directories.len() + value.files.len() + value.symlinks.len());
 
-            update_if_lt_prev(&mut last_directory_name, &directory_node.name)?;
-            insert_once(&mut seen_names, &directory_node.name)?;
+        for e in value.directories {
+            elems.push(
+                Node {
+                    node: Some(node::Node::Directory(e)),
+                }
+                .into_name_and_node()?,
+            );
         }
 
-        // check files
-        for file_node in &self.files {
-            node::Node::File(file_node.clone())
-                .validate()
-                .map_err(|e| ValidateDirectoryError::InvalidNode(file_node.name.to_vec(), e))?;
-
-            update_if_lt_prev(&mut last_file_name, &file_node.name)?;
-            insert_once(&mut seen_names, &file_node.name)?;
+        for e in value.files {
+            elems.push(
+                Node {
+                    node: Some(node::Node::File(e)),
+                }
+                .into_name_and_node()?,
+            )
         }
 
-        // check symlinks
-        for symlink_node in &self.symlinks {
-            node::Node::Symlink(symlink_node.clone())
-                .validate()
-                .map_err(|e| ValidateDirectoryError::InvalidNode(symlink_node.name.to_vec(), e))?;
-
-            update_if_lt_prev(&mut last_symlink_name, &symlink_node.name)?;
-            insert_once(&mut seen_names, &symlink_node.name)?;
+        for e in value.symlinks {
+            elems.push(
+                Node {
+                    node: Some(node::Node::Symlink(e)),
+                }
+                .into_name_and_node()?,
+            )
         }
 
-        self.size_checked()
-            .ok_or(ValidateDirectoryError::SizeOverflow)?;
-
-        Ok(())
+        crate::Directory::try_from_iter(elems)
     }
+}
 
-    /// Allows iterating over all three nodes ([DirectoryNode], [FileNode],
-    /// [SymlinkNode]) in an ordered fashion, as long as the individual lists
-    /// are sorted (which can be checked by the [Directory::validate]).
-    pub fn nodes(&self) -> DirectoryNodesIterator {
-        return DirectoryNodesIterator {
-            i_directories: self.directories.iter().peekable(),
-            i_files: self.files.iter().peekable(),
-            i_symlinks: self.symlinks.iter().peekable(),
-        };
-    }
+impl From<crate::Directory> for Directory {
+    fn from(value: crate::Directory) -> Self {
+        let mut directories = vec![];
+        let mut files = vec![];
+        let mut symlinks = vec![];
+
+        for (name, node) in value.into_nodes() {
+            match node {
+                crate::Node::File {
+                    digest,
+                    size,
+                    executable,
+                } => files.push(FileNode {
+                    name: name.into(),
+                    digest: digest.into(),
+                    size,
+                    executable,
+                }),
+                crate::Node::Directory { digest, size } => directories.push(DirectoryNode {
+                    name: name.into(),
+                    digest: digest.into(),
+                    size,
+                }),
+                crate::Node::Symlink { target } => {
+                    symlinks.push(SymlinkNode {
+                        name: name.into(),
+                        target: target.into(),
+                    });
+                }
+            }
+        }
 
-    /// Adds the specified [node::Node] to the [Directory], preserving sorted entries.
-    /// This assumes the [Directory] to be sorted prior to adding the node.
-    ///
-    /// Inserting an element that already exists with the same name in the directory is not
-    /// supported.
-    pub fn add(&mut self, node: node::Node) {
-        debug_assert!(
-            !self.files.iter().any(|x| x.get_name() == node.get_name()),
-            "name already exists in files"
-        );
-        debug_assert!(
-            !self
-                .directories
-                .iter()
-                .any(|x| x.get_name() == node.get_name()),
-            "name already exists in directories"
-        );
-        debug_assert!(
-            !self
-                .symlinks
-                .iter()
-                .any(|x| x.get_name() == node.get_name()),
-            "name already exists in symlinks"
-        );
+        Directory {
+            directories,
+            files,
+            symlinks,
+        }
+    }
+}
 
-        match node {
-            node::Node::File(node) => {
-                let pos = self
-                    .files
-                    .binary_search(&node)
-                    .expect_err("Tvix bug: dir entry with name already exists");
-                self.files.insert(pos, node);
+impl Node {
+    /// Converts a proto [Node] to a [crate::Node], and splits off the name.
+    pub fn into_name_and_node(self) -> Result<(PathComponent, crate::Node), DirectoryError> {
+        match self.node.ok_or_else(|| DirectoryError::NoNodeSet)? {
+            node::Node::Directory(n) => {
+                let name: PathComponent = n.name.try_into().map_err(DirectoryError::InvalidName)?;
+                let digest = B3Digest::try_from(n.digest)
+                    .map_err(|e| DirectoryError::InvalidNode(name.clone(), e.into()))?;
+
+                let node = crate::Node::Directory {
+                    digest,
+                    size: n.size,
+                };
+
+                Ok((name, node))
             }
-            node::Node::Directory(node) => {
-                let pos = self
-                    .directories
-                    .binary_search(&node)
-                    .expect_err("Tvix bug: dir entry with name already exists");
-                self.directories.insert(pos, node);
+            node::Node::File(n) => {
+                let name: PathComponent = n.name.try_into().map_err(DirectoryError::InvalidName)?;
+                let digest = B3Digest::try_from(n.digest)
+                    .map_err(|e| DirectoryError::InvalidNode(name.clone(), e.into()))?;
+
+                let node = crate::Node::File {
+                    digest,
+                    size: n.size,
+                    executable: n.executable,
+                };
+
+                Ok((name, node))
             }
-            node::Node::Symlink(node) => {
-                let pos = self
-                    .symlinks
-                    .binary_search(&node)
-                    .expect_err("Tvix bug: dir entry with name already exists");
-                self.symlinks.insert(pos, node);
+
+            node::Node::Symlink(n) => {
+                let name: PathComponent = n.name.try_into().map_err(DirectoryError::InvalidName)?;
+
+                let node = crate::Node::Symlink {
+                    target: n.target.try_into().map_err(|e| {
+                        DirectoryError::InvalidNode(
+                            name.clone(),
+                            crate::ValidateNodeError::InvalidSymlinkTarget(e),
+                        )
+                    })?,
+                };
+
+                Ok((name, node))
             }
         }
     }
+
+    /// Construsts a [Node] from a name and [crate::Node].
+    /// The name is a [bytes::Bytes], not a [PathComponent], as we have use an
+    /// empty name in some places.
+    pub fn from_name_and_node(name: bytes::Bytes, n: crate::Node) -> Self {
+        match n {
+            crate::Node::Directory { digest, size } => Self {
+                node: Some(node::Node::Directory(DirectoryNode {
+                    name,
+                    digest: digest.into(),
+                    size,
+                })),
+            },
+            crate::Node::File {
+                digest,
+                size,
+                executable,
+            } => Self {
+                node: Some(node::Node::File(FileNode {
+                    name,
+                    digest: digest.into(),
+                    size,
+                    executable,
+                })),
+            },
+            crate::Node::Symlink { target } => Self {
+                node: Some(node::Node::Symlink(SymlinkNode {
+                    name,
+                    target: target.into(),
+                })),
+            },
+        }
+    }
 }
 
 impl StatBlobResponse {
@@ -407,65 +286,3 @@ impl StatBlobResponse {
         Ok(())
     }
 }
-
-/// Struct to hold the state of an iterator over all nodes of a Directory.
-///
-/// Internally, this keeps peekable Iterators over all three lists of a
-/// Directory message.
-pub struct DirectoryNodesIterator<'a> {
-    // directory: &Directory,
-    i_directories: Peekable<std::slice::Iter<'a, DirectoryNode>>,
-    i_files: Peekable<std::slice::Iter<'a, FileNode>>,
-    i_symlinks: Peekable<std::slice::Iter<'a, SymlinkNode>>,
-}
-
-/// looks at two elements implementing NamedNode, and returns true if "left
-/// is smaller / comes first".
-///
-/// Some(_) is preferred over None.
-fn left_name_lt_right<A: NamedNode, B: NamedNode>(left: Option<&A>, right: Option<&B>) -> bool {
-    match left {
-        // if left is None, right always wins
-        None => false,
-        Some(left_inner) => {
-            // left is Some.
-            match right {
-                // left is Some, right is None - left wins.
-                None => true,
-                Some(right_inner) => {
-                    // both are Some - compare the name.
-                    return left_inner.get_name() < right_inner.get_name();
-                }
-            }
-        }
-    }
-}
-
-impl Iterator for DirectoryNodesIterator<'_> {
-    type Item = node::Node;
-
-    // next returns the next node in the Directory.
-    // we peek at all three internal iterators, and pick the one with the
-    // smallest name, to ensure lexicographical ordering.
-    // The individual lists are already known to be sorted.
-    fn next(&mut self) -> Option<Self::Item> {
-        if left_name_lt_right(self.i_directories.peek(), self.i_files.peek()) {
-            // i_directories is still in the game, compare with symlinks
-            if left_name_lt_right(self.i_directories.peek(), self.i_symlinks.peek()) {
-                self.i_directories
-                    .next()
-                    .cloned()
-                    .map(node::Node::Directory)
-            } else {
-                self.i_symlinks.next().cloned().map(node::Node::Symlink)
-            }
-        } else {
-            // i_files is still in the game, compare with symlinks
-            if left_name_lt_right(self.i_files.peek(), self.i_symlinks.peek()) {
-                self.i_files.next().cloned().map(node::Node::File)
-            } else {
-                self.i_symlinks.next().cloned().map(node::Node::Symlink)
-            }
-        }
-    }
-}
diff --git a/tvix/castore/src/proto/tests/directory.rs b/tvix/castore/src/proto/tests/directory.rs
index 81b73a048d52..efbc4e9f2af1 100644
--- a/tvix/castore/src/proto/tests/directory.rs
+++ b/tvix/castore/src/proto/tests/directory.rs
@@ -1,7 +1,5 @@
-use crate::proto::{
-    node, Directory, DirectoryNode, FileNode, SymlinkNode, ValidateDirectoryError,
-    ValidateNodeError,
-};
+use crate::proto::{Directory, DirectoryError, DirectoryNode, FileNode, SymlinkNode};
+use crate::ValidateNodeError;
 
 use hex_literal::hex;
 
@@ -149,7 +147,7 @@ fn digest() {
 #[test]
 fn validate_empty() {
     let d = Directory::default();
-    assert_eq!(d.validate(), Ok(()));
+    assert!(crate::Directory::try_from(d).is_ok());
 }
 
 #[test]
@@ -157,18 +155,15 @@ fn validate_invalid_names() {
     {
         let d = Directory {
             directories: vec![DirectoryNode {
-                name: "".into(),
+                name: b"\0"[..].into(),
                 digest: DUMMY_DIGEST.to_vec().into(),
                 size: 42,
             }],
             ..Default::default()
         };
-        match d.validate().expect_err("must fail") {
-            ValidateDirectoryError::InvalidNode(n, ValidateNodeError::InvalidName(_)) => {
-                assert_eq!(n, b"")
-            }
-            _ => panic!("unexpected error"),
-        };
+
+        let e = crate::Directory::try_from(d).expect_err("must fail");
+        assert!(matches!(e, DirectoryError::InvalidName(_)));
     }
 
     {
@@ -180,12 +175,8 @@ fn validate_invalid_names() {
             }],
             ..Default::default()
         };
-        match d.validate().expect_err("must fail") {
-            ValidateDirectoryError::InvalidNode(n, ValidateNodeError::InvalidName(_)) => {
-                assert_eq!(n, b".")
-            }
-            _ => panic!("unexpected error"),
-        };
+        let e = crate::Directory::try_from(d).expect_err("must fail");
+        assert!(matches!(e, DirectoryError::InvalidName(_)));
     }
 
     {
@@ -198,12 +189,8 @@ fn validate_invalid_names() {
             }],
             ..Default::default()
         };
-        match d.validate().expect_err("must fail") {
-            ValidateDirectoryError::InvalidNode(n, ValidateNodeError::InvalidName(_)) => {
-                assert_eq!(n, b"..")
-            }
-            _ => panic!("unexpected error"),
-        };
+        let e = crate::Directory::try_from(d).expect_err("must fail");
+        assert!(matches!(e, DirectoryError::InvalidName(_)));
     }
 
     {
@@ -214,12 +201,8 @@ fn validate_invalid_names() {
             }],
             ..Default::default()
         };
-        match d.validate().expect_err("must fail") {
-            ValidateDirectoryError::InvalidNode(n, ValidateNodeError::InvalidName(_)) => {
-                assert_eq!(n, b"\x00")
-            }
-            _ => panic!("unexpected error"),
-        };
+        let e = crate::Directory::try_from(d).expect_err("must fail");
+        assert!(matches!(e, DirectoryError::InvalidName(_)));
     }
 
     {
@@ -230,12 +213,20 @@ fn validate_invalid_names() {
             }],
             ..Default::default()
         };
-        match d.validate().expect_err("must fail") {
-            ValidateDirectoryError::InvalidNode(n, ValidateNodeError::InvalidName(_)) => {
-                assert_eq!(n, b"foo/bar")
-            }
-            _ => panic!("unexpected error"),
+        let e = crate::Directory::try_from(d).expect_err("must fail");
+        assert!(matches!(e, DirectoryError::InvalidName(_)));
+    }
+
+    {
+        let d = Directory {
+            symlinks: vec![SymlinkNode {
+                name: bytes::Bytes::copy_from_slice("X".repeat(500).into_bytes().as_slice()),
+                target: "foo".into(),
+            }],
+            ..Default::default()
         };
+        let e = crate::Directory::try_from(d).expect_err("must fail");
+        assert!(matches!(e, DirectoryError::InvalidName(_)));
     }
 }
 
@@ -249,8 +240,8 @@ fn validate_invalid_digest() {
         }],
         ..Default::default()
     };
-    match d.validate().expect_err("must fail") {
-        ValidateDirectoryError::InvalidNode(_, ValidateNodeError::InvalidDigestLen(n)) => {
+    match crate::Directory::try_from(d).expect_err("must fail") {
+        DirectoryError::InvalidNode(_, ValidateNodeError::InvalidDigestLen(n)) => {
             assert_eq!(n, 2)
         }
         _ => panic!("unexpected error"),
@@ -276,15 +267,15 @@ fn validate_sorting() {
             ],
             ..Default::default()
         };
-        match d.validate().expect_err("must fail") {
-            ValidateDirectoryError::WrongSorting(s) => {
-                assert_eq!(s, b"a");
+        match crate::Directory::try_from(d).expect_err("must fail") {
+            DirectoryError::WrongSorting(s) => {
+                assert_eq!(s.as_ref(), b"a");
             }
             _ => panic!("unexpected error"),
         }
     }
 
-    // "a" exists twice, bad.
+    // "a" exists twice (same types), bad.
     {
         let d = Directory {
             directories: vec![
@@ -301,9 +292,31 @@ fn validate_sorting() {
             ],
             ..Default::default()
         };
-        match d.validate().expect_err("must fail") {
-            ValidateDirectoryError::DuplicateName(s) => {
-                assert_eq!(s, b"a");
+        match crate::Directory::try_from(d).expect_err("must fail") {
+            DirectoryError::DuplicateName(s) => {
+                assert_eq!(s.as_ref(), b"a");
+            }
+            _ => panic!("unexpected error"),
+        }
+    }
+
+    // "a" exists twice (different types), bad.
+    {
+        let d = Directory {
+            directories: vec![DirectoryNode {
+                name: "a".into(),
+                digest: DUMMY_DIGEST.to_vec().into(),
+                size: 42,
+            }],
+            symlinks: vec![SymlinkNode {
+                name: "a".into(),
+                target: "b".into(),
+            }],
+            ..Default::default()
+        };
+        match crate::Directory::try_from(d).expect_err("must fail") {
+            DirectoryError::DuplicateName(s) => {
+                assert_eq!(s.as_ref(), b"a");
             }
             _ => panic!("unexpected error"),
         }
@@ -327,7 +340,7 @@ fn validate_sorting() {
             ..Default::default()
         };
 
-        d.validate().expect("validate shouldn't error");
+        crate::Directory::try_from(d).expect("validate shouldn't error");
     }
 
     // [b, c] and [a] are both properly sorted.
@@ -352,101 +365,6 @@ fn validate_sorting() {
             ..Default::default()
         };
 
-        d.validate().expect("validate shouldn't error");
+        crate::Directory::try_from(d).expect("validate shouldn't error");
     }
 }
-
-#[test]
-fn validate_overflow() {
-    let d = Directory {
-        directories: vec![DirectoryNode {
-            name: "foo".into(),
-            digest: DUMMY_DIGEST.to_vec().into(),
-            size: u64::MAX,
-        }],
-        ..Default::default()
-    };
-
-    match d.validate().expect_err("must fail") {
-        ValidateDirectoryError::SizeOverflow => {}
-        _ => panic!("unexpected error"),
-    }
-}
-
-#[test]
-fn add_nodes_to_directory() {
-    let mut d = Directory {
-        ..Default::default()
-    };
-
-    d.add(node::Node::Directory(DirectoryNode {
-        name: "b".into(),
-        digest: DUMMY_DIGEST.to_vec().into(),
-        size: 1,
-    }));
-    d.add(node::Node::Directory(DirectoryNode {
-        name: "a".into(),
-        digest: DUMMY_DIGEST.to_vec().into(),
-        size: 1,
-    }));
-    d.add(node::Node::Directory(DirectoryNode {
-        name: "z".into(),
-        digest: DUMMY_DIGEST.to_vec().into(),
-        size: 1,
-    }));
-
-    d.add(node::Node::File(FileNode {
-        name: "f".into(),
-        digest: DUMMY_DIGEST.to_vec().into(),
-        size: 1,
-        executable: true,
-    }));
-    d.add(node::Node::File(FileNode {
-        name: "c".into(),
-        digest: DUMMY_DIGEST.to_vec().into(),
-        size: 1,
-        executable: true,
-    }));
-    d.add(node::Node::File(FileNode {
-        name: "g".into(),
-        digest: DUMMY_DIGEST.to_vec().into(),
-        size: 1,
-        executable: true,
-    }));
-
-    d.add(node::Node::Symlink(SymlinkNode {
-        name: "t".into(),
-        target: "a".into(),
-    }));
-    d.add(node::Node::Symlink(SymlinkNode {
-        name: "o".into(),
-        target: "a".into(),
-    }));
-    d.add(node::Node::Symlink(SymlinkNode {
-        name: "e".into(),
-        target: "a".into(),
-    }));
-
-    d.validate().expect("directory should be valid");
-}
-
-#[test]
-#[cfg_attr(not(debug_assertions), ignore)]
-#[should_panic = "name already exists in directories"]
-fn add_duplicate_node_to_directory_panics() {
-    let mut d = Directory {
-        ..Default::default()
-    };
-
-    d.add(node::Node::Directory(DirectoryNode {
-        name: "a".into(),
-        digest: DUMMY_DIGEST.to_vec().into(),
-        size: 1,
-    }));
-    d.add(node::Node::File(FileNode {
-        name: "a".into(),
-        digest: DUMMY_DIGEST.to_vec().into(),
-        size: 1,
-        executable: true,
-    }));
-}
diff --git a/tvix/castore/src/proto/tests/directory_nodes_iterator.rs b/tvix/castore/src/proto/tests/directory_nodes_iterator.rs
deleted file mode 100644
index 68f147a33210..000000000000
--- a/tvix/castore/src/proto/tests/directory_nodes_iterator.rs
+++ /dev/null
@@ -1,78 +0,0 @@
-use crate::proto::Directory;
-use crate::proto::DirectoryNode;
-use crate::proto::FileNode;
-use crate::proto::NamedNode;
-use crate::proto::SymlinkNode;
-
-#[test]
-fn iterator() {
-    let d = Directory {
-        directories: vec![
-            DirectoryNode {
-                name: "c".into(),
-                ..DirectoryNode::default()
-            },
-            DirectoryNode {
-                name: "d".into(),
-                ..DirectoryNode::default()
-            },
-            DirectoryNode {
-                name: "h".into(),
-                ..DirectoryNode::default()
-            },
-            DirectoryNode {
-                name: "l".into(),
-                ..DirectoryNode::default()
-            },
-        ],
-        files: vec![
-            FileNode {
-                name: "b".into(),
-                ..FileNode::default()
-            },
-            FileNode {
-                name: "e".into(),
-                ..FileNode::default()
-            },
-            FileNode {
-                name: "g".into(),
-                ..FileNode::default()
-            },
-            FileNode {
-                name: "j".into(),
-                ..FileNode::default()
-            },
-        ],
-        symlinks: vec![
-            SymlinkNode {
-                name: "a".into(),
-                ..SymlinkNode::default()
-            },
-            SymlinkNode {
-                name: "f".into(),
-                ..SymlinkNode::default()
-            },
-            SymlinkNode {
-                name: "i".into(),
-                ..SymlinkNode::default()
-            },
-            SymlinkNode {
-                name: "k".into(),
-                ..SymlinkNode::default()
-            },
-        ],
-    };
-
-    // We keep this strings here and convert to string to make the comparison
-    // less messy.
-    let mut node_names: Vec<String> = vec![];
-
-    for node in d.nodes() {
-        node_names.push(String::from_utf8(node.get_name().to_vec()).unwrap());
-    }
-
-    assert_eq!(
-        vec!["a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l"],
-        node_names
-    );
-}
diff --git a/tvix/castore/src/proto/tests/mod.rs b/tvix/castore/src/proto/tests/mod.rs
index 8d903bacb6c5..74334029e84c 100644
--- a/tvix/castore/src/proto/tests/mod.rs
+++ b/tvix/castore/src/proto/tests/mod.rs
@@ -1,2 +1 @@
 mod directory;
-mod directory_nodes_iterator;
diff --git a/tvix/castore/src/tests/import.rs b/tvix/castore/src/tests/import.rs
index 8b3bd5ce0ffc..32c2c363689f 100644
--- a/tvix/castore/src/tests/import.rs
+++ b/tvix/castore/src/tests/import.rs
@@ -2,15 +2,11 @@ use crate::blobservice::{self, BlobService};
 use crate::directoryservice;
 use crate::fixtures::*;
 use crate::import::fs::ingest_path;
-use crate::proto;
+use crate::Node;
 
-use std::sync::Arc;
 use tempfile::TempDir;
 
 #[cfg(target_family = "unix")]
-use std::os::unix::ffi::OsStrExt;
-
-#[cfg(target_family = "unix")]
 #[tokio::test]
 async fn symlink() {
     let blob_service = blobservice::from_addr("memory://").await.unwrap();
@@ -26,7 +22,7 @@ async fn symlink() {
     .unwrap();
 
     let root_node = ingest_path(
-        Arc::from(blob_service),
+        blob_service,
         directory_service,
         tmpdir.path().join("doesntmatter"),
     )
@@ -34,18 +30,16 @@ async fn symlink() {
     .expect("must succeed");
 
     assert_eq!(
-        proto::node::Node::Symlink(proto::SymlinkNode {
-            name: "doesntmatter".into(),
-            target: "/nix/store/somewhereelse".into(),
-        }),
+        Node::Symlink {
+            target: "/nix/store/somewhereelse".try_into().unwrap()
+        },
         root_node,
     )
 }
 
 #[tokio::test]
 async fn single_file() {
-    let blob_service =
-        Arc::from(blobservice::from_addr("memory://").await.unwrap()) as Arc<dyn BlobService>;
+    let blob_service = blobservice::from_addr("memory://").await.unwrap();
     let directory_service = directoryservice::from_addr("memory://").await.unwrap();
 
     let tmpdir = TempDir::new().unwrap();
@@ -61,12 +55,11 @@ async fn single_file() {
     .expect("must succeed");
 
     assert_eq!(
-        proto::node::Node::File(proto::FileNode {
-            name: "root".into(),
-            digest: HELLOWORLD_BLOB_DIGEST.clone().into(),
+        Node::File {
+            digest: HELLOWORLD_BLOB_DIGEST.clone(),
             size: HELLOWORLD_BLOB_CONTENTS.len() as u64,
             executable: false,
-        }),
+        },
         root_node,
     );
 
@@ -77,8 +70,7 @@ async fn single_file() {
 #[cfg(target_family = "unix")]
 #[tokio::test]
 async fn complicated() {
-    let blob_service =
-        Arc::from(blobservice::from_addr("memory://").await.unwrap()) as Arc<dyn BlobService>;
+    let blob_service = blobservice::from_addr("memory://").await.unwrap();
     let directory_service = directoryservice::from_addr("memory://").await.unwrap();
 
     let tmpdir = TempDir::new().unwrap();
@@ -98,17 +90,10 @@ async fn complicated() {
 
     // ensure root_node matched expectations
     assert_eq!(
-        proto::node::Node::Directory(proto::DirectoryNode {
-            name: tmpdir
-                .path()
-                .file_name()
-                .unwrap()
-                .as_bytes()
-                .to_owned()
-                .into(),
-            digest: DIRECTORY_COMPLICATED.digest().into(),
+        Node::Directory {
+            digest: DIRECTORY_COMPLICATED.digest().clone(),
             size: DIRECTORY_COMPLICATED.size(),
-        }),
+        },
         root_node,
     );
 
diff --git a/tvix/castore/src/tonic.rs b/tvix/castore/src/tonic.rs
index 4b65d6b028ef..e63e1ad7aab8 100644
--- a/tvix/castore/src/tonic.rs
+++ b/tvix/castore/src/tonic.rs
@@ -1,3 +1,4 @@
+use hyper_util::rt::TokioIo;
 use tokio::net::UnixStream;
 use tonic::transport::{Channel, Endpoint};
 
@@ -25,7 +26,10 @@ pub async fn channel_from_url(url: &url::Url) -> Result<Channel, self::Error> {
 
             let connector = tower::service_fn({
                 let url = url.clone();
-                move |_: tonic::transport::Uri| UnixStream::connect(url.path().to_string().clone())
+                move |_: tonic::transport::Uri| {
+                    let unix = UnixStream::connect(url.path().to_string().clone());
+                    async move { Ok::<_, std::io::Error>(TokioIo::new(unix.await?)) }
+                }
             });
 
             // the URL doesn't matter