refactor(tvix/store/fs): mv sparse -> populated directories
Do this upgrade whenever someone is actually interested in the children of a directory, but that directory doesn't contain a more detailed listing. This is much more predictable, and removes a bunch of confusing code from the inode tracker itself. Change-Id: Ib3a13694d6d5d22887d2d04ae429592137f39cb4 Reviewed-on: https://cl.tvl.fyi/c/depot/+/9982 Autosubmit: flokli <flokli@flokli.de> Tested-by: BuildkiteCI Reviewed-by: Connor Brewster <cbrewster@hey.com>
This commit is contained in:
parent
a778b855d2
commit
9cd2e92065
3 changed files with 54 additions and 317 deletions
|
|
@ -1,7 +1,6 @@
|
||||||
use std::{collections::HashMap, sync::Arc};
|
use std::{collections::HashMap, sync::Arc};
|
||||||
|
|
||||||
use super::inodes::{DirectoryInodeData, InodeData};
|
use super::inodes::{DirectoryInodeData, InodeData};
|
||||||
use tvix_castore::proto as castorepb;
|
|
||||||
use tvix_castore::B3Digest;
|
use tvix_castore::B3Digest;
|
||||||
|
|
||||||
/// InodeTracker keeps track of inodes, stores data being these inodes and deals
|
/// InodeTracker keeps track of inodes, stores data being these inodes and deals
|
||||||
|
|
@ -43,6 +42,14 @@ impl InodeTracker {
|
||||||
self.data.get(&ino).cloned()
|
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.
|
// Stores data and returns the inode for it.
|
||||||
// In case an inode has already been allocated for the same data, that inode
|
// In case an inode has already been allocated for the same data, that inode
|
||||||
// is returned, otherwise a new one is allocated.
|
// is returned, otherwise a new one is allocated.
|
||||||
|
|
@ -81,85 +88,13 @@ impl InodeTracker {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
// Inserting [DirectoryInodeData::Populated] usually replaces an
|
// Inserting [DirectoryInodeData::Populated] doesn't normally happen,
|
||||||
// existing [DirectoryInodeData::Sparse] one.
|
// only via [replace].
|
||||||
InodeData::Directory(DirectoryInodeData::Populated(ref digest, ref children)) => {
|
InodeData::Directory(DirectoryInodeData::Populated(..)) => {
|
||||||
let dir_ino = self.directory_digest_to_inode.get(digest);
|
unreachable!("should never be called with DirectoryInodeData::Populated")
|
||||||
if let Some(dir_ino) = dir_ino {
|
|
||||||
let dir_ino = *dir_ino;
|
|
||||||
|
|
||||||
// We know the data must exist, as we found it in [directory_digest_to_inode].
|
|
||||||
let needs_update = match **self.data.get(&dir_ino).unwrap() {
|
|
||||||
InodeData::Regular(..) | InodeData::Symlink(_) => {
|
|
||||||
panic!("unexpected type at inode {}", dir_ino);
|
|
||||||
}
|
|
||||||
// already populated, nothing to do
|
|
||||||
InodeData::Directory(DirectoryInodeData::Populated(..)) => false,
|
|
||||||
// in case the actual data is sparse, replace it with the populated one.
|
|
||||||
// this allocates inodes for new children in the process.
|
|
||||||
InodeData::Directory(DirectoryInodeData::Sparse(
|
|
||||||
ref old_digest,
|
|
||||||
ref _old_size,
|
|
||||||
)) => {
|
|
||||||
// sanity checking to ensure we update the right node
|
|
||||||
debug_assert_eq!(old_digest, digest);
|
|
||||||
|
|
||||||
true
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
if needs_update {
|
|
||||||
// populate inode fields in children
|
|
||||||
let children = self.allocate_inodes_for_children(children.to_vec());
|
|
||||||
|
|
||||||
// update sparse data with populated data
|
|
||||||
self.data.insert(
|
|
||||||
dir_ino,
|
|
||||||
Arc::new(InodeData::Directory(DirectoryInodeData::Populated(
|
|
||||||
digest.clone(),
|
|
||||||
children,
|
|
||||||
))),
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
dir_ino
|
|
||||||
} else {
|
|
||||||
// populate inode fields in children
|
|
||||||
let children = self.allocate_inodes_for_children(children.to_vec());
|
|
||||||
// insert and return InodeData
|
|
||||||
self.insert_and_increment(InodeData::Directory(DirectoryInodeData::Populated(
|
|
||||||
digest.clone(),
|
|
||||||
children,
|
|
||||||
)))
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
// Consume a list of children with zeroed inodes, and allocate (or fetch existing) inodes.
|
|
||||||
fn allocate_inodes_for_children(
|
|
||||||
&mut self,
|
|
||||||
children: Vec<(u64, castorepb::node::Node)>,
|
|
||||||
) -> Vec<(u64, castorepb::node::Node)> {
|
|
||||||
// allocate new inodes for all children
|
|
||||||
let mut children_new: Vec<(u64, castorepb::node::Node)> = Vec::new();
|
|
||||||
|
|
||||||
for (child_ino, ref child_node) in children {
|
|
||||||
debug_assert_eq!(0, child_ino, "expected child inode to be 0");
|
|
||||||
let child_ino = match child_node {
|
|
||||||
castorepb::node::Node::Directory(directory_node) => {
|
|
||||||
// Try putting the sparse data in. If we already have a
|
|
||||||
// populated version, it'll not update it.
|
|
||||||
self.put(directory_node.into())
|
|
||||||
}
|
|
||||||
castorepb::node::Node::File(file_node) => self.put(file_node.into()),
|
|
||||||
castorepb::node::Node::Symlink(symlink_node) => self.put(symlink_node.into()),
|
|
||||||
};
|
|
||||||
|
|
||||||
children_new.push((child_ino, child_node.clone()))
|
|
||||||
}
|
|
||||||
children_new
|
|
||||||
}
|
|
||||||
|
|
||||||
// Inserts the data and returns the inode it was stored at, while
|
// Inserts the data and returns the inode it was stored at, while
|
||||||
// incrementing next_inode.
|
// incrementing next_inode.
|
||||||
|
|
@ -197,9 +132,7 @@ impl InodeTracker {
|
||||||
|
|
||||||
#[cfg(test)]
|
#[cfg(test)]
|
||||||
mod tests {
|
mod tests {
|
||||||
use crate::fs::inodes::DirectoryInodeData;
|
|
||||||
use crate::tests::fixtures;
|
use crate::tests::fixtures;
|
||||||
use tvix_castore::proto as castorepb;
|
|
||||||
|
|
||||||
use super::InodeData;
|
use super::InodeData;
|
||||||
use super::InodeTracker;
|
use super::InodeTracker;
|
||||||
|
|
@ -271,187 +204,4 @@ mod tests {
|
||||||
// inserting another file should return a different ino
|
// inserting another file should return a different ino
|
||||||
assert_ne!(ino, inode_tracker.put(InodeData::Symlink("target2".into())));
|
assert_ne!(ino, inode_tracker.put(InodeData::Symlink("target2".into())));
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO: put sparse directory
|
|
||||||
|
|
||||||
/// Put a directory into the inode tracker, which refers to a file not seen yet.
|
|
||||||
#[test]
|
|
||||||
fn put_directory_leaf() {
|
|
||||||
let mut inode_tracker = InodeTracker::default();
|
|
||||||
|
|
||||||
// this is a directory with a single item, a ".keep" file pointing to a 0 bytes blob.
|
|
||||||
let dir: InodeData = fixtures::DIRECTORY_WITH_KEEP.clone().into();
|
|
||||||
|
|
||||||
// put it in
|
|
||||||
let dir_ino = inode_tracker.put(dir);
|
|
||||||
|
|
||||||
// a get should return the right data
|
|
||||||
let data = inode_tracker.get(dir_ino).expect("must be some");
|
|
||||||
match *data {
|
|
||||||
InodeData::Directory(super::DirectoryInodeData::Sparse(..)) => {
|
|
||||||
panic!("wrong type");
|
|
||||||
}
|
|
||||||
InodeData::Directory(super::DirectoryInodeData::Populated(
|
|
||||||
ref directory_digest,
|
|
||||||
ref children,
|
|
||||||
)) => {
|
|
||||||
// ensure the directory digest matches
|
|
||||||
assert_eq!(&fixtures::DIRECTORY_WITH_KEEP.digest(), directory_digest);
|
|
||||||
|
|
||||||
// ensure the child is populated, with a different inode than
|
|
||||||
// the parent, and the data matches expectations.
|
|
||||||
assert_eq!(1, children.len());
|
|
||||||
let (child_ino, child_node) = children.first().unwrap();
|
|
||||||
assert_ne!(dir_ino, *child_ino);
|
|
||||||
assert_eq!(
|
|
||||||
&castorepb::node::Node::File(
|
|
||||||
fixtures::DIRECTORY_WITH_KEEP.files.first().unwrap().clone()
|
|
||||||
),
|
|
||||||
child_node
|
|
||||||
);
|
|
||||||
|
|
||||||
// ensure looking up that inode directly returns the data
|
|
||||||
let child_data = inode_tracker.get(*child_ino).expect("must exist");
|
|
||||||
match *child_data {
|
|
||||||
InodeData::Regular(ref digest, size, executable) => {
|
|
||||||
assert_eq!(&fixtures::EMPTY_BLOB_DIGEST.clone(), digest);
|
|
||||||
assert_eq!(0, size);
|
|
||||||
assert!(!executable);
|
|
||||||
}
|
|
||||||
InodeData::Symlink(_) | InodeData::Directory(..) => panic!("wrong type"),
|
|
||||||
}
|
|
||||||
}
|
|
||||||
InodeData::Symlink(_) | InodeData::Regular(..) => panic!("wrong type"),
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Put a directory into the inode tracker, referring to files, directories
|
|
||||||
/// and symlinks not seen yet.
|
|
||||||
#[test]
|
|
||||||
fn put_directory_complicated() {
|
|
||||||
let mut inode_tracker = InodeTracker::default();
|
|
||||||
|
|
||||||
// this is a directory with a single item, a ".keep" file pointing to a 0 bytes blob.
|
|
||||||
let dir_complicated: InodeData = fixtures::DIRECTORY_COMPLICATED.clone().into();
|
|
||||||
|
|
||||||
// put it in
|
|
||||||
let dir_complicated_ino = inode_tracker.put(dir_complicated);
|
|
||||||
|
|
||||||
// a get should return the right data
|
|
||||||
let dir_data = inode_tracker
|
|
||||||
.get(dir_complicated_ino)
|
|
||||||
.expect("must be some");
|
|
||||||
|
|
||||||
let child_dir_ino = match *dir_data {
|
|
||||||
InodeData::Directory(DirectoryInodeData::Sparse(..)) => {
|
|
||||||
panic!("wrong type");
|
|
||||||
}
|
|
||||||
InodeData::Directory(DirectoryInodeData::Populated(
|
|
||||||
ref directory_digest,
|
|
||||||
ref children,
|
|
||||||
)) => {
|
|
||||||
// assert the directory digest matches
|
|
||||||
assert_eq!(&fixtures::DIRECTORY_COMPLICATED.digest(), directory_digest);
|
|
||||||
|
|
||||||
// ensure there's three children, all with different inodes
|
|
||||||
assert_eq!(3, children.len());
|
|
||||||
let mut seen_inodes = Vec::from([dir_complicated_ino]);
|
|
||||||
|
|
||||||
// check the first child (.keep)
|
|
||||||
{
|
|
||||||
let (child_ino, child_node) = &children[0];
|
|
||||||
assert!(!seen_inodes.contains(child_ino));
|
|
||||||
assert_eq!(
|
|
||||||
&castorepb::node::Node::File(
|
|
||||||
fixtures::DIRECTORY_COMPLICATED.files[0].clone()
|
|
||||||
),
|
|
||||||
child_node
|
|
||||||
);
|
|
||||||
seen_inodes.push(*child_ino);
|
|
||||||
}
|
|
||||||
|
|
||||||
// check the second child (aa)
|
|
||||||
{
|
|
||||||
let (child_ino, child_node) = &children[1];
|
|
||||||
assert!(!seen_inodes.contains(child_ino));
|
|
||||||
assert_eq!(
|
|
||||||
&castorepb::node::Node::Symlink(
|
|
||||||
fixtures::DIRECTORY_COMPLICATED.symlinks[0].clone()
|
|
||||||
),
|
|
||||||
child_node
|
|
||||||
);
|
|
||||||
seen_inodes.push(*child_ino);
|
|
||||||
}
|
|
||||||
|
|
||||||
// check the third child (keep)
|
|
||||||
{
|
|
||||||
let (child_ino, child_node) = &children[2];
|
|
||||||
assert!(!seen_inodes.contains(child_ino));
|
|
||||||
assert_eq!(
|
|
||||||
&castorepb::node::Node::Directory(
|
|
||||||
fixtures::DIRECTORY_COMPLICATED.directories[0].clone()
|
|
||||||
),
|
|
||||||
child_node
|
|
||||||
);
|
|
||||||
seen_inodes.push(*child_ino);
|
|
||||||
|
|
||||||
// return the child_ino
|
|
||||||
*child_ino
|
|
||||||
}
|
|
||||||
}
|
|
||||||
InodeData::Regular(..) | InodeData::Symlink(_) => panic!("wrong type"),
|
|
||||||
};
|
|
||||||
|
|
||||||
// get of the inode for child_ino
|
|
||||||
let child_dir_data = inode_tracker.get(child_dir_ino).expect("must be some");
|
|
||||||
// it should be a sparse InodeData::Directory with the right digest.
|
|
||||||
match *child_dir_data {
|
|
||||||
InodeData::Directory(DirectoryInodeData::Sparse(
|
|
||||||
ref child_dir_digest,
|
|
||||||
child_dir_size,
|
|
||||||
)) => {
|
|
||||||
assert_eq!(&fixtures::DIRECTORY_WITH_KEEP.digest(), child_dir_digest);
|
|
||||||
assert_eq!(fixtures::DIRECTORY_WITH_KEEP.size(), child_dir_size);
|
|
||||||
}
|
|
||||||
InodeData::Directory(DirectoryInodeData::Populated(..))
|
|
||||||
| InodeData::Regular(..)
|
|
||||||
| InodeData::Symlink(_) => {
|
|
||||||
panic!("wrong type")
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// put DIRECTORY_WITH_KEEP, which should return the same ino as [child_dir_ino],
|
|
||||||
// but update the sparse object to a populated one at the same time.
|
|
||||||
let child_dir_ino2 = inode_tracker.put(fixtures::DIRECTORY_WITH_KEEP.clone().into());
|
|
||||||
assert_eq!(child_dir_ino, child_dir_ino2);
|
|
||||||
|
|
||||||
// get the data
|
|
||||||
match *inode_tracker.get(child_dir_ino).expect("must be some") {
|
|
||||||
// it should be a populated InodeData::Directory with the right digest!
|
|
||||||
InodeData::Directory(DirectoryInodeData::Populated(
|
|
||||||
ref directory_digest,
|
|
||||||
ref children,
|
|
||||||
)) => {
|
|
||||||
// ensure the directory digest matches
|
|
||||||
assert_eq!(&fixtures::DIRECTORY_WITH_KEEP.digest(), directory_digest);
|
|
||||||
|
|
||||||
// ensure the child is populated, with a different inode than
|
|
||||||
// the parent, and the data matches expectations.
|
|
||||||
assert_eq!(1, children.len());
|
|
||||||
let (child_node_inode, child_node) = children.first().unwrap();
|
|
||||||
assert_ne!(dir_complicated_ino, *child_node_inode);
|
|
||||||
assert_eq!(
|
|
||||||
&castorepb::node::Node::File(
|
|
||||||
fixtures::DIRECTORY_WITH_KEEP.files.first().unwrap().clone()
|
|
||||||
),
|
|
||||||
child_node
|
|
||||||
);
|
|
||||||
}
|
|
||||||
InodeData::Directory(DirectoryInodeData::Sparse(..))
|
|
||||||
| InodeData::Regular(..)
|
|
||||||
| InodeData::Symlink(_) => panic!("wrong type"),
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO: add test inserting a populated one first, then ensure an update doesn't degrade it back to sparse.
|
|
||||||
|
|
|
||||||
|
|
@ -55,16 +55,3 @@ impl From<&castorepb::DirectoryNode> for InodeData {
|
||||||
))
|
))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/// converts a proto::Directory to a InodeData::Directory(DirectoryInodeData::Populated(..)).
|
|
||||||
/// The inodes for each child are 0, because it's up to the InodeTracker to allocate them.
|
|
||||||
impl From<castorepb::Directory> for InodeData {
|
|
||||||
fn from(value: castorepb::Directory) -> Self {
|
|
||||||
let digest = value.digest();
|
|
||||||
|
|
||||||
let children: Vec<(u64, castorepb::node::Node)> =
|
|
||||||
value.nodes().map(|node| (0, node)).collect();
|
|
||||||
|
|
||||||
InodeData::Directory(DirectoryInodeData::Populated(digest, children))
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
|
||||||
|
|
@ -30,11 +30,12 @@ use tokio::{
|
||||||
sync::mpsc,
|
sync::mpsc,
|
||||||
};
|
};
|
||||||
use tracing::{debug, info_span, instrument, warn};
|
use tracing::{debug, info_span, instrument, warn};
|
||||||
|
use tvix_castore::proto as castorepb;
|
||||||
use tvix_castore::{
|
use tvix_castore::{
|
||||||
blobservice::{BlobReader, BlobService},
|
blobservice::{BlobReader, BlobService},
|
||||||
directoryservice::DirectoryService,
|
directoryservice::DirectoryService,
|
||||||
proto::{node::Node, NamedNode},
|
proto::{node::Node, NamedNode},
|
||||||
B3Digest, Error,
|
B3Digest,
|
||||||
};
|
};
|
||||||
|
|
||||||
use self::{
|
use self::{
|
||||||
|
|
@ -146,51 +147,50 @@ impl TvixStoreFs {
|
||||||
ref parent_digest,
|
ref parent_digest,
|
||||||
ref children,
|
ref children,
|
||||||
)) => Ok((parent_digest.clone(), children.clone())),
|
)) => Ok((parent_digest.clone(), children.clone())),
|
||||||
// if it's sparse, fetch data using fetch_directory_inode_data and insert.
|
// 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, _)) => {
|
InodeData::Directory(DirectoryInodeData::Sparse(ref parent_digest, _)) => {
|
||||||
|
let directory = self
|
||||||
|
.tokio_handle
|
||||||
|
.block_on(self.tokio_handle.spawn({
|
||||||
let directory_service = self.directory_service.clone();
|
let directory_service = self.directory_service.clone();
|
||||||
let parent_digest = parent_digest.to_owned();
|
let parent_digest = parent_digest.to_owned();
|
||||||
match self
|
async move { directory_service.get(&parent_digest).await }
|
||||||
.tokio_handle
|
|
||||||
.block_on(self.tokio_handle.spawn(async move {
|
|
||||||
match directory_service.get(&parent_digest).await? {
|
|
||||||
// If the Directory can't be found, this is a hole, bail out.
|
|
||||||
None => {
|
|
||||||
warn!(directory.digest=%parent_digest, "directory not found");
|
|
||||||
Err(Error::StorageError(format!(
|
|
||||||
"directory {} not found",
|
|
||||||
parent_digest
|
|
||||||
)))
|
|
||||||
}
|
|
||||||
Some(directory) => Ok(directory.into()),
|
|
||||||
}
|
|
||||||
}))
|
}))
|
||||||
.unwrap()
|
.unwrap()?
|
||||||
{
|
.ok_or_else(|| {
|
||||||
Err(_e) => Err(io::Error::from_raw_os_error(libc::EIO)),
|
warn!(directory.digest=%parent_digest, "directory not found");
|
||||||
Ok(
|
// If the Directory can't be found, this is a hole, bail out.
|
||||||
ref data @ InodeData::Directory(DirectoryInodeData::Populated(
|
io::Error::from_raw_os_error(libc::EIO)
|
||||||
ref parent_digest,
|
})?;
|
||||||
_,
|
|
||||||
)),
|
// Turn the retrieved directory into a InodeData::Directory(DirectoryInodeData::Populated(..)),
|
||||||
) => {
|
// allocating inodes for the children on the way.
|
||||||
// update data in [self.inode_tracker] with populated variant.
|
let children = {
|
||||||
// we need to round-trip via self.inode_tracker so
|
let mut inode_tracker = self.inode_tracker.write();
|
||||||
// inodes for children are populated.
|
|
||||||
self.inode_tracker.write().put(data.clone());
|
let children: Vec<(u64, castorepb::node::Node)> = directory
|
||||||
let children = match *self.inode_tracker.read().get(ino).unwrap() {
|
.nodes()
|
||||||
InodeData::Directory(DirectoryInodeData::Populated(
|
.map(|child_node| {
|
||||||
_,
|
let child_ino = inode_tracker.put((&child_node).into());
|
||||||
ref children,
|
(child_ino, child_node)
|
||||||
)) => children.to_owned(),
|
})
|
||||||
_ => unreachable!(),
|
.collect();
|
||||||
|
|
||||||
|
// replace.
|
||||||
|
inode_tracker.replace(
|
||||||
|
ino,
|
||||||
|
Arc::new(InodeData::Directory(DirectoryInodeData::Populated(
|
||||||
|
parent_digest.clone(),
|
||||||
|
children.clone(),
|
||||||
|
))),
|
||||||
|
);
|
||||||
|
|
||||||
|
children
|
||||||
};
|
};
|
||||||
|
|
||||||
Ok((parent_digest.clone(), children))
|
Ok((parent_digest.clone(), children))
|
||||||
}
|
}
|
||||||
// we know fetch_directory_inode_data only returns InodeData::Directory(DirectoryInodeData::Populated(..))
|
|
||||||
Ok(_) => panic!("unexpected type"),
|
|
||||||
}
|
|
||||||
}
|
|
||||||
// if the parent inode was not a directory, this doesn't make sense
|
// if the parent inode was not a directory, this doesn't make sense
|
||||||
InodeData::Regular(..) | InodeData::Symlink(_) => {
|
InodeData::Regular(..) | InodeData::Symlink(_) => {
|
||||||
Err(io::Error::from_raw_os_error(libc::ENOTDIR))
|
Err(io::Error::from_raw_os_error(libc::ENOTDIR))
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue