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/chunked_reader.rs496
-rw-r--r--tvix/castore/src/blobservice/combinator.rs128
-rw-r--r--tvix/castore/src/blobservice/from_addr.rs88
-rw-r--r--tvix/castore/src/blobservice/grpc.rs388
-rw-r--r--tvix/castore/src/blobservice/memory.rs155
-rw-r--r--tvix/castore/src/blobservice/mod.rs112
-rw-r--r--tvix/castore/src/blobservice/object_store.rs617
-rw-r--r--tvix/castore/src/blobservice/tests/mod.rs253
-rw-r--r--tvix/castore/src/blobservice/tests/utils.rs42
-rw-r--r--tvix/castore/src/composition.rs541
-rw-r--r--tvix/castore/src/digests.rs97
-rw-r--r--tvix/castore/src/directoryservice/bigtable.rs388
-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.rs136
-rw-r--r--tvix/castore/src/directoryservice/grpc.rs386
-rw-r--r--tvix/castore/src/directoryservice/memory.rs101
-rw-r--r--tvix/castore/src/directoryservice/mod.rs147
-rw-r--r--tvix/castore/src/directoryservice/object_store.rs327
-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.rs80
-rw-r--r--tvix/castore/src/directoryservice/sled.rs263
-rw-r--r--tvix/castore/src/directoryservice/tests/mod.rs238
-rw-r--r--tvix/castore/src/directoryservice/tests/utils.rs47
-rw-r--r--tvix/castore/src/directoryservice/traverse.rs180
-rw-r--r--tvix/castore/src/directoryservice/utils.rs75
-rw-r--r--tvix/castore/src/errors.rs138
-rw-r--r--tvix/castore/src/fixtures.rs104
-rw-r--r--tvix/castore/src/fs/file_attr.rs29
-rw-r--r--tvix/castore/src/fs/fuse/mod.rs137
-rw-r--r--tvix/castore/src/fs/fuse/tests.rs1236
-rw-r--r--tvix/castore/src/fs/inode_tracker.rs207
-rw-r--r--tvix/castore/src/fs/inodes.rs89
-rw-r--r--tvix/castore/src/fs/mod.rs881
-rw-r--r--tvix/castore/src/fs/root_nodes.rs39
-rw-r--r--tvix/castore/src/fs/virtiofs.rs238
-rw-r--r--tvix/castore/src/hashing_reader.rs89
-rw-r--r--tvix/castore/src/import/archive.rs373
-rw-r--r--tvix/castore/src/import/blobs.rs190
-rw-r--r--tvix/castore/src/import/error.rs20
-rw-r--r--tvix/castore/src/import/fs.rs216
-rw-r--r--tvix/castore/src/import/mod.rs338
-rw-r--r--tvix/castore/src/lib.rs34
-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.rs470
-rw-r--r--tvix/castore/src/proto/grpc_blobservice_wrapper.rs175
-rw-r--r--tvix/castore/src/proto/grpc_directoryservice_wrapper.rs113
-rw-r--r--tvix/castore/src/proto/mod.rs288
-rw-r--r--tvix/castore/src/proto/tests/directory.rs370
-rw-r--r--tvix/castore/src/proto/tests/mod.rs1
-rw-r--r--tvix/castore/src/tests/import.rs114
-rw-r--r--tvix/castore/src/tests/mod.rs1
-rw-r--r--tvix/castore/src/tonic.rs126
57 files changed, 13212 insertions, 0 deletions
diff --git a/tvix/castore/src/blobservice/chunked_reader.rs b/tvix/castore/src/blobservice/chunked_reader.rs
new file mode 100644
index 000000000000..6e8355874bca
--- /dev/null
+++ b/tvix/castore/src/blobservice/chunked_reader.rs
@@ -0,0 +1,496 @@
+use futures::{ready, TryStreamExt};
+use pin_project_lite::pin_project;
+use tokio::io::{AsyncRead, AsyncSeekExt};
+use tokio_stream::StreamExt;
+use tokio_util::io::{ReaderStream, StreamReader};
+use tracing::{instrument, trace, warn};
+
+use crate::B3Digest;
+use std::{cmp::Ordering, pin::Pin};
+
+use super::{BlobReader, BlobService};
+
+pin_project! {
+    /// ChunkedReader provides a chunk-aware [BlobReader], so allows reading and
+    /// seeking into a blob.
+    /// It internally holds a [ChunkedBlob], which is storing chunk information
+    /// able to emit a reader seeked to a specific position whenever we need to seek.
+    pub struct ChunkedReader<BS> {
+        chunked_blob: ChunkedBlob<BS>,
+
+        #[pin]
+        r: Box<dyn AsyncRead + Unpin + Send>,
+
+        pos: u64,
+    }
+}
+
+impl<BS> ChunkedReader<BS>
+where
+    BS: AsRef<dyn BlobService> + Clone + 'static + Send,
+{
+    /// Construct a new [ChunkedReader], by retrieving a list of chunks (their
+    /// blake3 digests and chunk sizes)
+    pub fn from_chunks(chunks_it: impl Iterator<Item = (B3Digest, u64)>, blob_service: BS) -> Self {
+        let chunked_blob = ChunkedBlob::from_iter(chunks_it, blob_service);
+        let r = chunked_blob.reader_skipped_offset(0);
+
+        Self {
+            chunked_blob,
+            r,
+            pos: 0,
+        }
+    }
+}
+
+/// ChunkedReader implements BlobReader.
+impl<BS> BlobReader for ChunkedReader<BS> where BS: Send + Clone + 'static + AsRef<dyn BlobService> {}
+
+impl<BS> tokio::io::AsyncRead for ChunkedReader<BS>
+where
+    BS: AsRef<dyn BlobService> + Clone + 'static,
+{
+    fn poll_read(
+        self: std::pin::Pin<&mut Self>,
+        cx: &mut std::task::Context<'_>,
+        buf: &mut tokio::io::ReadBuf<'_>,
+    ) -> std::task::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;
+
+        Ok(()).into()
+    }
+}
+
+impl<BS> tokio::io::AsyncSeek for ChunkedReader<BS>
+where
+    BS: AsRef<dyn BlobService> + Clone + Send + 'static,
+{
+    #[instrument(skip(self), err(Debug))]
+    fn start_seek(self: Pin<&mut Self>, position: std::io::SeekFrom) -> std::io::Result<()> {
+        let total_len = self.chunked_blob.blob_length();
+        let mut this = self.project();
+
+        let absolute_offset: u64 = match position {
+            std::io::SeekFrom::Start(from_start) => from_start,
+            std::io::SeekFrom::End(from_end) => {
+                // note from_end is i64, not u64, so this is usually negative.
+                total_len.checked_add_signed(from_end).ok_or_else(|| {
+                    std::io::Error::new(
+                        std::io::ErrorKind::InvalidInput,
+                        "over/underflow while seeking",
+                    )
+                })?
+            }
+            std::io::SeekFrom::Current(from_current) => {
+                // note from_end is i64, not u64, so this can be positive or negative.
+                (*this.pos)
+                    .checked_add_signed(from_current)
+                    .ok_or_else(|| {
+                        std::io::Error::new(
+                            std::io::ErrorKind::InvalidInput,
+                            "over/underflow while seeking",
+                        )
+                    })?
+            }
+        };
+
+        // check if the position actually did change.
+        if absolute_offset != *this.pos {
+            // ensure the new position still is inside the file.
+            if absolute_offset > total_len {
+                Err(std::io::Error::new(
+                    std::io::ErrorKind::InvalidInput,
+                    "seeked beyond EOF",
+                ))?
+            }
+
+            // Update the position and the internal reader.
+            *this.pos = absolute_offset;
+
+            // FUTUREWORK: if we can seek forward, avoid re-assembling.
+            // At least if it's still in the same chunk?
+            *this.r = this.chunked_blob.reader_skipped_offset(absolute_offset);
+        }
+
+        Ok(())
+    }
+
+    fn poll_complete(
+        self: Pin<&mut Self>,
+        _cx: &mut std::task::Context<'_>,
+    ) -> std::task::Poll<std::io::Result<u64>> {
+        std::task::Poll::Ready(Ok(self.pos))
+    }
+}
+
+/// Holds a list of blake3 digest for individual chunks (and their sizes).
+/// Is able to construct a Reader that seeked to a certain offset, which
+/// is useful to construct a BlobReader (that implements AsyncSeek).
+/// - the current chunk index, and a Custor<Vec<u8>> holding the data of that chunk.
+struct ChunkedBlob<BS> {
+    blob_service: BS,
+    chunks: Vec<(u64, u64, B3Digest)>,
+}
+
+impl<BS> ChunkedBlob<BS>
+where
+    BS: AsRef<dyn BlobService> + Clone + 'static + Send,
+{
+    /// Constructs [Self] from a list of blake3 digests of chunks and their
+    /// sizes, and a reference to a blob service.
+    /// Initializing it with an empty list is disallowed.
+    fn from_iter(chunks_it: impl Iterator<Item = (B3Digest, u64)>, blob_service: BS) -> Self {
+        let mut chunks = Vec::new();
+        let mut offset: u64 = 0;
+
+        for (chunk_digest, chunk_size) in chunks_it {
+            chunks.push((offset, chunk_size, chunk_digest));
+            offset += chunk_size;
+        }
+
+        assert!(
+            !chunks.is_empty(),
+            "Chunks must be provided, don't use this for blobs without chunks"
+        );
+
+        Self {
+            blob_service,
+            chunks,
+        }
+    }
+
+    /// Returns the length of the blob.
+    fn blob_length(&self) -> u64 {
+        self.chunks
+            .last()
+            .map(|(chunk_offset, chunk_size, _)| chunk_offset + chunk_size)
+            .unwrap_or(0)
+    }
+
+    /// For a given position pos, return the chunk containing the data.
+    /// In case this would range outside the blob, None is returned.
+    #[instrument(level = "trace", skip(self), ret)]
+    fn get_chunk_idx_for_position(&self, pos: u64) -> Option<usize> {
+        // FUTUREWORK: benchmark when to use linear search, binary_search and BTreeSet
+        self.chunks
+            .binary_search_by(|(chunk_start_pos, chunk_size, _)| {
+                if chunk_start_pos + chunk_size <= pos {
+                    Ordering::Less
+                } else if *chunk_start_pos > pos {
+                    Ordering::Greater
+                } else {
+                    Ordering::Equal
+                }
+            })
+            .ok()
+    }
+
+    /// Returns a stream of bytes of the data in that blob.
+    /// It internally assembles a stream reading from each chunk (skipping over
+    /// chunks containing irrelevant data).
+    /// From the first relevant chunk, the irrelevant bytes are skipped too.
+    /// The returned boxed thing does not implement AsyncSeek on its own, but
+    /// ChunkedReader does.
+    #[instrument(level = "trace", skip(self))]
+    fn reader_skipped_offset(&self, offset: u64) -> Box<dyn tokio::io::AsyncRead + Send + Unpin> {
+        if offset == self.blob_length() {
+            return Box::new(std::io::Cursor::new(vec![]));
+        }
+        // construct a stream of all chunks starting with the given offset
+        let start_chunk_idx = self
+            .get_chunk_idx_for_position(offset)
+            .expect("outside of blob");
+        // It's ok to panic here, we can only reach this by seeking, and seeking should already reject out-of-file seeking.
+
+        let skip_first_chunk_bytes = (offset - self.chunks[start_chunk_idx].0) as usize;
+
+        let blob_service = self.blob_service.clone();
+        let chunks: Vec<_> = self.chunks[start_chunk_idx..].to_vec();
+        let readers_stream = tokio_stream::iter(chunks.into_iter().enumerate()).map(
+            move |(nth_chunk, (_chunk_start_offset, chunk_size, chunk_digest))| {
+                let chunk_digest = chunk_digest.to_owned();
+                let blob_service = blob_service.clone();
+                async move {
+                    trace!(chunk_size=%chunk_size, chunk_digest=%chunk_digest, "open_read on chunk in stream");
+                    let mut blob_reader = blob_service
+                        .as_ref()
+                        .open_read(&chunk_digest.to_owned())
+                        .await?
+                        .ok_or_else(|| {
+                            warn!(chunk.digest = %chunk_digest, "chunk not found");
+                            std::io::Error::new(std::io::ErrorKind::NotFound, "chunk not found")
+                        })?;
+
+                    // iff this is the first chunk in the stream, skip by skip_first_chunk_bytes
+                    if nth_chunk == 0 && skip_first_chunk_bytes > 0 {
+                        blob_reader
+                            .seek(std::io::SeekFrom::Start(skip_first_chunk_bytes as u64))
+                            .await?;
+                    }
+                    Ok::<_, std::io::Error>(blob_reader)
+                }
+            },
+        );
+
+        // 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();
+
+        // convert into AsyncRead
+        Box::new(StreamReader::new(Box::pin(bytes_stream)))
+    }
+}
+
+#[cfg(test)]
+mod test {
+    use std::{io::SeekFrom, sync::Arc};
+
+    use crate::{
+        blobservice::{chunked_reader::ChunkedReader, BlobService, MemoryBlobService},
+        B3Digest,
+    };
+    use hex_literal::hex;
+    use lazy_static::lazy_static;
+    use tokio::io::{AsyncReadExt, AsyncSeekExt};
+
+    const CHUNK_1: [u8; 2] = hex!("0001");
+    const CHUNK_2: [u8; 4] = hex!("02030405");
+    const CHUNK_3: [u8; 1] = hex!("06");
+    const CHUNK_4: [u8; 2] = hex!("0708");
+    const CHUNK_5: [u8; 7] = hex!("090a0b0c0d0e0f");
+
+    lazy_static! {
+        // `[ 0 1 ] [ 2 3 4 5 ] [ 6 ] [ 7 8 ] [ 9 10 11 12 13 14 15 ]`
+        pub static ref CHUNK_1_DIGEST: B3Digest = blake3::hash(&CHUNK_1).as_bytes().into();
+        pub static ref CHUNK_2_DIGEST: B3Digest = blake3::hash(&CHUNK_2).as_bytes().into();
+        pub static ref CHUNK_3_DIGEST: B3Digest = blake3::hash(&CHUNK_3).as_bytes().into();
+        pub static ref CHUNK_4_DIGEST: B3Digest = blake3::hash(&CHUNK_4).as_bytes().into();
+        pub static ref CHUNK_5_DIGEST: B3Digest = blake3::hash(&CHUNK_5).as_bytes().into();
+        pub static ref BLOB_1_LIST: [(B3Digest, u64); 5] = [
+            (CHUNK_1_DIGEST.clone(), 2),
+            (CHUNK_2_DIGEST.clone(), 4),
+            (CHUNK_3_DIGEST.clone(), 1),
+            (CHUNK_4_DIGEST.clone(), 2),
+            (CHUNK_5_DIGEST.clone(), 7),
+        ];
+    }
+
+    use super::ChunkedBlob;
+
+    /// ensure the start offsets are properly calculated.
+    #[test]
+    fn from_iter() {
+        let cb = ChunkedBlob::from_iter(
+            BLOB_1_LIST.clone().into_iter(),
+            Arc::new(MemoryBlobService::default()) as Arc<dyn BlobService>,
+        );
+
+        assert_eq!(
+            cb.chunks,
+            Vec::from_iter([
+                (0, 2, CHUNK_1_DIGEST.clone()),
+                (2, 4, CHUNK_2_DIGEST.clone()),
+                (6, 1, CHUNK_3_DIGEST.clone()),
+                (7, 2, CHUNK_4_DIGEST.clone()),
+                (9, 7, CHUNK_5_DIGEST.clone()),
+            ])
+        );
+    }
+
+    /// ensure ChunkedBlob can't be used with an empty list of chunks
+    #[test]
+    #[should_panic]
+    fn from_iter_empty() {
+        ChunkedBlob::from_iter(
+            [].into_iter(),
+            Arc::new(MemoryBlobService::default()) as Arc<dyn BlobService>,
+        );
+    }
+
+    /// ensure the right chunk is selected
+    #[test]
+    fn chunk_idx_for_position() {
+        let cb = ChunkedBlob::from_iter(
+            BLOB_1_LIST.clone().into_iter(),
+            Arc::new(MemoryBlobService::default()) as Arc<dyn BlobService>,
+        );
+
+        assert_eq!(Some(0), cb.get_chunk_idx_for_position(0), "start of blob");
+
+        assert_eq!(
+            Some(0),
+            cb.get_chunk_idx_for_position(1),
+            "middle of first chunk"
+        );
+        assert_eq!(
+            Some(1),
+            cb.get_chunk_idx_for_position(2),
+            "beginning of second chunk"
+        );
+
+        assert_eq!(
+            Some(4),
+            cb.get_chunk_idx_for_position(15),
+            "right before the end of the blob"
+        );
+        assert_eq!(
+            None,
+            cb.get_chunk_idx_for_position(16),
+            "right outside the blob"
+        );
+        assert_eq!(
+            None,
+            cb.get_chunk_idx_for_position(100),
+            "way outside the blob"
+        );
+    }
+
+    /// returns a blobservice with all chunks in BLOB_1 present.
+    async fn gen_blobservice_blob1() -> Arc<dyn BlobService> {
+        let blob_service = Arc::new(MemoryBlobService::default()) as Arc<dyn BlobService>;
+
+        // seed blob service with all chunks
+        for blob_contents in [
+            CHUNK_1.to_vec(),
+            CHUNK_2.to_vec(),
+            CHUNK_3.to_vec(),
+            CHUNK_4.to_vec(),
+            CHUNK_5.to_vec(),
+        ] {
+            let mut bw = blob_service.open_write().await;
+            tokio::io::copy(&mut std::io::Cursor::new(blob_contents), &mut bw)
+                .await
+                .expect("writing blob");
+            bw.close().await.expect("close blobwriter");
+        }
+
+        blob_service
+    }
+
+    #[tokio::test]
+    async fn test_read() {
+        let blob_service = gen_blobservice_blob1().await;
+        let mut chunked_reader =
+            ChunkedReader::from_chunks(BLOB_1_LIST.clone().into_iter(), blob_service);
+
+        // read all data
+        let mut buf = Vec::new();
+        tokio::io::copy(&mut chunked_reader, &mut buf)
+            .await
+            .expect("copy");
+
+        assert_eq!(
+            hex!("000102030405060708090a0b0c0d0e0f").to_vec(),
+            buf,
+            "read data must match"
+        );
+    }
+
+    #[tokio::test]
+    async fn test_seek() {
+        let blob_service = gen_blobservice_blob1().await;
+        let mut chunked_reader =
+            ChunkedReader::from_chunks(BLOB_1_LIST.clone().into_iter(), blob_service);
+
+        // seek to the end
+        // expect to read 0 bytes
+        {
+            chunked_reader
+                .seek(SeekFrom::End(0))
+                .await
+                .expect("seek to end");
+
+            let mut buf = Vec::new();
+            chunked_reader
+                .read_to_end(&mut buf)
+                .await
+                .expect("read to end");
+
+            assert_eq!(hex!("").to_vec(), buf);
+        }
+
+        // seek one bytes before the end
+        {
+            chunked_reader.seek(SeekFrom::End(-1)).await.expect("seek");
+
+            let mut buf = Vec::new();
+            chunked_reader
+                .read_to_end(&mut buf)
+                .await
+                .expect("read to end");
+
+            assert_eq!(hex!("0f").to_vec(), buf);
+        }
+
+        // seek back three bytes, but using relative positioning
+        // read two bytes
+        {
+            chunked_reader
+                .seek(SeekFrom::Current(-3))
+                .await
+                .expect("seek");
+
+            let mut buf = [0b0; 2];
+            chunked_reader
+                .read_exact(&mut buf)
+                .await
+                .expect("read exact");
+
+            assert_eq!(hex!("0d0e"), buf);
+        }
+    }
+
+    // seeds a blob service with only the first two chunks, reads a bit in the
+    // front (which succeeds), but then tries to seek past and read more (which
+    // should fail).
+    #[tokio::test]
+    async fn test_read_missing_chunks() {
+        let blob_service = Arc::new(MemoryBlobService::default()) as Arc<dyn BlobService>;
+
+        for blob_contents in [CHUNK_1.to_vec(), CHUNK_2.to_vec()] {
+            let mut bw = blob_service.open_write().await;
+            tokio::io::copy(&mut std::io::Cursor::new(blob_contents), &mut bw)
+                .await
+                .expect("writing blob");
+
+            bw.close().await.expect("close blobwriter");
+        }
+
+        let mut chunked_reader =
+            ChunkedReader::from_chunks(BLOB_1_LIST.clone().into_iter(), blob_service);
+
+        // read a bit from the front (5 bytes out of 6 available)
+        let mut buf = [0b0; 5];
+        chunked_reader
+            .read_exact(&mut buf)
+            .await
+            .expect("read exact");
+
+        assert_eq!(hex!("0001020304"), buf);
+
+        // seek 2 bytes forward, into an area where we don't have chunks
+        chunked_reader
+            .seek(SeekFrom::Current(2))
+            .await
+            .expect("seek");
+
+        let mut buf = Vec::new();
+        chunked_reader
+            .read_to_end(&mut buf)
+            .await
+            .expect_err("must fail");
+
+        // FUTUREWORK: check semantics on errorkinds. Should this be InvalidData
+        // or NotFound?
+    }
+}
diff --git a/tvix/castore/src/blobservice/combinator.rs b/tvix/castore/src/blobservice/combinator.rs
new file mode 100644
index 000000000000..6a964c8a8440
--- /dev/null
+++ b/tvix/castore/src/blobservice/combinator.rs
@@ -0,0 +1,128 @@
+use std::sync::Arc;
+
+use tonic::async_trait;
+use tracing::instrument;
+
+use crate::composition::{CompositionContext, ServiceBuilder};
+use crate::{B3Digest, Error};
+
+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
+/// things are not present there, the remote BlobService is queried.
+/// In case the local blobservice doesn't have the blob, we ask the remote
+/// blobservice for chunks, and try to read each of these chunks from the local
+/// blobservice again, before falling back to the remote one.
+/// The remote BlobService is never written to.
+pub struct CombinedBlobService<BL, BR> {
+    local: BL,
+    remote: BR,
+}
+
+impl<BL, BR> Clone for CombinedBlobService<BL, BR>
+where
+    BL: Clone,
+    BR: Clone,
+{
+    fn clone(&self) -> Self {
+        Self {
+            local: self.local.clone(),
+            remote: self.remote.clone(),
+        }
+    }
+}
+
+#[async_trait]
+impl<BL, BR> BlobService for CombinedBlobService<BL, BR>
+where
+    BL: AsRef<dyn BlobService> + Clone + Send + Sync + 'static,
+    BR: AsRef<dyn BlobService> + Clone + Send + Sync + 'static,
+{
+    #[instrument(skip(self, digest), fields(blob.digest=%digest))]
+    async fn has(&self, digest: &B3Digest) -> std::io::Result<bool> {
+        Ok(self.local.as_ref().has(digest).await? || self.remote.as_ref().has(digest).await?)
+    }
+
+    #[instrument(skip(self, digest), fields(blob.digest=%digest), err)]
+    async fn open_read(&self, digest: &B3Digest) -> std::io::Result<Option<Box<dyn BlobReader>>> {
+        if self.local.as_ref().has(digest).await? {
+            // local store has the blob, so we can assume it also has all chunks.
+            self.local.as_ref().open_read(digest).await
+        } else {
+            // Local store doesn't have the blob.
+            // Ask the remote one for the list of chunks,
+            // and create a chunked reader that uses self.open_read() for
+            // individual chunks. There's a chance we already have some chunks
+            // locally, meaning we don't need to fetch them all from the remote
+            // BlobService.
+            match self.remote.as_ref().chunks(digest).await? {
+                // blob doesn't exist on the remote side either, nothing we can do.
+                None => Ok(None),
+                Some(remote_chunks) => {
+                    // if there's no more granular chunks, or the remote
+                    // blobservice doesn't support chunks, read the blob from
+                    // the remote blobservice directly.
+                    if remote_chunks.is_empty() {
+                        return self.remote.as_ref().open_read(digest).await;
+                    }
+                    // otherwise, a chunked reader, which will always try the
+                    // local backend first.
+
+                    let chunked_reader = ChunkedReader::from_chunks(
+                        remote_chunks.into_iter().map(|chunk| {
+                            (
+                                chunk.digest.try_into().expect("invalid b3 digest"),
+                                chunk.size,
+                            )
+                        }),
+                        Arc::new(self.clone()) as Arc<dyn BlobService>,
+                    );
+                    Ok(Some(Box::new(chunked_reader)))
+                }
+            }
+        }
+    }
+
+    #[instrument(skip_all)]
+    async fn open_write(&self) -> Box<dyn BlobWriter> {
+        // direct writes to the local one.
+        self.local.as_ref().open_write().await
+    }
+}
+
+#[derive(serde::Deserialize, Debug, Clone)]
+#[serde(deny_unknown_fields)]
+pub struct CombinedBlobServiceConfig {
+    local: String,
+    remote: String,
+}
+
+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())
+    }
+}
+
+#[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
new file mode 100644
index 000000000000..c5cabaa9d945
--- /dev/null
+++ b/tvix/castore/src/blobservice/from_addr.rs
@@ -0,0 +1,88 @@
+use std::sync::Arc;
+
+use url::Url;
+
+use crate::composition::{
+    with_registry, CompositionContext, DeserializeWithRegistry, ServiceBuilder, REG,
+};
+
+use super::BlobService;
+
+/// Constructs a new instance of a [BlobService] from an URI.
+///
+/// The following schemes are supported by the following services:
+/// - `memory://` ([MemoryBlobService])
+/// - `grpc+*://` ([GRPCBlobService])
+/// - `objectstore+*://` ([ObjectStoreBlobService])
+///
+/// See their `from_url` methods for more details about their syntax.
+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_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)
+}
+
+#[cfg(test)]
+mod tests {
+    use super::from_addr;
+    use rstest::rstest;
+
+    #[rstest]
+    /// This uses an unsupported scheme.
+    #[case::unsupported_scheme("http://foo.example/test", false)]
+    /// This correctly sets the scheme, and doesn't set a path.
+    #[case::memory_valid("memory://", true)]
+    /// This sets a memory url host to `foo`
+    #[case::memory_invalid_host("memory://foo", false)]
+    /// This sets a memory url path to "/", which is invalid.
+    #[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)]
+    /// 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.
+    #[case::grpc_invalid_unix_socket_and_host("grpc+unix://host.example/path/to/somewhere", false)]
+    /// Correct scheme to connect to localhost, with port 12345
+    #[case::grpc_valid_ipv6_localhost_port_12345("grpc+http://[::1]:12345", true)]
+    /// Correct scheme to connect to localhost over http, without specifying a port.
+    #[case::grpc_valid_http_host_without_port("grpc+http://localhost", true)]
+    /// Correct scheme to connect to localhost over http, without specifying a port.
+    #[case::grpc_valid_https_host_without_port("grpc+https://localhost", true)]
+    /// Correct scheme to connect to localhost over http, but with additional path, which is invalid.
+    #[case::grpc_invalid_has_path("grpc+http://localhost/some-path", false)]
+    /// An example for object store (InMemory)
+    #[case::objectstore_valid_memory("objectstore+memory:///", true)]
+    /// An example for object store (LocalFileSystem)
+    #[case::objectstore_valid_file("objectstore+file:///foo/bar", true)]
+    // An example for object store (HTTP / WebDAV)
+    #[case::objectstore_valid_http_url("objectstore+https://localhost:8080/some-path", true)]
+    /// An example for object store (S3)
+    #[cfg_attr(
+        feature = "cloud",
+        case::objectstore_valid_s3_url("objectstore+s3://bucket/path", true)
+    )]
+    /// An example for object store (GCS)
+    #[cfg_attr(
+        feature = "cloud",
+        case::objectstore_valid_gcs_url("objectstore+gs://bucket/path", true)
+    )]
+    #[tokio::test]
+    async fn test_from_addr_tokio(#[case] uri_str: &str, #[case] exp_succeed: bool) {
+        if exp_succeed {
+            from_addr(uri_str).await.expect("should succeed");
+        } else {
+            assert!(from_addr(uri_str).await.is_err(), "should fail");
+        }
+    }
+}
diff --git a/tvix/castore/src/blobservice/grpc.rs b/tvix/castore/src/blobservice/grpc.rs
new file mode 100644
index 000000000000..0db3dfea4ad8
--- /dev/null
+++ b/tvix/castore/src/blobservice/grpc.rs
@@ -0,0 +1,388 @@
+use super::{BlobReader, BlobService, BlobWriter, ChunkedReader};
+use crate::composition::{CompositionContext, ServiceBuilder};
+use crate::{
+    proto::{self, stat_blob_response::ChunkMeta},
+    B3Digest,
+};
+use futures::sink::SinkExt;
+use std::{
+    io::{self, Cursor},
+    pin::pin,
+    sync::Arc,
+    task::Poll,
+};
+use tokio::io::AsyncWriteExt;
+use tokio::task::JoinHandle;
+use tokio_stream::{wrappers::ReceiverStream, StreamExt};
+use tokio_util::{
+    io::{CopyToBytes, SinkWriter},
+    sync::PollSender,
+};
+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<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<T>,
+}
+
+impl<T> GRPCBlobService<T> {
+    /// construct a [GRPCBlobService] from a [proto::blob_service_client::BlobServiceClient].
+    pub fn from_client(grpc_client: proto::blob_service_client::BlobServiceClient<T>) -> Self {
+        Self { grpc_client }
+    }
+}
+
+#[async_trait]
+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> {
+        match self
+            .grpc_client
+            .clone()
+            .stat(proto::StatBlobRequest {
+                digest: digest.clone().into(),
+                ..Default::default()
+            })
+            .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)),
+        }
+    }
+
+    #[instrument(skip(self, digest), fields(blob.digest=%digest), err)]
+    async fn open_read(&self, digest: &B3Digest) -> io::Result<Option<Box<dyn BlobReader>>> {
+        // First try to get a list of chunks. In case there's only one chunk returned,
+        // buffer its data into a Vec, otherwise use a ChunkedReader.
+        // We previously used NaiveSeeker here, but userland likes to seek backwards too often,
+        // and without store composition this will get very noisy.
+        // FUTUREWORK: use CombinedBlobService and store composition.
+        match self.chunks(digest).await {
+            Ok(None) => Ok(None),
+            Ok(Some(chunks)) => {
+                if chunks.is_empty() || chunks.len() == 1 {
+                    // No more granular chunking info, treat this as an individual chunk.
+                    // Get a stream of [proto::BlobChunk], or return an error if the blob
+                    // doesn't exist.
+                    return match self
+                        .grpc_client
+                        .clone()
+                        .read(proto::ReadBlobRequest {
+                            digest: digest.clone().into(),
+                        })
+                        .await
+                    {
+                        Ok(stream) => {
+                            let data_stream = stream.into_inner().map(|e| {
+                                e.map(|c| c.data)
+                                    .map_err(|s| std::io::Error::new(io::ErrorKind::InvalidData, s))
+                            });
+
+                            // Use StreamReader::new to convert to an AsyncRead.
+                            let mut data_reader = tokio_util::io::StreamReader::new(data_stream);
+
+                            let mut buf = Vec::new();
+                            // TODO: only do this up to a certain limit.
+                            tokio::io::copy(&mut data_reader, &mut buf).await?;
+
+                            Ok(Some(Box::new(Cursor::new(buf))))
+                        }
+                        Err(e) if e.code() == Code::NotFound => Ok(None),
+                        Err(e) => Err(io::Error::new(io::ErrorKind::Other, e)),
+                    };
+                }
+
+                // The chunked case. Let ChunkedReader do individual reads.
+                // TODO: we should store the chunking data in some local cache,
+                // so `ChunkedReader` doesn't call `self.chunks` *again* for every chunk.
+                // Think about how store composition will fix this.
+                let chunked_reader = ChunkedReader::from_chunks(
+                    chunks.into_iter().map(|chunk| {
+                        (
+                            chunk.digest.try_into().expect("invalid b3 digest"),
+                            chunk.size,
+                        )
+                    }),
+                    Arc::new(self.clone()) as Arc<dyn BlobService>,
+                );
+                Ok(Some(Box::new(chunked_reader)))
+            }
+            Err(e) => Err(e)?,
+        }
+    }
+
+    /// Returns a BlobWriter, that'll internally wrap each write in a
+    /// [proto::BlobChunk], which is send to the gRPC server.
+    #[instrument(skip_all)]
+    async fn open_write(&self) -> Box<dyn BlobWriter> {
+        // set up an mpsc channel passing around Bytes.
+        let (tx, rx) = tokio::sync::mpsc::channel::<bytes::Bytes>(10);
+
+        // bytes arriving on the RX side are wrapped inside a
+        // [proto::BlobChunk], and a [ReceiverStream] is constructed.
+        let blobchunk_stream = ReceiverStream::new(rx).map(|x| proto::BlobChunk { data: x });
+
+        // spawn the gRPC put request, which will read from blobchunk_stream.
+        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.
+        let sink = PollSender::new(tx)
+            .sink_map_err(|e| std::io::Error::new(std::io::ErrorKind::BrokenPipe, e));
+
+        // โ€ฆ which is turned into an [tokio::io::AsyncWrite].
+        let writer = SinkWriter::new(CopyToBytes::new(sink));
+
+        Box::new(GRPCBlobWriter {
+            task_and_writer: Some((task, writer)),
+            digest: None,
+        })
+    }
+
+    #[instrument(skip(self, digest), fields(blob.digest=%digest), err)]
+    async fn chunks(&self, digest: &B3Digest) -> io::Result<Option<Vec<ChunkMeta>>> {
+        let resp = self
+            .grpc_client
+            .clone()
+            .stat(proto::StatBlobRequest {
+                digest: digest.clone().into(),
+                send_chunks: true,
+                ..Default::default()
+            })
+            .await;
+
+        match resp {
+            Err(e) if e.code() == Code::NotFound => Ok(None),
+            Err(e) => Err(io::Error::new(io::ErrorKind::Other, e)),
+            Ok(resp) => {
+                let resp = resp.into_inner();
+
+                resp.validate()
+                    .map_err(|e| std::io::Error::new(io::ErrorKind::InvalidData, e))?;
+
+                Ok(Some(resp.chunks))
+            }
+        }
+    }
+}
+
+#[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)>,
+
+    /// The digest that has been returned, if we successfully closed.
+    digest: Option<B3Digest>,
+}
+
+#[async_trait]
+impl<W: tokio::io::AsyncWrite + Send + Sync + Unpin + 'static> BlobWriter for GRPCBlobWriter<W> {
+    async fn close(&mut self) -> io::Result<B3Digest> {
+        if self.task_and_writer.is_none() {
+            // if we're already closed, return the b3 digest, which must exist.
+            // If it doesn't, we already closed and failed once, and didn't handle the error.
+            match &self.digest {
+                Some(digest) => Ok(digest.clone()),
+                None => Err(io::Error::new(io::ErrorKind::BrokenPipe, "already closed")),
+            }
+        } else {
+            let (task, mut writer) = self.task_and_writer.take().unwrap();
+
+            // invoke shutdown, so the inner writer closes its internal tx side of
+            // the channel.
+            writer.shutdown().await?;
+
+            // block on the RPC call to return.
+            // This ensures all chunks are sent out, and have been received by the
+            // backend.
+
+            match task.await? {
+                Ok(resp) => {
+                    // return the digest from the response, and store it in self.digest for subsequent closes.
+                    let digest_len = resp.digest.len();
+                    let digest: B3Digest = resp.digest.try_into().map_err(|_| {
+                        io::Error::new(
+                            io::ErrorKind::Other,
+                            format!("invalid root digest length {} in response", digest_len),
+                        )
+                    })?;
+                    self.digest = Some(digest.clone());
+                    Ok(digest)
+                }
+                Err(e) => Err(io::Error::new(io::ErrorKind::Other, e.to_string())),
+            }
+        }
+    }
+}
+
+impl<W: tokio::io::AsyncWrite + Unpin> tokio::io::AsyncWrite for GRPCBlobWriter<W> {
+    fn poll_write(
+        mut self: std::pin::Pin<&mut Self>,
+        cx: &mut std::task::Context<'_>,
+        buf: &[u8],
+    ) -> std::task::Poll<Result<usize, io::Error>> {
+        match &mut self.task_and_writer {
+            None => Poll::Ready(Err(io::Error::new(
+                io::ErrorKind::NotConnected,
+                "already closed",
+            ))),
+            Some((_, ref mut writer)) => {
+                let pinned_writer = pin!(writer);
+                pinned_writer.poll_write(cx, buf)
+            }
+        }
+    }
+
+    fn poll_flush(
+        mut self: std::pin::Pin<&mut Self>,
+        cx: &mut std::task::Context<'_>,
+    ) -> std::task::Poll<Result<(), io::Error>> {
+        match &mut self.task_and_writer {
+            None => Poll::Ready(Err(io::Error::new(
+                io::ErrorKind::NotConnected,
+                "already closed",
+            ))),
+            Some((_, ref mut writer)) => {
+                let pinned_writer = pin!(writer);
+                pinned_writer.poll_flush(cx)
+            }
+        }
+    }
+
+    fn poll_shutdown(
+        self: std::pin::Pin<&mut Self>,
+        _cx: &mut std::task::Context<'_>,
+    ) -> std::task::Poll<Result<(), io::Error>> {
+        // TODO(raitobezarius): this might not be a graceful shutdown of the
+        // channel inside the gRPC connection.
+        Poll::Ready(Ok(()))
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use std::time::Duration;
+
+    use tempfile::TempDir;
+    use tokio::net::UnixListener;
+    use tokio_retry::strategy::ExponentialBackoff;
+    use tokio_retry::Retry;
+    use tokio_stream::wrappers::UnixListenerStream;
+
+    use crate::blobservice::MemoryBlobService;
+    use crate::fixtures;
+    use crate::proto::blob_service_client::BlobServiceClient;
+    use crate::proto::GRPCBlobServiceWrapper;
+
+    use super::BlobService;
+    use super::GRPCBlobService;
+
+    /// This ensures connecting via gRPC works as expected.
+    #[tokio::test]
+    async fn test_valid_unix_path_ping_pong() {
+        let tmpdir = TempDir::new().unwrap();
+        let socket_path = tmpdir.path().join("daemon");
+
+        let path_clone = socket_path.clone();
+
+        // Spin up a server
+        tokio::spawn(async {
+            let uds = UnixListener::bind(path_clone).unwrap();
+            let uds_stream = UnixListenerStream::new(uds);
+
+            // spin up a new server
+            let mut server = tonic::transport::Server::builder();
+            let router =
+                server.add_service(crate::proto::blob_service_server::BlobServiceServer::new(
+                    GRPCBlobServiceWrapper::new(
+                        Box::<MemoryBlobService>::default() as Box<dyn BlobService>
+                    ),
+                ));
+            router.serve_with_incoming(uds_stream).await
+        });
+
+        // wait for the socket to be created
+        Retry::spawn(
+            ExponentialBackoff::from_millis(20).max_delay(Duration::from_secs(10)),
+            || async {
+                if socket_path.exists() {
+                    Ok(())
+                } else {
+                    Err(())
+                }
+            },
+        )
+        .await
+        .expect("failed to wait for socket");
+
+        // prepare a client
+        let grpc_client = {
+            let url = url::Url::parse(&format!(
+                "grpc+unix://{}?wait-connect=1",
+                socket_path.display()
+            ))
+            .expect("must parse");
+            let client = BlobServiceClient::new(
+                crate::tonic::channel_from_url(&url)
+                    .await
+                    .expect("must succeed"),
+            );
+            GRPCBlobService::from_client(client)
+        };
+
+        let has = grpc_client
+            .has(&fixtures::BLOB_A_DIGEST)
+            .await
+            .expect("must not be err");
+
+        assert!(!has);
+    }
+}
diff --git a/tvix/castore/src/blobservice/memory.rs b/tvix/castore/src/blobservice/memory.rs
new file mode 100644
index 000000000000..3d733f950470
--- /dev/null
+++ b/tvix/castore/src/blobservice/memory.rs
@@ -0,0 +1,155 @@
+use parking_lot::RwLock;
+use std::io::{self, Cursor, Write};
+use std::task::Poll;
+use std::{collections::HashMap, sync::Arc};
+use tonic::async_trait;
+use tracing::instrument;
+
+use super::{BlobReader, BlobService, BlobWriter};
+use crate::composition::{CompositionContext, ServiceBuilder};
+use crate::{B3Digest, Error};
+
+#[derive(Clone, Default)]
+pub struct MemoryBlobService {
+    db: Arc<RwLock<HashMap<B3Digest, Vec<u8>>>>,
+}
+
+#[async_trait]
+impl BlobService for MemoryBlobService {
+    #[instrument(skip_all, ret, err, fields(blob.digest=%digest))]
+    async fn has(&self, digest: &B3Digest) -> io::Result<bool> {
+        let db = self.db.read();
+        Ok(db.contains_key(digest))
+    }
+
+    #[instrument(skip_all, err, fields(blob.digest=%digest))]
+    async fn open_read(&self, digest: &B3Digest) -> io::Result<Option<Box<dyn BlobReader>>> {
+        let db = self.db.read();
+
+        match db.get(digest).map(|x| Cursor::new(x.clone())) {
+            Some(result) => Ok(Some(Box::new(result))),
+            None => Ok(None),
+        }
+    }
+
+    #[instrument(skip_all)]
+    async fn open_write(&self) -> Box<dyn BlobWriter> {
+        Box::new(MemoryBlobWriter::new(self.db.clone()))
+    }
+}
+
+#[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>>>>,
+
+    /// Contains the buffer Vec and hasher, or None if already closed
+    writers: Option<(Vec<u8>, blake3::Hasher)>,
+
+    /// The digest that has been returned, if we successfully closed.
+    digest: Option<B3Digest>,
+}
+
+impl MemoryBlobWriter {
+    fn new(db: Arc<RwLock<HashMap<B3Digest, Vec<u8>>>>) -> Self {
+        Self {
+            db,
+            writers: Some((Vec::new(), blake3::Hasher::new())),
+            digest: None,
+        }
+    }
+}
+impl tokio::io::AsyncWrite for MemoryBlobWriter {
+    fn poll_write(
+        mut self: std::pin::Pin<&mut Self>,
+        _cx: &mut std::task::Context<'_>,
+        b: &[u8],
+    ) -> std::task::Poll<Result<usize, io::Error>> {
+        Poll::Ready(match &mut self.writers {
+            None => Err(io::Error::new(
+                io::ErrorKind::NotConnected,
+                "already closed",
+            )),
+            Some((ref mut buf, ref mut hasher)) => {
+                let bytes_written = buf.write(b)?;
+                hasher.write(&b[..bytes_written])
+            }
+        })
+    }
+
+    fn poll_flush(
+        self: std::pin::Pin<&mut Self>,
+        _cx: &mut std::task::Context<'_>,
+    ) -> std::task::Poll<Result<(), io::Error>> {
+        Poll::Ready(match self.writers {
+            None => Err(io::Error::new(
+                io::ErrorKind::NotConnected,
+                "already closed",
+            )),
+            Some(_) => Ok(()),
+        })
+    }
+
+    fn poll_shutdown(
+        self: std::pin::Pin<&mut Self>,
+        _cx: &mut std::task::Context<'_>,
+    ) -> std::task::Poll<Result<(), io::Error>> {
+        // shutdown is "instantaneous", we only write to memory.
+        Poll::Ready(Ok(()))
+    }
+}
+
+#[async_trait]
+impl BlobWriter for MemoryBlobWriter {
+    async fn close(&mut self) -> io::Result<B3Digest> {
+        if self.writers.is_none() {
+            match &self.digest {
+                Some(digest) => Ok(digest.clone()),
+                None => Err(io::Error::new(io::ErrorKind::BrokenPipe, "already closed")),
+            }
+        } else {
+            let (buf, hasher) = self.writers.take().unwrap();
+
+            let digest: B3Digest = hasher.finalize().as_bytes().into();
+
+            // Only insert if the blob doesn't already exist.
+            let mut db = self.db.upgradable_read();
+            if !db.contains_key(&digest) {
+                // open the database for writing.
+                db.with_upgraded(|db| {
+                    // and put buf in there. This will move buf out.
+                    db.insert(digest.clone(), buf);
+                });
+            }
+
+            self.digest = Some(digest.clone());
+
+            Ok(digest)
+        }
+    }
+}
diff --git a/tvix/castore/src/blobservice/mod.rs b/tvix/castore/src/blobservice/mod.rs
new file mode 100644
index 000000000000..85292722fa7e
--- /dev/null
+++ b/tvix/castore/src/blobservice/mod.rs
@@ -0,0 +1,112 @@
+use std::io;
+
+use tonic::async_trait;
+
+use crate::composition::{Registry, ServiceBuilder};
+use crate::proto::stat_blob_response::ChunkMeta;
+use crate::B3Digest;
+
+mod chunked_reader;
+mod combinator;
+mod from_addr;
+mod grpc;
+mod memory;
+mod object_store;
+
+#[cfg(test)]
+pub mod tests;
+
+pub use self::chunked_reader::ChunkedReader;
+pub use self::combinator::{CombinedBlobService, CombinedBlobServiceConfig};
+pub use self::from_addr::from_addr;
+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,
+/// a way to read (and seek) a blob, and a method to create a blobwriter handle,
+/// which will implement a writer interface, and also provides a close funtion,
+/// to finalize a blob and get its digest.
+#[async_trait]
+pub trait BlobService: Send + Sync {
+    /// Check if the service has the blob, by its content hash.
+    /// On implementations returning chunks, this must also work for chunks.
+    async fn has(&self, digest: &B3Digest) -> io::Result<bool>;
+
+    /// Request a blob from the store, by its content hash.
+    /// On implementations returning chunks, this must also work for chunks.
+    async fn open_read(&self, digest: &B3Digest) -> io::Result<Option<Box<dyn BlobReader>>>;
+
+    /// Insert a new blob into the store. Returns a [BlobWriter], which
+    /// implements [tokio::io::AsyncWrite] and a [BlobWriter::close] to finalize
+    /// the blob and get its digest.
+    async fn open_write(&self) -> Box<dyn BlobWriter>;
+
+    /// Return a list of chunks for a given blob.
+    /// There's a distinction between returning Ok(None) and Ok(Some(vec![])).
+    /// The former return value is sent in case the blob is not present at all,
+    /// while the second one is sent in case there's no more granular chunks (or
+    /// the backend does not support chunking).
+    /// A default implementation checking for existence and then returning it
+    /// does not have more granular chunks available is provided.
+    async fn chunks(&self, digest: &B3Digest) -> io::Result<Option<Vec<ChunkMeta>>> {
+        if !self.has(digest).await? {
+            return Ok(None);
+        }
+        // default implementation, signalling the backend does not have more
+        // granular chunks available.
+        Ok(Some(vec![]))
+    }
+}
+
+#[async_trait]
+impl<A> BlobService for A
+where
+    A: AsRef<dyn BlobService> + Send + Sync,
+{
+    async fn has(&self, digest: &B3Digest) -> io::Result<bool> {
+        self.as_ref().has(digest).await
+    }
+
+    async fn open_read(&self, digest: &B3Digest) -> io::Result<Option<Box<dyn BlobReader>>> {
+        self.as_ref().open_read(digest).await
+    }
+
+    async fn open_write(&self) -> Box<dyn BlobWriter> {
+        self.as_ref().open_write().await
+    }
+
+    async fn chunks(&self, digest: &B3Digest) -> io::Result<Option<Vec<ChunkMeta>>> {
+        self.as_ref().chunks(digest).await
+    }
+}
+
+/// A [tokio::io::AsyncWrite] that the user needs to close() afterwards for persist.
+/// On success, it returns the digest of the written blob.
+#[async_trait]
+pub trait BlobWriter: tokio::io::AsyncWrite + Send + Unpin {
+    /// Signal there's no more data to be written, and return the digest of the
+    /// contents written.
+    ///
+    /// Closing a already-closed BlobWriter is a no-op.
+    async fn close(&mut self) -> io::Result<B3Digest>;
+}
+
+/// BlobReader is a [tokio::io::AsyncRead] that also allows seeking.
+pub trait BlobReader: tokio::io::AsyncRead + tokio::io::AsyncSeek + Send + Unpin + 'static {}
+
+/// A [`io::Cursor<Vec<u8>>`] can be used as a BlobReader.
+impl BlobReader for io::Cursor<&'static [u8]> {}
+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/object_store.rs b/tvix/castore/src/blobservice/object_store.rs
new file mode 100644
index 000000000000..5bb05cf26123
--- /dev/null
+++ b/tvix/castore/src/blobservice/object_store.rs
@@ -0,0 +1,617 @@
+use std::{
+    collections::HashMap,
+    io::{self, Cursor},
+    pin::pin,
+    sync::Arc,
+    task::Poll,
+};
+
+use data_encoding::HEXLOWER;
+use fastcdc::v2020::AsyncStreamCDC;
+use futures::Future;
+use object_store::{path::Path, ObjectStore};
+use pin_project_lite::pin_project;
+use prost::Message;
+use tokio::io::{AsyncRead, AsyncWrite, AsyncWriteExt};
+use tokio_stream::StreamExt;
+use tonic::async_trait;
+use tracing::{debug, instrument, trace, Level};
+use url::Url;
+
+use crate::{
+    composition::{CompositionContext, ServiceBuilder},
+    proto::{stat_blob_response::ChunkMeta, StatBlobResponse},
+    B3Digest, B3HashingReader, Error,
+};
+
+use super::{BlobReader, BlobService, BlobWriter, ChunkedReader};
+
+/// Uses any object storage supported by the [object_store] crate to provide a
+/// tvix-castore [BlobService].
+///
+/// # Data format
+/// Data is organized in "blobs" and "chunks".
+/// Blobs don't hold the actual data, but instead contain a list of more
+/// granular chunks that assemble to the contents requested.
+/// This allows clients to seek, and not download chunks they already have
+/// locally, as it's referred to from other files.
+/// Check `rpc_blobstore` and more general BlobStore docs on that.
+///
+/// ## Blobs
+/// Stored at `${base_path}/blobs/b3/$digest_key`. They contains the serialized
+/// StatBlobResponse for the blob with the digest.
+///
+/// ## Chunks
+/// Chunks are stored at `${base_path}/chunks/b3/$digest_key`. They contain
+/// the literal contents of the chunk, but are zstd-compressed.
+///
+/// ## Digest key sharding
+/// The blake3 digest encoded in lower hex, and sharded after the second
+/// character.
+/// The blob for "Hello World" is stored at
+/// `${base_path}/blobs/b3/41/41f8394111eb713a22165c46c90ab8f0fd9399c92028fd6d288944b23ff5bf76`.
+///
+/// This reduces the number of files in the same directory, which would be a
+/// problem at least when using [object_store::local::LocalFileSystem].
+///
+/// # Future changes
+/// There's no guarantees about this being a final format yet.
+/// Once object_store gets support for additional metadata / content-types,
+/// we can eliminate some requests (small blobs only consisting of a single
+/// chunk can be stored as-is, without the blob index file).
+/// 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 ;-)
+#[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,
+}
+
+#[instrument(level=Level::TRACE, skip_all,fields(base_path=%base_path,blob.digest=%digest),ret(Display))]
+fn derive_blob_path(base_path: &Path, digest: &B3Digest) -> Path {
+    base_path
+        .child("blobs")
+        .child("b3")
+        .child(HEXLOWER.encode(&digest.as_slice()[..2]))
+        .child(HEXLOWER.encode(digest.as_slice()))
+}
+
+#[instrument(level=Level::TRACE, skip_all,fields(base_path=%base_path,chunk.digest=%digest),ret(Display))]
+fn derive_chunk_path(base_path: &Path, digest: &B3Digest) -> Path {
+    base_path
+        .child("chunks")
+        .child("b3")
+        .child(HEXLOWER.encode(&digest.as_slice()[..2]))
+        .child(HEXLOWER.encode(digest.as_slice()))
+}
+
+#[async_trait]
+impl BlobService for ObjectStoreBlobService {
+    #[instrument(skip_all, ret, err, fields(blob.digest=%digest))]
+    async fn has(&self, digest: &B3Digest) -> io::Result<bool> {
+        // TODO: clarify if this should work for chunks or not, and explicitly
+        // document in the proto docs.
+        let p = derive_blob_path(&self.base_path, digest);
+
+        match self.object_store.head(&p).await {
+            Ok(_) => Ok(true),
+            Err(object_store::Error::NotFound { .. }) => {
+                let p = derive_chunk_path(&self.base_path, digest);
+                match self.object_store.head(&p).await {
+                    Ok(_) => Ok(true),
+                    Err(object_store::Error::NotFound { .. }) => Ok(false),
+                    Err(e) => Err(e)?,
+                }
+            }
+            Err(e) => Err(e)?,
+        }
+    }
+
+    #[instrument(skip_all, err, fields(blob.digest=%digest))]
+    async fn open_read(&self, digest: &B3Digest) -> io::Result<Option<Box<dyn BlobReader>>> {
+        // handle reading the empty blob.
+        if digest.as_slice() == blake3::hash(b"").as_bytes() {
+            return Ok(Some(Box::new(Cursor::new(b"")) as Box<dyn BlobReader>));
+        }
+        match self
+            .object_store
+            .get(&derive_chunk_path(&self.base_path, digest))
+            .await
+        {
+            Ok(res) => {
+                // handle reading blobs that are small enough to fit inside a single chunk:
+                // fetch the entire chunk into memory, decompress, ensure the b3 digest matches,
+                // and return a io::Cursor over that data.
+                // FUTUREWORK: use zstd::bulk to prevent decompression bombs
+
+                let chunk_raw_bytes = res.bytes().await?;
+                let chunk_contents = zstd::stream::decode_all(Cursor::new(chunk_raw_bytes))?;
+
+                if *digest != blake3::hash(&chunk_contents).as_bytes().into() {
+                    Err(io::Error::other("chunk contents invalid"))?;
+                }
+
+                Ok(Some(Box::new(Cursor::new(chunk_contents))))
+            }
+            Err(object_store::Error::NotFound { .. }) => {
+                // NOTE: For public-facing things, we would want to stop here.
+                // Clients should fetch granularly, so they can make use of
+                // chunks they have locally.
+                // However, if this is used directly, without any caches, do the
+                // assembly here.
+                // This is subject to change, once we have store composition.
+                // TODO: make this configurable, and/or clarify behaviour for
+                // the gRPC server surface (explicitly document behaviour in the
+                // proto docs)
+                if let Some(chunks) = self.chunks(digest).await? {
+                    let chunked_reader = ChunkedReader::from_chunks(
+                        chunks.into_iter().map(|chunk| {
+                            (
+                                chunk.digest.try_into().expect("invalid b3 digest"),
+                                chunk.size,
+                            )
+                        }),
+                        Arc::new(self.clone()) as Arc<dyn BlobService>,
+                    );
+
+                    Ok(Some(Box::new(chunked_reader)))
+                } else {
+                    // This is neither a chunk nor a blob, return None.
+                    Ok(None)
+                }
+            }
+            Err(e) => Err(e.into()),
+        }
+    }
+
+    #[instrument(skip_all)]
+    async fn open_write(&self) -> Box<dyn BlobWriter> {
+        // ObjectStoreBlobWriter implements AsyncWrite, but all the chunking
+        // needs an AsyncRead, so we create a pipe here.
+        // In its `AsyncWrite` implementation, `ObjectStoreBlobWriter` delegates
+        // writes to w. It periodically polls the future that's reading from the
+        // other side.
+        let (w, r) = tokio::io::duplex(self.avg_chunk_size as usize * 10);
+
+        Box::new(ObjectStoreBlobWriter {
+            writer: Some(w),
+            fut: Some(Box::pin(chunk_and_upload(
+                r,
+                self.object_store.clone(),
+                self.base_path.clone(),
+                self.avg_chunk_size / 2,
+                self.avg_chunk_size,
+                self.avg_chunk_size * 2,
+            ))),
+            fut_output: None,
+        })
+    }
+
+    #[instrument(skip_all, err, fields(blob.digest=%digest))]
+    async fn chunks(&self, digest: &B3Digest) -> io::Result<Option<Vec<ChunkMeta>>> {
+        match self
+            .object_store
+            .get(&derive_blob_path(&self.base_path, digest))
+            .await
+        {
+            Ok(get_result) => {
+                // fetch the data at the blob path
+                let blob_data = get_result.bytes().await?;
+                // parse into StatBlobResponse
+                let stat_blob_response: StatBlobResponse = StatBlobResponse::decode(blob_data)?;
+
+                debug!(
+                    chunk.count = stat_blob_response.chunks.len(),
+                    blob.size = stat_blob_response
+                        .chunks
+                        .iter()
+                        .map(|x| x.size)
+                        .sum::<u64>(),
+                    "found more granular chunks"
+                );
+
+                Ok(Some(stat_blob_response.chunks))
+            }
+            Err(object_store::Error::NotFound { .. }) => {
+                // If there's only a chunk, we must return the empty vec here, rather than None.
+                match self
+                    .object_store
+                    .head(&derive_chunk_path(&self.base_path, digest))
+                    .await
+                {
+                    Ok(_) => {
+                        // present, but no more chunks available
+                        debug!("found a single chunk");
+                        Ok(Some(vec![]))
+                    }
+                    Err(object_store::Error::NotFound { .. }) => {
+                        // Neither blob nor single chunk found
+                        debug!("not found");
+                        Ok(None)
+                    }
+                    // error checking for chunk
+                    Err(e) => Err(e.into()),
+                }
+            }
+            // error checking for blob
+            Err(err) => Err(err.into()),
+        }
+    }
+}
+
+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)]
+async fn chunk_and_upload<R: AsyncRead + Unpin>(
+    r: R,
+    object_store: Arc<dyn ObjectStore>,
+    base_path: Path,
+    min_chunk_size: u32,
+    avg_chunk_size: u32,
+    max_chunk_size: u32,
+) -> io::Result<B3Digest> {
+    // wrap reader with something calculating the blake3 hash of all data read.
+    let mut b3_r = B3HashingReader::from(r);
+    // set up a fastcdc chunker
+    let mut chunker =
+        AsyncStreamCDC::new(&mut b3_r, min_chunk_size, avg_chunk_size, max_chunk_size);
+
+    /// This really should just belong into the closure at
+    /// `chunker.as_stream().then(|_| { โ€ฆ })``, but if we try to, rustc spits
+    /// higher-ranked lifetime errors at us.
+    async fn fastcdc_chunk_uploader(
+        resp: Result<fastcdc::v2020::ChunkData, fastcdc::v2020::Error>,
+        base_path: Path,
+        object_store: Arc<dyn ObjectStore>,
+    ) -> std::io::Result<ChunkMeta> {
+        let chunk_data = resp?;
+        let chunk_digest: B3Digest = blake3::hash(&chunk_data.data).as_bytes().into();
+        let chunk_path = derive_chunk_path(&base_path, &chunk_digest);
+
+        upload_chunk(object_store, chunk_digest, chunk_path, chunk_data.data).await
+    }
+
+    // Use the fastcdc chunker to produce a stream of chunks, and upload these
+    // that don't exist to the backend.
+    let chunks = chunker
+        .as_stream()
+        .then(|resp| fastcdc_chunk_uploader(resp, base_path.clone(), object_store.clone()))
+        .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
+    };
+
+    // check for Blob, if it doesn't exist, persist.
+    let blob_digest: B3Digest = b3_r.digest().into();
+    let blob_path = derive_blob_path(&base_path, &blob_digest);
+
+    match object_store.head(&blob_path).await {
+        // blob already exists, nothing to do
+        Ok(_) => {
+            trace!(
+                blob.digest = %blob_digest,
+                blob.path = %blob_path,
+                "blob already exists on backend"
+            );
+        }
+        // chunk does not yet exist, upload first
+        Err(object_store::Error::NotFound { .. }) => {
+            debug!(
+                blob.digest = %blob_digest,
+                blob.path = %blob_path,
+                "uploading blob"
+            );
+            object_store
+                .put(&blob_path, stat_blob_response.encode_to_vec().into())
+                .await?;
+        }
+        Err(err) => {
+            // other error
+            Err(err)?
+        }
+    }
+
+    Ok(blob_digest)
+}
+
+/// upload chunk if it doesn't exist yet.
+#[instrument(skip_all, fields(chunk.digest = %chunk_digest, chunk.size = chunk_data.len(), chunk.path = %chunk_path), err)]
+async fn upload_chunk(
+    object_store: Arc<dyn ObjectStore>,
+    chunk_digest: B3Digest,
+    chunk_path: Path,
+    chunk_data: Vec<u8>,
+) -> std::io::Result<ChunkMeta> {
+    let chunk_size = chunk_data.len();
+    match object_store.head(&chunk_path).await {
+        // chunk already exists, nothing to do
+        Ok(_) => {
+            debug!("chunk already exists");
+        }
+
+        // chunk does not yet exist, compress and upload.
+        Err(object_store::Error::NotFound { .. }) => {
+            let chunk_data_compressed =
+                zstd::encode_all(Cursor::new(chunk_data), zstd::DEFAULT_COMPRESSION_LEVEL)?;
+
+            debug!(chunk.compressed_size=%chunk_data_compressed.len(), "uploading chunk");
+
+            object_store
+                .as_ref()
+                .put(&chunk_path, chunk_data_compressed.into())
+                .await?;
+        }
+        // other error
+        Err(err) => Err(err)?,
+    }
+
+    Ok(ChunkMeta {
+        digest: chunk_digest.into(),
+        size: chunk_size as u64,
+    })
+}
+
+pin_project! {
+    /// Takes care of blob uploads.
+    /// All writes are relayed to self.writer, and we continuously poll the
+    /// future (which will internally read from the other side of the pipe and
+    /// upload chunks).
+    /// Our BlobWriter::close() needs to drop self.writer, so the other side
+    /// will read EOF and can finalize the blob.
+    /// The future should then resolve and return the blob digest.
+    pub struct ObjectStoreBlobWriter<W, Fut>
+    where
+        W: AsyncWrite,
+        Fut: Future,
+    {
+        #[pin]
+        writer: Option<W>,
+
+        #[pin]
+        fut: Option<Fut>,
+
+        fut_output: Option<io::Result<B3Digest>>
+    }
+}
+
+impl<W, Fut> tokio::io::AsyncWrite for ObjectStoreBlobWriter<W, Fut>
+where
+    W: AsyncWrite + Send + Unpin,
+    Fut: Future,
+{
+    fn poll_write(
+        self: std::pin::Pin<&mut Self>,
+        cx: &mut std::task::Context<'_>,
+        buf: &[u8],
+    ) -> std::task::Poll<Result<usize, io::Error>> {
+        let this = self.project();
+        // poll the future.
+        let fut = this.fut.as_pin_mut().expect("not future");
+        let fut_p = fut.poll(cx);
+        // if it's ready, the only way this could have happened is that the
+        // upload failed, because we're only closing `self.writer` after all
+        // writes happened.
+        if fut_p.is_ready() {
+            return Poll::Ready(Err(io::Error::other("upload failed")));
+        }
+
+        // write to the underlying writer
+        this.writer
+            .as_pin_mut()
+            .expect("writer must be some")
+            .poll_write(cx, buf)
+    }
+
+    fn poll_flush(
+        self: std::pin::Pin<&mut Self>,
+        cx: &mut std::task::Context<'_>,
+    ) -> std::task::Poll<Result<(), io::Error>> {
+        let this = self.project();
+        // poll the future.
+        let fut = this.fut.as_pin_mut().expect("not future");
+        let fut_p = fut.poll(cx);
+        // if it's ready, the only way this could have happened is that the
+        // upload failed, because we're only closing `self.writer` after all
+        // writes happened.
+        if fut_p.is_ready() {
+            return Poll::Ready(Err(io::Error::other("upload failed")));
+        }
+
+        // Call poll_flush on the writer
+        this.writer
+            .as_pin_mut()
+            .expect("writer must be some")
+            .poll_flush(cx)
+    }
+
+    fn poll_shutdown(
+        self: std::pin::Pin<&mut Self>,
+        _cx: &mut std::task::Context<'_>,
+    ) -> std::task::Poll<Result<(), io::Error>> {
+        // There's nothing to do on shutdown. We might have written some chunks
+        // that are nowhere else referenced, but cleaning them up here would be racy.
+        std::task::Poll::Ready(Ok(()))
+    }
+}
+
+#[async_trait]
+impl<W, Fut> BlobWriter for ObjectStoreBlobWriter<W, Fut>
+where
+    W: AsyncWrite + Send + Unpin,
+    Fut: Future<Output = io::Result<B3Digest>> + Send + Unpin,
+{
+    async fn close(&mut self) -> io::Result<B3Digest> {
+        match self.writer.take() {
+            Some(mut writer) => {
+                // shut down the writer, so the other side will read EOF.
+                writer.shutdown().await?;
+
+                // take out the future.
+                let fut = self.fut.take().expect("fut must be some");
+                // await it.
+                let resp = pin!(fut).await;
+
+                match resp.as_ref() {
+                    // In the case of an Ok value, we store it in self.fut_output,
+                    // so future calls to close can return that.
+                    Ok(b3_digest) => {
+                        self.fut_output = Some(Ok(b3_digest.clone()));
+                    }
+                    Err(e) => {
+                        // for the error type, we need to cheat a bit, as
+                        // they're not clone-able.
+                        // Simply store a sloppy clone, with the same ErrorKind and message there.
+                        self.fut_output = Some(Err(std::io::Error::new(e.kind(), e.to_string())))
+                    }
+                }
+                resp
+            }
+            None => {
+                // called a second time, return self.fut_output.
+                match self.fut_output.as_ref().unwrap() {
+                    Ok(ref b3_digest) => Ok(b3_digest.clone()),
+                    Err(e) => Err(std::io::Error::new(e.kind(), e.to_string())),
+                }
+            }
+        }
+    }
+}
+
+#[cfg(test)]
+mod test {
+    use super::{chunk_and_upload, default_avg_chunk_size};
+    use crate::{
+        blobservice::{BlobService, ObjectStoreBlobService},
+        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(
+        #[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,
+            1024 * 2,
+        )
+        .await
+        .expect("chunk_and_upload succeeds");
+
+        assert_eq!(blob_digest.clone(), inserted_blob_digest);
+
+        // Now we should have the blob
+        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/mod.rs b/tvix/castore/src/blobservice/tests/mod.rs
new file mode 100644
index 000000000000..0280faebb171
--- /dev/null
+++ b/tvix/castore/src/blobservice/tests/mod.rs
@@ -0,0 +1,253 @@
+//! This contains test scenarios that a given [BlobService] needs to pass.
+//! We use [rstest] and [rstest_reuse] to provide all services we want to test
+//! against, and then apply this template to all test functions.
+
+use rstest::*;
+use rstest_reuse::{self, *};
+use std::io;
+use tokio::io::AsyncReadExt;
+use tokio::io::AsyncSeekExt;
+
+use super::BlobService;
+use crate::blobservice;
+use crate::fixtures::BLOB_A;
+use crate::fixtures::BLOB_A_DIGEST;
+use crate::fixtures::BLOB_B;
+use crate::fixtures::BLOB_B_DIGEST;
+
+mod utils;
+use self::utils::make_grpc_blob_service_client;
+
+/// This produces a template, which will be applied to all individual test functions.
+/// See https://github.com/la10736/rstest/issues/130#issuecomment-968864832
+#[template]
+#[rstest]
+#[case::grpc(make_grpc_blob_service_client().await)]
+#[case::memory(blobservice::from_addr("memory://").await.unwrap())]
+#[case::objectstore_memory(blobservice::from_addr("objectstore+memory://").await.unwrap())]
+pub fn blob_services(#[case] blob_service: impl BlobService) {}
+
+/// Using [BlobService::has] on a non-existing blob should return false.
+#[apply(blob_services)]
+#[tokio::test]
+async fn has_nonexistent_false(blob_service: impl BlobService) {
+    assert!(!blob_service
+        .has(&BLOB_A_DIGEST)
+        .await
+        .expect("must not fail"));
+}
+
+/// Using [BlobService::chunks] on a non-existing blob should return Ok(None)
+#[apply(blob_services)]
+#[tokio::test]
+async fn chunks_nonexistent_false(blob_service: impl BlobService) {
+    assert!(blob_service
+        .chunks(&BLOB_A_DIGEST)
+        .await
+        .expect("must be ok")
+        .is_none());
+}
+
+// TODO: do tests with `chunks`
+
+/// Trying to read a non-existing blob should return a None instead of a reader.
+#[apply(blob_services)]
+#[tokio::test]
+async fn not_found_read(blob_service: impl BlobService) {
+    assert!(blob_service
+        .open_read(&BLOB_A_DIGEST)
+        .await
+        .expect("must not fail")
+        .is_none())
+}
+
+/// Put a blob in the store, check has, get it back.
+#[apply(blob_services)]
+// #[case::small(&fixtures::BLOB_A, &fixtures::BLOB_A_DIGEST)]
+// #[case::big(&fixtures::BLOB_B, &fixtures::BLOB_B_DIGEST)]
+#[tokio::test]
+async fn put_has_get(blob_service: impl BlobService) {
+    // TODO: figure out how to instantiate this with BLOB_A and BLOB_B, as two separate cases
+    for (blob_contents, blob_digest) in &[
+        (&*BLOB_A, BLOB_A_DIGEST.clone()),
+        (&*BLOB_B, BLOB_B_DIGEST.clone()),
+    ] {
+        let mut w = blob_service.open_write().await;
+
+        let l = tokio::io::copy(&mut io::Cursor::new(blob_contents), &mut w)
+            .await
+            .expect("copy must succeed");
+        assert_eq!(
+            blob_contents.len(),
+            l as usize,
+            "written bytes must match blob length"
+        );
+
+        let digest = w.close().await.expect("close must succeed");
+
+        assert_eq!(*blob_digest, digest, "returned digest must be correct");
+
+        assert!(
+            blob_service.has(blob_digest).await.expect("must not fail"),
+            "blob service should now have the blob"
+        );
+
+        let mut r = blob_service
+            .open_read(blob_digest)
+            .await
+            .expect("open_read must succeed")
+            .expect("must be some");
+
+        let mut buf: Vec<u8> = Vec::new();
+        let mut pinned_reader = std::pin::pin!(r);
+        let l = tokio::io::copy(&mut pinned_reader, &mut buf)
+            .await
+            .expect("copy must succeed");
+
+        assert_eq!(
+            blob_contents.len(),
+            l as usize,
+            "read bytes must match blob length"
+        );
+
+        assert_eq!(&blob_contents[..], &buf, "read blob contents must match");
+    }
+}
+
+/// Put a blob in the store, and seek inside it a bit.
+#[apply(blob_services)]
+#[tokio::test]
+async fn put_seek(blob_service: impl BlobService) {
+    let mut w = blob_service.open_write().await;
+
+    tokio::io::copy(&mut io::Cursor::new(&BLOB_B.to_vec()), &mut w)
+        .await
+        .expect("copy must succeed");
+    w.close().await.expect("close must succeed");
+
+    // open a blob for reading
+    let mut r = blob_service
+        .open_read(&BLOB_B_DIGEST)
+        .await
+        .expect("open_read must succeed")
+        .expect("must be some");
+
+    let mut pos: u64 = 0;
+
+    // read the first 10 bytes, they must match the data in the fixture.
+    {
+        let mut buf = [0; 10];
+        r.read_exact(&mut buf).await.expect("must succeed");
+
+        assert_eq!(
+            &BLOB_B[pos as usize..pos as usize + buf.len()],
+            buf,
+            "expected first 10 bytes to match"
+        );
+
+        pos += buf.len() as u64;
+    }
+    // seek by 0 bytes, using SeekFrom::Start.
+    let p = r
+        .seek(io::SeekFrom::Start(pos))
+        .await
+        .expect("must not fail");
+    assert_eq!(pos, p);
+
+    // read the next 10 bytes, they must match the data in the fixture.
+    {
+        let mut buf = [0; 10];
+        r.read_exact(&mut buf).await.expect("must succeed");
+
+        assert_eq!(
+            &BLOB_B[pos as usize..pos as usize + buf.len()],
+            buf,
+            "expected data to match"
+        );
+
+        pos += buf.len() as u64;
+    }
+
+    // seek by 5 bytes, using SeekFrom::Start.
+    let p = r
+        .seek(io::SeekFrom::Start(pos + 5))
+        .await
+        .expect("must not fail");
+    pos += 5;
+    assert_eq!(pos, p);
+
+    // read the next 10 bytes, they must match the data in the fixture.
+    {
+        let mut buf = [0; 10];
+        r.read_exact(&mut buf).await.expect("must succeed");
+
+        assert_eq!(
+            &BLOB_B[pos as usize..pos as usize + buf.len()],
+            buf,
+            "expected data to match"
+        );
+
+        pos += buf.len() as u64;
+    }
+
+    // seek by 12345 bytes, using SeekFrom::
+    let p = r
+        .seek(io::SeekFrom::Current(12345))
+        .await
+        .expect("must not fail");
+    pos += 12345;
+    assert_eq!(pos, p);
+
+    // read the next 10 bytes, they must match the data in the fixture.
+    {
+        let mut buf = [0; 10];
+        r.read_exact(&mut buf).await.expect("must succeed");
+
+        assert_eq!(
+            &BLOB_B[pos as usize..pos as usize + buf.len()],
+            buf,
+            "expected data to match"
+        );
+
+        #[allow(unused_assignments)]
+        {
+            pos += buf.len() as u64;
+        }
+    }
+
+    // seeking to the end is okayโ€ฆ
+    let p = r
+        .seek(io::SeekFrom::Start(BLOB_B.len() as u64))
+        .await
+        .expect("must not fail");
+    pos = BLOB_B.len() as u64;
+    assert_eq!(pos, p);
+
+    {
+        // but it returns no more data.
+        let mut buf: Vec<u8> = Vec::new();
+        r.read_to_end(&mut buf).await.expect("must not fail");
+        assert!(buf.is_empty(), "expected no more data to be read");
+    }
+
+    // seeking past the endโ€ฆ
+    // should either be ok, but then return 0 bytes.
+    // this matches the behaviour or a Cursor<Vec<u8>>.
+    if let Ok(_pos) = r.seek(io::SeekFrom::Start(BLOB_B.len() as u64 + 1)).await {
+        let mut buf: Vec<u8> = Vec::new();
+        r.read_to_end(&mut buf).await.expect("must not fail");
+        assert!(buf.is_empty(), "expected no more data to be read");
+    }
+    // or not be okay.
+
+    // TODO: this is only broken for the gRPC version
+    // We expect seeking backwards or relative to the end to fail.
+    // r.seek(io::SeekFrom::Current(-1))
+    //     .expect_err("SeekFrom::Current(-1) expected to fail");
+
+    // r.seek(io::SeekFrom::Start(pos - 1))
+    //     .expect_err("SeekFrom::Start(pos-1) expected to fail");
+
+    // r.seek(io::SeekFrom::End(0))
+    //     .expect_err("SeekFrom::End(_) expected to fail");
+}
diff --git a/tvix/castore/src/blobservice/tests/utils.rs b/tvix/castore/src/blobservice/tests/utils.rs
new file mode 100644
index 000000000000..7df4f00d3a09
--- /dev/null
+++ b/tvix/castore/src/blobservice/tests/utils.rs
@@ -0,0 +1,42 @@
+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.
+/// The server part is a [MemoryBlobService], exposed via the
+/// [GRPCBlobServiceWrapper], and connected through a DuplexStream
+pub async fn make_grpc_blob_service_client() -> Box<dyn BlobService> {
+    let (left, right) = tokio::io::duplex(64);
+
+    // spin up a server, which will only connect once, to the left side.
+    tokio::spawn(async {
+        let blob_service = Box::<MemoryBlobService>::default() as Box<dyn BlobService>;
+
+        // spin up a new DirectoryService
+        let mut server = Server::builder();
+        let router = server.add_service(BlobServiceServer::new(GRPCBlobServiceWrapper::new(
+            blob_service,
+        )));
+
+        router
+            .serve_with_incoming(tokio_stream::once(Ok::<_, std::io::Error>(left)))
+            .await
+    });
+
+    // Create a client, connecting to the right side. The URI is unused.
+    let mut maybe_right = Some(right);
+
+    Box::new(GRPCBlobService::from_client(BlobServiceClient::new(
+        Endpoint::try_from("http://[::]:50051")
+            .unwrap()
+            .connect_with_connector(tower::service_fn(move |_: Uri| {
+                let right = maybe_right.take().unwrap();
+                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
new file mode 100644
index 000000000000..4d919ff0d873
--- /dev/null
+++ b/tvix/castore/src/digests.rs
@@ -0,0 +1,97 @@
+use bytes::Bytes;
+use data_encoding::BASE64;
+use thiserror::Error;
+
+#[derive(PartialEq, Eq, Hash)]
+pub struct B3Digest(Bytes);
+
+// TODO: allow converting these errors to crate::Error
+#[derive(Error, Debug, PartialEq)]
+pub enum Error {
+    #[error("invalid digest length: {0}")]
+    InvalidDigestLen(usize),
+}
+
+pub const B3_LEN: usize = 32;
+
+impl B3Digest {
+    pub fn as_slice(&self) -> &[u8] {
+        &self.0[..]
+    }
+}
+
+impl From<B3Digest> for bytes::Bytes {
+    fn from(val: B3Digest) -> Self {
+        val.0
+    }
+}
+
+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);
+        Self(Bytes::copy_from_slice(&v))
+    }
+}
+
+impl TryFrom<Vec<u8>> for B3Digest {
+    type Error = Error;
+
+    // constructs a [B3Digest] from a [Vec<u8>].
+    // Returns an error if the digest has the wrong length.
+    fn try_from(value: Vec<u8>) -> Result<Self, Self::Error> {
+        if value.len() != B3_LEN {
+            Err(Error::InvalidDigestLen(value.len()))
+        } else {
+            Ok(Self(value.into()))
+        }
+    }
+}
+
+impl TryFrom<bytes::Bytes> for B3Digest {
+    type Error = Error;
+
+    // constructs a [B3Digest] from a [bytes::Bytes].
+    // Returns an error if the digest has the wrong length.
+    fn try_from(value: bytes::Bytes) -> Result<Self, Self::Error> {
+        if value.len() != B3_LEN {
+            Err(Error::InvalidDigestLen(value.len()))
+        } else {
+            Ok(Self(value))
+        }
+    }
+}
+
+impl From<&[u8; B3_LEN]> for B3Digest {
+    fn from(value: &[u8; B3_LEN]) -> Self {
+        Self(value.to_vec().into())
+    }
+}
+
+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())
+    }
+}
+
+impl std::fmt::Display for B3Digest {
+    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+        write!(f, "b3:{}", BASE64.encode(&self.0))
+    }
+}
+
+impl std::fmt::Debug for B3Digest {
+    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+        write!(f, "b3:{}", BASE64.encode(&self.0))
+    }
+}
diff --git a/tvix/castore/src/directoryservice/bigtable.rs b/tvix/castore/src/directoryservice/bigtable.rs
new file mode 100644
index 000000000000..73ab4342d832
--- /dev/null
+++ b/tvix/castore/src/directoryservice/bigtable.rs
@@ -0,0 +1,388 @@
+use bigtable_rs::{bigtable, google::bigtable::v2 as bigtable_v2};
+use bytes::Bytes;
+use data_encoding::HEXLOWER;
+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, 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.
+/// https://cloud.google.com/bigtable/docs/schema-design#cells
+const CELL_SIZE_LIMIT: u64 = 10 * 1024 * 1024;
+
+/// Provides a [DirectoryService] implementation using
+/// [Bigtable](https://cloud.google.com/bigtable/docs/)
+/// as an underlying K/V store.
+///
+/// # Data format
+/// We use Bigtable as a plain K/V store.
+/// The row key is the digest of the directory, in hexlower.
+/// Inside the row, we currently have a single column/cell, again using the
+/// hexlower directory digest.
+/// Its value is the Directory message, serialized in canonical protobuf.
+/// We currently only populate this column.
+///
+/// In the future, we might want to introduce "bucketing", essentially storing
+/// all directories inserted via `put_multiple_start` in a batched form.
+/// This will prevent looking up intermediate Directories, which are not
+/// directly at the root, so rely on store composition.
+#[derive(Clone)]
+pub struct BigtableDirectoryService {
+    client: bigtable::BigTable,
+    params: BigtableParameters,
+
+    #[cfg(test)]
+    #[allow(dead_code)]
+    /// Holds the temporary directory containing the unix socket, and the
+    /// spawned emulator process.
+    emulator: std::sync::Arc<(tempfile::TempDir, async_process::Child)>,
+}
+
+impl BigtableDirectoryService {
+    #[cfg(not(test))]
+    pub async fn connect(params: BigtableParameters) -> Result<Self, bigtable::Error> {
+        let connection = bigtable::BigTableConnection::new(
+            &params.project_id,
+            &params.instance_name,
+            params.is_read_only,
+            params.channel_size,
+            params.timeout,
+        )
+        .await?;
+
+        Ok(Self {
+            client: connection.client(),
+            params,
+        })
+    }
+
+    #[cfg(test)]
+    pub async fn connect(params: BigtableParameters) -> Result<Self, bigtable::Error> {
+        use std::time::Duration;
+
+        use async_process::{Command, Stdio};
+        use tempfile::TempDir;
+        use tokio_retry::{strategy::ExponentialBackoff, Retry};
+
+        let tmpdir = TempDir::new().unwrap();
+
+        let socket_path = tmpdir.path().join("cbtemulator.sock");
+
+        let emulator_process = Command::new("cbtemulator")
+            .arg("-address")
+            .arg(socket_path.clone())
+            .stderr(Stdio::piped())
+            .stdout(Stdio::piped())
+            .kill_on_drop(true)
+            .spawn()
+            .expect("failed to spawn emulator");
+
+        Retry::spawn(
+            ExponentialBackoff::from_millis(20)
+                .max_delay(Duration::from_secs(1))
+                .take(3),
+            || async {
+                if socket_path.exists() {
+                    Ok(())
+                } else {
+                    Err(())
+                }
+            },
+        )
+        .await
+        .expect("failed to wait for socket");
+
+        // populate the emulator
+        for cmd in &[
+            vec!["createtable", &params.table_name],
+            vec!["createfamily", &params.table_name, &params.family_name],
+        ] {
+            Command::new("cbt")
+                .args({
+                    let mut args = vec![
+                        "-instance",
+                        &params.instance_name,
+                        "-project",
+                        &params.project_id,
+                    ];
+                    args.extend_from_slice(cmd);
+                    args
+                })
+                .env(
+                    "BIGTABLE_EMULATOR_HOST",
+                    format!("unix://{}", socket_path.to_string_lossy()),
+                )
+                .output()
+                .await
+                .expect("failed to run cbt setup command");
+        }
+
+        let connection = bigtable_rs::bigtable::BigTableConnection::new_with_emulator(
+            &format!("unix://{}", socket_path.to_string_lossy()),
+            &params.project_id,
+            &params.instance_name,
+            params.is_read_only,
+            params.timeout,
+        )?;
+
+        Ok(Self {
+            client: connection.client(),
+            params,
+            emulator: (tmpdir, emulator_process).into(),
+        })
+    }
+}
+
+/// Derives the row/column key for a given blake3 digest.
+/// We use hexlower encoding, also because it can't be misinterpreted as RE2.
+fn derive_directory_key(digest: &B3Digest) -> String {
+    HEXLOWER.encode(digest.as_slice())
+}
+
+#[async_trait]
+impl DirectoryService for BigtableDirectoryService {
+    #[instrument(skip(self, digest), err, fields(directory.digest = %digest))]
+    async fn get(&self, digest: &B3Digest) -> Result<Option<Directory>, Error> {
+        let mut client = self.client.clone();
+        let directory_key = derive_directory_key(digest);
+
+        let request = bigtable_v2::ReadRowsRequest {
+            app_profile_id: self.params.app_profile_id.to_string(),
+            table_name: client.get_full_table_name(&self.params.table_name),
+            rows_limit: 1,
+            rows: Some(bigtable_v2::RowSet {
+                row_keys: vec![directory_key.clone().into()],
+                row_ranges: vec![],
+            }),
+            // Filter selected family name, and column qualifier matching our digest.
+            // This is to ensure we don't fail once we start bucketing.
+            filter: Some(bigtable_v2::RowFilter {
+                filter: Some(bigtable_v2::row_filter::Filter::Chain(
+                    bigtable_v2::row_filter::Chain {
+                        filters: vec![
+                            bigtable_v2::RowFilter {
+                                filter: Some(
+                                    bigtable_v2::row_filter::Filter::FamilyNameRegexFilter(
+                                        self.params.family_name.to_string(),
+                                    ),
+                                ),
+                            },
+                            bigtable_v2::RowFilter {
+                                filter: Some(
+                                    bigtable_v2::row_filter::Filter::ColumnQualifierRegexFilter(
+                                        directory_key.clone().into(),
+                                    ),
+                                ),
+                            },
+                        ],
+                    },
+                )),
+            }),
+            ..Default::default()
+        };
+
+        let mut response = client
+            .read_rows(request)
+            .await
+            .map_err(|e| Error::StorageError(format!("unable to read rows: {}", e)))?;
+
+        if response.len() != 1 {
+            if response.len() > 1 {
+                // This shouldn't happen, we limit number of rows to 1
+                return Err(Error::StorageError(
+                    "got more than one row from bigtable".into(),
+                ));
+            }
+            // else, this is simply a "not found".
+            return Ok(None);
+        }
+
+        let (row_key, mut row_cells) = response.pop().unwrap();
+        if row_key != directory_key.as_bytes() {
+            // This shouldn't happen, we requested this row key.
+            return Err(Error::StorageError(
+                "got wrong row key from bigtable".into(),
+            ));
+        }
+
+        let row_cell = row_cells
+            .pop()
+            .ok_or_else(|| Error::StorageError("found no cells".into()))?;
+
+        // Ensure there's only one cell (so no more left after the pop())
+        // This shouldn't happen, We filter out other cells in our query.
+        if !row_cells.is_empty() {
+            return Err(Error::StorageError(
+                "more than one cell returned from bigtable".into(),
+            ));
+        }
+
+        // We also require the qualifier to be correct in the filter above,
+        // so this shouldn't happen.
+        if directory_key.as_bytes() != row_cell.qualifier {
+            return Err(Error::StorageError("unexpected cell qualifier".into()));
+        }
+
+        // For the data in that cell, ensure the digest matches what's requested, before parsing.
+        let got_digest = B3Digest::from(blake3::hash(&row_cell.value).as_bytes());
+        if got_digest != *digest {
+            return Err(Error::StorageError(format!(
+                "invalid digest: {}",
+                got_digest
+            )));
+        }
+
+        // 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)))?
+            .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: Directory) -> Result<B3Digest, Error> {
+        let directory_digest = directory.digest();
+        let mut client = self.client.clone();
+        let directory_key = derive_directory_key(&directory_digest);
+
+        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(),
+            ));
+        }
+
+        let resp = client
+            .check_and_mutate_row(bigtable_v2::CheckAndMutateRowRequest {
+                table_name: client.get_full_table_name(&self.params.table_name),
+                app_profile_id: self.params.app_profile_id.to_string(),
+                row_key: directory_key.clone().into(),
+                predicate_filter: Some(bigtable_v2::RowFilter {
+                    filter: Some(bigtable_v2::row_filter::Filter::ColumnQualifierRegexFilter(
+                        directory_key.clone().into(),
+                    )),
+                }),
+                // If the column was already found, do nothing.
+                true_mutations: vec![],
+                // Else, do the insert.
+                false_mutations: vec![
+                    // https://cloud.google.com/bigtable/docs/writes
+                    bigtable_v2::Mutation {
+                        mutation: Some(bigtable_v2::mutation::Mutation::SetCell(
+                            bigtable_v2::mutation::SetCell {
+                                family_name: self.params.family_name.to_string(),
+                                column_qualifier: directory_key.clone().into(),
+                                timestamp_micros: -1, // use server time to fill timestamp
+                                value: data,
+                            },
+                        )),
+                    },
+                ],
+            })
+            .await
+            .map_err(|e| Error::StorageError(format!("unable to mutate rows: {}", e)))?;
+
+        if resp.predicate_matched {
+            trace!("already existed")
+        }
+
+        Ok(directory_digest)
+    }
+
+    #[instrument(skip_all, fields(directory.digest = %root_directory_digest))]
+    fn get_recursive(
+        &self,
+        root_directory_digest: &B3Digest,
+    ) -> BoxStream<'static, Result<Directory, Error>> {
+        traverse_directory(self.clone(), root_directory_digest)
+    }
+
+    #[instrument(skip_all)]
+    fn put_multiple_start(&self) -> Box<(dyn DirectoryPutter + 'static)>
+    where
+        Self: Clone,
+    {
+        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/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
new file mode 100644
index 000000000000..3feb8f3509fe
--- /dev/null
+++ b/tvix/castore/src/directoryservice/from_addr.rs
@@ -0,0 +1,136 @@
+use std::sync::Arc;
+
+use url::Url;
+
+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:
+/// - `memory:`
+///   Uses a in-memory implementation.
+/// - `sled:`
+///   Uses a in-memory sled implementation.
+/// - `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<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_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?;
+
+    Ok(directory_service)
+}
+
+#[cfg(test)]
+mod tests {
+    use super::from_addr;
+    use lazy_static::lazy_static;
+    use rstest::rstest;
+    use tempfile::TempDir;
+
+    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]
+    /// This uses an unsupported scheme.
+    #[case::unsupported_scheme("http://foo.example/test", false)]
+    /// This configures sled in temporary mode.
+    #[case::sled_valid_temporary("sled://", true)]
+    /// This configures sled with /, which should fail.
+    #[case::sled_invalid_root("sled:///", false)]
+    /// This configures sled with a host, not path, which should fail.
+    #[case::sled_invalid_host("sled://foo.example", false)]
+    /// This configures sled with a valid path path, which should succeed.
+    #[case::sled_valid_path(&format!("sled://{}", &TMPDIR_SLED_1.path().to_str().unwrap()), true)]
+    /// This configures sled with a host, and a valid path path, which should fail.
+    #[case::sled_invalid_host_with_valid_path(&format!("sled://foo.example{}", &TMPDIR_SLED_2.path().to_str().unwrap()), false)]
+    /// This correctly sets the scheme, and doesn't set a path.
+    #[case::memory_valid("memory://", true)]
+    /// This sets a memory url host to `foo`
+    #[case::memory_invalid_host("memory://foo", false)]
+    /// This sets a memory url path to "/", which is invalid.
+    #[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.
+    #[case::grpc_invalid_unix_socket_and_host("grpc+unix://host.example/path/to/somewhere", false)]
+    /// Correct scheme to connect to localhost, with port 12345
+    #[case::grpc_valid_ipv6_localhost_port_12345("grpc+http://[::1]:12345", true)]
+    /// Correct scheme to connect to localhost over http, without specifying a port.
+    #[case::grpc_valid_http_host_without_port("grpc+http://localhost", true)]
+    /// Correct scheme to connect to localhost over http, without specifying a port.
+    #[case::grpc_valid_https_host_without_port("grpc+https://localhost", true)]
+    /// Correct scheme to connect to localhost over http, but with additional path, which is invalid.
+    #[case::grpc_invalid_host_and_path("grpc+http://localhost/some-path", false)]
+    /// A valid example for Bigtable
+    #[cfg_attr(
+        all(feature = "cloud", feature = "integration"),
+        case::bigtable_valid_url(
+            "bigtable://instance-1?project_id=project-1&table_name=table-1&family_name=cf1",
+            true
+        )
+    )]
+    /// A valid example for Bigtable, specifying a custom channel size and timeout
+    #[cfg_attr(
+        all(feature = "cloud", feature = "integration"),
+        case::bigtable_valid_url(
+            "bigtable://instance-1?project_id=project-1&table_name=table-1&family_name=cf1&channel_size=10&timeout=10",
+            true
+        )
+    )]
+    /// A invalid Bigtable example (missing fields)
+    #[cfg_attr(
+        all(feature = "cloud", feature = "integration"),
+        case::bigtable_invalid_url("bigtable://instance-1", false)
+    )]
+    #[tokio::test]
+    async fn test_from_addr_tokio(#[case] uri_str: &str, #[case] exp_succeed: bool) {
+        if exp_succeed {
+            from_addr(uri_str).await.expect("should succeed");
+        } else {
+            assert!(from_addr(uri_str).await.is_err(), "should fail");
+        }
+    }
+}
diff --git a/tvix/castore/src/directoryservice/grpc.rs b/tvix/castore/src/directoryservice/grpc.rs
new file mode 100644
index 000000000000..9696c5631949
--- /dev/null
+++ b/tvix/castore/src/directoryservice/grpc.rs
@@ -0,0 +1,386 @@
+use std::collections::HashSet;
+
+use super::{Directory, DirectoryPutter, DirectoryService};
+use crate::composition::{CompositionContext, ServiceBuilder};
+use crate::proto::{self, get_directory_request::ByWhat};
+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, Code, Status};
+use tracing::{instrument, warn, Instrument as _};
+
+/// Connects to a (remote) tvix-store DirectoryService over gRPC.
+#[derive(Clone)]
+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<T>,
+}
+
+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<T>,
+    ) -> Self {
+        Self { grpc_client }
+    }
+}
+
+#[async_trait]
+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<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();
+        let message = async move {
+            let mut s = grpc_client
+                .get(proto::GetDirectoryRequest {
+                    recursive: false,
+                    by_what: Some(ByWhat::Digest(digest_cpy.into())),
+                })
+                .await?
+                .into_inner();
+
+            // Retrieve the first message only, then close the stream (we set recursive to false)
+            s.message().await
+        };
+
+        let digest = digest.clone();
+        match message.await {
+            Ok(Some(directory)) => {
+                // Validate the retrieved Directory indeed has the
+                // digest we expect it to have, to detect corruptions.
+                let actual_digest = directory.digest();
+                if actual_digest != digest {
+                    Err(crate::Error::StorageError(format!(
+                        "requested directory with digest {}, but got {}",
+                        digest, actual_digest
+                    )))
+                } else {
+                    Ok(Some(directory.try_into().map_err(|_| {
+                        Error::StorageError("invalid root digest length in response".to_string())
+                    })?))
+                }
+            }
+            Ok(None) => Ok(None),
+            Err(e) if e.code() == Code::NotFound => Ok(None),
+            Err(e) => Err(crate::Error::StorageError(e.to_string())),
+        }
+    }
+
+    #[instrument(level = "trace", skip_all, fields(directory.digest = %directory.digest()))]
+    async fn put(&self, directory: Directory) -> Result<B3Digest, crate::Error> {
+        let resp = self
+            .grpc_client
+            .clone()
+            .put(tokio_stream::once(proto::Directory::from(directory)))
+            .await;
+
+        match resp {
+            Ok(put_directory_resp) => Ok(put_directory_resp
+                .into_inner()
+                .root_digest
+                .try_into()
+                .map_err(|_| {
+                    Error::StorageError("invalid root digest length in response".to_string())
+                })?),
+            Err(e) => Err(crate::Error::StorageError(e.to_string())),
+        }
+    }
+
+    #[instrument(level = "trace", skip_all, fields(directory.digest = %root_directory_digest))]
+    fn get_recursive(
+        &self,
+        root_directory_digest: &B3Digest,
+    ) -> BoxStream<'static, Result<Directory, Error>> {
+        let mut grpc_client = self.grpc_client.clone();
+        let root_directory_digest = root_directory_digest.clone();
+
+        let stream = try_stream! {
+            let mut stream = grpc_client
+                .get(proto::GetDirectoryRequest {
+                    recursive: true,
+                    by_what: Some(ByWhat::Digest(root_directory_digest.clone().into())),
+                })
+                .await
+                .map_err(|e| crate::Error::StorageError(e.to_string()))?
+                .into_inner();
+
+            // 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.clone()]);
+
+            loop {
+                match stream.message().await {
+                    Ok(Some(directory)) => {
+                        // 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);
+                        if !was_expected {
+                            // FUTUREWORK: dumb clients might send the same stuff twice.
+                            // as a fallback, we might want to tolerate receiving
+                            // it if it's in received_directory_digests (as that
+                            // means it once was in expected_directory_digests)
+                            Err(crate::Error::StorageError(format!(
+                                "received unexpected directory {}",
+                                directory_digest
+                            )))?;
+                        }
+                        received_directory_digests.insert(directory_digest);
+
+                        // register all children in expected_directory_digests.
+                        for child_directory in &directory.directories {
+                            // We ran validate() above, so we know these digests must be correct.
+                            let child_directory_digest =
+                                child_directory.digest.clone().try_into().unwrap();
+
+                            expected_directory_digests
+                                .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) => {
+                        // 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",
+                                diff_len
+                            )))?
+                        } else {
+                            return
+                        }
+                    },
+                    Err(e) => {
+                        Err(crate::Error::StorageError(e.to_string()))?;
+                    },
+                }
+            }
+        };
+
+        Box::pin(stream)
+    }
+
+    #[instrument(skip_all)]
+    fn put_multiple_start(&self) -> Box<(dyn DirectoryPutter + 'static)>
+    where
+        Self: Clone,
+    {
+        let mut grpc_client = self.grpc_client.clone();
+
+        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();
+
+                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)),
+        })
+    }
+}
+
+#[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
+    /// of the channel.
+    /// The tx part of the pipe is used to send [proto::Directory] to the ongoing request.
+    /// The task will yield a [proto::PutDirectoryResponse] once the stream is closed.
+    #[allow(clippy::type_complexity)] // lol
+    rq: Option<(
+        JoinHandle<Result<proto::PutDirectoryResponse, Status>>,
+        UnboundedSender<proto::Directory>,
+    )>,
+}
+
+#[async_trait]
+impl DirectoryPutter for GRPCPutter {
+    #[instrument(level = "trace", skip_all, fields(directory.digest=%directory.digest()), err)]
+    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.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.
+                    self.close().await?;
+                }
+                Ok(())
+            }
+            // If self.close() was already called, we can't put again.
+            None => Err(Error::StorageError(
+                "DirectoryPutter already closed".to_string(),
+            )),
+        }
+    }
+
+    /// Closes the stream for sending, and returns the value.
+    #[instrument(level = "trace", skip_all, ret, err)]
+    async fn close(&mut self) -> Result<B3Digest, crate::Error> {
+        // get self.rq, and replace it with None.
+        // This ensures we can only close it once.
+        match std::mem::take(&mut self.rq) {
+            None => Err(Error::StorageError("already closed".to_string())),
+            Some((task, directory_sender)) => {
+                // close directory_sender, so blocking on task will finish.
+                drop(directory_sender);
+
+                let root_digest = task
+                    .await?
+                    .map_err(|e| Error::StorageError(e.to_string()))?
+                    .root_digest;
+
+                root_digest.try_into().map_err(|_| {
+                    Error::StorageError("invalid root digest length in response".to_string())
+                })
+            }
+        }
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use std::time::Duration;
+    use tempfile::TempDir;
+    use tokio::net::UnixListener;
+    use tokio_retry::{strategy::ExponentialBackoff, Retry};
+    use tokio_stream::wrappers::UnixListenerStream;
+
+    use crate::{
+        directoryservice::{DirectoryService, GRPCDirectoryService, MemoryDirectoryService},
+        fixtures,
+        proto::{directory_service_client::DirectoryServiceClient, GRPCDirectoryServiceWrapper},
+    };
+
+    /// This ensures connecting via gRPC works as expected.
+    #[tokio::test]
+    async fn test_valid_unix_path_ping_pong() {
+        let tmpdir = TempDir::new().unwrap();
+        let socket_path = tmpdir.path().join("daemon");
+
+        let path_clone = socket_path.clone();
+
+        // Spin up a server
+        tokio::spawn(async {
+            let uds = UnixListener::bind(path_clone).unwrap();
+            let uds_stream = UnixListenerStream::new(uds);
+
+            // spin up a new server
+            let mut server = tonic::transport::Server::builder();
+            let router = server.add_service(
+                crate::proto::directory_service_server::DirectoryServiceServer::new(
+                    GRPCDirectoryServiceWrapper::new(
+                        Box::<MemoryDirectoryService>::default() as Box<dyn DirectoryService>
+                    ),
+                ),
+            );
+            router.serve_with_incoming(uds_stream).await
+        });
+
+        // wait for the socket to be created
+        Retry::spawn(
+            ExponentialBackoff::from_millis(20).max_delay(Duration::from_secs(10)),
+            || async {
+                if socket_path.exists() {
+                    Ok(())
+                } else {
+                    Err(())
+                }
+            },
+        )
+        .await
+        .expect("failed to wait for socket");
+
+        // prepare a client
+        let grpc_client = {
+            let url = url::Url::parse(&format!(
+                "grpc+unix://{}?wait-connect=1",
+                socket_path.display()
+            ))
+            .expect("must parse");
+            let client = DirectoryServiceClient::new(
+                crate::tonic::channel_from_url(&url)
+                    .await
+                    .expect("must succeed"),
+            );
+            GRPCDirectoryService::from_client(client)
+        };
+
+        assert!(grpc_client
+            .get(&fixtures::DIRECTORY_A.digest())
+            .await
+            .expect("must not fail")
+            .is_none())
+    }
+}
diff --git a/tvix/castore/src/directoryservice/memory.rs b/tvix/castore/src/directoryservice/memory.rs
new file mode 100644
index 000000000000..b039d9bc7d84
--- /dev/null
+++ b/tvix/castore/src/directoryservice/memory.rs
@@ -0,0 +1,101 @@
+use crate::{B3Digest, Error};
+use futures::stream::BoxStream;
+use std::collections::HashMap;
+use std::sync::Arc;
+use tokio::sync::RwLock;
+use tonic::async_trait;
+use tracing::{instrument, warn};
+
+use super::utils::traverse_directory;
+use super::{Directory, DirectoryPutter, DirectoryService, SimplePutter};
+use crate::composition::{CompositionContext, ServiceBuilder};
+use crate::proto;
+
+#[derive(Clone, Default)]
+pub struct MemoryDirectoryService {
+    db: Arc<RwLock<HashMap<B3Digest, proto::Directory>>>,
+}
+
+#[async_trait]
+impl DirectoryService for MemoryDirectoryService {
+    #[instrument(skip(self, digest), fields(directory.digest = %digest))]
+    async fn get(&self, digest: &B3Digest) -> Result<Option<Directory>, Error> {
+        let db = self.db.read().await;
+
+        match db.get(digest) {
+            // The directory was not found, return
+            None => Ok(None),
+
+            // The directory was found, try to parse the data as Directory message
+            Some(directory) => {
+                // Validate the retrieved Directory indeed has the
+                // digest we expect it to have, to detect corruptions.
+                let actual_digest = directory.digest();
+                if actual_digest != *digest {
+                    return Err(Error::StorageError(format!(
+                        "requested directory with digest {}, but got {}",
+                        digest, actual_digest
+                    )));
+                }
+
+                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: Directory) -> Result<B3Digest, Error> {
+        let digest = directory.digest();
+
+        // store it
+        let mut db = self.db.write().await;
+        db.insert(digest.clone(), directory.into());
+
+        Ok(digest)
+    }
+
+    #[instrument(skip_all, fields(directory.digest = %root_directory_digest))]
+    fn get_recursive(
+        &self,
+        root_directory_digest: &B3Digest,
+    ) -> BoxStream<'static, Result<Directory, Error>> {
+        traverse_directory(self.clone(), root_directory_digest)
+    }
+
+    #[instrument(skip_all)]
+    fn put_multiple_start(&self) -> Box<(dyn DirectoryPutter + 'static)>
+    where
+        Self: Clone,
+    {
+        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
new file mode 100644
index 000000000000..25162e4de853
--- /dev/null
+++ b/tvix/castore/src/directoryservice/mod.rs
@@ -0,0 +1,147 @@
+use crate::composition::{Registry, ServiceBuilder};
+use crate::{B3Digest, Directory, Error};
+
+use futures::stream::BoxStream;
+use tonic::async_trait;
+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)]
+pub mod tests;
+mod traverse;
+mod utils;
+
+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, 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, SledDirectoryServiceConfig};
+pub use self::traverse::descend_to;
+pub use self::utils::traverse_directory;
+
+#[cfg(feature = "cloud")]
+mod bigtable;
+
+#[cfg(feature = "cloud")]
+pub use self::bigtable::{BigtableDirectoryService, BigtableParameters};
+
+/// The base trait all Directory services need to implement.
+/// This is a simple get and put of [Directory], returning their
+/// digest.
+#[async_trait]
+pub trait DirectoryService: Send + Sync {
+    /// Looks up a single Directory message by its digest.
+    /// The returned Directory message *must* be valid.
+    /// In case the directory is not found, Ok(None) is returned.
+    ///
+    /// It is okay for certain implementations to only allow retrieval of
+    /// 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<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: Directory) -> Result<B3Digest, 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.
+    ///
+    /// This returns a pinned, boxed stream. The pinning allows for it to be polled easily,
+    /// and the box allows different underlying stream implementations to be returned since
+    /// Rust doesn't support this as a generic in traits yet. This is the same thing that
+    /// [async_trait] generates, but for streams instead of futures.
+    ///
+    /// The individually returned Directory messages *must* be valid.
+    /// 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<Directory, Error>>;
+
+    /// Allows persisting a closure of [Directory], which is a graph of
+    /// connected Directory messages.
+    fn put_multiple_start(&self) -> Box<dyn DirectoryPutter>;
+}
+
+#[async_trait]
+impl<A> DirectoryService for A
+where
+    A: AsRef<dyn DirectoryService> + Send + Sync,
+{
+    async fn get(&self, digest: &B3Digest) -> Result<Option<Directory>, Error> {
+        self.as_ref().get(digest).await
+    }
+
+    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<Directory, Error>> {
+        self.as_ref().get_recursive(root_directory_digest)
+    }
+
+    fn put_multiple_start(&self) -> Box<dyn DirectoryPutter> {
+        self.as_ref().put_multiple_start()
+    }
+}
+
+/// 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
+/// retrieve the root digest (or an error).
+///
+/// DirectoryPutters might be created without a single [DirectoryPutter::put],
+/// and then dropped without calling [DirectoryPutter::close],
+/// for example when ingesting a path that ends up not pointing to a directory,
+/// but a single file or symlink.
+#[async_trait]
+pub trait DirectoryPutter: Send {
+    /// 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: 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
new file mode 100644
index 000000000000..5b5281abcd2f
--- /dev/null
+++ b/tvix/castore/src/directoryservice/object_store.rs
@@ -0,0 +1,327 @@
+use std::collections::HashMap;
+use std::sync::Arc;
+
+use data_encoding::HEXLOWER;
+use futures::future::Either;
+use futures::stream::BoxStream;
+use futures::SinkExt;
+use futures::StreamExt;
+use futures::TryFutureExt;
+use futures::TryStreamExt;
+use object_store::{path::Path, ObjectStore};
+use prost::Message;
+use tokio::io::AsyncWriteExt;
+use tokio_util::codec::LengthDelimitedCodec;
+use tonic::async_trait;
+use tracing::{instrument, trace, warn, Level};
+use url::Url;
+
+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
+/// fetching them recursively via the top-level directory, since all batched writes
+/// (using `put_multiple_start`) are stored in a single object.
+/// Directories are stored in a length-delimited format with a 1MiB limit. The length field is a
+/// u32 and the directories are stored in root-to-leaves topological order, the same way they will
+/// be returned to the client in get_recursive.
+#[derive(Clone)]
+pub struct ObjectStoreDirectoryService {
+    object_store: Arc<dyn ObjectStore>,
+    base_path: Path,
+}
+
+#[instrument(level=Level::TRACE, skip_all,fields(base_path=%base_path,blob.digest=%digest),ret(Display))]
+fn derive_dirs_path(base_path: &Path, digest: &B3Digest) -> Path {
+    base_path
+        .child("dirs")
+        .child("b3")
+        .child(HEXLOWER.encode(&digest.as_slice()[..2]))
+        .child(HEXLOWER.encode(digest.as_slice()))
+}
+
+#[allow(clippy::identity_op)]
+const MAX_FRAME_LENGTH: usize = 1 * 1024 * 1024 * 1000; // 1 MiB
+                                                        //
+impl ObjectStoreDirectoryService {
+    /// 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
+    /// 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,
+        })
+    }
+
+    /// 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())
+    }
+}
+
+#[async_trait]
+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<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: 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(),
+            ));
+        }
+
+        let mut handle = self.put_multiple_start();
+        handle.put(directory).await?;
+        handle.close().await
+    }
+
+    #[instrument(skip_all, fields(directory.digest = %root_directory_digest))]
+    fn get_recursive(
+        &self,
+        root_directory_digest: &B3Digest,
+    ) -> 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();
+
+        Box::pin(
+            (async move {
+                let stream = match object_store.get(&dir_path).await {
+                    Ok(v) => v.into_stream(),
+                    Err(object_store::Error::NotFound { .. }) => {
+                        return Ok(Either::Left(futures::stream::empty()))
+                    }
+                    Err(e) => return Err(std::io::Error::from(e).into()),
+                };
+
+                // get a reader of the response body.
+                let r = tokio_util::io::StreamReader::new(stream);
+                let decompressed_stream = async_compression::tokio::bufread::ZstdDecoder::new(r);
+
+                // the subdirectories are stored in a length delimited format
+                let delimited_stream = LengthDelimitedCodec::builder()
+                    .max_frame_length(MAX_FRAME_LENGTH)
+                    .length_field_type::<u32>()
+                    .new_read(decompressed_stream);
+
+                let dirs_stream = delimited_stream.map_err(Error::from).and_then(move |buf| {
+                    futures::future::ready((|| {
+                        let mut hasher = blake3::Hasher::new();
+                        let digest: B3Digest = hasher.update(&buf).finalize().as_bytes().into();
+
+                        // Ensure to only decode the directory objects whose digests we trust
+                        if !order_validator.digest_allowed(&digest) {
+                            return Err(crate::Error::StorageError(format!(
+                                "received unexpected directory {}",
+                                digest
+                            )));
+                        }
+
+                        let directory = proto::Directory::decode(&*buf).map_err(|e| {
+                            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())
+                        })?;
+
+                        // Allow the children to appear next
+                        order_validator.add_directory_unchecked(&directory);
+
+                        Ok(directory)
+                    })())
+                });
+
+                Ok(Either::Right(dirs_stream))
+            })
+            .try_flatten_stream(),
+        )
+    }
+
+    #[instrument(skip_all)]
+    fn put_multiple_start(&self) -> Box<(dyn DirectoryPutter + 'static)>
+    where
+        Self: Clone,
+    {
+        Box::new(ObjectStoreDirectoryPutter::new(
+            self.object_store.clone(),
+            self.base_path.clone(),
+        ))
+    }
+}
+
+#[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<DirectoryGraph<LeavesToRootValidator>>,
+}
+
+impl ObjectStoreDirectoryPutter {
+    fn new(object_store: Arc<dyn ObjectStore>, base_path: Path) -> Self {
+        Self {
+            object_store,
+            base_path,
+            directory_validator: Some(Default::default()),
+        }
+    }
+}
+
+#[async_trait]
+impl DirectoryPutter for ObjectStoreDirectoryPutter {
+    #[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> {
+        let validator = match self.directory_validator.take() {
+            None => return Err(Error::InvalidRequest("already closed".to_string())),
+            Some(validator) => validator,
+        };
+
+        // 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
+            .validate()
+            .map_err(|e| Error::StorageError(e.to_string()))?
+            .drain_root_to_leaves()
+            .collect::<Vec<_>>();
+
+        // Get the root digest
+        let root_digest = directories
+            .first()
+            .ok_or_else(|| Error::InvalidRequest("got no directories".to_string()))?
+            .digest();
+
+        let dir_path = derive_dirs_path(&self.base_path, &root_digest);
+
+        match self.object_store.head(&dir_path).await {
+            // directory tree already exists, nothing to do
+            Ok(_) => {
+                trace!("directory tree already exists");
+            }
+
+            // directory tree does not yet exist, compress and upload.
+            Err(object_store::Error::NotFound { .. }) => {
+                trace!("uploading directory tree");
+
+                let object_store_writer =
+                    object_store::buffered::BufWriter::new(self.object_store.clone(), dir_path);
+                let compressed_writer =
+                    async_compression::tokio::write::ZstdEncoder::new(object_store_writer);
+                let mut directories_sink = LengthDelimitedCodec::builder()
+                    .max_frame_length(MAX_FRAME_LENGTH)
+                    .length_field_type::<u32>()
+                    .new_write(compressed_writer);
+
+                for directory in directories {
+                    directories_sink
+                        .send(proto::Directory::from(directory).encode_to_vec().into())
+                        .await?;
+                }
+
+                let mut compressed_writer = directories_sink.into_inner();
+                compressed_writer.shutdown().await?;
+            }
+            // other error
+            Err(err) => Err(std::io::Error::from(err))?,
+        }
+
+        Ok(root_digest)
+    }
+}
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
new file mode 100644
index 000000000000..b4daaee61b22
--- /dev/null
+++ b/tvix/castore/src/directoryservice/simple_putter.rs
@@ -0,0 +1,80 @@
+use super::DirectoryPutter;
+use super::DirectoryService;
+use super::{Directory, DirectoryGraph, LeavesToRootValidator};
+use crate::B3Digest;
+use crate::Error;
+use tonic::async_trait;
+use tracing::instrument;
+use tracing::warn;
+
+/// This is an implementation of DirectoryPutter that simply
+/// inserts individual Directory messages one by one, on close, after
+/// they successfully validated.
+pub struct SimplePutter<DS: DirectoryService> {
+    directory_service: DS,
+
+    directory_validator: Option<DirectoryGraph<LeavesToRootValidator>>,
+}
+
+impl<DS: DirectoryService> SimplePutter<DS> {
+    pub fn new(directory_service: DS) -> Self {
+        Self {
+            directory_service,
+            directory_validator: Some(Default::default()),
+        }
+    }
+}
+
+#[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: 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::InvalidRequest("already closed".to_string())),
+            Some(validator) => {
+                // 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::InvalidRequest("got no directories".to_string()))?
+                    .digest();
+
+                // call an individual put for each directory and await the insertion.
+                for directory in directories {
+                    let exp_digest = directory.digest();
+                    let actual_digest = self.directory_service.put(directory).await?;
+
+                    // ensure the digest the backend told us matches our expectations.
+                    if exp_digest != actual_digest {
+                        warn!(directory.digest_expected=%exp_digest, directory.digest_actual=%actual_digest, "unexpected digest");
+                        return Err(Error::StorageError(
+                            "got unexpected digest from backend during put".into(),
+                        ));
+                    }
+                }
+
+                Ok(root_digest)
+            }
+        }
+    }
+}
diff --git a/tvix/castore/src/directoryservice/sled.rs b/tvix/castore/src/directoryservice/sled.rs
new file mode 100644
index 000000000000..4f3a860d14e4
--- /dev/null
+++ b/tvix/castore/src/directoryservice/sled.rs
@@ -0,0 +1,263 @@
+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::{Directory, DirectoryGraph, DirectoryPutter, DirectoryService, LeavesToRootValidator};
+use crate::composition::{CompositionContext, ServiceBuilder};
+use crate::{proto, B3Digest, Error};
+
+#[derive(Clone)]
+pub struct SledDirectoryService {
+    db: sled::Db,
+}
+
+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);
+        let db = config.open()?;
+
+        Ok(Self { db })
+    }
+
+    pub fn new_temporary() -> Result<Self, sled::Error> {
+        let config = sled::Config::default().temporary(true);
+        let db = config.open()?;
+
+        Ok(Self { db })
+    }
+}
+
+#[async_trait]
+impl DirectoryService for SledDirectoryService {
+    #[instrument(skip(self, digest), fields(directory.digest = %digest))]
+    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();
+            move || db.get(digest.as_slice())
+        })
+        .await?
+        .map_err(|e| {
+            warn!("failed to retrieve directory: {}", e);
+            Error::StorageError(format!("failed to retrieve directory: {}", e))
+        })?;
+
+        match resp {
+            // The directory was not found, return
+            None => Ok(None),
+
+            // The directory was found, try to parse the data as Directory message
+            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.
+                    let actual_digest = directory.digest();
+                    if actual_digest != *digest {
+                        return Err(Error::StorageError(format!(
+                            "requested directory with digest {}, but got {}",
+                            digest, actual_digest
+                        )));
+                    }
+
+                    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))
+                }
+                Err(e) => {
+                    warn!("unable to parse directory {}: {}", digest, e);
+                    Err(Error::StorageError(e.to_string()))
+                }
+            },
+        }
+    }
+
+    #[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 it
+                db.insert(
+                    digest.as_slice(),
+                    proto::Directory::from(directory).encode_to_vec(),
+                )
+                .map_err(|e| Error::StorageError(e.to_string()))?;
+
+                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>> {
+        traverse_directory(self.clone(), root_directory_digest)
+    }
+
+    #[instrument(skip_all)]
+    fn put_multiple_start(&self) -> Box<(dyn DirectoryPutter + 'static)>
+    where
+        Self: Clone,
+    {
+        Box::new(SledDirectoryPutter {
+            tree: self.db.deref().clone(),
+            directory_validator: Some(Default::default()),
+        })
+    }
+}
+
+#[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 {
+    tree: sled::Tree,
+
+    /// 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 SledDirectoryPutter {
+    #[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::InvalidRequest("already closed".to_string())),
+            Some(validator) => {
+                // Insert all directories as a batch.
+                tokio::task::spawn_blocking({
+                    let tree = self.tree.clone();
+                    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::InvalidRequest("got no directories".to_string()))?
+                            .digest();
+
+                        let mut batch = sled::Batch::default();
+                        for directory in directories {
+                            batch.insert(
+                                directory.digest().as_slice(),
+                                proto::Directory::from(directory).encode_to_vec(),
+                            );
+                        }
+
+                        tree.apply_batch(batch).map_err(|e| {
+                            Error::StorageError(format!("unable to apply batch: {}", e))
+                        })?;
+
+                        Ok(root_digest)
+                    }
+                })
+                .await?
+            }
+        }
+    }
+}
diff --git a/tvix/castore/src/directoryservice/tests/mod.rs b/tvix/castore/src/directoryservice/tests/mod.rs
new file mode 100644
index 000000000000..ad189564bfe7
--- /dev/null
+++ b/tvix/castore/src/directoryservice/tests/mod.rs
@@ -0,0 +1,238 @@
+//! This contains test scenarios that a given [DirectoryService] needs to pass.
+//! We use [rstest] and [rstest_reuse] to provide all services we want to test
+//! against, and then apply this template to all test functions.
+
+use futures::StreamExt;
+use rstest::*;
+use rstest_reuse::{self, *};
+
+use super::DirectoryService;
+use crate::directoryservice;
+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;
+
+// TODO: add tests doing individual puts of a closure, then doing a get_recursive
+// (and figure out semantics if necessary)
+
+/// This produces a template, which will be applied to all individual test functions.
+/// See https://github.com/la10736/rstest/issues/130#issuecomment-968864832
+#[template]
+#[rstest]
+#[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), and a get_recursive
+/// returns an empty stream.
+#[apply(directory_services)]
+#[tokio::test]
+async fn test_non_exist(directory_service: impl DirectoryService) {
+    // 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
+/// `.get[_recursive]` should work.
+#[apply(directory_services)]
+#[tokio::test]
+async fn put_get(directory_service: impl DirectoryService) {
+    // Insert a Directory.
+    let digest = directory_service.put(DIRECTORY_A.clone()).await.unwrap();
+    assert_eq!(DIRECTORY_A.digest(), digest, "returned digest must match");
+
+    // single get
+    assert_eq!(
+        Some(DIRECTORY_A.clone()),
+        directory_service.get(&DIRECTORY_A.digest()).await.unwrap()
+    );
+
+    // recursive get
+    assert_eq!(
+        vec![Ok(DIRECTORY_A.clone())],
+        directory_service
+            .get_recursive(&DIRECTORY_A.digest())
+            .collect::<Vec<_>>()
+            .await
+    );
+}
+
+/// Putting a directory closure should work, and it should be possible to get
+/// back the root node both via .get[_recursive]. We don't check `.get` for the
+/// leaf node is possible, as it's Ok for stores to not support that.
+#[apply(directory_services)]
+#[tokio::test]
+async fn put_get_multiple_success(directory_service: impl DirectoryService) {
+    // Insert a Directory closure.
+    let mut handle = directory_service.put_multiple_start();
+    handle.put(DIRECTORY_A.clone()).await.unwrap();
+    handle.put(DIRECTORY_C.clone()).await.unwrap();
+    let root_digest = handle.close().await.unwrap();
+    assert_eq!(
+        DIRECTORY_C.digest(),
+        root_digest,
+        "root digest should match"
+    );
+
+    // Get the root node.
+    assert_eq!(
+        Some(DIRECTORY_C.clone()),
+        directory_service.get(&DIRECTORY_C.digest()).await.unwrap()
+    );
+
+    // Get the closure. Ensure it's sent from the root to the leaves.
+    assert_eq!(
+        vec![Ok(DIRECTORY_C.clone()), Ok(DIRECTORY_A.clone())],
+        directory_service
+            .get_recursive(&DIRECTORY_C.digest())
+            .collect::<Vec<_>>()
+            .await
+    )
+}
+
+/// Puts a directory closure, but simulates a dumb client not deduplicating
+/// its list. Ensure we still only get back a deduplicated list.
+#[apply(directory_services)]
+#[tokio::test]
+async fn put_get_multiple_dedup(directory_service: impl DirectoryService) {
+    // Insert a Directory closure.
+    let mut handle = directory_service.put_multiple_start();
+    handle.put(DIRECTORY_A.clone()).await.unwrap();
+    handle.put(DIRECTORY_A.clone()).await.unwrap();
+    handle.put(DIRECTORY_C.clone()).await.unwrap();
+    let root_digest = handle.close().await.unwrap();
+    assert_eq!(
+        DIRECTORY_C.digest(),
+        root_digest,
+        "root digest should match"
+    );
+
+    // Ensure the returned closure only contains `DIRECTORY_A` once.
+    assert_eq!(
+        vec![Ok(DIRECTORY_C.clone()), Ok(DIRECTORY_A.clone())],
+        directory_service
+            .get_recursive(&DIRECTORY_C.digest())
+            .collect::<Vec<_>>()
+            .await
+    )
+}
+
+/// 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)]
+#[tokio::test]
+async fn upload_reject_unconnected(directory_service: impl DirectoryService) {
+    let mut handle = directory_service.put_multiple_start();
+
+    handle.put(DIRECTORY_A.clone()).await.unwrap();
+    handle.put(DIRECTORY_C.clone()).await.unwrap();
+    handle.put(DIRECTORY_B.clone()).await.unwrap();
+
+    assert!(
+        handle.close().await.is_err(),
+        "closing handle should fail, as B would be left unconnected"
+    );
+}
+
+/// Uploading a directory that refers to another directory not yet uploaded
+/// should fail.
+#[apply(directory_services)]
+#[tokio::test]
+async fn upload_reject_dangling_pointer(directory_service: impl DirectoryService) {
+    let mut handle = directory_service.put_multiple_start();
+
+    // We insert DIRECTORY_A on its own, to ensure the check runs for the
+    // individual put_multiple session, not across the global DirectoryService
+    // contents.
+    directory_service.put(DIRECTORY_A.clone()).await.unwrap();
+
+    // DIRECTORY_B refers to DIRECTORY_A, which is not uploaded with this handle.
+    if handle.put(DIRECTORY_B.clone()).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::try_from_iter([(
+        "foo".try_into().unwrap(),
+        Node::Directory {
+            digest: DIRECTORY_A.digest(),
+            size: DIRECTORY_A.size() + 42, // wrong!
+        },
+    )])
+    .unwrap();
+
+    // Now upload both. Ensure it either fails during the second put, or during
+    // the close.
+    let mut handle = directory_service.put_multiple_start();
+    handle.put(DIRECTORY_A.clone()).await.unwrap();
+    if handle.put(wrong_parent_directory).await.is_ok() {
+        assert!(
+            handle.close().await.is_err(),
+            "when second put succeeds, close must fail"
+        )
+    }
+}
diff --git a/tvix/castore/src/directoryservice/tests/utils.rs b/tvix/castore/src/directoryservice/tests/utils.rs
new file mode 100644
index 000000000000..3d245ea412d5
--- /dev/null
+++ b/tvix/castore/src/directoryservice/tests/utils.rs
@@ -0,0 +1,47 @@
+use crate::directoryservice::{DirectoryService, GRPCDirectoryService};
+use crate::proto::directory_service_client::DirectoryServiceClient;
+use crate::proto::GRPCDirectoryServiceWrapper;
+use crate::{
+    directoryservice::MemoryDirectoryService,
+    proto::directory_service_server::DirectoryServiceServer,
+};
+
+use hyper_util::rt::TokioIo;
+use tonic::transport::{Endpoint, Server, Uri};
+
+/// Constructs and returns a gRPC DirectoryService.
+/// The server part is a [MemoryDirectoryService], exposed via the
+/// [GRPCDirectoryServiceWrapper], and connected through a DuplexStream.
+pub async fn make_grpc_directory_service_client() -> Box<dyn DirectoryService> {
+    let (left, right) = tokio::io::duplex(64);
+
+    // spin up a server, which will only connect once, to the left side.
+    tokio::spawn(async {
+        let directory_service =
+            Box::<MemoryDirectoryService>::default() as Box<dyn DirectoryService>;
+
+        let mut server = Server::builder();
+        let router = server.add_service(DirectoryServiceServer::new(
+            GRPCDirectoryServiceWrapper::new(directory_service),
+        ));
+
+        router
+            .serve_with_incoming(tokio_stream::once(Ok::<_, std::io::Error>(left)))
+            .await
+    });
+
+    // Create a client, connecting to the right side. The URI is unused.
+    let mut maybe_right = Some(right);
+    Box::new(GRPCDirectoryService::from_client(
+        DirectoryServiceClient::new(
+            Endpoint::try_from("http://[::]:50051")
+                .unwrap()
+                .connect_with_connector(tower::service_fn(move |_: Uri| {
+                    let right = maybe_right.take().unwrap();
+                    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
new file mode 100644
index 000000000000..0bd67e9bcf1f
--- /dev/null
+++ b/tvix/castore/src/directoryservice/traverse.rs
@@ -0,0 +1,180 @@
+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
+/// at that path, or none, if there's nothing at that path.
+#[instrument(skip(directory_service, path), fields(%path))]
+pub async fn descend_to<DS>(
+    directory_service: DS,
+    root_node: Node,
+    path: impl AsRef<Path> + std::fmt::Display,
+) -> Result<Option<Node>, Error>
+where
+    DS: AsRef<dyn DirectoryService>,
+{
+    let mut parent_node = root_node;
+    for component in path.as_ref().components_bytes() {
+        match parent_node {
+            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 { digest, .. } => {
+                // fetch the linked node from the directory_service.
+                let directory =
+                    directory_service
+                        .as_ref()
+                        .get(&digest)
+                        .await?
+                        .ok_or_else(|| {
+                            // If we didn't get the directory node that's linked, that's a store inconsistency, bail out!
+                            warn!("directory {} does not exist", digest);
+
+                            Error::StorageError(format!("directory {} does not exist", digest))
+                        })?;
+
+                // look for the component in the [Directory].
+                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.clone();
+                } else {
+                    // child node not found means there's no such element inside the directory.
+                    return Ok(None);
+                };
+            }
+        }
+    }
+
+    // We traversed the entire path, so this must be the node.
+    Ok(Some(parent_node))
+}
+
+#[cfg(test)]
+mod tests {
+    use crate::{
+        directoryservice,
+        fixtures::{DIRECTORY_COMPLICATED, DIRECTORY_WITH_KEEP, EMPTY_BLOB_DIGEST},
+        Node, PathBuf,
+    };
+
+    use super::descend_to;
+
+    #[tokio::test]
+    async fn test_descend_to() {
+        let directory_service = directoryservice::from_addr("memory://").await.unwrap();
+
+        let mut handle = directory_service.put_multiple_start();
+        handle
+            .put(DIRECTORY_WITH_KEEP.clone())
+            .await
+            .expect("must succeed");
+        handle
+            .put(DIRECTORY_COMPLICATED.clone())
+            .await
+            .expect("must succeed");
+
+        handle.close().await.expect("must upload");
+
+        // construct the node for DIRECTORY_COMPLICATED
+        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 = Node::Directory {
+            digest: DIRECTORY_WITH_KEEP.digest(),
+            size: DIRECTORY_WITH_KEEP.size(),
+        };
+
+        // construct the node for the .keep file
+        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.
+        {
+            let resp = descend_to(
+                &directory_service,
+                node_directory_complicated.clone(),
+                "".parse::<PathBuf>().unwrap(),
+            )
+            .await
+            .expect("must succeed");
+
+            assert_eq!(Some(node_directory_complicated.clone()), resp);
+        }
+
+        // traversal to `keep` should return the node for DIRECTORY_WITH_KEEP
+        {
+            let resp = descend_to(
+                &directory_service,
+                node_directory_complicated.clone(),
+                "keep".parse::<PathBuf>().unwrap(),
+            )
+            .await
+            .expect("must succeed");
+
+            assert_eq!(Some(node_directory_with_keep), resp);
+        }
+
+        // traversal to `keep/.keep` should return the node for the .keep file
+        {
+            let resp = descend_to(
+                &directory_service,
+                node_directory_complicated.clone(),
+                "keep/.keep".parse::<PathBuf>().unwrap(),
+            )
+            .await
+            .expect("must succeed");
+
+            assert_eq!(Some(node_file_keep.clone()), resp);
+        }
+
+        // traversal to `void` should return None (doesn't exist)
+        {
+            let resp = descend_to(
+                &directory_service,
+                node_directory_complicated.clone(),
+                "void".parse::<PathBuf>().unwrap(),
+            )
+            .await
+            .expect("must succeed");
+
+            assert_eq!(None, resp);
+        }
+
+        // traversal to `v/oid` should return None (doesn't exist)
+        {
+            let resp = descend_to(
+                &directory_service,
+                node_directory_complicated.clone(),
+                "v/oid".parse::<PathBuf>().unwrap(),
+            )
+            .await
+            .expect("must succeed");
+
+            assert_eq!(None, resp);
+        }
+
+        // traversal to `keep/.keep/404` should return None (the path can't be
+        // reached, as keep/.keep already is a file)
+        {
+            let resp = descend_to(
+                &directory_service,
+                node_directory_complicated.clone(),
+                "keep/.keep/foo".parse::<PathBuf>().unwrap(),
+            )
+            .await
+            .expect("must succeed");
+
+            assert_eq!(None, resp);
+        }
+    }
+}
diff --git a/tvix/castore/src/directoryservice/utils.rs b/tvix/castore/src/directoryservice/utils.rs
new file mode 100644
index 000000000000..d073c2c3c8ec
--- /dev/null
+++ b/tvix/castore/src/directoryservice/utils.rs
@@ -0,0 +1,75 @@
+use super::Directory;
+use super::DirectoryService;
+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 [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<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.
+    let mut worklist_directory_digests: VecDeque<B3Digest> =
+        VecDeque::from([root_directory_digest.clone()]);
+    // The list of directory digests already sent to the consumer.
+    // 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 = 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
+                ))
+            })? {
+                // 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.
+            sent_directory_digests.insert(current_directory_digest);
+
+            // 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.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());
+                }
+            }
+
+            yield current_directory;
+        }
+    })
+}
diff --git a/tvix/castore/src/errors.rs b/tvix/castore/src/errors.rs
new file mode 100644
index 000000000000..7b5d1a422c99
--- /dev/null
+++ b/tvix/castore/src/errors.rs
@@ -0,0 +1,138 @@
+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 {
+    #[error("invalid request: {0}")]
+    InvalidRequest(String),
+
+    #[error("internal storage error: {0}")]
+    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())
+    }
+}
+
+impl From<Error> for Status {
+    fn from(value: Error) -> Self {
+        match value {
+            Error::InvalidRequest(msg) => Status::invalid_argument(msg),
+            Error::StorageError(msg) => Status::data_loss(format!("storage error: {}", msg)),
+        }
+    }
+}
+
+impl From<crate::tonic::Error> for Error {
+    fn from(value: crate::tonic::Error) -> Self {
+        Self::StorageError(value.to_string())
+    }
+}
+
+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 {
+            Error::InvalidRequest(value.to_string())
+        } else {
+            Error::StorageError(value.to_string())
+        }
+    }
+}
+
+// TODO: this should probably go somewhere else?
+impl From<Error> for std::io::Error {
+    fn from(value: Error) -> Self {
+        match value {
+            Error::InvalidRequest(msg) => Self::new(std::io::ErrorKind::InvalidInput, msg),
+            Error::StorageError(msg) => Self::new(std::io::ErrorKind::Other, msg),
+        }
+    }
+}
diff --git a/tvix/castore/src/fixtures.rs b/tvix/castore/src/fixtures.rs
new file mode 100644
index 000000000000..05bad916d55f
--- /dev/null
+++ b/tvix/castore/src/fixtures.rs
@@ -0,0 +1,104 @@
+use crate::{B3Digest, Directory, Node};
+use lazy_static::lazy_static;
+
+pub const HELLOWORLD_BLOB_CONTENTS: &[u8] = b"Hello World!";
+pub const EMPTY_BLOB_CONTENTS: &[u8] = b"";
+
+lazy_static! {
+    pub static ref DUMMY_DIGEST: B3Digest = {
+        let u = [0u8; 32];
+        (&u).into()
+    };
+    pub static ref DUMMY_DIGEST_2: B3Digest = {
+        let mut u = [0u8; 32];
+        u[0] = 0x10;
+        (&u).into()
+    };
+    pub static ref DUMMY_DATA_1: bytes::Bytes = vec![0x01, 0x02, 0x03].into();
+    pub static ref DUMMY_DATA_2: bytes::Bytes = vec![0x04, 0x05].into();
+
+    pub static ref HELLOWORLD_BLOB_DIGEST: B3Digest =
+        blake3::hash(HELLOWORLD_BLOB_CONTENTS).as_bytes().into();
+    pub static ref EMPTY_BLOB_DIGEST: B3Digest =
+        blake3::hash(EMPTY_BLOB_CONTENTS).as_bytes().into();
+
+    // 2 bytes
+    pub static ref BLOB_A: bytes::Bytes = vec![0x00, 0x01].into();
+    pub static ref BLOB_A_DIGEST: B3Digest = blake3::hash(&BLOB_A).as_bytes().into();
+
+    // 1MB
+    pub static ref BLOB_B: bytes::Bytes = (0..255).collect::<Vec<u8>>().repeat(4 * 1024).into();
+    pub static ref BLOB_B_DIGEST: B3Digest = blake3::hash(&BLOB_B).as_bytes().into();
+
+    // Directories
+    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
+    })]).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(),
+            }
+    )]).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(),
+            }
+        ),
+        (
+            "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/file_attr.rs b/tvix/castore/src/fs/file_attr.rs
new file mode 100644
index 000000000000..2e0e70e3cdae
--- /dev/null
+++ b/tvix/castore/src/fs/file_attr.rs
@@ -0,0 +1,29 @@
+#![allow(clippy::unnecessary_cast)] // libc::S_IFDIR is u32 on Linux and u16 on MacOS
+
+use fuse_backend_rs::abi::fuse_abi::Attr;
+
+/// The [Attr] describing the root
+pub const ROOT_FILE_ATTR: Attr = Attr {
+    ino: fuse_backend_rs::api::filesystem::ROOT_ID,
+    size: 0,
+    blksize: 1024,
+    blocks: 0,
+    mode: libc::S_IFDIR as u32 | 0o555,
+    atime: 0,
+    mtime: 0,
+    ctime: 0,
+    atimensec: 0,
+    mtimensec: 0,
+    ctimensec: 0,
+    nlink: 0,
+    uid: 0,
+    gid: 0,
+    rdev: 0,
+    flags: 0,
+    #[cfg(target_os = "macos")]
+    crtime: 0,
+    #[cfg(target_os = "macos")]
+    crtimensec: 0,
+    #[cfg(target_os = "macos")]
+    padding: 0,
+};
diff --git a/tvix/castore/src/fs/fuse/mod.rs b/tvix/castore/src/fs/fuse/mod.rs
new file mode 100644
index 000000000000..64ef29ed2aa1
--- /dev/null
+++ b/tvix/castore/src/fs/fuse/mod.rs
@@ -0,0 +1,137 @@
+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,
+{
+    server: Arc<fuse_backend_rs::api::server::Server<Arc<FS>>>,
+    channel: fuse_backend_rs::transport::FuseChannel,
+}
+
+#[cfg(target_os = "macos")]
+const BADFD: libc::c_int = libc::EBADF;
+#[cfg(target_os = "linux")]
+const BADFD: libc::c_int = libc::EBADFD;
+
+impl<FS> FuseServer<FS>
+where
+    FS: FileSystem + Sync + Send,
+{
+    fn start(&mut self) -> io::Result<()> {
+        while let Some((reader, writer)) = self
+            .channel
+            .get_request()
+            .map_err(|_| io::Error::from_raw_os_error(libc::EINVAL))?
+        {
+            if let Err(e) = self
+                .server
+                .handle_message(reader, writer.into(), None, None)
+            {
+                match e {
+                    // This indicates the session has been shut down.
+                    fuse_backend_rs::Error::EncodeMessage(e) if e.raw_os_error() == Some(BADFD) => {
+                        break;
+                    }
+                    error => {
+                        error!(?error, "failed to handle fuse request");
+                        continue;
+                    }
+                }
+            }
+        }
+        Ok(())
+    }
+}
+
+/// 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: Arc<Mutex<FuseSession>>,
+    threads: Arc<ThreadPool>,
+}
+
+impl FuseDaemon {
+    #[instrument(skip(fs, mountpoint), fields(mountpoint=?mountpoint), err)]
+    pub fn new<FS, P>(
+        fs: FS,
+        mountpoint: P,
+        num_threads: usize,
+        allow_other: bool,
+    ) -> Result<Self, io::Error>
+    where
+        FS: FileSystem + Sync + Send + 'static,
+        P: AsRef<Path> + std::fmt::Debug,
+    {
+        let server = Arc::new(fuse_backend_rs::api::server::Server::new(Arc::new(fs)));
+
+        let mut session = FuseSession::new(mountpoint.as_ref(), "tvix-store", "", true)
+            .map_err(|e| io::Error::new(io::ErrorKind::Other, e.to_string()))?;
+
+        #[cfg(target_os = "linux")]
+        session.set_allow_other(allow_other);
+        session
+            .mount()
+            .map_err(|e| io::Error::new(io::ErrorKind::Other, e.to_string()))?;
+
+        // 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()))?,
+            };
+
+            threads.execute(move || {
+                let _ = server.start();
+            });
+        }
+
+        Ok(FuseDaemon {
+            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(&self) -> Result<(), io::Error> {
+        // Send the unmount command.
+        self.session
+            .lock()
+            .umount()
+            .map_err(|e| io::Error::new(io::ErrorKind::Other, e.to_string()))?;
+
+        self.wait();
+        Ok(())
+    }
+}
+
+impl Drop for FuseDaemon {
+    fn drop(&mut self) {
+        if let Err(error) = self.unmount() {
+            error!(?error, "failed to unmont fuse filesystem")
+        }
+    }
+}
diff --git a/tvix/castore/src/fs/fuse/tests.rs b/tvix/castore/src/fs/fuse/tests.rs
new file mode 100644
index 000000000000..9e01204d5da7
--- /dev/null
+++ b/tvix/castore/src/fs/fuse/tests.rs
@@ -0,0 +1,1236 @@
+use bstr::ByteSlice;
+use std::{
+    collections::BTreeMap,
+    ffi::{OsStr, OsString},
+    io::{self, Cursor},
+    os::unix::{ffi::OsStrExt, fs::MetadataExt},
+    path::Path,
+    sync::Arc,
+};
+use tempfile::TempDir;
+use tokio_stream::{wrappers::ReadDirStream, StreamExt};
+
+use super::FuseDaemon;
+use crate::{
+    blobservice::{BlobService, MemoryBlobService},
+    directoryservice::{DirectoryService, MemoryDirectoryService},
+    fixtures, Node,
+};
+use crate::{
+    fs::{TvixStoreFs, XATTR_NAME_BLOB_DIGEST, XATTR_NAME_DIRECTORY_DIGEST},
+    PathComponent,
+};
+
+const BLOB_A_NAME: &str = "00000000000000000000000000000000-test";
+const BLOB_B_NAME: &str = "55555555555555555555555555555555-test";
+const HELLOWORLD_BLOB_NAME: &str = "66666666666666666666666666666666-test";
+const SYMLINK_NAME: &str = "11111111111111111111111111111111-test";
+const SYMLINK_NAME2: &str = "44444444444444444444444444444444-test";
+const DIRECTORY_WITH_KEEP_NAME: &str = "22222222222222222222222222222222-test";
+const DIRECTORY_COMPLICATED_NAME: &str = "33333333333333333333333333333333-test";
+
+fn gen_svcs() -> (Arc<dyn BlobService>, Arc<dyn DirectoryService>) {
+    (
+        Arc::new(MemoryBlobService::default()) as Arc<dyn BlobService>,
+        Arc::new(MemoryDirectoryService::default()) as Arc<dyn DirectoryService>,
+    )
+}
+
+fn do_mount<P: AsRef<Path>, BS, DS>(
+    blob_service: BS,
+    directory_service: DS,
+    root_nodes: BTreeMap<PathComponent, Node>,
+    mountpoint: P,
+    list_root: bool,
+    show_xattr: bool,
+) -> io::Result<FuseDaemon>
+where
+    BS: AsRef<dyn BlobService> + Send + Sync + Clone + 'static,
+    DS: AsRef<dyn DirectoryService> + Send + Sync + Clone + 'static,
+{
+    let fs = TvixStoreFs::new(
+        blob_service,
+        directory_service,
+        Arc::new(root_nodes),
+        list_root,
+        show_xattr,
+    );
+    FuseDaemon::new(Arc::new(fs), mountpoint.as_ref(), 4, false)
+}
+
+async fn populate_blob_a(
+    blob_service: &Arc<dyn BlobService>,
+    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)
+        .await
+        .expect("must succeed uploading");
+    bw.close().await.expect("must succeed closing");
+
+    root_nodes.insert(
+        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<PathComponent, Node>,
+) {
+    let mut bw = blob_service.open_write().await;
+    tokio::io::copy(&mut Cursor::new(fixtures::BLOB_B.to_vec()), &mut bw)
+        .await
+        .expect("must succeed uploading");
+    bw.close().await.expect("must succeed closing");
+
+    root_nodes.insert(
+        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<PathComponent, Node>,
+) {
+    let mut bw = blob_service.open_write().await;
+    tokio::io::copy(
+        &mut Cursor::new(fixtures::HELLOWORLD_BLOB_CONTENTS.to_vec()),
+        &mut bw,
+    )
+    .await
+    .expect("must succeed uploading");
+    bw.close().await.expect("must succeed closing");
+
+    root_nodes.insert(
+        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<PathComponent, Node>) {
+    root_nodes.insert(
+        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<PathComponent, Node>) {
+    root_nodes.insert(
+        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<PathComponent, Node>,
+) {
+    // upload empty blob
+    let mut bw = blob_service.open_write().await;
+    assert_eq!(
+        fixtures::EMPTY_BLOB_DIGEST.as_slice(),
+        bw.close().await.expect("must succeed closing").as_slice(),
+    );
+
+    // upload directory
+    directory_service
+        .put(fixtures::DIRECTORY_WITH_KEEP.clone())
+        .await
+        .expect("must succeed uploading");
+
+    root_nodes.insert(
+        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<PathComponent, Node>) {
+    root_nodes.insert(
+        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<PathComponent, Node>) {
+    root_nodes.insert(
+        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<PathComponent, Node>,
+) {
+    // upload empty blob
+    let mut bw = blob_service.open_write().await;
+    assert_eq!(
+        fixtures::EMPTY_BLOB_DIGEST.as_slice(),
+        bw.close().await.expect("must succeed closing").as_slice(),
+    );
+
+    // upload inner directory
+    directory_service
+        .put(fixtures::DIRECTORY_WITH_KEEP.clone())
+        .await
+        .expect("must succeed uploading");
+
+    // upload parent directory
+    directory_service
+        .put(fixtures::DIRECTORY_COMPLICATED.clone())
+        .await
+        .expect("must succeed uploading");
+
+    root_nodes.insert(
+        DIRECTORY_COMPLICATED_NAME.try_into().unwrap(),
+        Node::Directory {
+            digest: fixtures::DIRECTORY_COMPLICATED.digest(),
+            size: fixtures::DIRECTORY_COMPLICATED.size(),
+        },
+    );
+}
+
+/// Ensure mounting itself doesn't fail
+#[tokio::test]
+async fn mount() {
+    // https://plume.benboeckel.net/~/JustAnotherBlog/skipping-tests-in-rust
+    if !std::path::Path::new("/dev/fuse").exists() {
+        eprintln!("skipping test");
+        return;
+    }
+
+    let tmpdir = TempDir::new().unwrap();
+
+    let (blob_service, directory_service) = gen_svcs();
+
+    let fuse_daemon = do_mount(
+        blob_service,
+        directory_service,
+        BTreeMap::default(),
+        tmpdir.path(),
+        false,
+        false,
+    )
+    .expect("must succeed");
+
+    fuse_daemon.unmount().expect("unmount");
+}
+/// Ensure listing the root isn't allowed
+#[tokio::test]
+async fn root() {
+    // https://plume.benboeckel.net/~/JustAnotherBlog/skipping-tests-in-rust
+    if !std::path::Path::new("/dev/fuse").exists() {
+        eprintln!("skipping test");
+        return;
+    }
+    let tmpdir = TempDir::new().unwrap();
+
+    let (blob_service, directory_service) = gen_svcs();
+    let fuse_daemon = do_mount(
+        blob_service,
+        directory_service,
+        BTreeMap::default(),
+        tmpdir.path(),
+        false,
+        false,
+    )
+    .expect("must succeed");
+
+    {
+        // read_dir fails (as opendir fails).
+        let err = tokio::fs::read_dir(tmpdir).await.expect_err("must fail");
+        assert_eq!(std::io::ErrorKind::PermissionDenied, err.kind());
+    }
+
+    fuse_daemon.unmount().expect("unmount");
+}
+
+/// Ensure listing the root is allowed if configured explicitly
+#[tokio::test(flavor = "multi_thread", worker_threads = 2)]
+async fn root_with_listing() {
+    // https://plume.benboeckel.net/~/JustAnotherBlog/skipping-tests-in-rust
+    if !std::path::Path::new("/dev/fuse").exists() {
+        eprintln!("skipping test");
+        return;
+    }
+    let tmpdir = TempDir::new().unwrap();
+
+    let (blob_service, directory_service) = gen_svcs();
+    let mut root_nodes = BTreeMap::default();
+
+    populate_blob_a(&blob_service, &mut root_nodes).await;
+
+    let fuse_daemon = do_mount(
+        blob_service,
+        directory_service,
+        root_nodes,
+        tmpdir.path(),
+        true, /* allow listing */
+        false,
+    )
+    .expect("must succeed");
+
+    {
+        // read_dir succeeds, but getting the first element will fail.
+        let mut it = ReadDirStream::new(tokio::fs::read_dir(tmpdir).await.expect("must succeed"));
+
+        let e = it
+            .next()
+            .await
+            .expect("must be some")
+            .expect("must succeed");
+
+        let metadata = e.metadata().await.expect("must succeed");
+        assert!(metadata.is_file());
+        assert!(metadata.permissions().readonly());
+        assert_eq!(fixtures::BLOB_A.len() as u64, metadata.len());
+    }
+
+    fuse_daemon.unmount().expect("unmount");
+}
+
+/// Ensure we can stat a file at the root
+#[tokio::test]
+async fn stat_file_at_root() {
+    // https://plume.benboeckel.net/~/JustAnotherBlog/skipping-tests-in-rust
+    if !std::path::Path::new("/dev/fuse").exists() {
+        eprintln!("skipping test");
+        return;
+    }
+    let tmpdir = TempDir::new().unwrap();
+
+    let (blob_service, directory_service) = gen_svcs();
+    let mut root_nodes = BTreeMap::default();
+
+    populate_blob_a(&blob_service, &mut root_nodes).await;
+
+    let fuse_daemon = do_mount(
+        blob_service,
+        directory_service,
+        root_nodes,
+        tmpdir.path(),
+        false,
+        false,
+    )
+    .expect("must succeed");
+
+    let p = tmpdir.path().join(BLOB_A_NAME);
+
+    // peek at the file metadata
+    let metadata = tokio::fs::metadata(p).await.expect("must succeed");
+
+    assert!(metadata.is_file());
+    assert!(metadata.permissions().readonly());
+    assert_eq!(fixtures::BLOB_A.len() as u64, metadata.len());
+
+    fuse_daemon.unmount().expect("unmount");
+}
+
+/// Ensure we can read a file at the root
+#[tokio::test(flavor = "multi_thread", worker_threads = 2)]
+async fn read_file_at_root() {
+    // https://plume.benboeckel.net/~/JustAnotherBlog/skipping-tests-in-rust
+    if !std::path::Path::new("/dev/fuse").exists() {
+        eprintln!("skipping test");
+        return;
+    }
+    let tmpdir = TempDir::new().unwrap();
+
+    let (blob_service, directory_service) = gen_svcs();
+    let mut root_nodes = BTreeMap::default();
+
+    populate_blob_a(&blob_service, &mut root_nodes).await;
+
+    let fuse_daemon = do_mount(
+        blob_service,
+        directory_service,
+        root_nodes,
+        tmpdir.path(),
+        false,
+        false,
+    )
+    .expect("must succeed");
+
+    let p = tmpdir.path().join(BLOB_A_NAME);
+
+    // read the file contents
+    let data = tokio::fs::read(p).await.expect("must succeed");
+
+    // ensure size and contents match
+    assert_eq!(fixtures::BLOB_A.len(), data.len());
+    assert_eq!(fixtures::BLOB_A.to_vec(), data);
+
+    fuse_daemon.unmount().expect("unmount");
+}
+
+/// Ensure we can read a large file at the root
+#[tokio::test(flavor = "multi_thread", worker_threads = 2)]
+async fn read_large_file_at_root() {
+    // https://plume.benboeckel.net/~/JustAnotherBlog/skipping-tests-in-rust
+    if !std::path::Path::new("/dev/fuse").exists() {
+        eprintln!("skipping test");
+        return;
+    }
+    let tmpdir = TempDir::new().unwrap();
+
+    let (blob_service, directory_service) = gen_svcs();
+    let mut root_nodes = BTreeMap::default();
+
+    populate_blob_b(&blob_service, &mut root_nodes).await;
+
+    let fuse_daemon = do_mount(
+        blob_service,
+        directory_service,
+        root_nodes,
+        tmpdir.path(),
+        false,
+        false,
+    )
+    .expect("must succeed");
+
+    let p = tmpdir.path().join(BLOB_B_NAME);
+    {
+        // peek at the file metadata
+        let metadata = tokio::fs::metadata(&p).await.expect("must succeed");
+
+        assert!(metadata.is_file());
+        assert!(metadata.permissions().readonly());
+        assert_eq!(fixtures::BLOB_B.len() as u64, metadata.len());
+    }
+
+    // read the file contents
+    let data = tokio::fs::read(p).await.expect("must succeed");
+
+    // ensure size and contents match
+    assert_eq!(fixtures::BLOB_B.len(), data.len());
+    assert_eq!(fixtures::BLOB_B.to_vec(), data);
+
+    fuse_daemon.unmount().expect("unmount");
+}
+
+/// Read the target of a symlink
+#[tokio::test]
+async fn symlink_readlink() {
+    // https://plume.benboeckel.net/~/JustAnotherBlog/skipping-tests-in-rust
+    if !std::path::Path::new("/dev/fuse").exists() {
+        eprintln!("skipping test");
+        return;
+    }
+    let tmpdir = TempDir::new().unwrap();
+
+    let (blob_service, directory_service) = gen_svcs();
+    let mut root_nodes = BTreeMap::default();
+
+    populate_symlink(&mut root_nodes).await;
+
+    let fuse_daemon = do_mount(
+        blob_service,
+        directory_service,
+        root_nodes,
+        tmpdir.path(),
+        false,
+        false,
+    )
+    .expect("must succeed");
+
+    let p = tmpdir.path().join(SYMLINK_NAME);
+
+    let target = tokio::fs::read_link(&p).await.expect("must succeed");
+    assert_eq!(BLOB_A_NAME, target.to_str().unwrap());
+
+    // peek at the file metadata, which follows symlinks.
+    // this must fail, as we didn't populate the target.
+    let e = tokio::fs::metadata(&p).await.expect_err("must fail");
+    assert_eq!(std::io::ErrorKind::NotFound, e.kind());
+
+    // peeking at the file metadata without following symlinks will succeed.
+    let metadata = tokio::fs::symlink_metadata(&p).await.expect("must succeed");
+    assert!(metadata.is_symlink());
+
+    // reading from the symlink (which follows) will fail, because the target doesn't exist.
+    let e = tokio::fs::read(p).await.expect_err("must fail");
+    assert_eq!(std::io::ErrorKind::NotFound, e.kind());
+
+    fuse_daemon.unmount().expect("unmount");
+}
+
+/// Read and stat a regular file through a symlink pointing to it.
+#[tokio::test(flavor = "multi_thread", worker_threads = 2)]
+async fn read_stat_through_symlink() {
+    // https://plume.benboeckel.net/~/JustAnotherBlog/skipping-tests-in-rust
+    if !std::path::Path::new("/dev/fuse").exists() {
+        eprintln!("skipping test");
+        return;
+    }
+    let tmpdir = TempDir::new().unwrap();
+
+    let (blob_service, directory_service) = gen_svcs();
+    let mut root_nodes = BTreeMap::default();
+
+    populate_blob_a(&blob_service, &mut root_nodes).await;
+    populate_symlink(&mut root_nodes).await;
+
+    let fuse_daemon = do_mount(
+        blob_service,
+        directory_service,
+        root_nodes,
+        tmpdir.path(),
+        false,
+        false,
+    )
+    .expect("must succeed");
+
+    let p_symlink = tmpdir.path().join(SYMLINK_NAME);
+    let p_blob = tmpdir.path().join(SYMLINK_NAME);
+
+    // peek at the file metadata, which follows symlinks.
+    // this must now return the same metadata as when statting at the target directly.
+    let metadata_symlink = tokio::fs::metadata(&p_symlink).await.expect("must succeed");
+    let metadata_blob = tokio::fs::metadata(&p_blob).await.expect("must succeed");
+    assert_eq!(metadata_blob.file_type(), metadata_symlink.file_type());
+    assert_eq!(metadata_blob.len(), metadata_symlink.len());
+
+    // reading from the symlink (which follows) will return the same data as if
+    // we were reading from the file directly.
+    assert_eq!(
+        tokio::fs::read(p_blob).await.expect("must succeed"),
+        tokio::fs::read(p_symlink).await.expect("must succeed"),
+    );
+
+    fuse_daemon.unmount().expect("unmount");
+}
+
+/// Read a directory in the root, and validate some attributes.
+#[tokio::test]
+async fn read_stat_directory() {
+    // https://plume.benboeckel.net/~/JustAnotherBlog/skipping-tests-in-rust
+    if !std::path::Path::new("/dev/fuse").exists() {
+        eprintln!("skipping test");
+        return;
+    }
+    let tmpdir = TempDir::new().unwrap();
+
+    let (blob_service, directory_service) = gen_svcs();
+    let mut root_nodes = BTreeMap::default();
+
+    populate_directory_with_keep(&blob_service, &directory_service, &mut root_nodes).await;
+
+    let fuse_daemon = do_mount(
+        blob_service,
+        directory_service,
+        root_nodes,
+        tmpdir.path(),
+        false,
+        false,
+    )
+    .expect("must succeed");
+
+    let p = tmpdir.path().join(DIRECTORY_WITH_KEEP_NAME);
+
+    // peek at the metadata of the directory
+    let metadata = tokio::fs::metadata(p).await.expect("must succeed");
+    assert!(metadata.is_dir());
+    assert!(metadata.permissions().readonly());
+
+    fuse_daemon.unmount().expect("unmount");
+}
+
+/// Read a directory and file in the root, and ensure the xattrs expose blob or
+/// directory digests.
+#[tokio::test]
+async fn xattr() {
+    // https://plume.benboeckel.net/~/JustAnotherBlog/skipping-tests-in-rust
+    if !std::path::Path::new("/dev/fuse").exists() {
+        eprintln!("skipping test");
+        return;
+    }
+    let tmpdir = TempDir::new().unwrap();
+
+    let (blob_service, directory_service) = gen_svcs();
+    let mut root_nodes = BTreeMap::default();
+
+    populate_directory_with_keep(&blob_service, &directory_service, &mut root_nodes).await;
+    populate_blob_a(&blob_service, &mut root_nodes).await;
+
+    let fuse_daemon = do_mount(
+        blob_service,
+        directory_service,
+        root_nodes,
+        tmpdir.path(),
+        false,
+        true, /* support xattr */
+    )
+    .expect("must succeed");
+
+    // peek at the directory
+    {
+        let p = tmpdir.path().join(DIRECTORY_WITH_KEEP_NAME);
+
+        let xattr_names: Vec<OsString> = xattr::list(&p).expect("must succeed").collect();
+        // There should be 1 key, XATTR_NAME_DIRECTORY_DIGEST.
+        assert_eq!(1, xattr_names.len(), "there should be 1 xattr name");
+        assert_eq!(
+            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(XATTR_NAME_DIRECTORY_DIGEST))
+            .expect("must succeed")
+            .expect("must be some");
+        assert_eq!(
+            fixtures::DIRECTORY_WITH_KEEP
+                .digest()
+                .to_string()
+                .as_bytes()
+                .as_bstr(),
+            val.as_bstr()
+        );
+
+        // Reading another xattr key is gonna return None.
+        let val = xattr::get(&p, OsStr::from_bytes(b"user.cheesecake")).expect("must succeed");
+        assert_eq!(None, val);
+    }
+    // peek at the file
+    {
+        let p = tmpdir.path().join(BLOB_A_NAME);
+
+        let xattr_names: Vec<OsString> = xattr::list(&p).expect("must succeed").collect();
+        // There should be 1 key, XATTR_NAME_BLOB_DIGEST.
+        assert_eq!(1, xattr_names.len(), "there should be 1 xattr name");
+        assert_eq!(
+            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(XATTR_NAME_BLOB_DIGEST))
+            .expect("must succeed")
+            .expect("must be some");
+        assert_eq!(
+            fixtures::BLOB_A_DIGEST.to_string().as_bytes().as_bstr(),
+            val.as_bstr()
+        );
+
+        // Reading another xattr key is gonna return None.
+        let val = xattr::get(&p, OsStr::from_bytes(b"user.cheesecake")).expect("must succeed");
+        assert_eq!(None, val);
+    }
+
+    fuse_daemon.unmount().expect("unmount");
+}
+
+#[tokio::test(flavor = "multi_thread", worker_threads = 2)]
+/// Read a blob inside a directory. This ensures we successfully populate directory data.
+async fn read_blob_inside_dir() {
+    // https://plume.benboeckel.net/~/JustAnotherBlog/skipping-tests-in-rust
+    if !std::path::Path::new("/dev/fuse").exists() {
+        eprintln!("skipping test");
+        return;
+    }
+    let tmpdir = TempDir::new().unwrap();
+
+    let (blob_service, directory_service) = gen_svcs();
+    let mut root_nodes = BTreeMap::default();
+
+    populate_directory_with_keep(&blob_service, &directory_service, &mut root_nodes).await;
+
+    let fuse_daemon = do_mount(
+        blob_service,
+        directory_service,
+        root_nodes,
+        tmpdir.path(),
+        false,
+        false,
+    )
+    .expect("must succeed");
+
+    let p = tmpdir.path().join(DIRECTORY_WITH_KEEP_NAME).join(".keep");
+
+    // peek at metadata.
+    let metadata = tokio::fs::metadata(&p).await.expect("must succeed");
+    assert!(metadata.is_file());
+    assert!(metadata.permissions().readonly());
+
+    // read from it
+    let data = tokio::fs::read(&p).await.expect("must succeed");
+    assert_eq!(fixtures::EMPTY_BLOB_CONTENTS.to_vec(), data);
+
+    fuse_daemon.unmount().expect("unmount");
+}
+
+#[tokio::test(flavor = "multi_thread", worker_threads = 2)]
+/// Read a blob inside a directory inside a directory. This ensures we properly
+/// populate directories as we traverse down the structure.
+async fn read_blob_deep_inside_dir() {
+    // https://plume.benboeckel.net/~/JustAnotherBlog/skipping-tests-in-rust
+    if !std::path::Path::new("/dev/fuse").exists() {
+        eprintln!("skipping test");
+        return;
+    }
+    let tmpdir = TempDir::new().unwrap();
+
+    let (blob_service, directory_service) = gen_svcs();
+    let mut root_nodes = BTreeMap::default();
+
+    populate_directory_complicated(&blob_service, &directory_service, &mut root_nodes).await;
+
+    let fuse_daemon = do_mount(
+        blob_service,
+        directory_service,
+        root_nodes,
+        tmpdir.path(),
+        false,
+        false,
+    )
+    .expect("must succeed");
+
+    let p = tmpdir
+        .path()
+        .join(DIRECTORY_COMPLICATED_NAME)
+        .join("keep")
+        .join(".keep");
+
+    // peek at metadata.
+    let metadata = tokio::fs::metadata(&p).await.expect("must succeed");
+    assert!(metadata.is_file());
+    assert!(metadata.permissions().readonly());
+
+    // read from it
+    let data = tokio::fs::read(&p).await.expect("must succeed");
+    assert_eq!(fixtures::EMPTY_BLOB_CONTENTS.to_vec(), data);
+
+    fuse_daemon.unmount().expect("unmount");
+}
+
+/// Ensure readdir works.
+#[tokio::test]
+async fn readdir() {
+    // https://plume.benboeckel.net/~/JustAnotherBlog/skipping-tests-in-rust
+    if !std::path::Path::new("/dev/fuse").exists() {
+        eprintln!("skipping test");
+        return;
+    }
+    let tmpdir = TempDir::new().unwrap();
+
+    let (blob_service, directory_service) = gen_svcs();
+    let mut root_nodes = BTreeMap::default();
+
+    populate_directory_complicated(&blob_service, &directory_service, &mut root_nodes).await;
+
+    let fuse_daemon = do_mount(
+        blob_service,
+        directory_service,
+        root_nodes,
+        tmpdir.path(),
+        false,
+        false,
+    )
+    .expect("must succeed");
+
+    let p = tmpdir.path().join(DIRECTORY_COMPLICATED_NAME);
+
+    {
+        // read_dir should succeed. Collect all elements
+        let elements: Vec<_> =
+            ReadDirStream::new(tokio::fs::read_dir(p).await.expect("must succeed"))
+                .map(|e| e.expect("must not be err"))
+                .collect()
+                .await;
+
+        assert_eq!(3, elements.len(), "number of elements should be 3"); // rust skips . and ..
+
+        // We explicitly look at specific positions here, because we always emit
+        // them ordered.
+
+        // ".keep", 0 byte file.
+        let e = &elements[0];
+        assert_eq!(".keep", e.file_name());
+        assert!(e.file_type().await.expect("must succeed").is_file());
+        assert_eq!(0, e.metadata().await.expect("must succeed").len());
+
+        // "aa", symlink.
+        let e = &elements[1];
+        assert_eq!("aa", e.file_name());
+        assert!(e.file_type().await.expect("must succeed").is_symlink());
+
+        // "keep", directory
+        let e = &elements[2];
+        assert_eq!("keep", e.file_name());
+        assert!(e.file_type().await.expect("must succeed").is_dir());
+    }
+
+    fuse_daemon.unmount().expect("unmount");
+}
+
+#[tokio::test]
+/// Do a readdir deeper inside a directory, without doing readdir or stat in the parent directory.
+async fn readdir_deep() {
+    // https://plume.benboeckel.net/~/JustAnotherBlog/skipping-tests-in-rust
+    if !std::path::Path::new("/dev/fuse").exists() {
+        eprintln!("skipping test");
+        return;
+    }
+    let tmpdir = TempDir::new().unwrap();
+
+    let (blob_service, directory_service) = gen_svcs();
+    let mut root_nodes = BTreeMap::default();
+
+    populate_directory_complicated(&blob_service, &directory_service, &mut root_nodes).await;
+
+    let fuse_daemon = do_mount(
+        blob_service,
+        directory_service,
+        root_nodes,
+        tmpdir.path(),
+        false,
+        false,
+    )
+    .expect("must succeed");
+
+    let p = tmpdir.path().join(DIRECTORY_COMPLICATED_NAME).join("keep");
+
+    {
+        // read_dir should succeed. Collect all elements
+        let elements: Vec<_> =
+            ReadDirStream::new(tokio::fs::read_dir(p).await.expect("must succeed"))
+                .map(|e| e.expect("must not be err"))
+                .collect()
+                .await;
+
+        assert_eq!(1, elements.len(), "number of elements should be 1"); // rust skips . and ..
+
+        // ".keep", 0 byte file.
+        let e = &elements[0];
+        assert_eq!(".keep", e.file_name());
+        assert!(e.file_type().await.expect("must succeed").is_file());
+        assert_eq!(0, e.metadata().await.expect("must succeed").len());
+    }
+
+    fuse_daemon.unmount().expect("unmount");
+}
+
+/// Check attributes match how they show up in /nix/store normally.
+#[tokio::test]
+async fn check_attributes() {
+    // https://plume.benboeckel.net/~/JustAnotherBlog/skipping-tests-in-rust
+    if !std::path::Path::new("/dev/fuse").exists() {
+        eprintln!("skipping test");
+        return;
+    }
+    let tmpdir = TempDir::new().unwrap();
+
+    let (blob_service, directory_service) = gen_svcs();
+    let mut root_nodes = BTreeMap::default();
+
+    populate_blob_a(&blob_service, &mut root_nodes).await;
+    populate_directory_with_keep(&blob_service, &directory_service, &mut root_nodes).await;
+    populate_symlink(&mut root_nodes).await;
+    populate_blob_helloworld(&blob_service, &mut root_nodes).await;
+
+    let fuse_daemon = do_mount(
+        blob_service,
+        directory_service,
+        root_nodes,
+        tmpdir.path(),
+        false,
+        false,
+    )
+    .expect("must succeed");
+
+    let p_file = tmpdir.path().join(BLOB_A_NAME);
+    let p_directory = tmpdir.path().join(DIRECTORY_WITH_KEEP_NAME);
+    let p_symlink = tmpdir.path().join(SYMLINK_NAME);
+    let p_executable_file = tmpdir.path().join(HELLOWORLD_BLOB_NAME);
+
+    // peek at metadata. We use symlink_metadata to ensure we don't traverse a symlink by accident.
+    let metadata_file = tokio::fs::symlink_metadata(&p_file)
+        .await
+        .expect("must succeed");
+    let metadata_executable_file = tokio::fs::symlink_metadata(&p_executable_file)
+        .await
+        .expect("must succeed");
+    let metadata_directory = tokio::fs::symlink_metadata(&p_directory)
+        .await
+        .expect("must succeed");
+    let metadata_symlink = tokio::fs::symlink_metadata(&p_symlink)
+        .await
+        .expect("must succeed");
+
+    // modes should match. We & with 0o777 to remove any higher bits.
+    assert_eq!(0o444, metadata_file.mode() & 0o777);
+    assert_eq!(0o555, metadata_executable_file.mode() & 0o777);
+    assert_eq!(0o555, metadata_directory.mode() & 0o777);
+    assert_eq!(0o444, metadata_symlink.mode() & 0o777);
+
+    // files should have the correct filesize
+    assert_eq!(fixtures::BLOB_A.len() as u64, metadata_file.len());
+    // directories should have their "size" as filesize
+    assert_eq!(
+        { fixtures::DIRECTORY_WITH_KEEP.size() },
+        metadata_directory.size()
+    );
+
+    for metadata in &[&metadata_file, &metadata_directory, &metadata_symlink] {
+        // uid and gid should be 0.
+        assert_eq!(0, metadata.uid());
+        assert_eq!(0, metadata.gid());
+
+        // all times should be set to the unix epoch.
+        assert_eq!(0, metadata.atime());
+        assert_eq!(0, metadata.mtime());
+        assert_eq!(0, metadata.ctime());
+        // crtime seems MacOS only
+    }
+
+    fuse_daemon.unmount().expect("unmount");
+}
+
+#[tokio::test]
+/// Ensure we allocate the same inodes for the same directory contents.
+/// $DIRECTORY_COMPLICATED_NAME/keep contains the same data as $DIRECTORY_WITH_KEEP.
+async fn compare_inodes_directories() {
+    // https://plume.benboeckel.net/~/JustAnotherBlog/skipping-tests-in-rust
+    if !std::path::Path::new("/dev/fuse").exists() {
+        eprintln!("skipping test");
+        return;
+    }
+    let tmpdir = TempDir::new().unwrap();
+
+    let (blob_service, directory_service) = gen_svcs();
+    let mut root_nodes = BTreeMap::default();
+
+    populate_directory_with_keep(&blob_service, &directory_service, &mut root_nodes).await;
+    populate_directory_complicated(&blob_service, &directory_service, &mut root_nodes).await;
+
+    let fuse_daemon = do_mount(
+        blob_service,
+        directory_service,
+        root_nodes,
+        tmpdir.path(),
+        false,
+        false,
+    )
+    .expect("must succeed");
+
+    let p_dir_with_keep = tmpdir.path().join(DIRECTORY_WITH_KEEP_NAME);
+    let p_sibling_dir = tmpdir.path().join(DIRECTORY_COMPLICATED_NAME).join("keep");
+
+    // peek at metadata.
+    assert_eq!(
+        tokio::fs::metadata(p_dir_with_keep)
+            .await
+            .expect("must succeed")
+            .ino(),
+        tokio::fs::metadata(p_sibling_dir)
+            .await
+            .expect("must succeed")
+            .ino()
+    );
+
+    fuse_daemon.unmount().expect("unmount");
+}
+
+/// Ensure we allocate the same inodes for the same directory contents.
+/// $DIRECTORY_COMPLICATED_NAME/keep/,keep contains the same data as $DIRECTORY_COMPLICATED_NAME/.keep
+#[tokio::test]
+async fn compare_inodes_files() {
+    // https://plume.benboeckel.net/~/JustAnotherBlog/skipping-tests-in-rust
+    if !std::path::Path::new("/dev/fuse").exists() {
+        eprintln!("skipping test");
+        return;
+    }
+    let tmpdir = TempDir::new().unwrap();
+
+    let (blob_service, directory_service) = gen_svcs();
+    let mut root_nodes = BTreeMap::default();
+
+    populate_directory_complicated(&blob_service, &directory_service, &mut root_nodes).await;
+
+    let fuse_daemon = do_mount(
+        blob_service,
+        directory_service,
+        root_nodes,
+        tmpdir.path(),
+        false,
+        false,
+    )
+    .expect("must succeed");
+
+    let p_keep1 = tmpdir.path().join(DIRECTORY_COMPLICATED_NAME).join(".keep");
+    let p_keep2 = tmpdir
+        .path()
+        .join(DIRECTORY_COMPLICATED_NAME)
+        .join("keep")
+        .join(".keep");
+
+    // peek at metadata.
+    assert_eq!(
+        tokio::fs::metadata(p_keep1)
+            .await
+            .expect("must succeed")
+            .ino(),
+        tokio::fs::metadata(p_keep2)
+            .await
+            .expect("must succeed")
+            .ino()
+    );
+
+    fuse_daemon.unmount().expect("unmount");
+}
+
+/// Ensure we allocate the same inode for symlinks pointing to the same targets.
+/// $DIRECTORY_COMPLICATED_NAME/aa points to the same target as SYMLINK_NAME2.
+#[tokio::test]
+async fn compare_inodes_symlinks() {
+    // https://plume.benboeckel.net/~/JustAnotherBlog/skipping-tests-in-rust
+    if !std::path::Path::new("/dev/fuse").exists() {
+        eprintln!("skipping test");
+        return;
+    }
+    let tmpdir = TempDir::new().unwrap();
+
+    let (blob_service, directory_service) = gen_svcs();
+    let mut root_nodes = BTreeMap::default();
+
+    populate_directory_complicated(&blob_service, &directory_service, &mut root_nodes).await;
+    populate_symlink2(&mut root_nodes).await;
+
+    let fuse_daemon = do_mount(
+        blob_service,
+        directory_service,
+        root_nodes,
+        tmpdir.path(),
+        false,
+        false,
+    )
+    .expect("must succeed");
+
+    let p1 = tmpdir.path().join(DIRECTORY_COMPLICATED_NAME).join("aa");
+    let p2 = tmpdir.path().join(SYMLINK_NAME2);
+
+    // peek at metadata.
+    assert_eq!(
+        tokio::fs::symlink_metadata(p1)
+            .await
+            .expect("must succeed")
+            .ino(),
+        tokio::fs::symlink_metadata(p2)
+            .await
+            .expect("must succeed")
+            .ino()
+    );
+
+    fuse_daemon.unmount().expect("unmount");
+}
+
+/// Check we match paths exactly.
+#[tokio::test]
+async fn read_wrong_paths_in_root() {
+    // https://plume.benboeckel.net/~/JustAnotherBlog/skipping-tests-in-rust
+    if !std::path::Path::new("/dev/fuse").exists() {
+        eprintln!("skipping test");
+        return;
+    }
+    let tmpdir = TempDir::new().unwrap();
+
+    let (blob_service, directory_service) = gen_svcs();
+    let mut root_nodes = BTreeMap::default();
+
+    populate_blob_a(&blob_service, &mut root_nodes).await;
+
+    let fuse_daemon = do_mount(
+        blob_service,
+        directory_service,
+        root_nodes,
+        tmpdir.path(),
+        false,
+        false,
+    )
+    .expect("must succeed");
+
+    // wrong name
+    assert!(
+        tokio::fs::metadata(tmpdir.path().join("00000000000000000000000000000000-tes"))
+            .await
+            .is_err()
+    );
+
+    // invalid hash
+    assert!(
+        tokio::fs::metadata(tmpdir.path().join("0000000000000000000000000000000-test"))
+            .await
+            .is_err()
+    );
+
+    // right name, must exist
+    assert!(
+        tokio::fs::metadata(tmpdir.path().join("00000000000000000000000000000000-test"))
+            .await
+            .is_ok()
+    );
+
+    // now wrong name with right hash still may not exist
+    assert!(
+        tokio::fs::metadata(tmpdir.path().join("00000000000000000000000000000000-tes"))
+            .await
+            .is_err()
+    );
+
+    fuse_daemon.unmount().expect("unmount");
+}
+
+/// Make sure writes are not allowed
+#[tokio::test]
+async fn disallow_writes() {
+    // https://plume.benboeckel.net/~/JustAnotherBlog/skipping-tests-in-rust
+    if !std::path::Path::new("/dev/fuse").exists() {
+        eprintln!("skipping test");
+        return;
+    }
+
+    let tmpdir = TempDir::new().unwrap();
+
+    let (blob_service, directory_service) = gen_svcs();
+    let root_nodes = BTreeMap::default();
+
+    let fuse_daemon = do_mount(
+        blob_service,
+        directory_service,
+        root_nodes,
+        tmpdir.path(),
+        false,
+        false,
+    )
+    .expect("must succeed");
+
+    let p = tmpdir.path().join(BLOB_A_NAME);
+    let e = tokio::fs::File::create(p).await.expect_err("must fail");
+
+    assert_eq!(Some(libc::EROFS), e.raw_os_error());
+
+    fuse_daemon.unmount().expect("unmount");
+}
+
+#[tokio::test]
+/// Ensure we get an IO error if the directory service does not have the Directory object.
+async fn missing_directory() {
+    if !std::path::Path::new("/dev/fuse").exists() {
+        eprintln!("skipping test");
+        return;
+    }
+    let tmpdir = TempDir::new().unwrap();
+
+    let (blob_service, directory_service) = gen_svcs();
+    let mut root_nodes = BTreeMap::default();
+
+    populate_directorynode_without_directory(&mut root_nodes).await;
+
+    let fuse_daemon = do_mount(
+        blob_service,
+        directory_service,
+        root_nodes,
+        tmpdir.path(),
+        false,
+        false,
+    )
+    .expect("must succeed");
+
+    let p = tmpdir.path().join(DIRECTORY_WITH_KEEP_NAME);
+
+    {
+        // `stat` on the path should succeed, because it doesn't trigger the directory request.
+        tokio::fs::metadata(&p).await.expect("must succeed");
+
+        // However, calling either `readdir` or `stat` on a child should fail with an IO error.
+        // It fails when trying to pull the first entry, because we don't implement opendir separately
+        ReadDirStream::new(tokio::fs::read_dir(&p).await.unwrap())
+            .next()
+            .await
+            .expect("must be some")
+            .expect_err("must be err");
+
+        // rust currently sets e.kind() to Uncategorized, which isn't very
+        // helpful, so we don't look at the error more closely than that..
+        tokio::fs::metadata(p.join(".keep"))
+            .await
+            .expect_err("must fail");
+    }
+
+    fuse_daemon.unmount().expect("unmount");
+}
+
+#[tokio::test(flavor = "multi_thread", worker_threads = 2)]
+/// Ensure we get an IO error if the blob service does not have the blob
+async fn missing_blob() {
+    if !std::path::Path::new("/dev/fuse").exists() {
+        eprintln!("skipping test");
+        return;
+    }
+    let tmpdir = TempDir::new().unwrap();
+
+    let (blob_service, directory_service) = gen_svcs();
+    let mut root_nodes = BTreeMap::default();
+
+    populate_filenode_without_blob(&mut root_nodes).await;
+
+    let fuse_daemon = do_mount(
+        blob_service,
+        directory_service,
+        root_nodes,
+        tmpdir.path(),
+        false,
+        false,
+    )
+    .expect("must succeed");
+
+    let p = tmpdir.path().join(BLOB_A_NAME);
+
+    {
+        // `stat` on the blob should succeed, because it doesn't trigger a request to the blob service.
+        tokio::fs::metadata(&p).await.expect("must succeed");
+
+        // However, calling read on the blob should fail.
+        // rust currently sets e.kind() to Uncategorized, which isn't very
+        // helpful, so we don't look at the error more closely than that..
+        tokio::fs::read(p).await.expect_err("must fail");
+    }
+
+    fuse_daemon.unmount().expect("unmount");
+}
diff --git a/tvix/castore/src/fs/inode_tracker.rs b/tvix/castore/src/fs/inode_tracker.rs
new file mode 100644
index 000000000000..4a8283b6b144
--- /dev/null
+++ b/tvix/castore/src/fs/inode_tracker.rs
@@ -0,0 +1,207 @@
+use std::{collections::HashMap, sync::Arc};
+
+use super::inodes::{DirectoryInodeData, InodeData};
+use crate::B3Digest;
+
+/// InodeTracker keeps track of inodes, stores data being these inodes and deals
+/// with inode allocation.
+pub struct InodeTracker {
+    data: HashMap<u64, Arc<InodeData>>,
+
+    // lookup table for blobs by their B3Digest
+    blob_digest_to_inode: HashMap<B3Digest, u64>,
+
+    // lookup table for symlinks by their target
+    symlink_target_to_inode: HashMap<bytes::Bytes, u64>,
+
+    // lookup table for directories by their B3Digest.
+    // Note the corresponding directory may not be present in data yet.
+    directory_digest_to_inode: HashMap<B3Digest, u64>,
+
+    // the next inode to allocate
+    next_inode: u64,
+}
+
+impl Default for InodeTracker {
+    fn default() -> Self {
+        Self {
+            data: Default::default(),
+
+            blob_digest_to_inode: Default::default(),
+            symlink_target_to_inode: Default::default(),
+            directory_digest_to_inode: Default::default(),
+
+            next_inode: 2,
+        }
+    }
+}
+
+impl InodeTracker {
+    // Retrieves data for a given inode, if it exists.
+    pub fn get(&self, ino: u64) -> Option<Arc<InodeData>> {
+        self.data.get(&ino).cloned()
+    }
+
+    // Replaces data for a given inode.
+    // Panics if the inode doesn't already exist.
+    pub fn replace(&mut self, ino: u64, data: Arc<InodeData>) {
+        if self.data.insert(ino, data).is_none() {
+            panic!("replace called on unknown inode");
+        }
+    }
+
+    // Stores data and returns the inode for it.
+    // In case an inode has already been allocated for the same data, that inode
+    // is returned, otherwise a new one is allocated.
+    // In case data is a [InodeData::Directory], inodes for all items are looked
+    // up
+    pub fn put(&mut self, data: InodeData) -> u64 {
+        match data {
+            InodeData::Regular(ref digest, _, _) => {
+                match self.blob_digest_to_inode.get(digest) {
+                    Some(found_ino) => {
+                        // We already have it, return the inode.
+                        *found_ino
+                    }
+                    None => self.insert_and_increment(data),
+                }
+            }
+            InodeData::Symlink(ref target) => {
+                match self.symlink_target_to_inode.get(target) {
+                    Some(found_ino) => {
+                        // We already have it, return the inode.
+                        *found_ino
+                    }
+                    None => self.insert_and_increment(data),
+                }
+            }
+            InodeData::Directory(DirectoryInodeData::Sparse(ref digest, _size)) => {
+                // check the lookup table if the B3Digest is known.
+                match self.directory_digest_to_inode.get(digest) {
+                    Some(found_ino) => {
+                        // We already have it, return the inode.
+                        *found_ino
+                    }
+                    None => {
+                        // insert and return the inode
+                        self.insert_and_increment(data)
+                    }
+                }
+            }
+            // Inserting [DirectoryInodeData::Populated] doesn't normally happen,
+            // only via [replace].
+            InodeData::Directory(DirectoryInodeData::Populated(..)) => {
+                unreachable!("should never be called with DirectoryInodeData::Populated")
+            }
+        }
+    }
+
+    // Inserts the data and returns the inode it was stored at, while
+    // incrementing next_inode.
+    fn insert_and_increment(&mut self, data: InodeData) -> u64 {
+        let ino = self.next_inode;
+        // insert into lookup tables
+        match data {
+            InodeData::Regular(ref digest, _, _) => {
+                self.blob_digest_to_inode.insert(digest.clone(), ino);
+            }
+            InodeData::Symlink(ref target) => {
+                self.symlink_target_to_inode.insert(target.clone(), ino);
+            }
+            InodeData::Directory(DirectoryInodeData::Sparse(ref digest, _size)) => {
+                self.directory_digest_to_inode.insert(digest.clone(), ino);
+            }
+            // This is currently not used outside test fixtures.
+            // Usually a [DirectoryInodeData::Sparse] is inserted and later
+            // "upgraded" with more data.
+            // However, as a future optimization, a lookup for a PathInfo could trigger a
+            // [DirectoryService::get_recursive()] request that "forks into
+            // background" and prepopulates all Directories in a closure.
+            InodeData::Directory(DirectoryInodeData::Populated(ref digest, _)) => {
+                self.directory_digest_to_inode.insert(digest.clone(), ino);
+            }
+        }
+        // Insert data
+        self.data.insert(ino, Arc::new(data));
+
+        // increment inode counter and return old inode.
+        self.next_inode += 1;
+        ino
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use crate::fixtures;
+
+    use super::InodeData;
+    use super::InodeTracker;
+
+    /// Getting something non-existent should be none
+    #[test]
+    fn get_nonexistent() {
+        let inode_tracker = InodeTracker::default();
+        assert!(inode_tracker.get(1).is_none());
+    }
+
+    /// Put of a regular file should allocate a uid, which should be the same when inserting again.
+    #[test]
+    fn put_regular() {
+        let mut inode_tracker = InodeTracker::default();
+        let f = InodeData::Regular(
+            fixtures::BLOB_A_DIGEST.clone(),
+            fixtures::BLOB_A.len() as u64,
+            false,
+        );
+
+        // put it in
+        let ino = inode_tracker.put(f.clone());
+
+        // a get should return the right data
+        let data = inode_tracker.get(ino).expect("must be some");
+        match *data {
+            InodeData::Regular(ref digest, _, _) => {
+                assert_eq!(&fixtures::BLOB_A_DIGEST.clone(), digest);
+            }
+            InodeData::Symlink(_) | InodeData::Directory(..) => panic!("wrong type"),
+        }
+
+        // another put should return the same ino
+        assert_eq!(ino, inode_tracker.put(f));
+
+        // inserting another file should return a different ino
+        assert_ne!(
+            ino,
+            inode_tracker.put(InodeData::Regular(
+                fixtures::BLOB_B_DIGEST.clone(),
+                fixtures::BLOB_B.len() as u64,
+                false,
+            ))
+        );
+    }
+
+    // Put of a symlink should allocate a uid, which should be the same when inserting again
+    #[test]
+    fn put_symlink() {
+        let mut inode_tracker = InodeTracker::default();
+        let f = InodeData::Symlink("target".into());
+
+        // put it in
+        let ino = inode_tracker.put(f.clone());
+
+        // a get should return the right data
+        let data = inode_tracker.get(ino).expect("must be some");
+        match *data {
+            InodeData::Symlink(ref target) => {
+                assert_eq!(b"target".to_vec(), *target);
+            }
+            InodeData::Regular(..) | InodeData::Directory(..) => panic!("wrong type"),
+        }
+
+        // another put should return the same ino
+        assert_eq!(ino, inode_tracker.put(f));
+
+        // inserting another file should return a different ino
+        assert_ne!(ino, inode_tracker.put(InodeData::Symlink("target2".into())));
+    }
+}
diff --git a/tvix/castore/src/fs/inodes.rs b/tvix/castore/src/fs/inodes.rs
new file mode 100644
index 000000000000..2696fdede378
--- /dev/null
+++ b/tvix/castore/src/fs/inodes.rs
@@ -0,0 +1,89 @@
+//! This module contains all the data structures used to track information
+//! about inodes, which present tvix-castore nodes in a filesystem.
+use std::time::Duration;
+
+use crate::{path::PathComponent, B3Digest, Node};
+
+#[derive(Clone, Debug)]
+pub enum InodeData {
+    Regular(B3Digest, u64, bool),  // digest, size, executable
+    Symlink(bytes::Bytes),         // target
+    Directory(DirectoryInodeData), // either [DirectoryInodeData:Sparse] or [DirectoryInodeData:Populated]
+}
+
+/// This encodes the two different states of [InodeData::Directory].
+/// Either the data still is sparse (we only saw a [castorepb::DirectoryNode],
+/// but didn't fetch the [castorepb::Directory] struct yet, or we processed a
+/// lookup and did fetch the data.
+#[derive(Clone, Debug)]
+pub enum DirectoryInodeData {
+    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].
+    pub fn from_node(node: &Node) -> Self {
+        match node {
+            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()),
+        }
+    }
+
+    pub fn as_fuse_file_attr(&self, inode: u64) -> fuse_backend_rs::abi::fuse_abi::Attr {
+        fuse_backend_rs::abi::fuse_abi::Attr {
+            ino: inode,
+            // FUTUREWORK: play with this numbers, as it affects read sizes for client applications.
+            blocks: 1024,
+            size: match self {
+                InodeData::Regular(_, size, _) => *size,
+                InodeData::Symlink(target) => target.len() as u64,
+                InodeData::Directory(DirectoryInodeData::Sparse(_, size)) => *size,
+                InodeData::Directory(DirectoryInodeData::Populated(_, ref children)) => {
+                    children.len() as u64
+                }
+            },
+            mode: self.as_fuse_type() | self.mode(),
+            ..Default::default()
+        }
+    }
+
+    fn mode(&self) -> u32 {
+        match self {
+            InodeData::Regular(_, _, false) | InodeData::Symlink(_) => 0o444,
+            InodeData::Regular(_, _, true) | InodeData::Directory(_) => 0o555,
+        }
+    }
+
+    pub fn as_fuse_entry(&self, inode: u64) -> fuse_backend_rs::api::filesystem::Entry {
+        fuse_backend_rs::api::filesystem::Entry {
+            inode,
+            attr: self.as_fuse_file_attr(inode).into(),
+            attr_timeout: Duration::MAX,
+            entry_timeout: Duration::MAX,
+            ..Default::default()
+        }
+    }
+
+    /// Returns the u32 fuse type
+    pub fn as_fuse_type(&self) -> u32 {
+        #[allow(clippy::let_and_return)]
+        let ty = match self {
+            InodeData::Regular(_, _, _) => libc::S_IFREG,
+            InodeData::Symlink(_) => libc::S_IFLNK,
+            InodeData::Directory(_) => libc::S_IFDIR,
+        };
+        // libc::S_IFDIR is u32 on Linux and u16 on MacOS
+        #[cfg(target_os = "macos")]
+        let ty = ty as u32;
+
+        ty
+    }
+}
diff --git a/tvix/castore/src/fs/mod.rs b/tvix/castore/src/fs/mod.rs
new file mode 100644
index 000000000000..e700a25d3966
--- /dev/null
+++ b/tvix/castore/src/fs/mod.rs
@@ -0,0 +1,881 @@
+mod file_attr;
+mod inode_tracker;
+mod inodes;
+mod root_nodes;
+
+#[cfg(feature = "fuse")]
+pub mod fuse;
+
+#[cfg(feature = "virtiofs")]
+pub mod virtiofs;
+
+pub use self::root_nodes::RootNodes;
+use self::{
+    file_attr::ROOT_FILE_ATTR,
+    inode_tracker::InodeTracker,
+    inodes::{DirectoryInodeData, InodeData},
+};
+use crate::{
+    blobservice::{BlobReader, BlobService},
+    directoryservice::DirectoryService,
+    path::PathComponent,
+    B3Digest, Node,
+};
+use bstr::ByteVec;
+use fuse_backend_rs::abi::fuse_abi::{stat64, OpenOptions};
+use fuse_backend_rs::api::filesystem::{
+    Context, FileSystem, FsOptions, GetxattrReply, ListxattrReply, ROOT_ID,
+};
+use futures::StreamExt;
+use parking_lot::RwLock;
+use std::sync::Mutex;
+use std::{
+    collections::HashMap,
+    io,
+    sync::atomic::AtomicU64,
+    sync::{atomic::Ordering, Arc},
+    time::Duration,
+};
+use std::{ffi::CStr, io::Cursor};
+use tokio::{
+    io::{AsyncReadExt, AsyncSeekExt},
+    sync::mpsc,
+};
+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].
+///
+/// Linux uses inodes in filesystems. When implementing FUSE, most calls are
+/// *for* a given inode.
+///
+/// This means, we need to have a stable mapping of inode numbers to the
+/// corresponding store nodes.
+///
+/// We internally delegate all inode allocation and state keeping to the
+/// inode tracker.
+/// We store a mapping from currently "explored" names in the root to their
+/// inode.
+///
+/// There's some places where inodes are allocated / data inserted into
+/// the inode tracker, if not allocated before already:
+///  - Processing a `lookup` request, either in the mount root, or somewhere
+///    deeper.
+///  - Processing a `readdir` request
+///
+///  Things pointing to the same contents get the same inodes, irrespective of
+///  their own location.
+///  This means:
+///  - Symlinks with the same target will get the same inode.
+///  - Regular/executable files with the same contents will get the same inode
+///  - Directories with the same contents will get the same inode.
+///
+/// Due to the above being valid across the whole store, and considering the
+/// merkle structure is a DAG, not a tree, this also means we can't do "bucketed
+/// allocation", aka reserve Directory.size inodes for each directory node we
+/// explore.
+/// Tests for this live in the tvix-store crate.
+pub struct TvixStoreFs<BS, DS, RN> {
+    blob_service: BS,
+    directory_service: DS,
+    root_nodes_provider: RN,
+
+    /// Whether to (try) listing elements in the root.
+    list_root: bool,
+
+    /// Whether to expose blob and directory digests as extended attributes.
+    show_xattr: bool,
+
+    /// This maps a given basename in the root to the inode we allocated for the node.
+    root_nodes: RwLock<HashMap<PathComponent, u64>>,
+
+    /// This keeps track of inodes and data alongside them.
+    inode_tracker: RwLock<InodeTracker>,
+
+    // FUTUREWORK: have a generic container type for dir/file handles and handle
+    // allocation.
+    /// Maps from the handle returned from an opendir to
+    /// This holds all opendir handles (for the root inode)
+    /// They point to the rx part of the channel producing the listing.
+    #[allow(clippy::type_complexity)]
+    dir_handles: RwLock<
+        HashMap<
+            u64,
+            (
+                Span,
+                Arc<Mutex<mpsc::Receiver<(usize, Result<(PathComponent, Node), crate::Error>)>>>,
+            ),
+        >,
+    >,
+
+    next_dir_handle: AtomicU64,
+
+    /// This holds all open file handles
+    #[allow(clippy::type_complexity)]
+    file_handles: RwLock<HashMap<u64, (Span, Arc<Mutex<Box<dyn BlobReader>>>)>>,
+
+    next_file_handle: AtomicU64,
+
+    tokio_handle: tokio::runtime::Handle,
+}
+
+impl<BS, DS, RN> TvixStoreFs<BS, DS, RN>
+where
+    BS: AsRef<dyn BlobService> + Clone + Send,
+    DS: AsRef<dyn DirectoryService> + Clone + Send + 'static,
+    RN: RootNodes + Clone + 'static,
+{
+    pub fn new(
+        blob_service: BS,
+        directory_service: DS,
+        root_nodes_provider: RN,
+        list_root: bool,
+        show_xattr: bool,
+    ) -> Self {
+        Self {
+            blob_service,
+            directory_service,
+            root_nodes_provider,
+
+            list_root,
+            show_xattr,
+
+            root_nodes: RwLock::new(HashMap::default()),
+            inode_tracker: RwLock::new(Default::default()),
+
+            dir_handles: RwLock::new(Default::default()),
+            next_dir_handle: AtomicU64::new(1),
+
+            file_handles: RwLock::new(Default::default()),
+            next_file_handle: AtomicU64::new(1),
+            tokio_handle: tokio::runtime::Handle::current(),
+        }
+    }
+
+    /// 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: &PathComponent) -> Option<u64> {
+        self.root_nodes.read().get(name).cloned()
+    }
+
+    /// For a given inode, look up the given directory behind it (from
+    /// self.inode_tracker), and return its children.
+    /// The inode_tracker MUST know about this inode already, and it MUST point
+    /// to a [InodeData::Directory].
+    /// 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, PathComponent, Node)>)> {
+        let data = self.inode_tracker.read().get(ino).unwrap();
+        match *data {
+            // if it's populated already, return children.
+            InodeData::Directory(DirectoryInodeData::Populated(
+                ref parent_digest,
+                ref children,
+            )) => Ok((parent_digest.clone(), children.clone())),
+            // if it's sparse, fetch data using directory_service, populate child nodes
+            // and update it in [self.inode_tracker].
+            InodeData::Directory(DirectoryInodeData::Sparse(ref parent_digest, _)) => {
+                let directory = self
+                    .tokio_handle
+                    .block_on({
+                        let directory_service = self.directory_service.clone();
+                        let parent_digest = parent_digest.to_owned();
+                        async move { directory_service.as_ref().get(&parent_digest).await }
+                    })?
+                    .ok_or_else(|| {
+                        warn!(directory.digest=%parent_digest, "directory not found");
+                        // If the Directory can't be found, this is a hole, bail out.
+                        io::Error::from_raw_os_error(libc::EIO)
+                    })?;
+
+                // Turn the retrieved directory into a InodeData::Directory(DirectoryInodeData::Populated(..)),
+                // allocating inodes for the children on the way.
+                // FUTUREWORK: there's a bunch of cloning going on here, which we can probably avoid.
+                let children = {
+                    let mut inode_tracker = self.inode_tracker.write();
+
+                    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_name, child_node)
+                        })
+                        .collect();
+
+                    // replace.
+                    inode_tracker.replace(
+                        ino,
+                        Arc::new(InodeData::Directory(DirectoryInodeData::Populated(
+                            parent_digest.clone(),
+                            children.clone(),
+                        ))),
+                    );
+
+                    children
+                };
+
+                Ok((parent_digest.clone(), children))
+            }
+            // if the parent inode was not a directory, this doesn't make sense
+            InodeData::Regular(..) | InodeData::Symlink(_) => {
+                Err(io::Error::from_raw_os_error(libc::ENOTDIR))
+            }
+        }
+    }
+
+    /// This will turn a lookup request for a name in the root to a ino and
+    /// [InodeData].
+    /// It will peek in [self.root_nodes], and then either look it up from
+    /// [self.inode_tracker],
+    /// or otherwise fetch from [self.root_nodes], and then insert into
+    /// [self.inode_tracker].
+    /// In the case the name can't be found, a libc::ENOENT is returned.
+    fn name_in_root_to_ino_and_data(
+        &self,
+        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) {
+            return Ok((
+                inode,
+                self.inode_tracker
+                    .read()
+                    .get(inode)
+                    .expect("must exist")
+                    .to_owned(),
+            ));
+        }
+
+        // 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();
+            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)),
+            // the root node doesn't exist, so the file doesn't exist.
+            Ok(None) => Err(io::Error::from_raw_os_error(libc::ENOENT)),
+            // The root node does exist
+            Ok(Some(root_node)) => {
+                // 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) {
+                    return Ok((
+                        *ino,
+                        self.inode_tracker.read().get(*ino).expect("must exist"),
+                    ));
+                }
+
+                // Only in case it doesn't, lock [self.root_nodes] and
+                // [self.inode_tracker] for writing.
+                let mut root_nodes = self.root_nodes.write();
+                let mut inode_tracker = self.inode_tracker.write();
+
+                // insert the (sparse) inode data and register in
+                // self.root_nodes.
+                let inode_data = InodeData::from_node(&root_node);
+                let ino = inode_tracker.put(inode_data.clone());
+                root_nodes.insert(name.to_owned(), ino);
+
+                Ok((ino, Arc::new(inode_data)))
+            }
+        }
+    }
+}
+
+/// Buffer size of the channel providing nodes in the mount root
+const ROOT_NODES_BUFFER_SIZE: usize = 16;
+
+const XATTR_NAME_DIRECTORY_DIGEST: &[u8] = b"user.tvix.castore.directory.digest";
+const XATTR_NAME_BLOB_DIGEST: &[u8] = b"user.tvix.castore.blob.digest";
+
+impl<BS, DS, RN> FileSystem for TvixStoreFs<BS, DS, RN>
+where
+    BS: AsRef<dyn BlobService> + Clone + Send + 'static,
+    DS: AsRef<dyn DirectoryService> + Send + Clone + 'static,
+    RN: RootNodes + Clone + 'static,
+{
+    type Handle = u64;
+    type Inode = u64;
+
+    fn init(&self, _capable: FsOptions) -> io::Result<FsOptions> {
+        Ok(FsOptions::empty())
+    }
+
+    #[tracing::instrument(skip_all, fields(rq.inode = inode))]
+    fn getattr(
+        &self,
+        _ctx: &Context,
+        inode: Self::Inode,
+        _handle: Option<Self::Handle>,
+    ) -> io::Result<(stat64, Duration)> {
+        if inode == ROOT_ID {
+            return Ok((ROOT_FILE_ATTR.into(), Duration::MAX));
+        }
+
+        match self.inode_tracker.read().get(inode) {
+            None => Err(io::Error::from_raw_os_error(libc::ENOENT)),
+            Some(inode_data) => {
+                debug!(inode_data = ?inode_data, "found node");
+                Ok((inode_data.as_fuse_file_attr(inode).into(), Duration::MAX))
+            }
+        }
+    }
+
+    #[tracing::instrument(skip_all, fields(rq.parent_inode = parent, rq.name = ?name))]
+    fn lookup(
+        &self,
+        _ctx: &Context,
+        parent: Self::Inode,
+        name: &std::ffi::CStr,
+    ) -> 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)?;
+
+            debug!(inode_data=?&inode_data, ino=ino, "Some");
+            return Ok(inode_data.as_fuse_entry(ino));
+        }
+        // This is the "lookup for "a" inside inode 42.
+        // We already know that inode 42 must be a directory.
+        let (parent_digest, children) = self.get_directory_children(parent)?;
+
+        Span::current().record("directory.digest", parent_digest.to_string());
+        // 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(|(_, 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();
+
+            // Reply with the file attributes for the child.
+            // For child directories, we still have all data we need to reply.
+            Ok(child_inode_data.as_fuse_entry(*child_ino))
+        } else {
+            // Child not found, return ENOENT.
+            Err(io::Error::from_raw_os_error(libc::ENOENT))
+        }
+    }
+
+    #[tracing::instrument(skip_all, fields(rq.inode = inode))]
+    fn opendir(
+        &self,
+        _ctx: &Context,
+        inode: Self::Inode,
+        _flags: u32,
+    ) -> io::Result<(Option<Self::Handle>, OpenOptions)> {
+        // In case opendir on the root is called, we provide the handle, as re-entering that listing is expensive.
+        // For all other directory inodes we just let readdir take care of it.
+        if inode == ROOT_ID {
+            if !self.list_root {
+                return Err(io::Error::from_raw_os_error(libc::EPERM)); // same error code as ipfs/kubo
+            }
+
+            let root_nodes_provider = self.root_nodes_provider.clone();
+            let (tx, rx) = mpsc::channel(ROOT_NODES_BUFFER_SIZE);
+
+            // 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(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,
+            // which is fine for now.
+            // See https://cl.tvl.fyi/c/depot/+/8834/comment/a6684ce0_d72469d1
+            // for the discussion on alternatives.
+            let dh = self.next_dir_handle.fetch_add(1, Ordering::SeqCst);
+
+            self.dir_handles
+                .write()
+                .insert(dh, (Span::current(), Arc::new(Mutex::new(rx))));
+
+            return Ok((
+                Some(dh),
+                fuse_backend_rs::api::filesystem::OpenOptions::empty(), // TODO: non-seekable
+            ));
+        }
+
+        Ok((None, OpenOptions::empty()))
+    }
+
+    #[tracing::instrument(skip_all, fields(rq.inode = inode, rq.handle = handle, rq.offset = offset), parent = self.dir_handles.read().get(&handle).and_then(|x| x.0.id()))]
+    fn readdir(
+        &self,
+        _ctx: &Context,
+        inode: Self::Inode,
+        handle: Self::Handle,
+        _size: u32,
+        offset: u64,
+        add_entry: &mut dyn FnMut(fuse_backend_rs::api::filesystem::DirEntry) -> io::Result<usize>,
+    ) -> io::Result<()> {
+        debug!("readdir");
+
+        if inode == ROOT_ID {
+            if !self.list_root {
+                return Err(io::Error::from_raw_os_error(libc::EPERM)); // same error code as ipfs/kubo
+            }
+
+            // get the handle from [self.dir_handles]
+            let (_span, rx) = match self.dir_handles.read().get(&handle) {
+                Some(rx) => rx.clone(),
+                None => {
+                    warn!("dir handle {} unknown", handle);
+                    return Err(io::Error::from_raw_os_error(libc::EIO));
+                }
+            };
+
+            let mut rx = rx
+                .lock()
+                .map_err(|_| crate::Error::StorageError("mutex poisoned".into()))?;
+
+            while let Some((i, n)) = rx.blocking_recv() {
+                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 = InodeData::from_node(&node);
+
+                // obtain the inode, or allocate a new one.
+                let ino = self.get_inode_for_root_name(&name).unwrap_or_else(|| {
+                    // insert the (sparse) inode data and register in
+                    // self.root_nodes.
+                    let ino = self.inode_tracker.write().put(inode_data.clone());
+                    self.root_nodes.write().insert(name.clone(), ino);
+                    ino
+                });
+
+                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.as_ref(),
+                })?;
+                // If the buffer is full, add_entry will return `Ok(0)`.
+                if written == 0 {
+                    break;
+                }
+            }
+            return Ok(());
+        }
+
+        // Non root-node case: lookup the children, or return an error if it's not a directory.
+        let (parent_digest, children) = self.get_directory_children(inode)?;
+        Span::current().record("directory.digest", parent_digest.to_string());
+
+        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: child_name.as_ref(),
+            })?;
+            // If the buffer is full, add_entry will return `Ok(0)`.
+            if written == 0 {
+                break;
+            }
+        }
+
+        Ok(())
+    }
+
+    #[tracing::instrument(skip_all, fields(rq.inode = inode, rq.handle = handle), parent = self.dir_handles.read().get(&handle).and_then(|x| x.0.id()))]
+    fn readdirplus(
+        &self,
+        _ctx: &Context,
+        inode: Self::Inode,
+        handle: Self::Handle,
+        _size: u32,
+        offset: u64,
+        add_entry: &mut dyn FnMut(
+            fuse_backend_rs::api::filesystem::DirEntry,
+            fuse_backend_rs::api::filesystem::Entry,
+        ) -> io::Result<usize>,
+    ) -> io::Result<()> {
+        debug!("readdirplus");
+
+        if inode == ROOT_ID {
+            if !self.list_root {
+                return Err(io::Error::from_raw_os_error(libc::EPERM)); // same error code as ipfs/kubo
+            }
+
+            // get the handle from [self.dir_handles]
+            let (_span, rx) = match self.dir_handles.read().get(&handle) {
+                Some(rx) => rx.clone(),
+                None => {
+                    warn!("dir handle {} unknown", handle);
+                    return Err(io::Error::from_raw_os_error(libc::EIO));
+                }
+            };
+
+            let mut rx = rx
+                .lock()
+                .map_err(|_| crate::Error::StorageError("mutex poisoned".into()))?;
+
+            while let Some((i, n)) = rx.blocking_recv() {
+                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 = InodeData::from_node(&node);
+
+                // obtain the inode, or allocate a new one.
+                let ino = self.get_inode_for_root_name(&name).unwrap_or_else(|| {
+                    // insert the (sparse) inode data and register in
+                    // self.root_nodes.
+                    let ino = self.inode_tracker.write().put(inode_data.clone());
+                    self.root_nodes.write().insert(name.clone(), ino);
+                    ino
+                });
+
+                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.as_ref(),
+                    },
+                    inode_data.as_fuse_entry(ino),
+                )?;
+                // If the buffer is full, add_entry will return `Ok(0)`.
+                if written == 0 {
+                    break;
+                }
+            }
+            return Ok(());
+        }
+
+        // Non root-node case: lookup the children, or return an error if it's not a directory.
+        let (parent_digest, children) = self.get_directory_children(inode)?;
+        Span::current().record("directory.digest", parent_digest.to_string());
+
+        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(
+                fuse_backend_rs::api::filesystem::DirEntry {
+                    ino,
+                    offset: offset + (i as u64) + 1,
+                    type_: inode_data.as_fuse_type(),
+                    name: name.as_ref(),
+                },
+                inode_data.as_fuse_entry(ino),
+            )?;
+            // If the buffer is full, add_entry will return `Ok(0)`.
+            if written == 0 {
+                break;
+            }
+        }
+
+        Ok(())
+    }
+
+    #[tracing::instrument(skip_all, fields(rq.inode = inode, rq.handle = handle), parent = self.dir_handles.read().get(&handle).and_then(|x| x.0.id()))]
+    fn releasedir(
+        &self,
+        _ctx: &Context,
+        inode: Self::Inode,
+        _flags: u32,
+        handle: Self::Handle,
+    ) -> io::Result<()> {
+        if inode == ROOT_ID {
+            // drop the rx part of the channel.
+            match self.dir_handles.write().remove(&handle) {
+                // drop it, which will close it.
+                Some(rx) => drop(rx),
+                None => {
+                    warn!("dir handle not found");
+                }
+            }
+        }
+
+        Ok(())
+    }
+
+    #[tracing::instrument(skip_all, fields(rq.inode = inode))]
+    fn open(
+        &self,
+        _ctx: &Context,
+        inode: Self::Inode,
+        _flags: u32,
+        _fuse_flags: u32,
+    ) -> io::Result<(
+        Option<Self::Handle>,
+        fuse_backend_rs::api::filesystem::OpenOptions,
+    )> {
+        if inode == ROOT_ID {
+            return Err(io::Error::from_raw_os_error(libc::ENOSYS));
+        }
+
+        // lookup the inode
+        match *self.inode_tracker.read().get(inode).unwrap() {
+            // read is invalid on non-files.
+            InodeData::Directory(..) | InodeData::Symlink(_) => {
+                warn!("is directory");
+                Err(io::Error::from_raw_os_error(libc::EISDIR))
+            }
+            InodeData::Regular(ref blob_digest, _blob_size, _) => {
+                Span::current().record("blob.digest", blob_digest.to_string());
+
+                match self.tokio_handle.block_on({
+                    let blob_service = self.blob_service.clone();
+                    let blob_digest = blob_digest.clone();
+                    async move { blob_service.as_ref().open_read(&blob_digest).await }
+                }) {
+                    Ok(None) => {
+                        warn!("blob not found");
+                        Err(io::Error::from_raw_os_error(libc::EIO))
+                    }
+                    Err(e) => {
+                        warn!(e=?e, "error opening blob");
+                        Err(io::Error::from_raw_os_error(libc::EIO))
+                    }
+                    Ok(Some(blob_reader)) => {
+                        // get a new file handle
+                        // TODO: this will overflow after 2**64 operations,
+                        // which is fine for now.
+                        // See https://cl.tvl.fyi/c/depot/+/8834/comment/a6684ce0_d72469d1
+                        // for the discussion on alternatives.
+                        let fh = self.next_file_handle.fetch_add(1, Ordering::SeqCst);
+
+                        self.file_handles
+                            .write()
+                            .insert(fh, (Span::current(), Arc::new(Mutex::new(blob_reader))));
+
+                        Ok((
+                            Some(fh),
+                            fuse_backend_rs::api::filesystem::OpenOptions::empty(),
+                        ))
+                    }
+                }
+            }
+        }
+    }
+
+    #[tracing::instrument(skip_all, fields(rq.inode = inode, rq.handle = handle), parent = self.file_handles.read().get(&handle).and_then(|x| x.0.id()))]
+    fn release(
+        &self,
+        _ctx: &Context,
+        inode: Self::Inode,
+        _flags: u32,
+        handle: Self::Handle,
+        _flush: bool,
+        _flock_release: bool,
+        _lock_owner: Option<u64>,
+    ) -> io::Result<()> {
+        match self.file_handles.write().remove(&handle) {
+            // drop the blob reader, which will close it.
+            Some(blob_reader) => drop(blob_reader),
+            None => {
+                // These might already be dropped if a read error occured.
+                warn!("file handle not found");
+            }
+        }
+
+        Ok(())
+    }
+
+    #[tracing::instrument(skip_all, fields(rq.inode = inode, rq.handle = handle, rq.offset = offset, rq.size = size), parent = self.file_handles.read().get(&handle).and_then(|x| x.0.id()))]
+    fn read(
+        &self,
+        _ctx: &Context,
+        inode: Self::Inode,
+        handle: Self::Handle,
+        w: &mut dyn fuse_backend_rs::api::filesystem::ZeroCopyWriter,
+        size: u32,
+        offset: u64,
+        _lock_owner: Option<u64>,
+        _flags: u32,
+    ) -> io::Result<usize> {
+        debug!("read");
+
+        // We need to take out the blob reader from self.file_handles, so we can
+        // interact with it in the separate task.
+        // On success, we pass it back out of the task, so we can put it back in self.file_handles.
+        let (_span, blob_reader) = self
+            .file_handles
+            .read()
+            .get(&handle)
+            .ok_or_else(|| {
+                warn!("file handle {} unknown", handle);
+                io::Error::from_raw_os_error(libc::EIO)
+            })
+            .cloned()?;
+
+        let mut blob_reader = blob_reader
+            .lock()
+            .map_err(|_| crate::Error::StorageError("mutex poisoned".into()))?;
+
+        let buf = self.tokio_handle.block_on(async move {
+            // seek to the offset specified, which is relative to the start of the file.
+            let pos = blob_reader
+                .seek(io::SeekFrom::Start(offset))
+                .await
+                .map_err(|e| {
+                    warn!("failed to seek to offset {}: {}", offset, e);
+                    io::Error::from_raw_os_error(libc::EIO)
+                })?;
+
+            debug_assert_eq!(offset, pos);
+
+            // As written in the fuse docs, read should send exactly the number
+            // of bytes requested except on EOF or error.
+
+            let mut buf: Vec<u8> = Vec::with_capacity(size as usize);
+
+            // copy things from the internal buffer into buf to fill it till up until size
+            tokio::io::copy(&mut blob_reader.as_mut().take(size as u64), &mut buf).await?;
+
+            Ok::<_, std::io::Error>(buf)
+        })?;
+
+        // We cannot use w.write() here, we're required to call write multiple
+        // times until we wrote the entirety of the buffer (which is `size`, except on EOF).
+        let buf_len = buf.len();
+        let bytes_written = io::copy(&mut Cursor::new(buf), w)?;
+        if bytes_written != buf_len as u64 {
+            error!(bytes_written=%bytes_written, "unable to write all of buf to kernel");
+            return Err(io::Error::from_raw_os_error(libc::EIO));
+        }
+
+        Ok(bytes_written as usize)
+    }
+
+    #[tracing::instrument(skip_all, fields(rq.inode = inode))]
+    fn readlink(&self, _ctx: &Context, inode: Self::Inode) -> io::Result<Vec<u8>> {
+        if inode == ROOT_ID {
+            return Err(io::Error::from_raw_os_error(libc::ENOSYS));
+        }
+
+        // lookup the inode
+        match *self.inode_tracker.read().get(inode).unwrap() {
+            InodeData::Directory(..) | InodeData::Regular(..) => {
+                Err(io::Error::from_raw_os_error(libc::EINVAL))
+            }
+            InodeData::Symlink(ref target) => Ok(target.to_vec()),
+        }
+    }
+
+    #[tracing::instrument(skip_all, fields(rq.inode = inode, name=?name))]
+    fn getxattr(
+        &self,
+        _ctx: &Context,
+        inode: Self::Inode,
+        name: &CStr,
+        size: u32,
+    ) -> io::Result<GetxattrReply> {
+        if !self.show_xattr {
+            return Err(io::Error::from_raw_os_error(libc::ENOSYS));
+        }
+
+        // Peek at the inode requested, and construct the response.
+        let digest_str = match *self
+            .inode_tracker
+            .read()
+            .get(inode)
+            .ok_or_else(|| io::Error::from_raw_os_error(libc::ENODATA))?
+        {
+            InodeData::Directory(DirectoryInodeData::Sparse(ref digest, _))
+            | InodeData::Directory(DirectoryInodeData::Populated(ref digest, _))
+                if name.to_bytes() == XATTR_NAME_DIRECTORY_DIGEST =>
+            {
+                digest.to_string()
+            }
+            InodeData::Regular(ref digest, _, _) if name.to_bytes() == XATTR_NAME_BLOB_DIGEST => {
+                digest.to_string()
+            }
+            _ => {
+                return Err(io::Error::from_raw_os_error(libc::ENODATA));
+            }
+        };
+
+        if size == 0 {
+            Ok(GetxattrReply::Count(digest_str.len() as u32))
+        } else if size < digest_str.len() as u32 {
+            Err(io::Error::from_raw_os_error(libc::ERANGE))
+        } else {
+            Ok(GetxattrReply::Value(digest_str.into_bytes()))
+        }
+    }
+
+    #[tracing::instrument(skip_all, fields(rq.inode = inode))]
+    fn listxattr(
+        &self,
+        _ctx: &Context,
+        inode: Self::Inode,
+        size: u32,
+    ) -> io::Result<ListxattrReply> {
+        if !self.show_xattr {
+            return Err(io::Error::from_raw_os_error(libc::ENOSYS));
+        }
+
+        // determine the (\0-terminated list) to of xattr keys present, depending on the type of the inode.
+        let xattrs_names = {
+            let mut out = Vec::new();
+            if let Some(inode_data) = self.inode_tracker.read().get(inode) {
+                match *inode_data {
+                    InodeData::Directory(_) => {
+                        out.extend_from_slice(XATTR_NAME_DIRECTORY_DIGEST);
+                        out.push_byte(b'\x00');
+                    }
+                    InodeData::Regular(..) => {
+                        out.extend_from_slice(XATTR_NAME_BLOB_DIGEST);
+                        out.push_byte(b'\x00');
+                    }
+                    _ => {}
+                }
+            }
+            out
+        };
+
+        if size == 0 {
+            Ok(ListxattrReply::Count(xattrs_names.len() as u32))
+        } else if size < xattrs_names.len() as u32 {
+            Err(io::Error::from_raw_os_error(libc::ERANGE))
+        } else {
+            Ok(ListxattrReply::Names(xattrs_names.to_vec()))
+        }
+    }
+}
diff --git a/tvix/castore/src/fs/root_nodes.rs b/tvix/castore/src/fs/root_nodes.rs
new file mode 100644
index 000000000000..5ed1a4d8d6c0
--- /dev/null
+++ b/tvix/castore/src/fs/root_nodes.rs
@@ -0,0 +1,39 @@
+use std::collections::BTreeMap;
+
+use crate::{path::PathComponent, Error, Node};
+use futures::stream::BoxStream;
+use tonic::async_trait;
+
+/// Provides an interface for looking up root nodes  in tvix-castore by given
+/// a lookup key (usually the basename), and optionally allow a listing.
+#[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: &PathComponent) -> Result<Option<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]
+/// Implements RootNodes for something deref'ing to a BTreeMap of Nodes, where
+/// the key is the node name.
+impl<T> RootNodes for T
+where
+    T: AsRef<BTreeMap<PathComponent, Node>> + Send + Sync,
+{
+    async fn get_by_basename(&self, name: &PathComponent) -> Result<Option<Node>, Error> {
+        Ok(self.as_ref().get(name).cloned())
+    }
+
+    fn list(&self) -> BoxStream<Result<(PathComponent, Node), Error>> {
+        Box::pin(tokio_stream::iter(
+            self.as_ref()
+                .iter()
+                .map(|(name, node)| Ok((name.to_owned(), node.to_owned()))),
+        ))
+    }
+}
diff --git a/tvix/castore/src/fs/virtiofs.rs b/tvix/castore/src/fs/virtiofs.rs
new file mode 100644
index 000000000000..d63e2f2bdd87
--- /dev/null
+++ b/tvix/castore/src/fs/virtiofs.rs
@@ -0,0 +1,238 @@
+use std::{
+    convert, error, fmt, io,
+    ops::Deref,
+    path::Path,
+    sync::{Arc, MutexGuard, RwLock},
+};
+
+use fuse_backend_rs::{
+    api::{filesystem::FileSystem, server::Server},
+    transport::{FsCacheReqHandler, Reader, VirtioFsWriter},
+};
+use tracing::error;
+use vhost::vhost_user::{
+    Listener, SlaveFsCacheReq, VhostUserProtocolFeatures, VhostUserVirtioFeatures,
+};
+use vhost_user_backend::{VhostUserBackendMut, VhostUserDaemon, VringMutex, VringState, VringT};
+use virtio_bindings::bindings::virtio_ring::{
+    VIRTIO_RING_F_EVENT_IDX, VIRTIO_RING_F_INDIRECT_DESC,
+};
+use virtio_queue::QueueT;
+use vm_memory::{GuestAddressSpace, GuestMemoryAtomic, GuestMemoryMmap};
+use vmm_sys_util::epoll::EventSet;
+
+const VIRTIO_F_VERSION_1: u32 = 32;
+const NUM_QUEUES: usize = 2;
+const QUEUE_SIZE: usize = 1024;
+
+#[derive(Debug)]
+enum Error {
+    /// Failed to handle non-input event.
+    HandleEventNotEpollIn,
+    /// Failed to handle unknown event.
+    HandleEventUnknownEvent,
+    /// Invalid descriptor chain.
+    InvalidDescriptorChain,
+    /// Failed to handle filesystem requests.
+    #[allow(dead_code)]
+    HandleRequests(fuse_backend_rs::Error),
+    /// Failed to construct new vhost user daemon.
+    NewDaemon,
+    /// Failed to start the vhost user daemon.
+    StartDaemon,
+    /// Failed to wait for the vhost user daemon.
+    WaitDaemon,
+}
+
+impl fmt::Display for Error {
+    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
+        write!(f, "vhost_user_fs_error: {self:?}")
+    }
+}
+
+impl error::Error for Error {}
+
+impl convert::From<Error> for io::Error {
+    fn from(e: Error) -> Self {
+        io::Error::new(io::ErrorKind::Other, e)
+    }
+}
+
+struct VhostUserFsBackend<FS>
+where
+    FS: FileSystem + Send + Sync,
+{
+    server: Arc<Server<Arc<FS>>>,
+    event_idx: bool,
+    guest_mem: GuestMemoryAtomic<GuestMemoryMmap>,
+    cache_req: Option<SlaveFsCacheReq>,
+}
+
+impl<FS> VhostUserFsBackend<FS>
+where
+    FS: FileSystem + Send + Sync,
+{
+    fn process_queue(&mut self, vring: &mut MutexGuard<VringState>) -> std::io::Result<bool> {
+        let mut used_descs = false;
+
+        while let Some(desc_chain) = vring
+            .get_queue_mut()
+            .pop_descriptor_chain(self.guest_mem.memory())
+        {
+            let memory = desc_chain.memory();
+            let reader = Reader::from_descriptor_chain(memory, desc_chain.clone())
+                .map_err(|_| Error::InvalidDescriptorChain)?;
+            let writer = VirtioFsWriter::new(memory, desc_chain.clone())
+                .map_err(|_| Error::InvalidDescriptorChain)?;
+
+            self.server
+                .handle_message(
+                    reader,
+                    writer.into(),
+                    self.cache_req
+                        .as_mut()
+                        .map(|req| req as &mut dyn FsCacheReqHandler),
+                    None,
+                )
+                .map_err(Error::HandleRequests)?;
+
+            // TODO: Is len 0 correct?
+            if let Err(error) = vring
+                .get_queue_mut()
+                .add_used(memory, desc_chain.head_index(), 0)
+            {
+                error!(?error, "failed to add desc back to ring");
+            }
+
+            // TODO: What happens if we error out before here?
+            used_descs = true;
+        }
+
+        let needs_notification = if self.event_idx {
+            match vring
+                .get_queue_mut()
+                .needs_notification(self.guest_mem.memory().deref())
+            {
+                Ok(needs_notification) => needs_notification,
+                Err(error) => {
+                    error!(?error, "failed to check if queue needs notification");
+                    true
+                }
+            }
+        } else {
+            true
+        };
+
+        if needs_notification {
+            if let Err(error) = vring.signal_used_queue() {
+                error!(?error, "failed to signal used queue");
+            }
+        }
+
+        Ok(used_descs)
+    }
+}
+
+impl<FS> VhostUserBackendMut<VringMutex> for VhostUserFsBackend<FS>
+where
+    FS: FileSystem + Send + Sync,
+{
+    fn num_queues(&self) -> usize {
+        NUM_QUEUES
+    }
+
+    fn max_queue_size(&self) -> usize {
+        QUEUE_SIZE
+    }
+
+    fn features(&self) -> u64 {
+        1 << VIRTIO_F_VERSION_1
+            | 1 << VIRTIO_RING_F_INDIRECT_DESC
+            | 1 << VIRTIO_RING_F_EVENT_IDX
+            | VhostUserVirtioFeatures::PROTOCOL_FEATURES.bits()
+    }
+
+    fn protocol_features(&self) -> VhostUserProtocolFeatures {
+        VhostUserProtocolFeatures::MQ | VhostUserProtocolFeatures::SLAVE_REQ
+    }
+
+    fn set_event_idx(&mut self, enabled: bool) {
+        self.event_idx = enabled;
+    }
+
+    fn update_memory(&mut self, _mem: GuestMemoryAtomic<GuestMemoryMmap>) -> std::io::Result<()> {
+        // This is what most the vhost user implementations do...
+        Ok(())
+    }
+
+    fn set_slave_req_fd(&mut self, cache_req: SlaveFsCacheReq) {
+        self.cache_req = Some(cache_req);
+    }
+
+    fn handle_event(
+        &mut self,
+        device_event: u16,
+        evset: vmm_sys_util::epoll::EventSet,
+        vrings: &[VringMutex],
+        _thread_id: usize,
+    ) -> std::io::Result<bool> {
+        if evset != EventSet::IN {
+            return Err(Error::HandleEventNotEpollIn.into());
+        }
+
+        let mut queue = match device_event {
+            // High priority queue
+            0 => vrings[0].get_mut(),
+            // Regurlar priority queue
+            1 => vrings[1].get_mut(),
+            _ => {
+                return Err(Error::HandleEventUnknownEvent.into());
+            }
+        };
+
+        if self.event_idx {
+            loop {
+                queue
+                    .get_queue_mut()
+                    .enable_notification(self.guest_mem.memory().deref())
+                    .map_err(|e| io::Error::new(io::ErrorKind::Other, e.to_string()))?;
+                if !self.process_queue(&mut queue)? {
+                    break;
+                }
+            }
+        } else {
+            self.process_queue(&mut queue)?;
+        }
+
+        Ok(false)
+    }
+}
+
+pub fn start_virtiofs_daemon<FS, P>(fs: FS, socket: P) -> io::Result<()>
+where
+    FS: FileSystem + Send + Sync + 'static,
+    P: AsRef<Path>,
+{
+    let guest_mem = GuestMemoryAtomic::new(GuestMemoryMmap::new());
+
+    let server = Arc::new(fuse_backend_rs::api::server::Server::new(Arc::new(fs)));
+
+    let backend = Arc::new(RwLock::new(VhostUserFsBackend {
+        server,
+        guest_mem: guest_mem.clone(),
+        event_idx: false,
+        cache_req: None,
+    }));
+
+    let listener = Listener::new(socket, true).unwrap();
+
+    let mut fs_daemon =
+        VhostUserDaemon::new(String::from("vhost-user-fs-tvix-store"), backend, guest_mem)
+            .map_err(|_| Error::NewDaemon)?;
+
+    fs_daemon.start(listener).map_err(|_| Error::StartDaemon)?;
+
+    fs_daemon.wait().map_err(|_| Error::WaitDaemon)?;
+
+    Ok(())
+}
diff --git a/tvix/castore/src/hashing_reader.rs b/tvix/castore/src/hashing_reader.rs
new file mode 100644
index 000000000000..7d78cae5877a
--- /dev/null
+++ b/tvix/castore/src/hashing_reader.rs
@@ -0,0 +1,89 @@
+use pin_project_lite::pin_project;
+use tokio::io::AsyncRead;
+
+pin_project! {
+    /// Wraps an existing AsyncRead, and allows querying for the digest of all
+    /// data read "through" it.
+    /// The hash function is configurable by type parameter.
+    pub struct HashingReader<R, H>
+    where
+        R: AsyncRead,
+        H: digest::Digest,
+    {
+        #[pin]
+        inner: R,
+        hasher: H,
+    }
+}
+
+pub type B3HashingReader<R> = HashingReader<R, blake3::Hasher>;
+
+impl<R, H> HashingReader<R, H>
+where
+    R: AsyncRead,
+    H: digest::Digest,
+{
+    pub fn from(r: R) -> Self {
+        Self {
+            inner: r,
+            hasher: H::new(),
+        }
+    }
+
+    /// Return the digest.
+    pub fn digest(self) -> digest::Output<H> {
+        self.hasher.finalize()
+    }
+}
+
+impl<R, H> tokio::io::AsyncRead for HashingReader<R, H>
+where
+    R: AsyncRead,
+    H: digest::Digest,
+{
+    fn poll_read(
+        self: std::pin::Pin<&mut Self>,
+        cx: &mut std::task::Context<'_>,
+        buf: &mut tokio::io::ReadBuf<'_>,
+    ) -> std::task::Poll<std::io::Result<()>> {
+        let buf_filled_len_before = buf.filled().len();
+
+        let this = self.project();
+        let ret = this.inner.poll_read(cx, buf);
+
+        // write everything new filled into the hasher.
+        this.hasher.update(&buf.filled()[buf_filled_len_before..]);
+
+        ret
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use std::io::Cursor;
+
+    use rstest::rstest;
+
+    use crate::fixtures::BLOB_A;
+    use crate::fixtures::BLOB_A_DIGEST;
+    use crate::fixtures::BLOB_B;
+    use crate::fixtures::BLOB_B_DIGEST;
+    use crate::fixtures::EMPTY_BLOB_DIGEST;
+    use crate::{B3Digest, B3HashingReader};
+
+    #[rstest]
+    #[case::blob_a(&BLOB_A, &BLOB_A_DIGEST)]
+    #[case::blob_b(&BLOB_B, &BLOB_B_DIGEST)]
+    #[case::empty_blob(&[], &EMPTY_BLOB_DIGEST)]
+    #[tokio::test]
+    async fn test_b3_hashing_reader(#[case] data: &[u8], #[case] b3_digest: &B3Digest) {
+        let r = Cursor::new(data);
+        let mut hr = B3HashingReader::from(r);
+
+        tokio::io::copy(&mut hr, &mut tokio::io::sink())
+            .await
+            .expect("read must succeed");
+
+        assert_eq!(*b3_digest, hr.digest().into());
+    }
+}
diff --git a/tvix/castore/src/import/archive.rs b/tvix/castore/src/import/archive.rs
new file mode 100644
index 000000000000..167f799efa0f
--- /dev/null
+++ b/tvix/castore/src/import/archive.rs
@@ -0,0 +1,373 @@
+//! Imports from an archive (tarballs)
+
+use std::collections::HashMap;
+
+use petgraph::graph::{DiGraph, NodeIndex};
+use petgraph::visit::{DfsPostOrder, EdgeRef};
+use petgraph::Direction;
+use tokio::io::AsyncRead;
+use tokio_stream::StreamExt;
+use tokio_tar::Archive;
+use tracing::{instrument, warn, Level};
+
+use crate::blobservice::BlobService;
+use crate::directoryservice::DirectoryService;
+use crate::import::{ingest_entries, IngestionEntry, IngestionError};
+use crate::Node;
+
+use super::blobs::{self, ConcurrentBlobUploader};
+
+type TarPathBuf = std::path::PathBuf;
+
+#[derive(Debug, thiserror::Error)]
+pub enum Error {
+    #[error("unable to construct stream of entries: {0}")]
+    Entries(std::io::Error),
+
+    #[error("unable to read next entry: {0}")]
+    NextEntry(std::io::Error),
+
+    #[error("unable to read path for entry: {0}")]
+    PathRead(std::io::Error),
+
+    #[error("unable to convert path {0} for entry: {1}")]
+    PathConvert(TarPathBuf, std::io::Error),
+
+    #[error("unable to read size field for {0}: {1}")]
+    Size(TarPathBuf, std::io::Error),
+
+    #[error("unable to read mode field for {0}: {1}")]
+    Mode(TarPathBuf, std::io::Error),
+
+    #[error("unable to read link name field for {0}: {1}")]
+    LinkName(TarPathBuf, std::io::Error),
+
+    #[error("unsupported tar entry {0} type: {1:?}")]
+    EntryType(TarPathBuf, tokio_tar::EntryType),
+
+    #[error("symlink missing target {0}")]
+    MissingSymlinkTarget(TarPathBuf),
+
+    #[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
+/// [`DirectoryService`].
+#[instrument(skip_all, ret(level = Level::TRACE), err)]
+pub async fn ingest_archive<BS, DS, R>(
+    blob_service: BS,
+    directory_service: DS,
+    mut archive: Archive<R>,
+) -> Result<Node, IngestionError<Error>>
+where
+    BS: BlobService + Clone + 'static,
+    DS: DirectoryService,
+    R: AsyncRead + Unpin,
+{
+    // Since tarballs can have entries in any arbitrary order, we need to
+    // buffer all of the directory metadata so we can reorder directory
+    // contents and entries to meet the requires of the castore.
+
+    // In the first phase, collect up all the regular files and symlinks.
+    let mut nodes = IngestionEntryGraph::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)? {
+        let tar_path: TarPathBuf = entry.path().map_err(Error::PathRead)?.into();
+
+        // construct a castore PathBuf, which we use in the produced IngestionEntry.
+        let path = crate::path::PathBuf::from_host_path(tar_path.as_path(), true)
+            .map_err(|e| Error::PathConvert(tar_path.clone(), e))?;
+
+        let header = entry.header();
+        let entry = match header.entry_type() {
+            tokio_tar::EntryType::Regular
+            | tokio_tar::EntryType::GNUSparse
+            | tokio_tar::EntryType::Continuous => {
+                let size = header
+                    .size()
+                    .map_err(|e| Error::Size(tar_path.clone(), e))?;
+
+                let digest = blob_uploader
+                    .upload(&path, size, &mut entry)
+                    .await
+                    .map_err(Error::BlobUploadError)?;
+
+                let executable = entry
+                    .header()
+                    .mode()
+                    .map_err(|e| Error::Mode(tar_path, e))?
+                    & 64
+                    != 0;
+
+                IngestionEntry::Regular {
+                    path,
+                    size,
+                    executable,
+                    digest,
+                }
+            }
+            tokio_tar::EntryType::Symlink => IngestionEntry::Symlink {
+                target: entry
+                    .link_name()
+                    .map_err(|e| Error::LinkName(tar_path.clone(), e))?
+                    .ok_or_else(|| Error::MissingSymlinkTarget(tar_path.clone()))?
+                    .into_owned()
+                    .into_os_string()
+                    .into_encoded_bytes(),
+                path,
+            },
+            // Push a bogus directory marker so we can make sure this directoy gets
+            // created. We don't know the digest and size until after reading the full
+            // tarball.
+            tokio_tar::EntryType::Directory => IngestionEntry::Dir { path },
+
+            tokio_tar::EntryType::XGlobalHeader | tokio_tar::EntryType::XHeader => continue,
+
+            entry_type => return Err(Error::EntryType(tar_path, entry_type).into()),
+        };
+
+        nodes.add(entry)?;
+    }
+
+    blob_uploader.join().await.map_err(Error::BlobUploadError)?;
+
+    let root_node = ingest_entries(
+        directory_service,
+        futures::stream::iter(nodes.finalize()?.into_iter().map(Ok)),
+    )
+    .await?;
+
+    Ok(root_node)
+}
+
+/// Keep track of the directory structure of a file tree being ingested. This is used
+/// for ingestion sources which do not provide any ordering or uniqueness guarantees
+/// like tarballs.
+///
+/// If we ingest multiple entries with the same paths and both entries are not directories,
+/// the newer entry will replace the latter entry, disconnecting the old node's children
+/// from the graph.
+///
+/// Once all nodes are ingested a call to [IngestionEntryGraph::finalize] will return
+/// a list of entries compute by performaing a DFS post order traversal of the graph
+/// from the top-level directory entry.
+///
+/// This expects the directory structure to contain a single top-level directory entry.
+/// An error is returned if this is not the case and ingestion will fail.
+struct IngestionEntryGraph {
+    graph: DiGraph<IngestionEntry, ()>,
+    path_to_index: HashMap<crate::path::PathBuf, NodeIndex>,
+    root_node: Option<NodeIndex>,
+}
+
+impl Default for IngestionEntryGraph {
+    fn default() -> Self {
+        Self::new()
+    }
+}
+
+impl IngestionEntryGraph {
+    /// Creates a new ingestion entry graph.
+    pub fn new() -> Self {
+        IngestionEntryGraph {
+            graph: DiGraph::new(),
+            path_to_index: HashMap::new(),
+            root_node: None,
+        }
+    }
+
+    /// Adds a new entry to the graph. Parent directories are automatically inserted.
+    /// If a node exists in the graph with the same name as the new entry and both the old
+    /// and new nodes are not directories, the node is replaced and is disconnected from its
+    /// children.
+    pub fn add(&mut self, entry: IngestionEntry) -> Result<NodeIndex, Error> {
+        let path = entry.path().to_owned();
+
+        let index = match self.path_to_index.get(entry.path()) {
+            Some(&index) => {
+                // If either the old entry or new entry are not directories, we'll replace the old
+                // entry.
+                if !entry.is_dir() || !self.get_node(index).is_dir() {
+                    self.replace_node(index, entry);
+                }
+
+                index
+            }
+            None => self.graph.add_node(entry),
+        };
+
+        // for archives, a path with 1 component is the root node
+        if path.components().count() == 1 {
+            // We expect archives to contain a single root node, if there is another root node
+            // entry with a different path name, this is unsupported.
+            if let Some(root_node) = self.root_node {
+                if self.get_node(root_node).path() != path.as_ref() {
+                    return Err(Error::UnexpectedNumberOfTopLevelEntries);
+                }
+            }
+
+            self.root_node = Some(index)
+        } else if let Some(parent_path) = path.parent() {
+            // Recursively add the parent node until it hits the root node.
+            let parent_index = self.add(IngestionEntry::Dir {
+                path: parent_path.to_owned(),
+            })?;
+
+            // Insert an edge from the parent directory to the child entry.
+            self.graph.add_edge(parent_index, index, ());
+        }
+
+        self.path_to_index.insert(path, index);
+
+        Ok(index)
+    }
+
+    /// Traverses the graph in DFS post order and collects the entries into a [Vec<IngestionEntry>].
+    ///
+    /// Unreachable parts of the graph are not included in the result.
+    pub fn finalize(self) -> Result<Vec<IngestionEntry>, Error> {
+        // There must be a root node.
+        let Some(root_node_index) = self.root_node else {
+            return Err(Error::UnexpectedNumberOfTopLevelEntries);
+        };
+
+        // The root node must be a directory.
+        if !self.get_node(root_node_index).is_dir() {
+            return Err(Error::UnexpectedNumberOfTopLevelEntries);
+        }
+
+        let mut traversal = DfsPostOrder::new(&self.graph, root_node_index);
+        let mut nodes = Vec::with_capacity(self.graph.node_count());
+        while let Some(node_index) = traversal.next(&self.graph) {
+            nodes.push(self.get_node(node_index).clone());
+        }
+
+        Ok(nodes)
+    }
+
+    /// Replaces the node with the specified entry. The node's children are disconnected.
+    ///
+    /// This should never be called if both the old and new nodes are directories.
+    fn replace_node(&mut self, index: NodeIndex, new_entry: IngestionEntry) {
+        let entry = self
+            .graph
+            .node_weight_mut(index)
+            .expect("Tvix bug: missing node entry");
+
+        debug_assert!(!(entry.is_dir() && new_entry.is_dir()));
+
+        // Replace the node itself.
+        warn!(
+            "saw duplicate entry in archive at path {:?}. old: {:?} new: {:?}",
+            entry.path(),
+            &entry,
+            &new_entry
+        );
+        *entry = new_entry;
+
+        // Remove any outgoing edges to disconnect the old node's children.
+        let edges = self
+            .graph
+            .edges_directed(index, Direction::Outgoing)
+            .map(|edge| edge.id())
+            .collect::<Vec<_>>();
+        for edge in edges {
+            self.graph.remove_edge(edge);
+        }
+    }
+
+    fn get_node(&self, index: NodeIndex) -> &IngestionEntry {
+        self.graph
+            .node_weight(index)
+            .expect("Tvix bug: missing node entry")
+    }
+}
+
+#[cfg(test)]
+mod test {
+    use crate::import::IngestionEntry;
+    use crate::B3Digest;
+
+    use super::{Error, IngestionEntryGraph};
+
+    use lazy_static::lazy_static;
+    use rstest::rstest;
+
+    lazy_static! {
+        pub static ref EMPTY_DIGEST: B3Digest = blake3::hash(&[]).as_bytes().into();
+        pub static ref DIR_A: IngestionEntry = IngestionEntry::Dir {
+            path: "a".parse().unwrap()
+        };
+        pub static ref DIR_B: IngestionEntry = IngestionEntry::Dir {
+            path: "b".parse().unwrap()
+        };
+        pub static ref DIR_A_B: IngestionEntry = IngestionEntry::Dir {
+            path: "a/b".parse().unwrap()
+        };
+        pub static ref FILE_A: IngestionEntry = IngestionEntry::Regular {
+            path: "a".parse().unwrap(),
+            size: 0,
+            executable: false,
+            digest: EMPTY_DIGEST.clone(),
+        };
+        pub static ref FILE_A_B: IngestionEntry = IngestionEntry::Regular {
+            path: "a/b".parse().unwrap(),
+            size: 0,
+            executable: false,
+            digest: EMPTY_DIGEST.clone(),
+        };
+        pub static ref FILE_A_B_C: IngestionEntry = IngestionEntry::Regular {
+            path: "a/b/c".parse().unwrap(),
+            size: 0,
+            executable: false,
+            digest: EMPTY_DIGEST.clone(),
+        };
+    }
+
+    #[rstest]
+    #[case::implicit_directories(&[&*FILE_A_B_C], &[&*FILE_A_B_C, &*DIR_A_B, &*DIR_A])]
+    #[case::explicit_directories(&[&*DIR_A, &*DIR_A_B, &*FILE_A_B_C], &[&*FILE_A_B_C, &*DIR_A_B, &*DIR_A])]
+    #[case::inaccesible_tree(&[&*DIR_A, &*DIR_A_B, &*FILE_A_B], &[&*FILE_A_B, &*DIR_A])]
+    fn node_ingestion_success(
+        #[case] in_entries: &[&IngestionEntry],
+        #[case] exp_entries: &[&IngestionEntry],
+    ) {
+        let mut nodes = IngestionEntryGraph::new();
+
+        for entry in in_entries {
+            nodes.add((*entry).clone()).expect("failed to add entry");
+        }
+
+        let entries = nodes.finalize().expect("invalid entries");
+
+        let exp_entries: Vec<IngestionEntry> =
+            exp_entries.iter().map(|entry| (*entry).clone()).collect();
+
+        assert_eq!(entries, exp_entries);
+    }
+
+    #[rstest]
+    #[case::no_top_level_entries(&[], Error::UnexpectedNumberOfTopLevelEntries)]
+    #[case::multiple_top_level_dirs(&[&*DIR_A, &*DIR_B], Error::UnexpectedNumberOfTopLevelEntries)]
+    #[case::top_level_file_entry(&[&*FILE_A], Error::UnexpectedNumberOfTopLevelEntries)]
+    fn node_ingestion_error(#[case] in_entries: &[&IngestionEntry], #[case] exp_error: Error) {
+        let mut nodes = IngestionEntryGraph::new();
+
+        let result = (|| {
+            for entry in in_entries {
+                nodes.add((*entry).clone())?;
+            }
+            nodes.finalize()
+        })();
+
+        let error = result.expect_err("expected error");
+        assert_eq!(error.to_string(), exp_error.to_string());
+    }
+}
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/error.rs b/tvix/castore/src/import/error.rs
new file mode 100644
index 000000000000..e3fba617e08f
--- /dev/null
+++ b/tvix/castore/src/import/error.rs
@@ -0,0 +1,20 @@
+use super::PathBuf;
+
+use crate::Error as CastoreError;
+
+/// Represents all error types that emitted by ingest_entries.
+/// It can represent errors uploading individual Directories and finalizing
+/// the upload.
+/// It also contains a generic error kind that'll carry ingestion-method
+/// specific errors.
+#[derive(Debug, thiserror::Error)]
+pub enum IngestionError<E: std::fmt::Display> {
+    #[error("error from producer: {0}")]
+    Producer(#[from] E),
+
+    #[error("failed to upload directory at {0}: {1}")]
+    UploadDirectoryError(PathBuf, CastoreError),
+
+    #[error("failed to finalize directory upload: {0}")]
+    FinalizeDirectoryUpload(CastoreError),
+}
diff --git a/tvix/castore/src/import/fs.rs b/tvix/castore/src/import/fs.rs
new file mode 100644
index 000000000000..1332fdfe57b5
--- /dev/null
+++ b/tvix/castore/src/import/fs.rs
@@ -0,0 +1,216 @@
+//! Import from a real filesystem.
+
+use futures::stream::BoxStream;
+use futures::StreamExt;
+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::{B3Digest, Node};
+
+use super::ingest_entries;
+use super::IngestionEntry;
+use super::IngestionError;
+
+/// Ingests the contents at a given path into the tvix store, interacting with a [BlobService] and
+/// [DirectoryService]. It returns the root node or an error.
+///
+/// It does not follow symlinks at the root, they will be ingested as actual symlinks.
+///
+/// This function will walk the filesystem using `walkdir` and will consume
+/// `O(#number of entries)` space.
+#[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,
+    path: P,
+) -> Result<Node, IngestionError<Error>>
+where
+    P: AsRef<std::path::Path> + std::fmt::Debug,
+    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)
+        .contents_first(true)
+        .into_iter();
+
+    let entries = dir_entries_to_ingestion_stream(blob_service, iter, path.as_ref());
+    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.
+/// This can then be fed into [ingest_entries] to ingest all the entries into the castore.
+///
+/// The produced stream is buffered, so uploads can happen concurrently.
+///
+/// The root is the [Path] in the filesystem that is being ingested into the castore.
+pub fn dir_entries_to_ingestion_stream<'a, BS, I>(
+    blob_service: BS,
+    iter: I,
+    root: &'a std::path::Path,
+) -> BoxStream<'a, Result<IngestionEntry, Error>>
+where
+    BS: BlobService + Clone + 'a,
+    I: Iterator<Item = Result<DirEntry, walkdir::Error>> + Send + 'a,
+{
+    let prefix = root.parent().unwrap_or_else(|| std::path::Path::new(""));
+
+    Box::pin(
+        futures::stream::iter(iter)
+            .map(move |x| {
+                let blob_service = blob_service.clone();
+                async move {
+                    match x {
+                        Ok(dir_entry) => {
+                            dir_entry_to_ingestion_entry(blob_service, &dir_entry, prefix).await
+                        }
+                        Err(e) => Err(Error::Stat(
+                            prefix.to_path_buf(),
+                            e.into_io_error().expect("walkdir err must be some"),
+                        )),
+                    }
+                }
+            })
+            .buffered(50),
+    )
+}
+
+/// Converts a [walkdir::DirEntry] into an [IngestionEntry], uploading blobs to the
+/// provided [BlobService].
+///
+/// The prefix path is stripped from the path of each entry. This is usually the parent path
+/// of the path being ingested so that the last element of the stream only has one component.
+pub async fn dir_entry_to_ingestion_entry<BS>(
+    blob_service: BS,
+    entry: &DirEntry,
+    prefix: &std::path::Path,
+) -> Result<IngestionEntry, Error>
+where
+    BS: BlobService,
+{
+    let file_type = entry.file_type();
+
+    let fs_path = entry
+        .path()
+        .strip_prefix(prefix)
+        .expect("Tvix bug: failed to strip root path prefix");
+
+    // convert to castore PathBuf
+    let path = crate::path::PathBuf::from_host_path(fs_path, false)
+        .unwrap_or_else(|e| panic!("Tvix bug: walkdir direntry cannot be parsed: {}", e));
+
+    if file_type.is_dir() {
+        Ok(IngestionEntry::Dir { path })
+    } else if file_type.is_symlink() {
+        let target = std::fs::read_link(entry.path())
+            .map_err(|e| Error::Stat(entry.path().to_path_buf(), e))?
+            .into_os_string()
+            .into_vec();
+
+        Ok(IngestionEntry::Symlink { path, target })
+    } else if file_type.is_file() {
+        let metadata = entry
+            .metadata()
+            .map_err(|e| Error::Stat(entry.path().to_path_buf(), e.into()))?;
+
+        let digest = upload_blob(blob_service, entry.path().to_path_buf()).await?;
+
+        Ok(IngestionEntry::Regular {
+            path,
+            size: metadata.size(),
+            // If it's executable by the user, it'll become executable.
+            // This matches nix's dump() function behaviour.
+            executable: metadata.permissions().mode() & 64 != 0,
+            digest,
+        })
+    } else {
+        return Err(Error::FileType(fs_path.to_path_buf(), file_type));
+    }
+}
+
+/// Uploads the file at the provided [Path] the the [BlobService].
+#[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>,
+) -> Result<B3Digest, Error>
+where
+    BS: BlobService,
+{
+    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 file = tokio::fs::File::open(path.as_ref())
+        .await
+        .map_err(|e| 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()
+        .await
+        .map_err(|e| Error::BlobFinalize(path.as_ref().to_path_buf(), e))?;
+
+    Ok(digest)
+}
+
+#[derive(Debug, thiserror::Error)]
+pub enum Error {
+    #[error("unsupported file type at {0}: {1:?}")]
+    FileType(std::path::PathBuf, FileType),
+
+    #[error("unable to stat {0}: {1}")]
+    Stat(std::path::PathBuf, std::io::Error),
+
+    #[error("unable to open {0}: {1}")]
+    Open(std::path::PathBuf, std::io::Error),
+
+    #[error("unable to read {0}: {1}")]
+    BlobRead(std::path::PathBuf, std::io::Error),
+
+    // TODO: proper error for blob finalize
+    #[error("unable to finalize blob {0}: {1}")]
+    BlobFinalize(std::path::PathBuf, std::io::Error),
+}
diff --git a/tvix/castore/src/import/mod.rs b/tvix/castore/src/import/mod.rs
new file mode 100644
index 000000000000..6e10a64939a4
--- /dev/null
+++ b/tvix/castore/src/import/mod.rs
@@ -0,0 +1,338 @@
+//! The main library function here is [ingest_entries], receiving a stream of
+//! [IngestionEntry].
+//!
+//! Specific implementations, such as ingesting from the filesystem, live in
+//! child modules.
+
+use crate::directoryservice::{DirectoryPutter, DirectoryService};
+use crate::path::{Path, PathBuf};
+use crate::{B3Digest, Directory, Node, SymlinkTargetError};
+use futures::{Stream, StreamExt};
+use tracing::Level;
+
+use std::collections::HashMap;
+use tracing::instrument;
+
+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].
+/// On success, returns the root [Node].
+///
+/// The stream must have the following invariants:
+/// - All children entries must come before their parents.
+/// - The last entry must be the root node which must have a single path component.
+/// - Every entry should have a unique path, and only consist of normal components.
+///   This means, no windows path prefixes, absolute paths, `.` or `..`.
+/// - All referenced directories must have an associated directory entry in the stream.
+///   This means if there is a file entry for `foo/bar`, there must also be a `foo` directory
+///   entry.
+///
+/// Internally we maintain a [HashMap] of [PathBuf] to partially populated [Directory] at that
+/// path. Once we receive an [IngestionEntry] for the directory itself, we remove it from the
+/// map and upload it to the [DirectoryService] through a lazily created [DirectoryPutter].
+///
+/// On success, returns the root node.
+#[instrument(skip_all, ret(level = Level::TRACE), err)]
+pub async fn ingest_entries<DS, S, E>(
+    directory_service: DS,
+    mut entries: S,
+) -> Result<Node, IngestionError<E>>
+where
+    DS: DirectoryService,
+    S: Stream<Item = Result<IngestionEntry, E>> + Send + std::marker::Unpin,
+    E: std::error::Error,
+{
+    // For a given path, this holds the [Directory] structs as they are populated.
+    let mut directories: HashMap<PathBuf, Directory> = HashMap::default();
+    let mut maybe_directory_putter: Option<Box<dyn DirectoryPutter>> = None;
+
+    let root_node = loop {
+        let mut entry = entries
+            .next()
+            .await
+            // The last entry of the stream must have 1 path component, after which
+            // we break the loop manually.
+            .expect("Tvix bug: unexpected end of stream")?;
+
+        let node = match &mut entry {
+            IngestionEntry::Dir { .. } => {
+                // If the entry is a directory, we traversed all its children (and
+                // populated it in `directories`).
+                // If we don't have it in directories, it's a directory without
+                // children.
+                let directory = directories
+                    .remove(entry.path())
+                    // In that case, it contained no children
+                    .unwrap_or_default();
+
+                let directory_size = directory.size();
+                let directory_digest = directory.digest();
+
+                // Use the directory_putter to upload the directory.
+                // If we don't have one yet (as that's the first one to upload),
+                // initialize the putter.
+                maybe_directory_putter
+                    .get_or_insert_with(|| directory_service.put_multiple_start())
+                    .put(directory)
+                    .await
+                    .map_err(|e| {
+                        IngestionError::UploadDirectoryError(entry.path().to_owned(), e)
+                    })?;
+
+                Node::Directory {
+                    digest: directory_digest,
+                    size: directory_size,
+                }
+            }
+            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 {
+                digest: digest.clone(),
+                size: *size,
+                executable: *executable,
+            },
+        };
+
+        let parent = entry
+            .path()
+            .parent()
+            .expect("Tvix bug: got entry with root node");
+
+        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(name, node)
+                .map_err(|e| {
+                    IngestionError::UploadDirectoryError(
+                        entry.path().to_owned(),
+                        crate::Error::StorageError(e.to_string()),
+                    )
+                })?;
+        }
+    };
+
+    assert!(
+        entries.count().await == 0,
+        "Tvix bug: left over elements in the stream"
+    );
+
+    assert!(
+        directories.is_empty(),
+        "Tvix bug: left over directories after processing ingestion stream"
+    );
+
+    // if there were directories uploaded, make sure we flush the putter, so
+    // they're all persisted to the backend.
+    if let Some(mut directory_putter) = maybe_directory_putter {
+        #[cfg_attr(not(debug_assertions), allow(unused))]
+        let root_directory_digest = directory_putter
+            .close()
+            .await
+            .map_err(|e| IngestionError::FinalizeDirectoryUpload(e))?;
+
+        #[cfg(debug_assertions)]
+        {
+            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");
+            }
+        }
+    };
+
+    Ok(root_node)
+}
+
+#[derive(Debug, Clone, Eq, PartialEq)]
+pub enum IngestionEntry {
+    Regular {
+        path: PathBuf,
+        size: u64,
+        executable: bool,
+        digest: B3Digest,
+    },
+    Symlink {
+        path: PathBuf,
+        target: Vec<u8>,
+    },
+    Dir {
+        path: PathBuf,
+    },
+}
+
+impl IngestionEntry {
+    fn path(&self) -> &Path {
+        match self {
+            IngestionEntry::Regular { path, .. } => path,
+            IngestionEntry::Symlink { path, .. } => path,
+            IngestionEntry::Dir { path } => path,
+        }
+    }
+
+    fn is_dir(&self) -> bool {
+        matches!(self, IngestionEntry::Dir { .. })
+    }
+}
+
+#[cfg(test)]
+mod test {
+    use rstest::rstest;
+
+    use crate::fixtures::{DIRECTORY_COMPLICATED, DIRECTORY_WITH_KEEP, EMPTY_BLOB_DIGEST};
+    use crate::{directoryservice::MemoryDirectoryService, fixtures::DUMMY_DIGEST};
+    use crate::{Directory, Node};
+
+    use super::ingest_entries;
+    use super::IngestionEntry;
+
+    #[rstest]
+    #[case::single_file(vec![IngestionEntry::Regular {
+        path: "foo".parse().unwrap(),
+        size: 42,
+        executable: true,
+        digest: DUMMY_DIGEST.clone(),
+    }],
+        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{target: "blub".try_into().unwrap()}
+    )]
+    #[case::single_dir(vec![IngestionEntry::Dir {
+        path: "foo".parse().unwrap(),
+    }],
+        Node::Directory{digest: Directory::default().digest(), size: Directory::default().size()}
+    )]
+    #[case::dir_with_keep(vec![
+        IngestionEntry::Regular {
+            path: "foo/.keep".parse().unwrap(),
+            size: 0,
+            executable: false,
+            digest: EMPTY_BLOB_DIGEST.clone(),
+        },
+        IngestionEntry::Dir {
+            path: "foo".parse().unwrap(),
+        },
+    ],
+        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.
+    #[case::directory_complicated(vec![
+        IngestionEntry::Regular {
+            path: "blub/.keep".parse().unwrap(),
+            size: 0,
+            executable: false,
+            digest: EMPTY_BLOB_DIGEST.clone(),
+        },
+        IngestionEntry::Regular {
+            path: "blub/keep/.keep".parse().unwrap(),
+            size: 0,
+            executable: false,
+            digest: EMPTY_BLOB_DIGEST.clone(),
+        },
+        IngestionEntry::Dir {
+            path: "blub/keep".parse().unwrap(),
+        },
+        IngestionEntry::Symlink {
+            path: "blub/aa".parse().unwrap(),
+            target: b"/nix/store/somewhereelse".into(),
+        },
+        IngestionEntry::Dir {
+            path: "blub".parse().unwrap(),
+        },
+    ],
+    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) {
+        let directory_service = MemoryDirectoryService::default();
+
+        let root_node = ingest_entries(
+            directory_service.clone(),
+            futures::stream::iter(entries.into_iter().map(Ok::<_, std::io::Error>)),
+        )
+        .await
+        .expect("must succeed");
+
+        assert_eq!(exp_root_node, root_node, "root node should match");
+    }
+
+    #[rstest]
+    #[should_panic]
+    #[case::empty_entries(vec![])]
+    #[should_panic]
+    #[case::missing_intermediate_dir(vec![
+        IngestionEntry::Regular {
+            path: "blub/.keep".parse().unwrap(),
+            size: 0,
+            executable: false,
+            digest: EMPTY_BLOB_DIGEST.clone(),
+        },
+    ])]
+    #[should_panic]
+    #[case::leaf_after_parent(vec![
+        IngestionEntry::Dir {
+            path: "blub".parse().unwrap(),
+        },
+        IngestionEntry::Regular {
+            path: "blub/.keep".parse().unwrap(),
+            size: 0,
+            executable: false,
+            digest: EMPTY_BLOB_DIGEST.clone(),
+        },
+    ])]
+    #[should_panic]
+    #[case::root_in_entry(vec![
+        IngestionEntry::Regular {
+            path: ".keep".parse().unwrap(),
+            size: 0,
+            executable: false,
+            digest: EMPTY_BLOB_DIGEST.clone(),
+        },
+        IngestionEntry::Dir {
+            path: "".parse().unwrap(),
+        },
+    ])]
+    #[tokio::test]
+    async fn test_ingestion_fail(#[case] entries: Vec<IngestionEntry>) {
+        let directory_service = MemoryDirectoryService::default();
+
+        let _ = ingest_entries(
+            directory_service.clone(),
+            futures::stream::iter(entries.into_iter().map(Ok::<_, std::io::Error>)),
+        )
+        .await;
+    }
+}
diff --git a/tvix/castore/src/lib.rs b/tvix/castore/src/lib.rs
new file mode 100644
index 000000000000..8ac6ca3dd66a
--- /dev/null
+++ b/tvix/castore/src/lib.rs
@@ -0,0 +1,34 @@
+mod digests;
+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, PathComponent, PathComponentError};
+
+pub mod import;
+pub mod proto;
+pub mod tonic;
+
+pub use digests::{B3Digest, B3_LEN};
+pub use errors::{DirectoryError, Error, ValidateNodeError};
+pub use hashing_reader::{B3HashingReader, HashingReader};
+
+#[cfg(test)]
+mod tests;
+
+// That's what the rstest_reuse README asks us do, and fails about being unable
+// to find rstest_reuse in crate root.
+#[cfg(test)]
+#[allow(clippy::single_component_path_imports)]
+use rstest_reuse;
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/mod.rs b/tvix/castore/src/path/mod.rs
new file mode 100644
index 000000000000..15f31a570da9
--- /dev/null
+++ b/tvix/castore/src/path/mod.rs
@@ -0,0 +1,470 @@
+//! Contains data structures to deal with Paths in the tvix-castore model.
+use bstr::ByteSlice;
+use std::{
+    borrow::Borrow,
+    fmt::{self, Debug, Display},
+    mem,
+    ops::Deref,
+    str::FromStr,
+};
+
+mod component;
+pub use component::{PathComponent, PathComponentError};
+
+/// Represents a Path in the castore model.
+/// These are always relative, and platform-independent, which distinguishes
+/// them from the ones provided in the standard library.
+#[derive(Eq, Hash, PartialEq)]
+#[repr(transparent)] // SAFETY: Representation has to match [u8]
+pub struct Path {
+    // As node names in the castore model cannot contain slashes,
+    // we use them as component separators here.
+    inner: [u8],
+}
+
+#[allow(dead_code)]
+impl Path {
+    // SAFETY: The empty path is valid.
+    pub const ROOT: &'static Path = unsafe { Path::from_bytes_unchecked(&[]) };
+
+    /// Convert a byte slice to a path, without checking validity.
+    const unsafe fn from_bytes_unchecked(bytes: &[u8]) -> &Path {
+        // SAFETY: &[u8] and &Path have the same representation.
+        unsafe { mem::transmute(bytes) }
+    }
+
+    fn from_bytes(bytes: &[u8]) -> Option<&Path> {
+        if !bytes.is_empty() {
+            // Ensure all components are valid castore node names.
+            for component in bytes.split_str(b"/") {
+                if component::validate_name(component).is_err() {
+                    return None;
+                }
+            }
+        }
+
+        // SAFETY: We have verified that the path contains no empty components.
+        Some(unsafe { Path::from_bytes_unchecked(bytes) })
+    }
+
+    pub fn into_boxed_bytes(self: Box<Path>) -> Box<[u8]> {
+        // SAFETY: Box<Path> and Box<[u8]> have the same representation.
+        unsafe { mem::transmute(self) }
+    }
+
+    /// Returns the path without its final component, if there is one.
+    ///
+    /// Note that the parent of a bare file name is [Path::ROOT].
+    /// [Path::ROOT] is the only path without a parent.
+    pub fn parent(&self) -> Option<&Path> {
+        // The root does not have a parent.
+        if self.inner.is_empty() {
+            return None;
+        }
+
+        Some(
+            if let Some((parent, _file_name)) = self.inner.rsplit_once_str(b"/") {
+                // SAFETY: The parent of a valid Path is a valid Path.
+                unsafe { Path::from_bytes_unchecked(parent) }
+            } else {
+                // The parent of a bare file name is the root.
+                Path::ROOT
+            },
+        )
+    }
+
+    /// Creates a PathBuf with `name` adjoined to self.
+    pub fn try_join(&self, name: &[u8]) -> Result<PathBuf, std::io::Error> {
+        let mut v = PathBuf::with_capacity(self.inner.len() + name.len() + 1);
+        v.inner.extend_from_slice(&self.inner);
+        v.try_push(name)?;
+
+        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_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.
+        if self.inner.is_empty() {
+            let _ = iter.next();
+        }
+
+        iter
+    }
+
+    /// 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
+    }
+}
+
+impl Debug for Path {
+    fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
+        Debug::fmt(self.inner.as_bstr(), f)
+    }
+}
+
+impl Display for Path {
+    fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
+        Display::fmt(self.inner.as_bstr(), f)
+    }
+}
+
+impl AsRef<Path> for Path {
+    fn as_ref(&self) -> &Path {
+        self
+    }
+}
+
+/// Represents a owned PathBuf in the castore model.
+/// These are always relative, and platform-independent, which distinguishes
+/// them from the ones provided in the standard library.
+#[derive(Clone, Default, Eq, Hash, PartialEq)]
+pub struct PathBuf {
+    inner: Vec<u8>,
+}
+
+impl Deref for PathBuf {
+    type Target = Path;
+
+    fn deref(&self) -> &Self::Target {
+        // SAFETY: PathBuf always contains a valid Path.
+        unsafe { Path::from_bytes_unchecked(&self.inner) }
+    }
+}
+
+impl AsRef<Path> for PathBuf {
+    fn as_ref(&self) -> &Path {
+        self
+    }
+}
+
+impl ToOwned for Path {
+    type Owned = PathBuf;
+
+    fn to_owned(&self) -> Self::Owned {
+        PathBuf {
+            inner: self.inner.to_owned(),
+        }
+    }
+}
+
+impl Borrow<Path> for PathBuf {
+    fn borrow(&self) -> &Path {
+        self
+    }
+}
+
+impl From<Box<Path>> for PathBuf {
+    fn from(value: Box<Path>) -> Self {
+        // SAFETY: Box<Path> is always a valid path.
+        unsafe { PathBuf::from_bytes_unchecked(value.into_boxed_bytes().into_vec()) }
+    }
+}
+
+impl From<&Path> for PathBuf {
+    fn from(value: &Path) -> Self {
+        value.to_owned()
+    }
+}
+
+impl FromStr for PathBuf {
+    type Err = std::io::Error;
+
+    fn from_str(s: &str) -> Result<PathBuf, Self::Err> {
+        Ok(Path::from_bytes(s.as_bytes())
+            .ok_or(std::io::ErrorKind::InvalidData)?
+            .to_owned())
+    }
+}
+
+impl Debug for PathBuf {
+    fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
+        Debug::fmt(&**self, f)
+    }
+}
+
+impl Display for PathBuf {
+    fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
+        Display::fmt(&**self, f)
+    }
+}
+
+impl PathBuf {
+    pub fn new() -> PathBuf {
+        Self::default()
+    }
+
+    pub fn with_capacity(capacity: usize) -> PathBuf {
+        // SAFETY: The empty path is a valid path.
+        Self {
+            inner: Vec::with_capacity(capacity),
+        }
+    }
+
+    /// Adjoins `name` to self.
+    pub fn try_push(&mut self, name: &[u8]) -> Result<(), std::io::Error> {
+        if component::validate_name(name).is_err() {
+            return Err(std::io::ErrorKind::InvalidData.into());
+        }
+
+        if !self.inner.is_empty() {
+            self.inner.push(b'/');
+        }
+
+        self.inner.extend_from_slice(name);
+
+        Ok(())
+    }
+
+    /// Convert a byte vector to a PathBuf, without checking validity.
+    unsafe fn from_bytes_unchecked(bytes: Vec<u8>) -> PathBuf {
+        PathBuf { inner: bytes }
+    }
+
+    /// Convert from a [&std::path::Path] to [Self].
+    ///
+    /// - Self uses `/` as path separator.
+    /// - Absolute paths are always rejected, are are these with custom prefixes.
+    /// - Repeated separators are deduplicated.
+    /// - Occurrences of `.` are normalized away.
+    /// - A trailing slash is normalized away.
+    ///
+    /// A `canonicalize_dotdot` boolean controls whether `..` will get
+    /// canonicalized if possible, or should return an error.
+    ///
+    /// For more exotic paths, this conversion might produce different results
+    /// on different platforms, due to different underlying byte
+    /// representations, which is why it's restricted to unix for now.
+    #[cfg(unix)]
+    pub fn from_host_path(
+        host_path: &std::path::Path,
+        canonicalize_dotdot: bool,
+    ) -> Result<Self, std::io::Error> {
+        let mut p = PathBuf::with_capacity(host_path.as_os_str().len());
+
+        for component in host_path.components() {
+            match component {
+                std::path::Component::Prefix(_) | std::path::Component::RootDir => {
+                    return Err(std::io::Error::new(
+                        std::io::ErrorKind::InvalidData,
+                        "found disallowed prefix or rootdir",
+                    ))
+                }
+                std::path::Component::CurDir => continue, // ignore
+                std::path::Component::ParentDir => {
+                    if canonicalize_dotdot {
+                        // Try popping the last element from the path being constructed.
+                        // FUTUREWORK: pop method?
+                        p = p
+                            .parent()
+                            .ok_or_else(|| {
+                                std::io::Error::new(
+                                    std::io::ErrorKind::InvalidData,
+                                    "found .. going too far up",
+                                )
+                            })?
+                            .to_owned();
+                    } else {
+                        return Err(std::io::Error::new(
+                            std::io::ErrorKind::InvalidData,
+                            "found disallowed ..",
+                        ));
+                    }
+                }
+                std::path::Component::Normal(s) => {
+                    // append the new component to the path being constructed.
+                    p.try_push(s.as_encoded_bytes()).map_err(|_| {
+                        std::io::Error::new(
+                            std::io::ErrorKind::InvalidData,
+                            "encountered invalid node in sub_path component",
+                        )
+                    })?
+                }
+            }
+        }
+
+        Ok(p)
+    }
+
+    pub fn into_boxed_path(self) -> Box<Path> {
+        // SAFETY: Box<[u8]> and Box<Path> have the same representation,
+        // and PathBuf always contains a valid Path.
+        unsafe { mem::transmute(self.inner.into_boxed_slice()) }
+    }
+
+    pub fn into_bytes(self) -> Vec<u8> {
+        self.inner
+    }
+}
+
+#[cfg(test)]
+mod test {
+    use super::{Path, PathBuf};
+    use bstr::ByteSlice;
+    use rstest::rstest;
+
+    // TODO: add some manual tests including invalid UTF-8 (hard to express
+    // with rstest)
+
+    #[rstest]
+    #[case::empty("", 0)]
+    #[case("a", 1)]
+    #[case("a/b", 2)]
+    #[case("a/b/c", 3)]
+    // add two slightly more cursed variants.
+    // Technically nothing prevents us from representing this with castore,
+    // but maybe we want to disallow constructing paths like this as it's a
+    // bad idea.
+    #[case::cursed("C:\\a/b", 2)]
+    #[case::cursed("\\\\tvix-store", 1)]
+    pub fn from_str(#[case] s: &str, #[case] num_components: usize) {
+        let p: PathBuf = s.parse().expect("must parse");
+
+        assert_eq!(s.as_bytes(), p.as_bytes(), "inner bytes mismatch");
+        assert_eq!(
+            num_components,
+            p.components_bytes().count(),
+            "number of components mismatch"
+        );
+    }
+
+    #[rstest]
+    #[case::absolute("/a/b")]
+    #[case::two_forward_slashes_start("//a/b")]
+    #[case::two_forward_slashes_middle("a/b//c/d")]
+    #[case::trailing_slash("a/b/")]
+    #[case::dot(".")]
+    #[case::dotdot("..")]
+    #[case::dot_start("./a")]
+    #[case::dotdot_start("../a")]
+    #[case::dot_middle("a/./b")]
+    #[case::dotdot_middle("a/../b")]
+    #[case::dot_end("a/b/.")]
+    #[case::dotdot_end("a/b/..")]
+    #[case::null("fo\0o")]
+    pub fn from_str_fail(#[case] s: &str) {
+        s.parse::<PathBuf>().expect_err("must fail");
+    }
+
+    #[rstest]
+    #[case("foo", "")]
+    #[case("foo/bar", "foo")]
+    #[case("foo2/bar2", "foo2")]
+    #[case("foo/bar/baz", "foo/bar")]
+    pub fn parent(#[case] p: PathBuf, #[case] exp_parent: PathBuf) {
+        assert_eq!(Some(&*exp_parent), p.parent());
+    }
+
+    #[rstest]
+    pub fn no_parent() {
+        assert!(Path::ROOT.parent().is_none());
+    }
+
+    #[rstest]
+    #[case("a", "b", "a/b")]
+    #[case("a", "b", "a/b")]
+    pub fn join_push(#[case] mut p: PathBuf, #[case] name: &str, #[case] exp_p: PathBuf) {
+        assert_eq!(exp_p, p.try_join(name.as_bytes()).expect("join failed"));
+        p.try_push(name.as_bytes()).expect("push failed");
+        assert_eq!(exp_p, p);
+    }
+
+    #[rstest]
+    #[case("a", "/")]
+    #[case("a", "")]
+    #[case("a", "b/c")]
+    #[case("", "/")]
+    #[case("", "")]
+    #[case("", "b/c")]
+    #[case("", ".")]
+    #[case("", "..")]
+    pub fn join_push_fail(#[case] mut p: PathBuf, #[case] name: &str) {
+        p.try_join(name.as_bytes())
+            .expect_err("join succeeded unexpectedly");
+        p.try_push(name.as_bytes())
+            .expect_err("push succeeded unexpectedly");
+    }
+
+    #[rstest]
+    #[case::empty("", vec![])]
+    #[case("a", vec!["a"])]
+    #[case("a/b", vec!["a", "b"])]
+    #[case("a/b/c", vec!["a","b", "c"])]
+    pub fn components_bytes(#[case] p: PathBuf, #[case] exp_components: Vec<&str>) {
+        assert_eq!(
+            exp_components,
+            p.components_bytes()
+                .map(|x| x.to_str().unwrap())
+                .collect::<Vec<_>>()
+        );
+    }
+
+    #[rstest]
+    #[case::empty("", "", false)]
+    #[case::path("a", "a", false)]
+    #[case::path2("a/b", "a/b", false)]
+    #[case::double_slash_middle("a//b", "a/b", false)]
+    #[case::dot(".", "", false)]
+    #[case::dot_start("./a/b", "a/b", false)]
+    #[case::dot_middle("a/./b", "a/b", false)]
+    #[case::dot_end("a/b/.", "a/b", false)]
+    #[case::trailing_slash("a/b/", "a/b", false)]
+    #[case::dotdot_canonicalize("a/..", "", true)]
+    #[case::dotdot_canonicalize2("a/../b", "b", true)]
+    #[cfg_attr(unix, case::faux_prefix("\\\\nix-store", "\\\\nix-store", false))]
+    #[cfg_attr(unix, case::faux_letter("C:\\foo.txt", "C:\\foo.txt", false))]
+    pub fn from_host_path(
+        #[case] host_path: std::path::PathBuf,
+        #[case] exp_path: PathBuf,
+        #[case] canonicalize_dotdot: bool,
+    ) {
+        let p = PathBuf::from_host_path(&host_path, canonicalize_dotdot).expect("must succeed");
+
+        assert_eq!(exp_path, p);
+    }
+
+    #[rstest]
+    #[case::absolute("/", false)]
+    #[case::dotdot_root("..", false)]
+    #[case::dotdot_root_canonicalize("..", true)]
+    #[case::dotdot_root_no_canonicalize("a/..", false)]
+    #[case::invalid_name("foo/bar\0", false)]
+    // #[cfg_attr(windows, case::prefix("\\\\nix-store", false))]
+    // #[cfg_attr(windows, case::letter("C:\\foo.txt", false))]
+    pub fn from_host_path_fail(
+        #[case] host_path: std::path::PathBuf,
+        #[case] canonicalize_dotdot: bool,
+    ) {
+        PathBuf::from_host_path(&host_path, canonicalize_dotdot).expect_err("must fail");
+    }
+}
diff --git a/tvix/castore/src/proto/grpc_blobservice_wrapper.rs b/tvix/castore/src/proto/grpc_blobservice_wrapper.rs
new file mode 100644
index 000000000000..41bd0698ec98
--- /dev/null
+++ b/tvix/castore/src/proto/grpc_blobservice_wrapper.rs
@@ -0,0 +1,175 @@
+use crate::blobservice::BlobService;
+use core::pin::pin;
+use data_encoding::BASE64;
+use futures::{stream::BoxStream, TryFutureExt};
+use std::{
+    collections::VecDeque,
+    ops::{Deref, DerefMut},
+};
+use tokio_stream::StreamExt;
+use tokio_util::io::ReaderStream;
+use tonic::{async_trait, Request, Response, Status, Streaming};
+use tracing::{instrument, warn};
+
+pub struct GRPCBlobServiceWrapper<T> {
+    blob_service: T,
+}
+
+impl<T> GRPCBlobServiceWrapper<T> {
+    pub fn new(blob_service: T) -> Self {
+        Self { blob_service }
+    }
+}
+
+// This is necessary because bytes::BytesMut comes up with
+// a default 64 bytes capacity that cannot be changed
+// easily if you assume a bytes::BufMut trait implementation
+// Therefore, we override the Default implementation here
+// TODO(raitobezarius?): upstream me properly
+struct BytesMutWithDefaultCapacity<const N: usize> {
+    inner: bytes::BytesMut,
+}
+
+impl<const N: usize> Deref for BytesMutWithDefaultCapacity<N> {
+    type Target = bytes::BytesMut;
+    fn deref(&self) -> &Self::Target {
+        &self.inner
+    }
+}
+
+impl<const N: usize> DerefMut for BytesMutWithDefaultCapacity<N> {
+    fn deref_mut(&mut self) -> &mut Self::Target {
+        &mut self.inner
+    }
+}
+
+impl<const N: usize> Default for BytesMutWithDefaultCapacity<N> {
+    fn default() -> Self {
+        BytesMutWithDefaultCapacity {
+            inner: bytes::BytesMut::with_capacity(N),
+        }
+    }
+}
+
+impl<const N: usize> bytes::Buf for BytesMutWithDefaultCapacity<N> {
+    fn remaining(&self) -> usize {
+        self.inner.remaining()
+    }
+
+    fn chunk(&self) -> &[u8] {
+        self.inner.chunk()
+    }
+
+    fn advance(&mut self, cnt: usize) {
+        self.inner.advance(cnt);
+    }
+}
+
+unsafe impl<const N: usize> bytes::BufMut for BytesMutWithDefaultCapacity<N> {
+    fn remaining_mut(&self) -> usize {
+        self.inner.remaining_mut()
+    }
+
+    unsafe fn advance_mut(&mut self, cnt: usize) {
+        self.inner.advance_mut(cnt);
+    }
+
+    fn chunk_mut(&mut self) -> &mut bytes::buf::UninitSlice {
+        self.inner.chunk_mut()
+    }
+}
+
+#[async_trait]
+impl<T> super::blob_service_server::BlobService for GRPCBlobServiceWrapper<T>
+where
+    T: Deref<Target = dyn BlobService> + Send + Sync + 'static,
+{
+    // https://github.com/tokio-rs/tokio/issues/2723#issuecomment-1534723933
+    type ReadStream = BoxStream<'static, Result<super::BlobChunk, Status>>;
+
+    #[instrument(skip_all, fields(blob.digest=format!("b3:{}", BASE64.encode(&request.get_ref().digest))))]
+    async fn stat(
+        &self,
+        request: Request<super::StatBlobRequest>,
+    ) -> Result<Response<super::StatBlobResponse>, Status> {
+        let rq = request.into_inner();
+        let req_digest = rq
+            .digest
+            .try_into()
+            .map_err(|_e| Status::invalid_argument("invalid digest length"))?;
+
+        match self.blob_service.chunks(&req_digest).await {
+            Ok(None) => Err(Status::not_found(format!("blob {} not found", &req_digest))),
+            Ok(Some(chunk_metas)) => Ok(Response::new(super::StatBlobResponse {
+                chunks: chunk_metas,
+                ..Default::default()
+            })),
+            Err(e) => {
+                warn!(err=%e, "failed to request chunks");
+                Err(e.into())
+            }
+        }
+    }
+
+    #[instrument(skip_all, fields(blob.digest=format!("b3:{}", BASE64.encode(&request.get_ref().digest))))]
+    async fn read(
+        &self,
+        request: Request<super::ReadBlobRequest>,
+    ) -> Result<Response<Self::ReadStream>, Status> {
+        let rq = request.into_inner();
+
+        let req_digest = rq
+            .digest
+            .try_into()
+            .map_err(|_e| Status::invalid_argument("invalid digest length"))?;
+
+        match self.blob_service.open_read(&req_digest).await {
+            Ok(Some(r)) => {
+                let chunks_stream =
+                    ReaderStream::new(r).map(|chunk| Ok(super::BlobChunk { data: chunk? }));
+                Ok(Response::new(Box::pin(chunks_stream)))
+            }
+            Ok(None) => Err(Status::not_found(format!("blob {} not found", &req_digest))),
+            Err(e) => {
+                warn!(err=%e, "failed to call open_read");
+                Err(e.into())
+            }
+        }
+    }
+
+    #[instrument(skip_all)]
+    async fn put(
+        &self,
+        request: Request<Streaming<super::BlobChunk>>,
+    ) -> Result<Response<super::PutBlobResponse>, Status> {
+        let req_inner = request.into_inner();
+
+        let data_stream = req_inner.map(|x| {
+            x.map(|x| VecDeque::from(x.data.to_vec()))
+                .map_err(|e| std::io::Error::new(std::io::ErrorKind::InvalidInput, e))
+        });
+
+        let mut data_reader = tokio_util::io::StreamReader::new(data_stream);
+
+        let mut blob_writer = pin!(self.blob_service.open_write().await);
+
+        tokio::io::copy(&mut data_reader, &mut blob_writer)
+            .await
+            .map_err(|e| {
+                warn!("error copying: {}", e);
+                Status::internal("error copying")
+            })?;
+
+        let digest = blob_writer
+            .close()
+            .map_err(|e| {
+                warn!("error closing stream: {}", e);
+                Status::internal("error closing stream")
+            })
+            .await?;
+
+        Ok(Response::new(super::PutBlobResponse {
+            digest: digest.into(),
+        }))
+    }
+}
diff --git a/tvix/castore/src/proto/grpc_directoryservice_wrapper.rs b/tvix/castore/src/proto/grpc_directoryservice_wrapper.rs
new file mode 100644
index 000000000000..62fdb34a25a0
--- /dev/null
+++ b/tvix/castore/src/proto/grpc_directoryservice_wrapper.rs
@@ -0,0 +1,113 @@
+use crate::directoryservice::{DirectoryGraph, DirectoryService, LeavesToRootValidator};
+use crate::{proto, B3Digest, DirectoryError};
+use futures::stream::BoxStream;
+use futures::TryStreamExt;
+use std::ops::Deref;
+use tokio_stream::once;
+use tonic::{async_trait, Request, Response, Status, Streaming};
+use tracing::{instrument, warn};
+
+pub struct GRPCDirectoryServiceWrapper<T> {
+    directory_service: T,
+}
+
+impl<T> GRPCDirectoryServiceWrapper<T> {
+    pub fn new(directory_service: T) -> Self {
+        Self { directory_service }
+    }
+}
+
+#[async_trait]
+impl<T> proto::directory_service_server::DirectoryService for GRPCDirectoryServiceWrapper<T>
+where
+    T: Deref<Target = dyn DirectoryService> + Send + Sync + 'static,
+{
+    type GetStream = BoxStream<'static, tonic::Result<proto::Directory, Status>>;
+
+    #[instrument(skip_all)]
+    async fn get<'a>(
+        &'a self,
+        request: Request<proto::GetDirectoryRequest>,
+    ) -> Result<Response<Self::GetStream>, Status> {
+        let req_inner = request.into_inner();
+
+        let by_what = &req_inner
+            .by_what
+            .ok_or_else(|| Status::invalid_argument("invalid by_what"))?;
+
+        match by_what {
+            proto::get_directory_request::ByWhat::Digest(ref digest) => {
+                let digest: B3Digest = digest
+                    .clone()
+                    .try_into()
+                    .map_err(|_e| Status::invalid_argument("invalid digest length"))?;
+
+                Ok(tonic::Response::new({
+                    if !req_inner.recursive {
+                        let directory = self
+                            .directory_service
+                            .get(&digest)
+                            .await
+                            .map_err(|e| {
+                                warn!(err = %e, directory.digest=%digest, "failed to get directory");
+                                tonic::Status::new(tonic::Code::Internal, e.to_string())
+                            })?
+                            .ok_or_else(|| {
+                                Status::not_found(format!("directory {} not found", digest))
+                            })?;
+
+                        Box::pin(once(Ok(directory.into())))
+                    } else {
+                        // If recursive was requested, traverse via get_recursive.
+                        Box::pin(
+                            self.directory_service
+                                .get_recursive(&digest)
+                                .map_ok(proto::Directory::from)
+                                .map_err(|e| {
+                                    tonic::Status::new(tonic::Code::Internal, e.to_string())
+                                }),
+                        )
+                    }
+                }))
+            }
+        }
+    }
+
+    #[instrument(skip_all)]
+    async fn put(
+        &self,
+        request: Request<Streaming<proto::Directory>>,
+    ) -> Result<Response<proto::PutDirectoryResponse>, Status> {
+        let mut req_inner = request.into_inner();
+
+        // 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.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
+            .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 {
+            directory_putter.put(directory).await?;
+        }
+
+        // Properly close the directory putter. Peek at last_directory_digest
+        // and return it, or propagate errors.
+        let last_directory_dgst = directory_putter.close().await?;
+
+        Ok(Response::new(proto::PutDirectoryResponse {
+            root_digest: last_directory_dgst.into(),
+        }))
+    }
+}
diff --git a/tvix/castore/src/proto/mod.rs b/tvix/castore/src/proto/mod.rs
new file mode 100644
index 000000000000..8bc74b412676
--- /dev/null
+++ b/tvix/castore/src/proto/mod.rs
@@ -0,0 +1,288 @@
+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;
+
+tonic::include_proto!("tvix.castore.v1");
+
+#[cfg(feature = "tonic-reflection")]
+/// Compiled file descriptors for implementing [gRPC
+/// reflection](https://github.com/grpc/grpc/blob/master/doc/server-reflection.md) with e.g.
+/// [`tonic_reflection`](https://docs.rs/tonic-reflection).
+pub const FILE_DESCRIPTOR_SET: &[u8] = tonic::include_file_descriptor_set!("tvix.castore.v1");
+
+#[cfg(test)]
+mod tests;
+
+/// Errors that occur during StatBlobResponse validation
+#[derive(Debug, PartialEq, Eq, thiserror::Error)]
+pub enum ValidateStatBlobResponseError {
+    /// Invalid digest length encountered
+    #[error("Invalid digest length {0} for chunk #{1}")]
+    InvalidDigestLen(usize, usize),
+}
+
+fn checked_sum(iter: impl IntoIterator<Item = u64>) -> Option<u64> {
+    iter.into_iter().try_fold(0u64, |acc, i| acc.checked_add(i))
+}
+
+impl Directory {
+    /// 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 {
+        if cfg!(debug_assertions) {
+            self.size_checked()
+                .expect("Directory::size exceeds u64::MAX")
+        } else {
+            self.size_checked().unwrap_or(u64::MAX)
+        }
+    }
+
+    fn size_checked(&self) -> Option<u64> {
+        checked_sum([
+            self.files.len().try_into().ok()?,
+            self.symlinks.len().try_into().ok()?,
+            self.directories.len().try_into().ok()?,
+            checked_sum(self.directories.iter().map(|e| e.size))?,
+        ])
+    }
+
+    /// 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 {
+        let mut hasher = blake3::Hasher::new();
+
+        hasher
+            .update(&self.encode_to_vec())
+            .finalize()
+            .as_bytes()
+            .into()
+    }
+}
+
+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()),
+            }
+        })?;
+
+        // 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());
+
+        for e in value.directories {
+            elems.push(
+                Node {
+                    node: Some(node::Node::Directory(e)),
+                }
+                .into_name_and_node()?,
+            );
+        }
+
+        for e in value.files {
+            elems.push(
+                Node {
+                    node: Some(node::Node::File(e)),
+                }
+                .into_name_and_node()?,
+            )
+        }
+
+        for e in value.symlinks {
+            elems.push(
+                Node {
+                    node: Some(node::Node::Symlink(e)),
+                }
+                .into_name_and_node()?,
+            )
+        }
+
+        crate::Directory::try_from_iter(elems)
+    }
+}
+
+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(),
+                    });
+                }
+            }
+        }
+
+        Directory {
+            directories,
+            files,
+            symlinks,
+        }
+    }
+}
+
+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::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(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 {
+    /// Validates a StatBlobResponse. All chunks must have valid blake3 digests.
+    /// It is allowed to send an empty list, if no more granular chunking is
+    /// available.
+    pub fn validate(&self) -> Result<(), ValidateStatBlobResponseError> {
+        for (i, chunk) in self.chunks.iter().enumerate() {
+            if chunk.digest.len() != blake3::KEY_LEN {
+                return Err(ValidateStatBlobResponseError::InvalidDigestLen(
+                    chunk.digest.len(),
+                    i,
+                ));
+            }
+        }
+        Ok(())
+    }
+}
diff --git a/tvix/castore/src/proto/tests/directory.rs b/tvix/castore/src/proto/tests/directory.rs
new file mode 100644
index 000000000000..efbc4e9f2af1
--- /dev/null
+++ b/tvix/castore/src/proto/tests/directory.rs
@@ -0,0 +1,370 @@
+use crate::proto::{Directory, DirectoryError, DirectoryNode, FileNode, SymlinkNode};
+use crate::ValidateNodeError;
+
+use hex_literal::hex;
+
+const DUMMY_DIGEST: [u8; 32] = [0; 32];
+
+#[test]
+fn size() {
+    {
+        let d = Directory::default();
+        assert_eq!(d.size(), 0);
+    }
+    {
+        let d = Directory {
+            directories: vec![DirectoryNode {
+                name: "foo".into(),
+                digest: DUMMY_DIGEST.to_vec().into(),
+                size: 0,
+            }],
+            ..Default::default()
+        };
+        assert_eq!(d.size(), 1);
+    }
+    {
+        let d = Directory {
+            directories: vec![DirectoryNode {
+                name: "foo".into(),
+                digest: DUMMY_DIGEST.to_vec().into(),
+                size: 4,
+            }],
+            ..Default::default()
+        };
+        assert_eq!(d.size(), 5);
+    }
+    {
+        let d = Directory {
+            files: vec![FileNode {
+                name: "foo".into(),
+                digest: DUMMY_DIGEST.to_vec().into(),
+                size: 42,
+                executable: false,
+            }],
+            ..Default::default()
+        };
+        assert_eq!(d.size(), 1);
+    }
+    {
+        let d = Directory {
+            symlinks: vec![SymlinkNode {
+                name: "foo".into(),
+                target: "bar".into(),
+            }],
+            ..Default::default()
+        };
+        assert_eq!(d.size(), 1);
+    }
+}
+
+#[test]
+#[cfg_attr(not(debug_assertions), ignore)]
+#[should_panic = "Directory::size exceeds u64::MAX"]
+fn size_unchecked_panic() {
+    let d = Directory {
+        directories: vec![DirectoryNode {
+            name: "foo".into(),
+            digest: DUMMY_DIGEST.to_vec().into(),
+            size: u64::MAX,
+        }],
+        ..Default::default()
+    };
+
+    d.size();
+}
+
+#[test]
+#[cfg_attr(debug_assertions, ignore)]
+fn size_unchecked_saturate() {
+    let d = Directory {
+        directories: vec![DirectoryNode {
+            name: "foo".into(),
+            digest: DUMMY_DIGEST.to_vec().into(),
+            size: u64::MAX,
+        }],
+        ..Default::default()
+    };
+
+    assert_eq!(d.size(), u64::MAX);
+}
+
+#[test]
+fn size_checked() {
+    // We don't test the overflow cases that rely purely on immediate
+    // child count, since that would take an absurd amount of memory.
+    {
+        let d = Directory {
+            directories: vec![DirectoryNode {
+                name: "foo".into(),
+                digest: DUMMY_DIGEST.to_vec().into(),
+                size: u64::MAX - 1,
+            }],
+            ..Default::default()
+        };
+        assert_eq!(d.size_checked(), Some(u64::MAX));
+    }
+    {
+        let d = Directory {
+            directories: vec![DirectoryNode {
+                name: "foo".into(),
+                digest: DUMMY_DIGEST.to_vec().into(),
+                size: u64::MAX,
+            }],
+            ..Default::default()
+        };
+        assert_eq!(d.size_checked(), None);
+    }
+    {
+        let d = Directory {
+            directories: vec![
+                DirectoryNode {
+                    name: "foo".into(),
+                    digest: DUMMY_DIGEST.to_vec().into(),
+                    size: u64::MAX / 2,
+                },
+                DirectoryNode {
+                    name: "foo".into(),
+                    digest: DUMMY_DIGEST.to_vec().into(),
+                    size: u64::MAX / 2,
+                },
+            ],
+            ..Default::default()
+        };
+        assert_eq!(d.size_checked(), None);
+    }
+}
+
+#[test]
+fn digest() {
+    let d = Directory::default();
+
+    assert_eq!(
+        d.digest(),
+        (&hex!("af1349b9f5f9a1a6a0404dea36dcc9499bcb25c9adc112b7cc9a93cae41f3262")).into()
+    )
+}
+
+#[test]
+fn validate_empty() {
+    let d = Directory::default();
+    assert!(crate::Directory::try_from(d).is_ok());
+}
+
+#[test]
+fn validate_invalid_names() {
+    {
+        let d = Directory {
+            directories: vec![DirectoryNode {
+                name: b"\0"[..].into(),
+                digest: DUMMY_DIGEST.to_vec().into(),
+                size: 42,
+            }],
+            ..Default::default()
+        };
+
+        let e = crate::Directory::try_from(d).expect_err("must fail");
+        assert!(matches!(e, DirectoryError::InvalidName(_)));
+    }
+
+    {
+        let d = Directory {
+            directories: vec![DirectoryNode {
+                name: ".".into(),
+                digest: DUMMY_DIGEST.to_vec().into(),
+                size: 42,
+            }],
+            ..Default::default()
+        };
+        let e = crate::Directory::try_from(d).expect_err("must fail");
+        assert!(matches!(e, DirectoryError::InvalidName(_)));
+    }
+
+    {
+        let d = Directory {
+            files: vec![FileNode {
+                name: "..".into(),
+                digest: DUMMY_DIGEST.to_vec().into(),
+                size: 42,
+                executable: false,
+            }],
+            ..Default::default()
+        };
+        let e = crate::Directory::try_from(d).expect_err("must fail");
+        assert!(matches!(e, DirectoryError::InvalidName(_)));
+    }
+
+    {
+        let d = Directory {
+            symlinks: vec![SymlinkNode {
+                name: "\x00".into(),
+                target: "foo".into(),
+            }],
+            ..Default::default()
+        };
+        let e = crate::Directory::try_from(d).expect_err("must fail");
+        assert!(matches!(e, DirectoryError::InvalidName(_)));
+    }
+
+    {
+        let d = Directory {
+            symlinks: vec![SymlinkNode {
+                name: "foo/bar".into(),
+                target: "foo".into(),
+            }],
+            ..Default::default()
+        };
+        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(_)));
+    }
+}
+
+#[test]
+fn validate_invalid_digest() {
+    let d = Directory {
+        directories: vec![DirectoryNode {
+            name: "foo".into(),
+            digest: vec![0x00, 0x42].into(), // invalid length
+            size: 42,
+        }],
+        ..Default::default()
+    };
+    match crate::Directory::try_from(d).expect_err("must fail") {
+        DirectoryError::InvalidNode(_, ValidateNodeError::InvalidDigestLen(n)) => {
+            assert_eq!(n, 2)
+        }
+        _ => panic!("unexpected error"),
+    }
+}
+
+#[test]
+fn validate_sorting() {
+    // "b" comes before "a", bad.
+    {
+        let d = Directory {
+            directories: vec![
+                DirectoryNode {
+                    name: "b".into(),
+                    digest: DUMMY_DIGEST.to_vec().into(),
+                    size: 42,
+                },
+                DirectoryNode {
+                    name: "a".into(),
+                    digest: DUMMY_DIGEST.to_vec().into(),
+                    size: 42,
+                },
+            ],
+            ..Default::default()
+        };
+        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 (same types), bad.
+    {
+        let d = Directory {
+            directories: vec![
+                DirectoryNode {
+                    name: "a".into(),
+                    digest: DUMMY_DIGEST.to_vec().into(),
+                    size: 42,
+                },
+                DirectoryNode {
+                    name: "a".into(),
+                    digest: DUMMY_DIGEST.to_vec().into(),
+                    size: 42,
+                },
+            ],
+            ..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"),
+        }
+    }
+
+    // "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"),
+        }
+    }
+
+    // "a" comes before "b", all good.
+    {
+        let d = Directory {
+            directories: vec![
+                DirectoryNode {
+                    name: "a".into(),
+                    digest: DUMMY_DIGEST.to_vec().into(),
+                    size: 42,
+                },
+                DirectoryNode {
+                    name: "b".into(),
+                    digest: DUMMY_DIGEST.to_vec().into(),
+                    size: 42,
+                },
+            ],
+            ..Default::default()
+        };
+
+        crate::Directory::try_from(d).expect("validate shouldn't error");
+    }
+
+    // [b, c] and [a] are both properly sorted.
+    {
+        let d = Directory {
+            directories: vec![
+                DirectoryNode {
+                    name: "b".into(),
+                    digest: DUMMY_DIGEST.to_vec().into(),
+                    size: 42,
+                },
+                DirectoryNode {
+                    name: "c".into(),
+                    digest: DUMMY_DIGEST.to_vec().into(),
+                    size: 42,
+                },
+            ],
+            symlinks: vec![SymlinkNode {
+                name: "a".into(),
+                target: "foo".into(),
+            }],
+            ..Default::default()
+        };
+
+        crate::Directory::try_from(d).expect("validate shouldn't error");
+    }
+}
diff --git a/tvix/castore/src/proto/tests/mod.rs b/tvix/castore/src/proto/tests/mod.rs
new file mode 100644
index 000000000000..74334029e84c
--- /dev/null
+++ b/tvix/castore/src/proto/tests/mod.rs
@@ -0,0 +1 @@
+mod directory;
diff --git a/tvix/castore/src/tests/import.rs b/tvix/castore/src/tests/import.rs
new file mode 100644
index 000000000000..32c2c363689f
--- /dev/null
+++ b/tvix/castore/src/tests/import.rs
@@ -0,0 +1,114 @@
+use crate::blobservice::{self, BlobService};
+use crate::directoryservice;
+use crate::fixtures::*;
+use crate::import::fs::ingest_path;
+use crate::Node;
+
+use tempfile::TempDir;
+
+#[cfg(target_family = "unix")]
+#[tokio::test]
+async fn symlink() {
+    let blob_service = blobservice::from_addr("memory://").await.unwrap();
+    let directory_service = directoryservice::from_addr("memory://").await.unwrap();
+
+    let tmpdir = TempDir::new().unwrap();
+
+    std::fs::create_dir_all(&tmpdir).unwrap();
+    std::os::unix::fs::symlink(
+        "/nix/store/somewhereelse",
+        tmpdir.path().join("doesntmatter"),
+    )
+    .unwrap();
+
+    let root_node = ingest_path(
+        blob_service,
+        directory_service,
+        tmpdir.path().join("doesntmatter"),
+    )
+    .await
+    .expect("must succeed");
+
+    assert_eq!(
+        Node::Symlink {
+            target: "/nix/store/somewhereelse".try_into().unwrap()
+        },
+        root_node,
+    )
+}
+
+#[tokio::test]
+async fn single_file() {
+    let blob_service = blobservice::from_addr("memory://").await.unwrap();
+    let directory_service = directoryservice::from_addr("memory://").await.unwrap();
+
+    let tmpdir = TempDir::new().unwrap();
+
+    std::fs::write(tmpdir.path().join("root"), HELLOWORLD_BLOB_CONTENTS).unwrap();
+
+    let root_node = ingest_path(
+        blob_service.clone(),
+        directory_service,
+        tmpdir.path().join("root"),
+    )
+    .await
+    .expect("must succeed");
+
+    assert_eq!(
+        Node::File {
+            digest: HELLOWORLD_BLOB_DIGEST.clone(),
+            size: HELLOWORLD_BLOB_CONTENTS.len() as u64,
+            executable: false,
+        },
+        root_node,
+    );
+
+    // ensure the blob has been uploaded
+    assert!(blob_service.has(&HELLOWORLD_BLOB_DIGEST).await.unwrap());
+}
+
+#[cfg(target_family = "unix")]
+#[tokio::test]
+async fn complicated() {
+    let blob_service = blobservice::from_addr("memory://").await.unwrap();
+    let directory_service = directoryservice::from_addr("memory://").await.unwrap();
+
+    let tmpdir = TempDir::new().unwrap();
+
+    // File ``.keep`
+    std::fs::write(tmpdir.path().join(".keep"), vec![]).unwrap();
+    // Symlink `aa`
+    std::os::unix::fs::symlink("/nix/store/somewhereelse", tmpdir.path().join("aa")).unwrap();
+    // Directory `keep`
+    std::fs::create_dir(tmpdir.path().join("keep")).unwrap();
+    // File ``keep/.keep`
+    std::fs::write(tmpdir.path().join("keep").join(".keep"), vec![]).unwrap();
+
+    let root_node = ingest_path(blob_service.clone(), &directory_service, tmpdir.path())
+        .await
+        .expect("must succeed");
+
+    // ensure root_node matched expectations
+    assert_eq!(
+        Node::Directory {
+            digest: DIRECTORY_COMPLICATED.digest().clone(),
+            size: DIRECTORY_COMPLICATED.size(),
+        },
+        root_node,
+    );
+
+    // ensure DIRECTORY_WITH_KEEP and DIRECTORY_COMPLICATED have been uploaded
+    assert!(directory_service
+        .get(&DIRECTORY_WITH_KEEP.digest())
+        .await
+        .unwrap()
+        .is_some());
+    assert!(directory_service
+        .get(&DIRECTORY_COMPLICATED.digest())
+        .await
+        .unwrap()
+        .is_some());
+
+    // ensure EMPTY_BLOB_CONTENTS has been uploaded
+    assert!(blob_service.has(&EMPTY_BLOB_DIGEST).await.unwrap());
+}
diff --git a/tvix/castore/src/tests/mod.rs b/tvix/castore/src/tests/mod.rs
new file mode 100644
index 000000000000..d016f3e0aa55
--- /dev/null
+++ b/tvix/castore/src/tests/mod.rs
@@ -0,0 +1 @@
+mod import;
diff --git a/tvix/castore/src/tonic.rs b/tvix/castore/src/tonic.rs
new file mode 100644
index 000000000000..e63e1ad7aab8
--- /dev/null
+++ b/tvix/castore/src/tonic.rs
@@ -0,0 +1,126 @@
+use hyper_util::rt::TokioIo;
+use tokio::net::UnixStream;
+use tonic::transport::{Channel, Endpoint};
+
+fn url_wants_wait_connect(url: &url::Url) -> bool {
+    url.query_pairs()
+        .filter(|(k, v)| k == "wait-connect" && v == "1")
+        .count()
+        > 0
+}
+
+/// Turn a [url::Url] to a [Channel] if it can be parsed successfully.
+/// It supports the following schemes (and URLs):
+///  - `grpc+http://[::1]:8000`, connecting over unencrypted HTTP/2 (h2c)
+///  - `grpc+https://[::1]:8000`, connecting over encrypted HTTP/2
+///  - `grpc+unix:/path/to/socket`, connecting to a unix domain socket
+///
+/// All URLs support adding `wait-connect=1` as a URL parameter, in which case
+/// the connection is established lazily.
+pub async fn channel_from_url(url: &url::Url) -> Result<Channel, self::Error> {
+    match url.scheme() {
+        "grpc+unix" => {
+            if url.host_str().is_some() {
+                return Err(Error::HostSetForUnixSocket());
+            }
+
+            let connector = tower::service_fn({
+                let url = url.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
+            let endpoint = Endpoint::from_static("http://[::]:50051");
+            if url_wants_wait_connect(url) {
+                Ok(endpoint.connect_with_connector(connector).await?)
+            } else {
+                Ok(endpoint.connect_with_connector_lazy(connector))
+            }
+        }
+        _ => {
+            // ensure path is empty, not supported with gRPC.
+            if !url.path().is_empty() {
+                return Err(Error::PathMayNotBeSet());
+            }
+
+            // Stringify the URL and remove the grpc+ prefix.
+            // We can't use `url.set_scheme(rest)`, as it disallows
+            // setting something http(s) that previously wasn't.
+            let unprefixed_url_str = match url.to_string().strip_prefix("grpc+") {
+                None => return Err(Error::MissingGRPCPrefix()),
+                Some(url_str) => url_str.to_owned(),
+            };
+
+            // Use the regular tonic transport::Endpoint logic, but unprefixed_url_str,
+            // as tonic doesn't know about grpc+http[s].
+            let endpoint = Endpoint::try_from(unprefixed_url_str)?;
+            if url_wants_wait_connect(url) {
+                Ok(endpoint.connect().await?)
+            } else {
+                Ok(endpoint.connect_lazy())
+            }
+        }
+    }
+}
+
+/// Errors occuring when trying to connect to a backend
+#[derive(Debug, thiserror::Error)]
+pub enum Error {
+    #[error("grpc+ prefix is missing from URL")]
+    MissingGRPCPrefix(),
+
+    #[error("host may not be set for unix domain sockets")]
+    HostSetForUnixSocket(),
+
+    #[error("path may not be set")]
+    PathMayNotBeSet(),
+
+    #[error("transport error: {0}")]
+    TransportError(tonic::transport::Error),
+}
+
+impl From<tonic::transport::Error> for Error {
+    fn from(value: tonic::transport::Error) -> Self {
+        Self::TransportError(value)
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::channel_from_url;
+    use rstest::rstest;
+    use url::Url;
+
+    #[rstest]
+    /// Correct scheme to connect to a unix socket.
+    #[case::valid_unix_socket("grpc+unix:///path/to/somewhere", true)]
+    /// Connecting with wait-connect set to 0 succeeds, as that's the default.
+    #[case::valid_unix_socket_wait_connect_0("grpc+unix:///path/to/somewhere?wait-connect=0", true)]
+    /// Connecting with wait-connect set to 1 fails, as the path doesn't exist.
+    #[case::valid_unix_socket_wait_connect_1(
+        "grpc+unix:///path/to/somewhere?wait-connect=1",
+        false
+    )]
+    /// Correct scheme for unix socket, but setting a host too, which is invalid.
+    #[case::invalid_unix_socket_and_host("grpc+unix://host.example/path/to/somewhere", false)]
+    /// Correct scheme to connect to localhost, with port 12345
+    #[case::valid_ipv6_localhost_port_12345("grpc+http://[::1]:12345", true)]
+    /// Correct scheme to connect to localhost over http, without specifying a port.
+    #[case::valid_http_host_without_port("grpc+http://localhost", true)]
+    /// Correct scheme to connect to localhost over http, without specifying a port.
+    #[case::valid_https_host_without_port("grpc+https://localhost", true)]
+    /// Correct scheme to connect to localhost over http, but with additional path, which is invalid.
+    #[case::invalid_host_and_path("grpc+http://localhost/some-path", false)]
+    /// Connecting with wait-connect set to 0 succeeds, as that's the default.
+    #[case::valid_host_wait_connect_0("grpc+http://localhost?wait-connect=0", true)]
+    /// Connecting with wait-connect set to 1 fails, as the host doesn't exist.
+    #[case::valid_host_wait_connect_1_fails("grpc+http://nonexist.invalid?wait-connect=1", false)]
+    #[tokio::test]
+    async fn test_from_addr_tokio(#[case] uri_str: &str, #[case] is_ok: bool) {
+        let url = Url::parse(uri_str).expect("must parse");
+        assert_eq!(channel_from_url(&url).await.is_ok(), is_ok)
+    }
+}