0xPolygonMiden / miden-vm

STARK-based virtual machine
MIT License
611 stars 149 forks source link

Implement extensible subsystem for on-demand storage/provisioning of MAST objects #1226

Open bitwalker opened 5 months ago

bitwalker commented 5 months ago

This is the second, and most significant of the three issues I planned to open related to MAST. It is intended to build on the foundation laid by #1217, and - in my opinion anyway - is the lynchpin to a number of important features that are related to packaging/distribution/deployment.

Background

Miden is currently oriented around shipping Miden Assembly files (either in .masm form or bundled into a .masl library). These files must be placed on disk in a precise structure, and then the VM invoked with the right set of incantations to make sure everything necessary to assemble and run the program is available. The assembler then processes these into a MAST and executes the program, all in one shot.

This isn't ideal for a number of reasons:

  1. MASM provides no cryptographic or versioning/stability guarantees. Your program will be compiled with the semantics of whatever version of the VM happens to assemble and run your code. Similarly, the standard library and kernel you get is also linked to the VM, rather than the version you were targeting/compiling against.
  2. There is a chicken-and-egg problem, which is that in order to get the hashes of the functions in your program, so that you can use them elsewhere, or hand them out to others, you have to compile your MASM to MAST - but if you are deploying MASM and not MAST, those hashes are effectively useless, because they do not actually guarantee anything, other than what the hashes were at that one point.
  3. To clarify a bit: since the MAST you derive your hashes from is not the literal MAST that will be executed downstream, any differences between the toolchain you used to compile/test your program, and those that will be used to execute it, could result in different hashes being computed. I don't know how frequently such changes occur, but in my view, it would be unacceptable if existing programs could break due to a new release of the assembler or VM. At a minimum, any kind of breaking change of that sort would require some kind of gating mechanism to allow legacy programs to continue to function without modification.
  4. We made significant tradeoffs to use MAST as the basis for Miden, but not using it as our object code format is wasting one of its big value propositions.

Binary Size

I believe one of the primary reasons MASM ended up being the de-facto format for distribution was due to it being viewed as significantly smaller in terms of binary size - but I think the actual case for this is a bit flimsy in the big picture. While it is almost certainly true that serializing our current MAST structure would indeed be bloated in comparison to the original MASM, there are some counter-balancing properties here:

  1. It is very straightforward to serialize MAST to a structure that contains no duplicated nodes, the primary source of it's size. Additionally, due to how low-level it is, it can use a tightly packed binary encoding, on top of which existing off-the-shelf compression tools can remove any remaining wasted space.
  2. Storage of commonly-used code in MAST form only needs to occupy the space of a single copy of each unique MAST root. Initially, one would not observe any meaningful space savings from this, but this changes quickly under any decent volume. Consider what happens when you have tens of thousands of programs with a handful of very widely used libraries (e.g. Miden Standard Library, but it's quite likely that someone in the community develops something foundational that everyone uses, we'll call it libfoo). If everyone is shipping MASM artifacts (containing all of the MASM code that their program uses), you end up with thousands of redundant copies of libfoo being stored. With MAST however, regardless of how many times a given version of libfoo is used, only a single copy need be stored.
  3. Even with those optimizations, MAST might still be larger than its equivalent MASM code (hard to say without measuring), but whether it is or not, the difference is likely missing the forest for the trees.

Proposal

The background above isn't what this proposal is about though, it just is the necessary backdrop for it. For now, let's assume we're moving to a world where MAST is our object code format (i.e. both in-memory and on disk, MAST is the equivalent of e.g. ELF executables, static archives/dynamic libraries).

I think I can summarize the key things that this enables:

In short, my primary proposal here is to introduce some changes to the VM in how it provisions MAST for execution, both initially at program startup, as well as during execution of the program. Specifically, the introduction of two high-level abstractions, with the necessary set of interfaces for use in the various parts of the VM that need to interact with them:

Object Store

This component would be responsible for storage and provisioning of the binary objects corresponding to a given MAST root. Conceptually the storage is content-addressable, where the MAST root is the content hash. This component has the following responsibilities:

The primary users of this component are three different parties:

Package Registry

This component is not a full package management solution, nor a registry like crates.io, but a core piece of VM infrastructure to make it package-aware, and to allow straightforward integration between package management tooling, the object store, and the VM. It has the following responsibilities:

Putting It Together

These two components work in tandem to provide a unified and convenient experience for publishing/distributing/and executing Miden programs. It can be extended to support complex tools and centralized registries like Cargo/crates.io, or be kept completely offline and using the local filesystem, or somewhere in between. Fundamentally though, the VM will assume that all code is either part of the program MAST which it started executing (the status quo), or that it can be loaded just-in-time from the object store. Any failed attempt to load MAST from the object store should result in a trap (and appropriate error code).

TODOs

There are a few related items we'll want to also tackle along these lines:

Final Thoughts

I tried to avoid letting this go too long by getting into example code and things like that, but I'm happy to show how I think some of these interfaces will actually look like/work. Interested to hear your feedback!

bobbinth commented 5 months ago

This looks great! Thank you again for putting so much though into this. I agree with most things above and I think it describes a pretty good end state (there are some minor things we can discuss in the future, but these won't have an impact on the overall direction/design).

This is a pretty big change, however, and one thing we should try to do is break it down into small incremental updates. Below, are some of the first steps that I think we could take and then we can expand from there:

Step 1: update MAST representation to be table-based

This is basically what is described in #1217. This will involve:

  1. Defining the struct/interface which will replace the current Program struct.
  2. Updating the assembler to be able to build this new struct from a .masm file.
  3. Updating the VM to be able to execute programs defined by this new struct.

Regarding point 1: we already discussed some high-level approaches to this struct and I'm curious if you have more concrete thoughts on this. A couple of points from my end:

impl Component {
    pub fn get_node(&self, hash: Digest) -> Option<&MastNode> {
        todo!()
    }
}

With this, it would be pretty easy to migrate the VM from using Program to using Component.

Step 2: implement exec.HASH

Once the above is done, adding support for exec.HASH should be pretty straight-forward.

Step 3: define Registry trait

This would be the basically the interface for the object store you mentioned above (or how the VM would see it). The main purpose if this trait would be to provide a component for a given MAST root. For example, it could look like so:

pub trait Registry {
    fn get_component(&self, root: Digest) -> Option<&Component>;
}

The VM can then be instantiated with a Registry (more on this below), and as it executes a component, if component.get_node() returns None it would try to do registry.get_component() and would start executing the return component or error out if the component is not in the registry.

There are probably also other ways to do it - e.g., erase the difference between Component and Registry somehow (so that both return MastNode for a given hash. I'm curious how you thought about it.

Regarding instantiating the VM with a given registry, we have several options:

  1. We could provide it as another parameter into the execute() function.
  2. Expose it via the Host interface (i.e., we add Host::get_component() function and how the host implements the registry is an implementation detail as far as the VM is concerned).
  3. Make the VM a stateful object (i.e., a struct which can be instantiated with a given set of options, registry etc.).
  4. Maybe something else.

I think both 2 and 3 are interesting options, with 2 being much easier to implement.

Step 4: implement a default Registry

This could be something simple (e.g., file-system-based or even in-memory registry) to let us test out the basic functionality. The goal here would be to compile standard library into a format from which this registry can be instantiated, provide this registry to the VM, and make sure the VM correctly executes the programs which rely on components in the registry.

To make this work, we would also need to make the assembler "registry-aware". Basically, if the assembler sees that a root for a given call is in the registry, it wouldn't need to add the underlying nodes to the Component it is building.

Step 5 and beyond

After the above is working, we can work on efficient ways to serialize MAST, and more sophisticated registry implementations etc.

Also, these are just my initial thoughts - so, I might have missed something or there could be better ways to approach this.

bitwalker commented 5 months ago

Thanks @bobbinth! You're overall sketch of the process more or less aligns with what I was expecting in terms of roadmap, so I'm glad to see I haven't forgotten something significant.

This is a pretty big change, however, and one thing we should try to do is break it down into small incremental updates. Below, are some of the first steps that I think we could take and then we can expand from there:

Definitely, and though it wasn't especially clear from my writeup, I wasn't expecting this to be an all-or-nothing, or even all-at-once kind of change, but rather a roadmap for how to get from where we are now to a VM that has the functionality I described. The overall steps you've laid out are pretty much the lines along which I was expecting to divide up the tasks at a high-level, but some of those are more involved and may warrant further breakdown.

Step 1: update MAST representation to be table-based

Regarding point 1: we already discussed some high-level approaches to this struct and I'm curious if you have more concrete thoughts on this.

Sure, I can be more specific with my thoughts here:

First, the CodeBlockTable and Program structures would be rewritten/merged, so that it can both be used by Program (and Component if we make those things separate, at least temporarily), as well as provide an abstraction behind which we can make transparent whether a CodeBlock is part of the local MAST, or fetched from a future object store (this wouldn't be implemented as part of these changes, but these changes will make that change less invasive):

NOTE: I'm using more general terminology in the code below, i.e. MastNode, MastForest rather than CodeBlock and Program to better reflect the generality of the representation. However, Component is a perfectly suitable name, though we may want to reserve that for the time being, just in case we end up implementing something like the Component Model runtime, wherein a "component" might have more precise boundaries than the MAST itself.

/// Assume this is the more robust handle implementation I outlined in #1217
/// But the key points are:
///    * Has the same layout as a u32 when wrapped in an Option (due to niche optimization)
///    * Can be used to index slices just like a u32, but with some additional safety
///    * Could be modified to contain a generational index if we expect the MAST to be
///      mutated while handles are floating around
///    * Implements TryFrom<u32/usize>/Into<usize> for convenience
#[repr(transparent)]
#[derive(Copy, Clone, PartialEq, Eq, PartialOrd, Ord, Hash)]
pub struct MastNodeId(core::num::NonZeroU32);
impl MastNodeId {
    pub fn new(id: usize) -> Result<Self, InvalidNodeIdError> {
        if id == u32::MAX { 
            Err(InvalidNodeIdError) 
        } else { 
            unsafe { Self(NonZeroU32::new_unchecked(id + 1) } 
        }
    }
    pub fn as_usize(self) -> usize {
        (self.0.get() - 1) as usize
    }
}

/// A [MastForest] represents either a disjoint set of MASTs, or an arborescence,
/// the distinction depending both on whether the forest represents a library or a
/// program, as well as the dependency structure of the code itself. For example,
/// an exectuable program would almost always be an arborescence, since it has
/// a single entrypoint/root, and code not reachable from that entrypoint must be dead.
/// Whereas, a library might consist of one or more entrypoints/roots. 
///
/// In any case, this single data structure can represent both, and the semantics
/// would allow merging trees/forests into larger ones; as well as splitting a 
/// forest into smaller forests, or into its constituent trees. These operations
/// are of particular usefulness when it comes to things like serialization/deserialization,
/// as well as granular storage (and loading) of trees at runtime.
#[derive(Default)]
pub struct MastForest {
    /// All of the blocks local to the trees comprising the MAST forest
    nodes: Vec<MastNode>,
    /// Dense bitset where each set bit indicates that the corresponding 
    /// index in `nodes` is a MAST root. Multiple ways of representing
    /// this information is possible, but I'm choosing this one to illustrate.
    roots: SmallVec<[u64; 1]>,
    /// The "entrypoint", when set, is the root of the entire forest, i.e.
    /// a path exists from this node to all other roots in the forest. This
    /// corresponds to the executable entry point. When not set, the forest
    /// may or may not have such a root in `roots`, but is not required.
    /// Whether or not the entrypoint is set distinguishes a MAST which is
    /// executable, versus a MAST which represents a library.
    ///
    /// NOTE: The entrypoint is also present in `roots` if set
    entrypoint: Option<MastNodeId>,
    /// FUTURE: Used to fetch nodes which are referenced, but are not present
    /// in `nodes`. Doesn't necessarily have to be made available here though,
    /// this is just one way such a thing could be used.
    loader: Arc<dyn ObjectStore>,
}
impl Mast {
    pub fn is_root(&self, id: MastNodeId) -> bool {
        let id = id.as_usize();
        let chunk = id / 64;
        let bit = (id % 64) as u64;
        self.roots[chunk] & bit == bit
    }

    pub fn roots(&self) -> impl Iterator<Item = MastNodeId> {
        let last_chunk = self.roots.len().saturating_sub(1);
        let trailing = self.nodes.len() % 64;
        self.roots.iter().copied().enumerate().flat_map(move |(chunk, bitset)| {
            let start = (chunk * 64) as u32;
            let end = if chunk == last_chunk { start + trailing } else { start + 64 };
            (start..end).filter_map(|id| {
                let id = MastNodeId::new(id).unwrap();
                if self.is_root(id) { Some(id) } else { None }
            })
        })
    }

    /// We could also maintain a digest -> id map, but if we only expect to look
    /// for roots, this should be sufficient
    pub fn get(&self, hash: Digest) -> Option<&CodeBlock> {
        self.roots().map(|id| &self.nodes[id]).find(|node| node.digest() == hash)
            /// FUTURE: Fall back to loading from object store
            .or_else(|| self.loader.get(hash))
    }

    pub fn has(&self, hash: Digest) -> bool { self.get(hash).is_some() }

    pub fn insert_root(&mut self, node: MastNode) -> MastNodeId {
         let id = self.insert_node(node);
         let index = id.as_usize();
         let root_index = index / 64;
         let bit = (index % 64) as u64;
         if root_index == self.roots.len() {
             self.roots.push(1u64 << bit);
         } else {
             self.roots[root_index] |= (1u64 << bit);
         }
         id
    }

    pub fn insert_node(&mut self, node: MastNode) -> MastNodeId {
        let index = self.nodes.len();
        let id = MastNodeId::new(index).expect("too many nodes");
        self.nodes.push(node);
        id
    }

    pub fn build(&mut self) -> MastBuilder<'_> {
        MastBuilder::new(self)
    }
}

/// This structure is used to build a MastForest while following the
/// control flow of a program naturally.
///
/// # Example
///
///    let mut forest = MastForest::default();
///    let mut builder = forest.build();
///    # Start a new `begin .. end` block
///    let mut entry = builder.begin();
///    # Start building a `if.true .. else .. end` construct
///    let mut split = builder.split();
///    # The split builder starts in the "true" block
///    split.call(a);
///    # And must be explicitly switched to the "false" block
///    split.else();
///    split.call(b);
///    # When both blocks are built, control flow is explicitly rejoined,
///    # consuming the split builder.
///    split.end();
///    # Likewise with the `begin .. end` block
///    entry.end();
///    # Lastly, the MAST built so far is added to the underlying forest
///    builder.build();
pub struct MastBuilder<'mast> {
    mast: &'mast mut MastForest,
}
impl<'mast> MastBuilder<'mast> {
    fn new(mast: &'mast mut MastForest) -> Self {
        Self { mast }
    }

    /// Save the tree built so far to the underlying `MastForest`
    pub fn build(self) { ... }
}

The MastBuilder type (and it's associated sub-builder types) would be used to construct a MastForest iteratively, while preserving invariants of the tree/forest. It would also provide conveniences for lowering Miden Assembly directly to MAST.

The current CodeBlock type (and the variant types, e.g. Split) would be reworked to reference their children using MastNodeId rather than pointers. This of course means that these types are useless without the corresponding MastForest, but in practice that's a non-issue, since these types are only used in a context where the MAST is at hand anyway, and this seems to already be the case with CodeBlock and CodeBlockTable.

Beyond that, the only other "interesting" bits are in serialization/deserialization, as well as the object store/registry which were the subject of this proposal. I'll open a proposal for the binary representation of MastForest tomorrow/Wednesday so we can discuss that separately.

bobbinth commented 4 months ago

I'm using more general terminology in the code below, i.e. MastNode, MastForest rather than CodeBlock and Program to better reflect the generality of the representation. However, Component is a perfectly suitable name, though we may want to reserve that for the time being, just in case we end up implementing something like the Component Model runtime, wherein a "component" might have more precise boundaries than the MAST itself.

I like MastNode and MastForest - let's use them from now on.

pub struct MastForest {
    /// All of the blocks local to the trees comprising the MAST forest
    nodes: Vec<MastNode>,
    /// Dense bitset where each set bit indicates that the corresponding 
    /// index in `nodes` is a MAST root. Multiple ways of representing
    /// this information is possible, but I'm choosing this one to illustrate.
    roots: SmallVec<[u64; 1]>,
    /// The "entrypoint", when set, is the root of the entire forest, i.e.
    /// a path exists from this node to all other roots in the forest. This
    /// corresponds to the executable entry point. When not set, the forest
    /// may or may not have such a root in `roots`, but is not required.
    /// Whether or not the entrypoint is set distinguishes a MAST which is
    /// executable, versus a MAST which represents a library.
    ///
    /// NOTE: The entrypoint is also present in `roots` if set
    entrypoint: Option<MastNodeId>,
    /// FUTURE: Used to fetch nodes which are referenced, but are not present
    /// in `nodes`. Doesn't necessarily have to be made available here though,
    /// this is just one way such a thing could be used.
    loader: Arc<dyn ObjectStore>,
}

I like the idea of putting the loader inside of MastForest. I think that's a good way to make the source of MastNodes transparent to the consumer. I'm assuming we'd also have something like MastForest::set_loader() method. This could be useful for when the VM receives a MastForest and then sets its own internal object store as the loader for it.

btw, what is the reason for using dyn ObjectStore type for the loader? Could we not use a generic type here?

    /// We could also maintain a digest -> id map, but if we only expect to look
    /// for roots, this should be sufficient
    pub fn get(&self, hash: Digest) -> Option<&CodeBlock> {
        self.roots().map(|id| &self.nodes[id]).find(|node| node.digest() == hash)
            /// FUTURE: Fall back to loading from object store
            .or_else(|| self.loader.get(hash))
    }

I think we will probably have to maintain the digest |-> id map as here we'd probably want to search not just roots, but all nodes (at least that's how I'm imagining it working).

bitwalker commented 4 months ago

btw, what is the reason for using dyn ObjectStore type for the loader? Could we not use a generic type here?

Primarily to avoid monomorphizing all of the MastForest code for every possible ObjectStore implementation, since it is not going to be in the "hot" path for executing the code, there is little benefit to making the MastForest type generic (which also means all downstream code that consumes a MastForest must also be generic).

Of course, if the hypothetical ObjectStore trait cannot be made object-safe, then we may be forced to either make MastForest generic, or avoid storing a reference to it in the MastForest, and instead pass it as a bit of context to specific APIs. I don't have any reason to believe that trait wouldn't be object-safe though.

In any case, that was my rationale. I'd be inclined to switch to making it generic if it was important for performance that we monomorphize code paths which interact with it, but the overhead of loading objects from the store is going to dwarf the actual dynamic dispatch, so the ergonomic benefits seem to far outweigh the downsides.

It's also quite likely that the ObjectStore implementation will be used/stored in a few places (e.g. global configuration/context/whatever), so it makes sense to turn it into a trait object at the boundary of the system so that we don't pollute a bunch of code with generics that aren't necessary for performance.

I think we will probably have to maintain the digest |-> id map as here we'd probably want to search not just roots, but all nodes (at least that's how I'm imagining it working).

Could you elaborate on the instances where you expect to look up a node in the MastForest that isn't a root? I suspect we won't benefit too much by de-duplicating nodes in storage by any finer granularity than a procedure, and it would probably be beneficial for the hypothetical ObjectStore to only concern itself with roots.

For example, the basic storage implementation I'm imagining, breaks up a MastForest into its constituent trees (which would be the smallest "object" that can be represented) and stores them on disk individually using the hash of the root of that tree as the filename for that object. Then, when a call to a root is made which is not in the current MastForest, that root would be requested from the ObjectStore, which would (in this case), look at the directory containing all of the stored objects, find the file corresponding to that hash, load it and deserialize it into a MastForest, which then can be used as-is, or merged into the MastForest which made the call, whichever makes the most sense.

In practice, its likely there would be a little more involved there, e.g. we'd need to load whatever metadata is associated with that MAST root to allow us to initialize its memory in a new context (whether that's directly, or by placing the data into the advice provider). There is some trickiness involved there that I'm still mulling over, but in any case, it doesn't have any bearing on the specific thing we're talking about here.

In any case, I just want to make sure I understand the circumstances in which it would be useful/necessary to maintain the full digest/node id map.

bobbinth commented 4 months ago

Could you elaborate on the instances where you expect to look up a node in the MastForest that isn't a root?

I guess this depends on how we implement MAST traversal on the VM side. If it is based on NodeId, then mapping just the roots may be sufficient. For example, let's say the VM needs to process a JOIN node. Assuming we know the NodeId of this node, we'd need to do the following:

  1. Get the node itself from the MastForest - for this, we may need something like get_by_id() method.
  2. If the node is found: a. Get the left child of the node also using get_by_id() method. b. Get the right child of the node also using get_by_id() method.

If we do it this way, the only time we'd use the MAST-root-based get() method is to process dynamic calls - as everything else we should be able to get by node ID.

One thing to note, the implementation of get_by_id() would account for handling external references (and this may apply to the get()method as well). Specifically, if the found node is an external reference, we first need to load it from the store, and only if we can't find it in the store returnNone`.

Also, one thing I just realized is that with the approach we've been discussing so far, it seems like we'd need to assign node IDs to all nodes loaded from the store (or at least to re-compute them). This is because node IDs make sense only in the context of a given forest. So, for MAST trees coming from the store, "local" node IDs may conflict with the ones already in the forest.

This may not be a big deal, but it does prevent us from building a "hot cache" of MAST components - e.g., something that lives in memory and is available for traversal without any additional work.

bitwalker commented 4 months ago

Also, one thing I just realized is that with the approach we've been discussing so far, it seems like we'd need to assign node IDs to all nodes loaded from the store (or at least to re-compute them). This is because node IDs make sense only in the context of a given forest. So, for MAST trees coming from the store, "local" node IDs may conflict with the ones already in the forest.

It's true that trying to merge two MastForest (already expanded in memory), let's call them a and b, requires computing new node IDs for every node in b, but I think that's to be expected and not an issue per se. The expensive operation, relatively speaking, is decoding the MastForest from its binary representation; remapping the node IDs is fast in comparison.

That said, I think there are other reasons why we may not want to make this quite as transparent in the MastForest API. Rather than hiding the fact that we're loading an external node (i.e. a node in another MastForest), we could instead return something like Option<Either<MastNode, (NodeId, Arc<MastForest>)>>, where None indicates that the MAST root could not be found at all, Some(Left(_)) indicates that the node was found in the current MastForest, and Some(Right(_)) indicates that the node was found in another MastForest, returning a reference to that forest as well as the NodeId of the desired node in that MastForest.

The traversal of a program would then be decoupled from a specific MastForest, by representing the traversal state as a stack of continuation pointers, where a continuation pointer would look something like:

pub struct Continuation {
    /// The forest containing the node we'll continue at
    forest: Arc<MastForest>, 
    /// The node at which control will resume
    node: NodeId,
    /// If applicable, the child index in `node` where control should resume.
    /// For nodes with only a single child, or with no children, this would always be zero.
    index: usize,
}

As we traverse the MAST depth-first, we push a continuation on the stack when visiting a child of some node, and pop a continuation off the stack when we've finished executing the current node and need to continue somewhere further up the tree. In this conceptual model the "current continuation" is the continuation on top of the stack. For example:

graph TD;
    subgraph A
        direction TB
        subgraph A^1
            direction TB
            A1("Join { id: 0, digest: 0x0, children: [1, 10] }");
            B1("Join { id: 1, digest: ..., children: [2, 6] }");
            C1("Span { id: 2, digest: ..., children: [3, 4, 5] }");
            D1("Call { id: 3, digest: ..., callee: 0x2 }");
            E1("Span { id: 6, digest: ..., children: [7, 8, 9] }");
            F1("Call { id: 10, digest: ..., callee: 0x1 }");
        end
        subgraph A^2
            G1("Span { id: 11, digest: 0x1, children: [...] }");
        end
    end

    subgraph B
        A2("Join { id: 0, digest: 0x2, children: [1, 5] }");
        B2("Span { id: 1, digest: ..., children: [2, 3, 4] }");
        C2("Span { id: 5, digest: ..., children: [6, 7, 8] }");
    end

    A1-->B1;
    A1-->F1;
    B1-->C1;
    B1-->E1;
    C1-->D1;

    A2-->B2;
    A2-->C2;

Here we have two forests, A and B, where A contains two trees (A1 and A2), and B is a single tree. A has local cross-tree references, as well as an external reference to the tree in B. The continuation stack would look like the following as we execute 0x0 in A (using (forest, node, index) as shorthand for both the current position, and for continuations on the stack):

Current Position Action Stack
(A, 0, 0) Enter []
(A, 1, 0) Enter [(A, 0, 1)]
(A, 2, 0) Enter [(A, 1, 1), (A, 0, 1)]
(A, 3, 0) Enter [(A, 2, 1), (A, 1, 1), (A, 0, 1)]
(B, 0, 0) Call [(A, 2, 1), (A, 1, 1), (A, 0, 1)]
(B, 1, 0) Enter [(B, 0, 1), [(A, 2, 1), (A, 1, 1), (A, 0, 1)]
...skip.. .. ..
(B, 5, 2) Enter [(A, 2, 1), (A, 1, 1), (A, 0, 1)]
(A, 2, 1) Enter [(A, 1, 1), (A, 0, 1)]
..skip.. .. ..
(A, 1, 1) Enter [(A, 0, 1)]
..skip.. .. ..
(A, 0, 1) Enter []

Enter corresponds to visiting a node for the first time. Call corresponds to Enter-ing a node by executing a call/dyncall instruction, and has tail-call like semantics with regard to the continuation stack - i.e. the current continuation is reused for the callee such that when the callee returns, it returns "over" the call node. There is no Exit action, as that is implicit in the traversal (i.e. when resuming the current continuation, we're returning up the tree past all of the nodes between the current node and the continuation point).

So the bottom line here is that despite node IDs being local to a MastForest, that doesn't change much in terms of how MAST is traversed, since presumably the traversal already maintains a node stack of some kind to avoid a depth-first traversal causing a stack overflow. The implementation might change a bit, but the overall approach should be essentially the same.

The key thing about this approach is that it when we request a MAST root from a MastForest and it returns (Arc<MastForest>, NodeId), that also provides us an opportunity to make sure that any read-only data segments or other static initialization associated with the forest being returned, gets performed before we start executing the given node. The VM will be the one maintaining that state, and it will be context-sensitive state too, so we can't really hide it behind MastForest::get. I hadn't put too much thought into that aspect of things when I originally sketched out the MastForest structure, but given our other recent conversations, I think it's something we definitely need to keep in mind.

This may not be a big deal, but it does prevent us from building a "hot cache" of MAST components - e.g., something that lives in memory and is available for traversal without any additional work.

I think using the scheme I sketched out above, we get best of both worlds - efficient MastForest-local access and traversal, and the ability to cache a MastForest for use across programs.

bobbinth commented 4 months ago

I think the approach you've described with continuations will work, but it also wonder if we need to add this much complexity (at least initially). Specifically, a much simpler approach would be to do something like this:

pub struct MastForest {
    nodes: BTreeMap<Digest, MastNode>,
    roots: Vec<Digest>,
    entrypoint: Option<Digest>,
    loader: Arc<dyn ObjectStore>,
}

impl MastForest {
    pub fn get(&self, node_hash: Digest) -> Option<MastNode> {
        if let Some(node) = self.nodes.get(node_hash) {
            match node {
                MastNode::External(ext_hash) => self.loader.get(ext_hash),
                _ => node
            }
        } else {
            None
        }
    }
}

The main benefit of this approach is that changes on the VM side would be minimal (maybe like an hour or two of work) and we'd be able to implement the whole transition way faster.

The obvious downside is that this will be much less memory efficient and traversals will take longer, but:

Note, that this struct does not need to affect the serialization format. We can make serialization format more or less independent from the in-memory representation.

plafer commented 2 months ago

I just caught up to this whole discussion; it all sounds very exciting! I wanted to share 2 quick thoughts:

1. Being mindful of registry attacks

The registry idea sounds great. Only thing I'll say is we should be mindful of security, since registries can be used for attacks (see NPM). Specifically, the prover should always verify the MAST that it received for a given root. Theoretically, this wouldn't be needed, since the verifier should catch any unsound MAST and reject any proof that used an execution trace built using it. But in practice, an attacker could use a registry to exploit a bug in the AIR constraints, such that the prover would build a proof using this invalid MAST, and the verifier would accept it. Of course, such attacker could run its own prover to build a proof that exploits the constraints, but at least, this would prevent anyone using our prover to generate malicious proofs due to a registry attack.

2. Plan ahead for MAST breaking changes

We should plan ahead for any future breaking changes to MAST and/or the serialization format in our packaging solution.

For example, say we add a new node type in the MAST, anyone running an outdated prover would fail to deserialize that new MAST. Or similarly for the serialization format, I'd expect a registry to understand many (all?) different versions of the serialization format as it changed through time, and be able to deserialize whichever version into its current MAST (if compatible).

We should also keep in mind that theoretically, hash collisions are possible across breaking changes of the MAST. A dummy example is if from version A to version B, we swap the hash domain between join & split nodes. Then, a tree with a JOIN as root in version A would have the same hash as the same tree in version B, except for the root changed to a SPLIT. Although we (hopefully) would never do that literally, we could unintentionally introduce a bug in a similar but more complex manner, which would allow unintended code to be fetched from the registry and executed/proved. Now, we should expect the verifier to catch that invalid code was executed (it basically almost always will), so it's not a huge safety concern. But at the very least, it would be a very annoying bug to figure out and fix, which would probably require people purge their local registries, etc. So this suggests we should think of the "content unique IDs" (referring to the content-addressable property of MAST) no longer as only node_hash, but rather (node_hash, mast_version).

bobbinth commented 1 month ago

The registry idea sounds great. Only thing I'll say is we should be mindful of security, since registries can be used for attacks (see NPM). Specifically, the prover should always verify the MAST that it received for a given root. Theoretically, this wouldn't be needed, since the verifier should catch any unsound MAST and reject any proof that used an execution trace built using it. But in practice, an attacker could use a registry to exploit a bug in the AIR constraints, such that the prover would build a proof using this invalid MAST, and the verifier would accept it. Of course, such attacker could run its own prover to build a proof that exploits the constraints, but at least, this would prevent anyone using our prover to generate malicious proofs due to a registry attack.

In https://github.com/0xPolygonMiden/compiler/pull/132#pullrequestreview-1999498527 we discussed two ways of serializing MAST: (1) "fast" (trusted) and (2) "compact" (un-trusted). I'm imagining that the registry would contain the "compact" variant, and so, on download, the user would re-build the MAST to verify that the roots match.

There is another issue here though: while verifying that a given program has a given MAST root is enough to verify the correct execution of this program, it may not be enough to execute it. There are two reasons for this:

  1. Decorators are not included in MAST root computations. So, it is possible to add/remove decorators from a program without affecting MAST root.
  2. Some nodes in the MAST can be "hidden" - i.e., we know the node hash, but we don't know what's under that node. Having MAST root itself may not be enough to identify which nodes may be hidden. So, it is possible to hide/reveal different nodes in the MAST without affecting its root.

Given the above, we may need to provide two commitments for a given library: a set of MAST roots as well as sequential hash of the serialized MAST (including decorators). Upon download the user could then verify that both hashes are correct and be sure that they have the right program.

We should also keep in mind that theoretically, hash collisions are possible across breaking changes of the MAST.

This would be a potentially very serious security vulnerability and I think we should make sure we avoid such scenarios. One way to do this would be to change domain separators for MAST nodes any time we introduce breaking changes (say as we go from v1.0 to v2.0). This way, we'd be guaranteed that version MAST v1.0 will never have the same hash as MAST v2.0 - even for identical programs.

bobbinth commented 3 weeks ago

With #1349 we will have the first part of the MAST migration implemented. The next step would be to add support for the ObjectStore. I think there are two ways we can approach this:

  1. We can implement some in-memory registry which would work similar to how things work now. That is, we assemble libraries and add them to the registry at runtime and then run the VM with this registry.
  2. We can implement a "persistent" registry to which we can add libraries ahead of time and the results would be stored on disk. Then, at runtime, we instantiate the VM with this registry and it reads the required components from disk as needed.

The first approach may be a bit simpler, but in the long term term we'll need the second approach. At this point yet I'm not sure we should start with the 1st approach to build the registry iteratively or if we should go for the second approach right away. One other thing to consider: we may need an in-memory registry anyways for things like tests.

In either case though, I think we'll need to make the following changes:

Object store

First, we need to define a trait for ObjectStore. This could look something like this:

pub trait ObjectStore {
    /// Returns MAST forest corresponding to the specified digest, or None if the MAST forest for this
    /// digest could not be found in this [ObjectStore].
    fn get(&self, node_digest: Digest) -> Option<&MastForest>
}

External nodes

Second, we'll need to add External (or Proxy) node to the MAST and modify the assembler to produce these nodes. One of the main questions to resolve here, in my mind, is how to tell assembler with procedures are External and which are not. In some cases this should be easy:

  1. Any procedure invoked via a syscall should be in the kernel, and therefore, a CallNode corresponding to a syscall should always have its callee be an External node. This should also eliminate the need to track the kernel's MAST forest in the assembler (i.e., we should be able to get rid of Assembler.mast_forest field).
  2. Any procedure for which we can't resolve a MAST root should be an External node. These would be generated by call.0x... (and in the future exec.0x... instructions).

But what to do about the procedures which we can resolve to MAST roots but do not want to include into the assembled MAST forest? For example, let's say we are compiling a program that uses miden-stdlib. Our program makes a call to the std::crypto::hashes::blake3::hash_2to1 procedure (e.g., via an exec instruction). The desired behavior would be to output an External node for this assuming that the implementation of blake3::hash_2to1 will be fetched from the VM's object store at runtime. But without extra info, the assembler won't be able to figure this out.

One way to address the above is to instantiate the assembler with its own object store. And then, if a procedure is found in this object store, we'd output an External node for it. So, for example, if I want to compile a program in the context of miden-stdlib, I'd instantiate the assembler with the object store loaded with miden-stdlib. Then, all calls to things like std::crypto::hashes::blake3::hash_2to1 would be found in this object store and we could output External nodes for them.

The Assembler struct would then look something like this:

pub struct Assembler {
    /// The global [ModuleGraph] for this assembler. All new [AssemblyContext]s inherit this graph
    /// as a baseline.
    module_graph: Box<ModuleGraph>,
    /// The global procedure cache for this assembler.
    procedure_cache: ProcedureCache,
    /// Whether to treat warning diagnostics as errors
    warnings_as_errors: bool,
    /// Whether the assembler enables extra debugging information.
    in_debug_mode: bool,
    /// Whether the assembler allows unknown invocation targets in compiled code.
    allow_phantom_calls: bool,
    /// An object store loaded with MAST forests which we assume to be available to the VM at runtime.
    store: Arc<dyn ObjectStore>,
}

But this would also require a deeper refactoring of the assembler and is not something we need on the first iteration - so, maybe we do it as a follow up after the basics are implemented.

Loading the object store

Loading of the object store would left to specific implementations. Assuming we want to implement a simple in-memory object store, it could look something like this:

pub struct MemObjectStore {
    procedures: BTreeMap<Digest, MastForest>,
}

impl MemObjectStore {
    pub fn load(&mut self, mast: MastForest) -> Result<(), SomeError> {
        // here, we'd break the incoming MAST forest into individual procedures such that the entries that
        // we add to self.procedures correspond to individual procedure MASTs.
    }
}

To support the above functionality, we'll also need to refactor how we identify procedures in the MastForest structs. Specifically, I think we'll need to introduce roots field which would define which procedures are exposed by a given MAST forest. I also think we can probably get rid of node_id_by_hash struct from MastForest (we probably would still need it during assembly time, but after that, it may be discarded). So, the MastForest struct could look like so:


pub struct MastForest {
    /// All of the nodes local to the trees comprising the MAST forest.
    nodes: Vec<MastNode>,

    /// Roots of all procedures defined within this MAST forest .
    roots: Vec<MastNodeId>,

    /// This corresponds to the executable entry point. Whether or not the entrypoint is set distinguishes
    /// a MAST which is executable, versus a MAST which represents a library.
    entrypoint: Option<MastNodeId>,
    kernel: Kernel,
}

I'm also wondering if entrypoint and kernel should be the properties of MastForest or Program - but that's a separate discussion.

Processor changes

In the processor, we could add the object store directly to the Process struct to make it look something like this:

struct Process<H>
where
    H: Host,
{
    system: System,
    decoder: Decoder,
    stack: Stack,
    range: RangeChecker,
    chiplets: Chiplets,
    host: RefCell<H>,
    max_cycles: u32,
    enable_tracing: bool,
    store: Arc<dyn ObjectStore>,
}

Or we could add it to the Host trait - e.g., we could do something like this:

pub trait Host {
    ...

    /// Returns MAST forest corresponding to the specified digest, or None if the MAST forest for this
    /// digest could not be found in this [Host].
    fn get_mast_forest(&self, node_digest: Digest) -> Option<&MastForest>;

    ...
}

Intuitively, it feels like putting this functionality into the Host is better and we can treat it similarly to how we treat the advice provider now (i.e., AdviceProvider is a separate trait which is used internally by the DefaultHost and in this case we'd still need to define ObjectStore trait). But also, I didn't think through far enough to know if this would cause some complications.

When the VM encounters an External node, we'd do something like this:

MastNode::External(node_digest) => {
    let mast_forest = self.host.get_mast_forest(node_digest)?;
    let node_id = mast_forest.get_node_id_by_digest(node_digest)?;
    self.execute_mast_node(node_id, mast_forest)
}

In the above, I'm not handling the case when an External node could be in the current MastForest because I'm actually not sure how we'd insert an External node which refers to a node in the current MAST anywhere I described above. But we can easily change it if needed.

bitwalker commented 3 weeks ago

At this point yet I'm not sure we should start with the 1st approach to build the registry iteratively or if we should go for the second approach right away. One other thing to consider: we may need an in-memory registry anyways for things like tests.

I think some concepts might be getting conflated here: in the design I had in mind, we need two components, the registry (which manages packages), and the object store (which manages MAST objects). The latter needs both in-memory and on-disk stores, while the registry doesn't need its own storage at all (or rather, the storage associated with registry data depends on the specific registry implementation). Below I've quickly sketched out the broad strokes of these components and how I see them interacting with one another (and with the general workflow of executing programs):

Registry

The registry provides two things:

  1. Access to an index of available packages and their associated metadata (what versions are available, their dependencies, etc.).
  2. Installation of packages into the object store (not all indexed packages are installed, this occurs on an as-needed basis)

The registry basically exists to facilitate looking up a package and installing its objects (and those of its dependencies, if available) into the object store. The core registry API is dead simple, and tailored towards the needs of the VM with regards to packaging. Depending on the specific registry implementation, you might have additional tooling above and beyond that provided by the core registry API, e.g. publishing, checking for updates.

The simplest possible registry would look for package files in a directory on-disk (e.g. ~/.local/share/miden/registry), and generate an in-memory index from the metadata contained in all of the packages found in that directory. New packages can be made available simply by placing them in that directory.

A slightly more sophisticated registry might follow a similar approach, but instead require the directory to be managed via a CLI tool, which not only installs/removes packages from the directory, but also maintains an on-disk index (e.g. ~/.local/share/miden/registry/packages.json). When this registry is instantiated, it loads the index from the on-disk representation, and does not attempt to read the metadata of every package in the directory (much more scalable, at the cost of a bit more complexity).

When the time comes where we have an actual centralized registry akin to crates.io, the local registry might maintain just an index of packages it has installed (so as to avoid attempting to install objects from a package which was already installed), and route all metadata requests to the remote registry server. In this case, storage for the registry itself is purely optional.

Object Store

The object store provides two things:

  1. A set of MAST objects indexed by digest
  2. (Un)installation of MAST objects available in the store

The object store itself requires both in-memory (hot) storage and on-disk (cold) storage:

If the VM is instantiated and discarded after each program, then the in-memory store can be very naive, and never unload anything. This consumes more memory the more complicated the call graph of a program is, but doesn't matter too much for most programs. The on-disk store ensures that instantiating a VM frequently is inexpensive (as all the work of preparing objects to be loaded into memory has been done). In this use case, you could theoretically omit either the in-memory or the on-disk store, but doing so would impact runtime performance significantly. Because the in-memory store would be so simple in this case, it makes little sense to only implement one of them IMO.

If the VM is instantiated once, and run as a long-lived process like a server, then the in-memory store will need to select a caching algorithm (e.g. ARC or LRU), and use that to determine what objects to keep in-memory, and which objects to drop and reload from disk when needed. In this use case, both the in-memory and on-disk stores serve a critical role.

Putting It Together

So with the design for those components in mind, let's say you want to invoke a program you have assembled locally, with the VM, here's a general sketch of how I imagine that working:

  1. The VM is instantiated, using either the default or user-provided configuration for the registry and object store
  2. The package containing the desired program is loaded directly into the VM, bypassing the registry (i.e. we are not installing a package containing a program and running it, we're loading a program we assembled locally and running it). During this process, the VM ensures that all of the dependencies declared by the program package are installed via the registry, so that the code is available to the object store if needed.
  3. The program is executed by specifying the MAST root of the entrypoint, and providing all stack inputs, etc.
  4. The program executes until it reaches a reference to a MAST root which is not in the current MAST forest (e.g. exported by one of the program's dependencies), at which point the following occurs: a. The VM requests the referenced MAST root from the object store b. The object store hasn't loaded that object into its in-memory store yet, so it checks the on-disk storage c. The object store finds the object on-disk, (e.g. the dependency containing it was installed when the program was loaded) d. The object store loads the object, stores a copy in its in-memory cache, and returns the loaded object
  5. Step 4 is repeated until the program finishes executing

Similarly, a program could be executed by installing it via the registry, and then invoking the entrypoint as described in Step 3 above.

If at any point, a referenced MAST root was requested from the object store, but not found, execution of the program will trap, and an error will be returned as the result.

Whether or not the full dependency tree is installed when a package is installed isn't super important. If you defer installation of dependencies until they are actually used, then you need to ensure that when a given MAST root is requested and not found in the object store, that you can look up what package in the current program's dependency tree, exports that root, and install the relevant package at that point. Doing it this way allows you to treat missing dependencies as a non-issue until an execution trace actually hits that dependency. The simplest approach is to just require everything up front, so that you don't waste time executing a program that is doomed to fail if a key dependency is missing.

I'll follow up here on the rest of your comment, but wanted to make sure the vision for how all of this ties together was laid out in a shorter, more cohesive description, now that some of the work has been done in the assembler that lays the foundation for it. If you spot any issues with what I've sketched out, or have a different approach in mind, let me know.

bitwalker commented 3 weeks ago

First, we need to define a trait for ObjectStore. This could look something like this:

pub trait ObjectStore {
    /// Returns MAST forest corresponding to the specified digest, or None if the MAST forest for this
    /// digest could not be found in this [ObjectStore].
    fn get(&self, node_digest: Digest) -> Option<&MastForest>
}

I think we also need the following methods:

/// Inserts all of the objects comprising the given MAST forest to the object store
fn insert(&mut self, forest: &MastForest);

/// Removes the referenced object from the object store
fn remove(&mut self, node_digest: Digest) -> Result<(), InvalidObjectError>;

One of the main questions to resolve here, in my mind, is how to tell assembler with procedures are External and which are not. In some cases this should be easy:

  1. Any procedure invoked via a syscall should be in the kernel, and therefore, a CallNode corresponding to a syscall should always have its callee be an External node. This should also eliminate the need to track the kernel's MAST forest in the assembler (i.e., we should be able to get rid of Assembler.mast_forest field).

The main reason we need the kernel in the assembler is because we need to know the names of the kernel procedures and their corresponding MAST roots. Currently, we require the kernel to be assembled first, and then we use the information gleaned from that assembly to compile programs on top of that kernel. To avoid needing to do it this way, we need to get the same information by some other means (e.g. package metadata).

  1. Any procedure for which we can't resolve a MAST root should be an External node. These would be generated by call.0x... (and in the future exec.0x... instructions).

This was the case prior to @plafer's PR, but since we removed that node type, I think we just panic now. In any case, if we restore that code, this one is solved for.

But what to do about the procedures which we can resolve to MAST roots but do not want to include into the assembled MAST forest?

In the assembler refactoring, all procedure calls were done via external/proxy nodes, so this was also a non-issue. The small tweak I proposed in @plafer's PR (prior to us deciding to remove that node type entirely), was to check if the referenced MAST root is part of the current forest, and if so, resolve it as part of the forest, otherwise resolve it as an external/proxy node. This meant that the code local to a program would use forest-local references, while code from dependencies would use external/proxy references. Additional heuristics could be used to further refine how that choice is made (i.e. deliberately choose to use a proxy reference even when the node is available in the current forest), but I don't believe that is likely to be necessary.

For example, let's say we are compiling a program that uses miden-stdlib. Our program makes a call to the std::crypto::hashes::blake3::hash_2to1 procedure (e.g., via an exec instruction). The desired behavior would be to output an External node for this assuming that the implementation of blake3::hash_2to1 will be fetched from the VM's object store at runtime. But without extra info, the assembler won't be able to figure this out.

Just to be clear, no extra info is required. When assembling a given program, we aren't assembling the whole world, we're assembling just the code we are given. To the extent we need extra info in the assembler, it is package metadata for the dependencies of the program being compiled, so that we can resolve calls to those dependencies to their corresponding MAST roots. We don't have packaging implemented yet, so at this point there isn't even a way to compile a program that references dependencies by name, without also providing the code for the dependency at the same time. Once we implement packaging, that is no longer the case, but at that point the assembler can use the package metadata to resolve procedure references without having to compile the dependency, and thus will use external/proxy nodes for those references.

TL;DR - this is solved automatically as a side effect of implementing packages.

One way to address the above is to instantiate the assembler with its own object store. And then, if a procedure is found in this object store, we'd output an External node for it.

It isn't necessary for the assembler to have the actual objects at all - it need only know the set of exports from a given package (i.e. the fully-qualified procedure names and their corresponding MAST roots). We can already tell if a given MAST root refers to a concrete node that is part of the current compilation unit or not.

Loading the object store

Loading of the object store would left to specific implementations.

I don't think the object store has or needs user-provided implementations (user-configurable, sure/maybe). The registry is different, that very obviously needs to be pluggable. See the Object Store section of my previous comment for an elaboration on what its implementation consists of. To the extent that anything needs to be pluggable in the object store component, it would be the implementation of the on-disk/cold object storage. IMO that is unnecessary, and we could just provide configuration rather than a plug-in system for those details, but if anywhere made sense, it would be there.

To support the above functionality, we'll also need to refactor how we identify procedures in the MastForest structs. Specifically, I think we'll need to introduce roots field which would define which procedures are exposed by a given MAST forest.

This metadata is already supposed to be part of the MastForest structure AFAIK, as it is how we identify what objects are present in the forest (as "objects" here are tracked at procedure granularity).

I also think we can probably get rid of node_id_by_hash struct from MastForest (we probably would still need it during assembly time, but after that, it may be discarded). So, the MastForest struct could look like so:

We also need it to merge forests and/or progressively extend a forest. I think these operations will be quite common throughout the execution of a program, so removing that field would vastly limit the utility of the MastForest data structure. One could free the contents of the node_id_by_hash map if the containing MastForest is "frozen", which would have the effect of freeing up the memory that no longer serves a purpose, but I'd want to see how much memory is actually being consumed for complex programs before worrying about that too much.

I'm also wondering if entrypoint and kernel should be the properties of MastForest or Program - but that's a separate discussion.

The kernel field doesn't make much sense to me (that belongs more at a package level, or to a Program, the MastForest is too general for that. The entrypoint field I'd agree doesn't need to exist on MastForest if it is being provided by some other means (e.g. package metadata or as a field of Program).

As an aside, I prefer to think of MastForest as a subset of an unbounded forest of MAST nodes, which can grow or shrink to encompass what is needed by the current program or purpose. It can represent a full program, or just a subset of that program down to a single procedure. This makes it well suited for breaking apart into chunks, and then merging back together as chunks are progressively loaded for a given execution of the program. I think the ideal MastForest is one which can handle this dynamic resizing efficiently and ergonomically. Consequently, it doesn't make sense to track package-level metadata as part of the MastForest, because the MastForest is constantly subject to change, not to mention you will likely have forests with elements of multiple packages combined.

Or we could add it to the Host trait - e.g., we could do something like this:

pub trait Host {
    ...

    /// Returns MAST forest corresponding to the specified digest, or None if the MAST forest for this
    /// digest could not be found in this [Host].
    fn get_mast_forest(&self, node_digest: Digest) -> Option<&MastForest>;

    ...
}

I think rather than request specific operations like get_mast_forest, it would make more sense to ask the Host for an implementation of a given interface/trait, e.g. fn registry(&self) -> Arc<dyn Registry>. As mentioned earlier, I don't think the object store is one of those things, but you could have a trait that represents the on-disk storage, and then instantiate the internal object store with that.

This way, users can easily instantiate default impls (or third-party impls) for those traits without needing to implement a bunch of trait methods. Not to mention it ties all of those traits to the Host trait and vice versa, so extending the Registry trait, for example, is now potentially a breaking change (since you need to add the new method to Host as well).

Intuitively, it feels like putting this functionality into the Host is better and we can treat it similarly to how we treat the advice provider now (i.e., AdviceProvider is a separate trait which is used internally by the DefaultHost and in this case we'd still need to define ObjectStore trait).

I think this is basically what I was just describing above (assuming the AdviceProvider trait implementation is accessed directly and not indirectly by propagating the trait methods to the API of the Host trait) - so I'd say that's a pretty good sign that we're on to something!

When the VM encounters an External node, we'd do something like this:

MastNode::External(node_digest) => {
    let mast_forest = self.host.get_mast_forest(node_digest)?;
    let node_id = mast_forest.get_node_id_by_digest(node_digest)?;
    self.execute_mast_node(node_id, mast_forest)
}

The one thing missing here is merging the loaded object (a MastForest) with the current MAST forest stored in the processor state. Whether that is managed by the object store, or is handled in the processor, I don't think is super important, but the main thing is to ensure that as a program executes, it becomes increasingly less common that a desired MAST root is not in the current forest. If you always switch forests, that can technically work, but you will end up doing a lot of redundant loading of objects from the object store.

bobbinth commented 3 weeks ago

At this point yet I'm not sure we should start with the 1st approach to build the registry iteratively or if we should go for the second approach right away. One other thing to consider: we may need an in-memory registry anyways for things like tests.

I think some concepts might be getting conflated here: in the design I had in mind, we need two components, the registry (which manages packages), and the object store (which manages MAST objects).

Ah yes - I was sloppy with terminology here. Everything I wrote was about the object store (and not about the registry).

The object store itself requires both in-memory (hot) storage and on-disk (cold) storage

I'm not sure this is a requirement: we could have a purely in-memory implementation of the object store, and we can also have a more sophisticated implementation that works the way you've described.

I think we also need the following methods:

/// Inserts all of the objects comprising the given MAST forest to the object store
fn insert(&mut self, forest: &MastForest);

/// Removes the referenced object from the object store
fn remove(&mut self, node_digest: Digest) -> Result<(), InvalidObjectError>;

I think one of the main differences between how you and I think about it is that in my mind the VM is instantiated only with an ObjectStore - so, it doesn't know anything about registries. How to deal with registries is up to specific implementations of the ObjectStore.

For example, I think of the above methods as being methods on a specific implementation of the ObjectStore trait rather than a part of the trait itself. For example, an object store we use for testing many not even need remove() method.

During this process, the VM ensures that all of the dependencies declared by the program package are installed via the registry, so that the code is available to the object store if needed.

I think this may be tricky as depending on the execution path the program may require a different set of dependencies to be available. So, in a sense, the set of dependencies may be "dynamic" and depend on the input with which a given program is executed. In my mind, the dependencies would be fetched from the object store at runtime (rather than prior to program execution). This way, we don't need to fetch something that would not eventually be requested during execution. But I think you mention something similar a few paragraphs down from the above quote.

The main reason we need the kernel in the assembler is because we need to know the names of the kernel procedures and their corresponding MAST roots. Currently, we require the kernel to be assembled first, and then we use the information gleaned from that assembly to compile programs on top of that kernel. To avoid needing to do it this way, we need to get the same information by some other means (e.g. package metadata).

Right - I'm imagining that we'll eventually end up having kernel packages and so an assembler would be instantiated with a kernel package (or build one on the fly) and the only useful info from that package to the assembler would be a map procedure_name -> procedure_mast_root.

The one thing missing here is merging the loaded object (a MastForest) with the current MAST forest stored in the processor state. Whether that is managed by the object store, or is handled in the processor, I don't think is super important, but the main thing is to ensure that as a program executes, it becomes increasingly less common that a desired MAST root is not in the current forest. If you always switch forests, that can technically work, but you will end up doing a lot of redundant loading of objects from the object store.

I think this is another main point of difference: I'm still not sure we need to merge MAST forests and my thinking so far has been based on the idea of "switching forests". My reasoning for this is as follows:

So, to summarize the way I'm envisioning the process:

  1. The VM is instantiated with an object store. Which specific object store implementation to use, is fully up to the user. We will provide a couple of such implementations (e.g., a purely in-memory object store, an object store with "cold" storage on disk and and a "hot" storage in memory) but people would be free to build other implementations. a. One of the things that will need to be loaded into the object store prior to any program execution would be kernels of the programs to be executed by the VM.
  2. To execute a program on the VM, the user provides a MastForest (wrapped in a Program struct to define the entrypoint etc.). At runtime, this MastForest remains static - i.e., we never split it or merge anything into it.
  3. When the VM comes across an External node, it makes a request to the object store to fetch the MastForest in which this node is present, and starts executing nodes in the context of this MastForest. It is completely up to a specific implementation of the ObjectStore to manage MAST forests internally. For example, I can imagine an implementation that maintains one giant MastForest for the entire registry (or subset thereof). There could also be an implementation where there is a MastForest per procedure, and anything in-between. a. If MastForest for a given node cannot be found, the VM returns an error.
plafer commented 3 weeks ago

The current design of ObjectStore/MastForest has confused me a bit, at least from the standpoint of the VM's processor. I will lay down a sketch of how I see could be a simpler representation of programs for the purpose of execution (i.e. I won't bother with Arc and whatnot).

Today's MastForest would become Procedure, i.e. would look like

pub struct Procedure {
    nodes: Vec<MastNode>,
    node_id_by_hash: BTreeMap<RpoDigest, MastNodeId>,
    entrypoint: MastNodeId,
}

We would represent External nodes as the hash of the Procedure it represents:

pub enum MastNode {
    // ...
    External(Digest)
}

Then, a Program would be:

pub struct Program {
    procedures: BTreeMap<Digest, Procedure>
}

impl Program {
    pub fn insert(&mut self, proc: Procedure) {
        self.procedures.insert(proc.hash(), proc)
    }
}

The VM's processor would actually execute an AbstractProgram, which holds an ObjectStore and can query for external nodes when it encounters them:

pub struct AbstractProgram {
    store: ObjectStore,
    program: Program
}

impl AbstractProgram {
    // Note: If this fails, VM execution halts
    pub fn get_procedure_by_digest(&mut self, proc_digest: Digest) -> Option<Procedure> {
        if let Some(proc) = self.program.procedures.get(proc_digest) {
            // procedure already loaded in program, just return it
            Some(proc)
        } else {
            let proc = self.store.get(proc_digest)?;

            // Cache the procedure, and return. This has the effect of automatically resolving all 
            // `External` nodes with this hash in the program to this procedure.
            //
            // Note: we'd want procedures to be behind `Arc`
            self.program.insert(proc.clone());

            Some(proc) 
        }
    }
}

where the ObjectStore can use a registry or whatnot to handle the get() requests:

pub trait ObjectStore {
    // use `&mut self` to allow caching (without interior mutability at least)
    fn get(&mut self, proc_digest: Digest) -> Option<Procedure>;
}

For me, the main benefit of this set of abstraction is that it maps more closely to how I think about program execution than the other suggestions. For example, ObjectStore::get() now returns a procedure, which is the type I'd expect when the argument I pass in is a procedure root (as opposed to potentially a full library). Note that in this framework, there's no benefit in returning anything more than a Procedure anyways, since all calls to the ObjectStore get cached in the AbstractProgram anyways.

External calls are also resolved very cleanly. You only need to update the "procedure cache" (i.e. the Program) with the new procedure; all future External calls to that procedure will automatically use that cached procedure. Contrast that with the previous MastForest approach; my understanding of resolving such calls would mean to load all the new MastNodes in the MastForest, and then swap (i.e. mutate the forest) all External(Digest) nodes that called this procedure with the MastNodeId that represents the root of the actual procedure. If you don't do that "swap" stage, then every other call sites to that external procedure will load a new identical version of the procedure in the MastForest, which we want to avoid for obvious reasons.

Similarly, a Procedure is now immutable, so there's no risk of using a "dangling" MastNodeId that perhaps referred into the procedure before it got merged elsewhere.

Now, I'm not sure if using a different approach for "execution" and "package management" is a good idea; this is just meant as my idea of what would be the ideal representation strictly for execution.

bobbinth commented 3 weeks ago

A few comments about this approach:

If I understood it correctly, it is opinionated in a way that we assume that ObjectStore would always return MAST for a single procedure. While this may make sense for some implementations (e.g., when there is an "on-disk" component for the object store), I'm not sure we should force this to always be the case. For example, an in-memory implementation of ObjectStore does not need to break up MAST forests into individual procedures (e.g., a whole miden-stdlib could be a single MAST forest).

So, to sum up - I think introducing Procedure locks us into a specific implementation of ObjectStore which may not make sense in all cases, while keeping MastForest gives us more flexibility and does not expose the internal structure of the object store to the processor.

External calls are also resolved very cleanly. You only need to update the "procedure cache" (i.e. the Program) with the new procedure; all future External calls to that procedure will automatically use that cached procedure. Contrast that with the previous MastForest approach; my understanding of resolving such calls would mean to load all the new MastNodes in the MastForest, and then swap (i.e. mutate the forest) all External(Digest) nodes that called this procedure with the MastNodeId that represents the root of the actual procedure. If you don't do that "swap" stage, then every other call sites to that external procedure will load a new identical version of the procedure in the MastForest, which we want to avoid for obvious reasons.

The way I think about it, if we go with MastForest-based approach, we don't need to do any mutations to MAST forests at runtime. We may mutate MAST forests at the time they are loaded into the object store - but this is not a requirement. For example, for in-memory object store, no mutations would be needed at all. For on-disk object stores we may want to mutate some MAST forests - but again, this would only happen at load time rather than at runtime.

So, as far as the processor is concerned MAST forests are immutable, and whether MAST forests are mutable or immutable inside the object store, would be up to a specific implementation of the object store.


Here is how I'm thinking some of the structs could change concretely:

pub enum MastNode {
    // ...
    External(Digest)
}

pub struct MastForest {
    /// All of the nodes local to the trees comprising the MAST forest.
    nodes: Vec<MastNode>,

    /// Roots of all procedures defined within this MAST forest .
    roots: Vec<MastNodeId>,
}

pub struct Program {
    mast_forest: MastForest,
    entrypoint: MastNodeId,
    kernel: Kernel,
}

Note that we no longer have node_id_by_hash field in the MastForest.

For the processor, we'll need to make the following change:


pub trait ObjectStore {
    // if object store needs to be mutated, this would be done via interior mutability
    fn get(&self, node_digest: Digest) -> Option<Arc<MastForest>>;
}

pub trait Host {
    // ...

    /// Returns MAST forest corresponding to the specified digest, or None if the MAST forest for this
    /// digest could not be found in this [Host].
    fn get_mast_forest(&self, node_digest: Digest) -> Option<Arc<MastForest>>;

    // ...
}

pub struct DefaultHost<A, S> {
    adv_provider: A,
    store: S, // we could also have it as Arc<dyn ObjectStore>,
}

We'll also need to provide a very simple implementation of in-memory object store which could look something like this:

pub struct MemObjectStore {
    masts: BTreeMap<Digest, Arc<MastForest>>,
}

impl MemObjectStore {
    pub fn insert(&mut self, mast: MastForest) -> Result<(), SomeError> {
        for root in mast.roots() {
            self.masts.insert(root, Arc::new(mast));
        }
    }
}

impl ObjectStore for MemObjectStore {
    fn get(&self, node_digest: Digest) -> Option<Arc<MastForest>> {
        self.masts.get(node_digest).cloned()
    }
}

In the future, we'll also implement more sophisticated object stores which may read data from disk and/or request packages form registry - though, how exactly this will work we can figure out when we get there.

Lastly, the way the External nodes could be handled during execution would be:

MastNode::External(node_digest) => {
    let mast_forest = self.host.get_mast_forest(node_digest)?.as_ref();
    let node_id = mast_forest.find_root(node_digest)?;
    self.execute_mast_node(node_id, mast_forest)
}

The above assumes that there is a MastForest::find_root() function which could be a linear search through the roots vector. This assumes that roots would be a fairly small vector - which is think should be the case most of the time. But if not, we could always change roots into a BTreeMap in the future.

I've omitted the changes to the assembler from the above description. We can discuss them separately - but I think they shouldn't be too big.