wasmtime/runtime/vm/instance/allocator/pooling/
memory_pool.rs

1//! Implements a memory pool using a single allocated memory slab.
2//!
3//! The pooling instance allocator maps one large slab of memory in advance and
4//! allocates WebAssembly memories from this slab--a [`MemoryPool`]. Each
5//! WebAssembly memory is allocated in its own slot (see uses of `index` and
6//! [`SlotId`] in this module):
7//!
8//! ```text
9//! ┌──────┬──────┬──────┬──────┬──────┐
10//! │Slot 0│Slot 1│Slot 2│Slot 3│......│
11//! └──────┴──────┴──────┴──────┴──────┘
12//! ```
13//!
14//! Diving deeper, we note that a [`MemoryPool`] protects Wasmtime from
15//! out-of-bounds memory accesses by inserting inaccessible guard regions
16//! between memory slots. These guard regions are configured to raise a signal
17//! if they are accessed--a WebAssembly out-of-bounds (OOB) memory access. The
18//! [`MemoryPool`] documentation has a more detailed chart but one can think of
19//! memory slots being laid out like the following:
20//!
21//! ```text
22//! ┌─────┬─────┬─────┬─────┬─────┬─────┬─────┬─────┐
23//! │Guard│Mem 0│Guard│Mem 1│Guard│Mem 2│.....│Guard│
24//! └─────┴─────┴─────┴─────┴─────┴─────┴─────┴─────┘
25//! ```
26//!
27//! But we can be more efficient about guard regions: with memory protection
28//! keys (MPK) enabled, the interleaved guard regions can be smaller. If we
29//! surround a memory with memories from other instances and each instance is
30//! protected by different protection keys, the guard region can be smaller AND
31//! the pool will still raise a signal on an OOB access. This complicates how we
32//! lay out memory slots: we must store memories from the same instance in the
33//! same "stripe". Each stripe is protected by a different protection key.
34//!
35//! This concept, dubbed [ColorGuard] in the original paper, relies on careful
36//! calculation of the memory sizes to prevent any "overlapping access" (see
37//! [`calculate`]): there are limited protection keys available (15) so the next
38//! memory using the same key must be at least as far away as the guard region
39//! we would insert otherwise. This ends up looking like the following, where a
40//! store for instance 0 (`I0`) "stripes" two memories (`M0` and `M1`) with the
41//! same protection key 1 and far enough apart to signal an OOB access:
42//!
43//! ```text
44//! ┌─────┬─────┬─────┬─────┬────────────────┬─────┬─────┬─────┐
45//! │.....│I0:M1│.....│.....│.<enough slots>.│I0:M2│.....│.....│
46//! ├─────┼─────┼─────┼─────┼────────────────┼─────┼─────┼─────┤
47//! │.....│key 1│key 2│key 3│..<more keys>...│key 1│key 2│.....│
48//! └─────┴─────┴─────┴─────┴────────────────┴─────┴─────┴─────┘
49//! ```
50//!
51//! [ColorGuard]: https://plas2022.github.io/files/pdf/SegueColorGuard.pdf
52
53use super::{
54    index_allocator::{MemoryInModule, ModuleAffinityIndexAllocator, SlotId},
55    MemoryAllocationIndex,
56};
57use crate::prelude::*;
58use crate::runtime::vm::{
59    mmap::AlignedLength, CompiledModuleId, InstanceAllocationRequest, InstanceLimits, Memory,
60    MemoryBase, MemoryImageSlot, Mmap, MmapOffset, PoolingInstanceAllocatorConfig,
61};
62use crate::{
63    runtime::vm::mpk::{self, ProtectionKey, ProtectionMask},
64    vm::HostAlignedByteCount,
65    MpkEnabled,
66};
67use std::sync::atomic::{AtomicUsize, Ordering};
68use std::sync::{Arc, Mutex};
69use wasmtime_environ::{DefinedMemoryIndex, Module, Tunables};
70
71/// A set of allocator slots.
72///
73/// The allocated slots can be split by striping them: e.g., with two stripe
74/// colors 0 and 1, we would allocate all even slots using stripe 0 and all odd
75/// slots using stripe 1.
76///
77/// This is helpful for the use of protection keys: (a) if a request comes to
78/// allocate multiple instances, we can allocate them all from the same stripe
79/// and (b) if a store wants to allocate more from the same stripe it can.
80#[derive(Debug)]
81struct Stripe {
82    allocator: ModuleAffinityIndexAllocator,
83    pkey: Option<ProtectionKey>,
84}
85
86/// Represents a pool of WebAssembly linear memories.
87///
88/// A linear memory is divided into accessible pages and guard pages. A memory
89/// pool contains linear memories: each memory occupies a slot in an
90/// allocated slab (i.e., `mapping`):
91///
92/// ```text
93///          layout.max_memory_bytes                 layout.slot_bytes
94///                    |                                   |
95///              ◄─────┴────►                  ◄───────────┴──────────►
96/// ┌───────────┬────────────┬───────────┐     ┌───────────┬───────────┬───────────┐
97/// | PROT_NONE |            | PROT_NONE | ... |           | PROT_NONE | PROT_NONE |
98/// └───────────┴────────────┴───────────┘     └───────────┴───────────┴───────────┘
99/// |           |◄──────────────────┬─────────────────────────────────► ◄────┬────►
100/// |           |                   |                                        |
101/// mapping     |            `layout.num_slots` memories         layout.post_slab_guard_size
102///             |
103///   layout.pre_slab_guard_size
104/// ```
105#[derive(Debug)]
106pub struct MemoryPool {
107    mapping: Arc<Mmap<AlignedLength>>,
108    /// This memory pool is stripe-aware. If using  memory protection keys, this
109    /// will contain one stripe per available key; otherwise, a single stripe
110    /// with an empty key.
111    stripes: Vec<Stripe>,
112
113    /// If using a copy-on-write allocation scheme, the slot management. We
114    /// dynamically transfer ownership of a slot to a Memory when in use.
115    image_slots: Vec<Mutex<Option<MemoryImageSlot>>>,
116
117    /// A description of the various memory sizes used in allocating the
118    /// `mapping` slab.
119    layout: SlabLayout,
120
121    /// The maximum number of memories that a single core module instance may
122    /// use.
123    ///
124    /// NB: this is needed for validation but does not affect the pool's size.
125    memories_per_instance: usize,
126
127    /// How much linear memory, in bytes, to keep resident after resetting for
128    /// use with the next instance. This much memory will be `memset` to zero
129    /// when a linear memory is deallocated.
130    ///
131    /// Memory exceeding this amount in the wasm linear memory will be released
132    /// with `madvise` back to the kernel.
133    ///
134    /// Only applicable on Linux.
135    pub(super) keep_resident: HostAlignedByteCount,
136
137    /// Keep track of protection keys handed out to initialized stores; this
138    /// allows us to round-robin the assignment of stores to stripes.
139    next_available_pkey: AtomicUsize,
140}
141
142impl MemoryPool {
143    /// Create a new `MemoryPool`.
144    pub fn new(config: &PoolingInstanceAllocatorConfig, tunables: &Tunables) -> Result<Self> {
145        if u64::try_from(config.limits.max_memory_size).unwrap() > tunables.memory_reservation {
146            bail!(
147                "maximum memory size of {:#x} bytes exceeds the configured \
148                 memory reservation of {:#x} bytes",
149                config.limits.max_memory_size,
150                tunables.memory_reservation
151            );
152        }
153        let pkeys = match config.memory_protection_keys {
154            MpkEnabled::Auto => {
155                if mpk::is_supported() {
156                    mpk::keys(config.max_memory_protection_keys)
157                } else {
158                    &[]
159                }
160            }
161            MpkEnabled::Enable => {
162                if mpk::is_supported() {
163                    mpk::keys(config.max_memory_protection_keys)
164                } else {
165                    bail!("mpk is disabled on this system")
166                }
167            }
168            MpkEnabled::Disable => &[],
169        };
170
171        // This is a tricky bit of global state: when creating a memory pool
172        // that uses memory protection keys, we ensure here that any host code
173        // will have access to all keys (i.e., stripes). It's only when we enter
174        // the WebAssembly guest code (see `StoreInner::call_hook`) that we
175        // enforce which keys/stripes can be accessed. Be forewarned about the
176        // assumptions here:
177        // - we expect this "allow all" configuration to reset the default
178        //   process state (only allow key 0) _before_ any memories are accessed
179        // - and we expect no other code (e.g., host-side code) to modify this
180        //   global MPK configuration
181        if !pkeys.is_empty() {
182            mpk::allow(ProtectionMask::all());
183        }
184
185        // Create a slab layout and allocate it as a completely inaccessible
186        // region to start--`PROT_NONE`.
187        let constraints = SlabConstraints::new(&config.limits, tunables, pkeys.len())?;
188        let layout = calculate(&constraints)?;
189        log::debug!(
190            "creating memory pool: {constraints:?} -> {layout:?} (total: {})",
191            layout.total_slab_bytes()?
192        );
193        let mut mapping =
194            Mmap::accessible_reserved(HostAlignedByteCount::ZERO, layout.total_slab_bytes()?)
195                .context("failed to create memory pool mapping")?;
196
197        // Then, stripe the memory with the available protection keys. This is
198        // unnecessary if there is only one stripe color.
199        if layout.num_stripes >= 2 {
200            let mut cursor = layout.pre_slab_guard_bytes;
201            let pkeys = &pkeys[..layout.num_stripes];
202            for i in 0..constraints.num_slots {
203                let pkey = &pkeys[i % pkeys.len()];
204                let region = unsafe {
205                    mapping.slice_mut(
206                        cursor.byte_count()..cursor.byte_count() + layout.slot_bytes.byte_count(),
207                    )
208                };
209                pkey.protect(region)?;
210                cursor = cursor
211                    .checked_add(layout.slot_bytes)
212                    .context("cursor + slot_bytes overflows")?;
213            }
214            debug_assert_eq!(
215                cursor
216                    .checked_add(layout.post_slab_guard_bytes)
217                    .context("cursor + post_slab_guard_bytes overflows")?,
218                layout.total_slab_bytes()?
219            );
220        }
221
222        let image_slots: Vec<_> = std::iter::repeat_with(|| Mutex::new(None))
223            .take(constraints.num_slots)
224            .collect();
225
226        let create_stripe = |i| {
227            let num_slots = constraints.num_slots / layout.num_stripes
228                + usize::from(constraints.num_slots % layout.num_stripes > i);
229            let allocator = ModuleAffinityIndexAllocator::new(
230                num_slots.try_into().unwrap(),
231                config.max_unused_warm_slots,
232            );
233            Stripe {
234                allocator,
235                pkey: pkeys.get(i).cloned(),
236            }
237        };
238
239        debug_assert!(layout.num_stripes > 0);
240        let stripes: Vec<_> = (0..layout.num_stripes)
241            .into_iter()
242            .map(create_stripe)
243            .collect();
244
245        let pool = Self {
246            stripes,
247            mapping: Arc::new(mapping),
248            image_slots,
249            layout,
250            memories_per_instance: usize::try_from(config.limits.max_memories_per_module).unwrap(),
251            keep_resident: HostAlignedByteCount::new_rounded_up(
252                config.linear_memory_keep_resident,
253            )?,
254            next_available_pkey: AtomicUsize::new(0),
255        };
256
257        Ok(pool)
258    }
259
260    /// Return a protection key that stores can use for requesting new
261    pub fn next_available_pkey(&self) -> Option<ProtectionKey> {
262        let index = self.next_available_pkey.fetch_add(1, Ordering::SeqCst) % self.stripes.len();
263        debug_assert!(
264            self.stripes.len() < 2 || self.stripes[index].pkey.is_some(),
265            "if we are using stripes, we cannot have an empty protection key"
266        );
267        self.stripes[index].pkey
268    }
269
270    /// Validate whether this memory pool supports the given module.
271    pub fn validate(&self, module: &Module) -> Result<()> {
272        let memories = module.num_defined_memories();
273        if memories > usize::try_from(self.memories_per_instance).unwrap() {
274            bail!(
275                "defined memories count of {} exceeds the per-instance limit of {}",
276                memories,
277                self.memories_per_instance,
278            );
279        }
280
281        for (i, memory) in module.memories.iter().skip(module.num_imported_memories) {
282            let min = memory.minimum_byte_size().with_context(|| {
283                format!(
284                    "memory index {} has a minimum byte size that cannot be represented in a u64",
285                    i.as_u32()
286                )
287            })?;
288            if min > u64::try_from(self.layout.max_memory_bytes.byte_count()).unwrap() {
289                bail!(
290                    "memory index {} has a minimum byte size of {} which exceeds the limit of {} bytes",
291                    i.as_u32(),
292                    min,
293                    self.layout.max_memory_bytes,
294                );
295            }
296            if memory.shared {
297                // FIXME(#4244): since the pooling allocator owns the memory
298                // allocation (which is torn down with the instance), that
299                // can't be used with shared memory where threads or the host
300                // might persist the memory beyond the lifetime of the instance
301                // itself.
302                bail!(
303                    "memory index {} is shared which is not supported in the pooling allocator",
304                    i.as_u32(),
305                );
306            }
307        }
308        Ok(())
309    }
310
311    /// Are zero slots in use right now?
312    #[allow(unused)] // some cfgs don't use this
313    pub fn is_empty(&self) -> bool {
314        self.stripes.iter().all(|s| s.allocator.is_empty())
315    }
316
317    /// Allocate a single memory for the given instance allocation request.
318    pub fn allocate(
319        &self,
320        request: &mut InstanceAllocationRequest,
321        ty: &wasmtime_environ::Memory,
322        tunables: &Tunables,
323        memory_index: DefinedMemoryIndex,
324    ) -> Result<(MemoryAllocationIndex, Memory)> {
325        let stripe_index = if let Some(pkey) = &request.pkey {
326            pkey.as_stripe()
327        } else {
328            debug_assert!(self.stripes.len() < 2);
329            0
330        };
331
332        let striped_allocation_index = self.stripes[stripe_index]
333            .allocator
334            .alloc(
335                request
336                    .runtime_info
337                    .unique_id()
338                    .map(|id| MemoryInModule(id, memory_index)),
339            )
340            .map(|slot| StripedAllocationIndex(u32::try_from(slot.index()).unwrap()))
341            .ok_or_else(|| {
342                super::PoolConcurrencyLimitError::new(
343                    self.stripes[stripe_index].allocator.len(),
344                    format!("memory stripe {stripe_index}"),
345                )
346            })?;
347        let allocation_index =
348            striped_allocation_index.as_unstriped_slot_index(stripe_index, self.stripes.len());
349
350        match (|| {
351            // Double-check that the runtime requirements of the memory are
352            // satisfied by the configuration of this pooling allocator. This
353            // should be returned as an error through `validate_memory_plans`
354            // but double-check here to be sure.
355            assert!(
356                tunables.memory_reservation + tunables.memory_guard_size
357                    <= u64::try_from(self.layout.bytes_to_next_stripe_slot().byte_count()).unwrap()
358            );
359
360            let base = self.get_base(allocation_index);
361            let base_capacity = self.layout.max_memory_bytes;
362
363            let mut slot = self.take_memory_image_slot(allocation_index);
364            let image = request.runtime_info.memory_image(memory_index)?;
365            let initial_size = ty
366                .minimum_byte_size()
367                .expect("min size checked in validation");
368
369            // If instantiation fails, we can propagate the error
370            // upward and drop the slot. This will cause the Drop
371            // handler to attempt to map the range with PROT_NONE
372            // memory, to reserve the space while releasing any
373            // stale mappings. The next use of this slot will then
374            // create a new slot that will try to map over
375            // this, returning errors as well if the mapping
376            // errors persist. The unmap-on-drop is best effort;
377            // if it fails, then we can still soundly continue
378            // using the rest of the pool and allowing the rest of
379            // the process to continue, because we never perform a
380            // mmap that would leave an open space for someone
381            // else to come in and map something.
382            let initial_size = usize::try_from(initial_size).unwrap();
383            slot.instantiate(initial_size, image, ty, tunables)?;
384
385            Memory::new_static(
386                ty,
387                tunables,
388                MemoryBase::Mmap(base),
389                base_capacity.byte_count(),
390                slot,
391                unsafe { &mut *request.store.get().unwrap() },
392            )
393        })() {
394            Ok(memory) => Ok((allocation_index, memory)),
395            Err(e) => {
396                self.stripes[stripe_index]
397                    .allocator
398                    .free(SlotId(striped_allocation_index.0));
399                Err(e)
400            }
401        }
402    }
403
404    /// Deallocate a previously-allocated memory.
405    ///
406    /// # Safety
407    ///
408    /// The memory must have been previously allocated from this pool and
409    /// assigned the given index, must currently be in an allocated state, and
410    /// must never be used again.
411    ///
412    /// The caller must have already called `clear_and_remain_ready` on the
413    /// memory's image and flushed any enqueued decommits for this memory.
414    pub unsafe fn deallocate(
415        &self,
416        allocation_index: MemoryAllocationIndex,
417        image: MemoryImageSlot,
418    ) {
419        self.return_memory_image_slot(allocation_index, image);
420
421        let (stripe_index, striped_allocation_index) =
422            StripedAllocationIndex::from_unstriped_slot_index(allocation_index, self.stripes.len());
423        self.stripes[stripe_index]
424            .allocator
425            .free(SlotId(striped_allocation_index.0));
426    }
427
428    /// Purging everything related to `module`.
429    pub fn purge_module(&self, module: CompiledModuleId) {
430        // This primarily means clearing out all of its memory images present in
431        // the virtual address space. Go through the index allocator for slots
432        // affine to `module` and reset them, freeing up the index when we're
433        // done.
434        //
435        // Note that this is only called when the specified `module` won't be
436        // allocated further (the module is being dropped) so this shouldn't hit
437        // any sort of infinite loop since this should be the final operation
438        // working with `module`.
439        //
440        // TODO: We are given a module id, but key affinity by pair of module id
441        // and defined memory index. We are missing any defined memory index or
442        // count of how many memories the module defines here. Therefore, we
443        // probe up to the maximum number of memories per instance. This is fine
444        // because that maximum is generally relatively small. If this method
445        // somehow ever gets hot because of unnecessary probing, we should
446        // either pass in the actual number of defined memories for the given
447        // module to this method, or keep a side table of all slots that are
448        // associated with a module (not just module and memory). The latter
449        // would require care to make sure that its maintenance wouldn't be too
450        // expensive for normal allocation/free operations.
451        for stripe in &self.stripes {
452            for i in 0..self.memories_per_instance {
453                use wasmtime_environ::EntityRef;
454                let memory_index = DefinedMemoryIndex::new(i);
455                while let Some(id) = stripe
456                    .allocator
457                    .alloc_affine_and_clear_affinity(module, memory_index)
458                {
459                    // Clear the image from the slot and, if successful, return it back
460                    // to our state. Note that on failure here the whole slot will get
461                    // paved over with an anonymous mapping.
462                    let index = MemoryAllocationIndex(id.0);
463                    let mut slot = self.take_memory_image_slot(index);
464                    if slot.remove_image().is_ok() {
465                        self.return_memory_image_slot(index, slot);
466                    }
467
468                    stripe.allocator.free(id);
469                }
470            }
471        }
472    }
473
474    fn get_base(&self, allocation_index: MemoryAllocationIndex) -> MmapOffset {
475        assert!(allocation_index.index() < self.layout.num_slots);
476        let offset = self
477            .layout
478            .slot_bytes
479            .checked_mul(allocation_index.index())
480            .and_then(|c| c.checked_add(self.layout.pre_slab_guard_bytes))
481            .expect("slot_bytes * index + pre_slab_guard_bytes overflows");
482        self.mapping.offset(offset).expect("offset is in bounds")
483    }
484
485    /// Take ownership of the given image slot. Must be returned via
486    /// `return_memory_image_slot` when the instance is done using it.
487    fn take_memory_image_slot(&self, allocation_index: MemoryAllocationIndex) -> MemoryImageSlot {
488        let maybe_slot = self.image_slots[allocation_index.index()]
489            .lock()
490            .unwrap()
491            .take();
492
493        maybe_slot.unwrap_or_else(|| {
494            MemoryImageSlot::create(
495                self.get_base(allocation_index),
496                HostAlignedByteCount::ZERO,
497                self.layout.max_memory_bytes.byte_count(),
498            )
499        })
500    }
501
502    /// Return ownership of the given image slot.
503    fn return_memory_image_slot(
504        &self,
505        allocation_index: MemoryAllocationIndex,
506        slot: MemoryImageSlot,
507    ) {
508        assert!(!slot.is_dirty());
509        *self.image_slots[allocation_index.index()].lock().unwrap() = Some(slot);
510    }
511}
512
513impl Drop for MemoryPool {
514    fn drop(&mut self) {
515        // Clear the `clear_no_drop` flag (i.e., ask to *not* clear on
516        // drop) for all slots, and then drop them here. This is
517        // valid because the one `Mmap` that covers the whole region
518        // can just do its one munmap.
519        for mut slot in std::mem::take(&mut self.image_slots) {
520            if let Some(slot) = slot.get_mut().unwrap() {
521                slot.no_clear_on_drop();
522            }
523        }
524    }
525}
526
527/// The index of a memory allocation within an `InstanceAllocator`.
528#[derive(Clone, Copy, Debug, Eq, PartialEq, PartialOrd, Ord)]
529pub struct StripedAllocationIndex(u32);
530
531impl StripedAllocationIndex {
532    fn from_unstriped_slot_index(
533        index: MemoryAllocationIndex,
534        num_stripes: usize,
535    ) -> (usize, Self) {
536        let stripe_index = index.index() % num_stripes;
537        let num_stripes: u32 = num_stripes.try_into().unwrap();
538        let index_within_stripe = Self(index.0 / num_stripes);
539        (stripe_index, index_within_stripe)
540    }
541
542    fn as_unstriped_slot_index(self, stripe: usize, num_stripes: usize) -> MemoryAllocationIndex {
543        let num_stripes: u32 = num_stripes.try_into().unwrap();
544        let stripe: u32 = stripe.try_into().unwrap();
545        MemoryAllocationIndex(self.0 * num_stripes + stripe)
546    }
547}
548
549#[derive(Clone, Debug)]
550struct SlabConstraints {
551    /// Essentially, the `static_memory_bound`: this is an assumption that the
552    /// runtime and JIT compiler make about how much space will be guarded
553    /// between slots.
554    expected_slot_bytes: HostAlignedByteCount,
555    /// The maximum size of any memory in the pool. Always a non-zero multiple
556    /// of the page size.
557    max_memory_bytes: HostAlignedByteCount,
558    num_slots: usize,
559    num_pkeys_available: usize,
560    guard_bytes: HostAlignedByteCount,
561    guard_before_slots: bool,
562}
563
564impl SlabConstraints {
565    fn new(
566        limits: &InstanceLimits,
567        tunables: &Tunables,
568        num_pkeys_available: usize,
569    ) -> Result<Self> {
570        // `memory_reservation` is the configured number of bytes for a
571        // static memory slot (see `Config::memory_reservation`); even
572        // if the memory never grows to this size (e.g., it has a lower memory
573        // maximum), codegen will assume that this unused memory is mapped
574        // `PROT_NONE`. Typically `memory_reservation` is 4GiB which helps
575        // elide most bounds checks. `MemoryPool` must respect this bound,
576        // though not explicitly: if we can achieve the same effect via
577        // MPK-protected stripes, the slot size can be lower than the
578        // `memory_reservation`.
579        let expected_slot_bytes =
580            HostAlignedByteCount::new_rounded_up_u64(tunables.memory_reservation)
581                .context("memory reservation is too large")?;
582
583        // Page-align the maximum size of memory since that's the granularity that
584        // permissions are going to be controlled at.
585        let max_memory_bytes = HostAlignedByteCount::new_rounded_up(limits.max_memory_size)
586            .context("maximum size of memory is too large")?;
587
588        let guard_bytes = HostAlignedByteCount::new_rounded_up_u64(tunables.memory_guard_size)
589            .context("guard region is too large")?;
590
591        let num_slots = limits
592            .total_memories
593            .try_into()
594            .context("too many memories")?;
595
596        let constraints = SlabConstraints {
597            max_memory_bytes,
598            num_slots,
599            expected_slot_bytes,
600            num_pkeys_available,
601            guard_bytes,
602            guard_before_slots: tunables.guard_before_linear_memory,
603        };
604        Ok(constraints)
605    }
606}
607
608#[derive(Debug)]
609struct SlabLayout {
610    /// The total number of slots available in the memory pool slab.
611    num_slots: usize,
612    /// The size of each slot in the memory pool; this contains the maximum
613    /// memory size (i.e., from WebAssembly or Wasmtime configuration) plus any
614    /// guard region after the memory to catch OOB access. On these guard
615    /// regions, note that:
616    /// - users can configure how aggressively (or not) to elide bounds checks
617    ///   via `Config::memory_guard_size` (see also:
618    ///   `memory_and_guard_size`)
619    /// - memory protection keys can compress the size of the guard region by
620    ///   placing slots from a different key (i.e., a stripe) in the guard
621    ///   region; this means the slot itself can be smaller and we can allocate
622    ///   more of them.
623    slot_bytes: HostAlignedByteCount,
624    /// The maximum size that can become accessible, in bytes, for each linear
625    /// memory. Guaranteed to be a whole number of Wasm pages.
626    max_memory_bytes: HostAlignedByteCount,
627    /// If necessary, the number of bytes to reserve as a guard region at the
628    /// beginning of the slab.
629    pre_slab_guard_bytes: HostAlignedByteCount,
630    /// Like `pre_slab_guard_bytes`, but at the end of the slab.
631    post_slab_guard_bytes: HostAlignedByteCount,
632    /// The number of stripes needed in the slab layout.
633    num_stripes: usize,
634}
635
636impl SlabLayout {
637    /// Return the total size of the slab, using the final layout (where `n =
638    /// num_slots`):
639    ///
640    /// ```text
641    /// ┌────────────────────┬──────┬──────┬───┬──────┬─────────────────────┐
642    /// │pre_slab_guard_bytes│slot 1│slot 2│...│slot n│post_slab_guard_bytes│
643    /// └────────────────────┴──────┴──────┴───┴──────┴─────────────────────┘
644    /// ```
645    fn total_slab_bytes(&self) -> Result<HostAlignedByteCount> {
646        self.slot_bytes
647            .checked_mul(self.num_slots)
648            .and_then(|c| c.checked_add(self.pre_slab_guard_bytes))
649            .and_then(|c| c.checked_add(self.post_slab_guard_bytes))
650            .context("total size of memory reservation exceeds addressable memory")
651    }
652
653    /// Returns the number of Wasm bytes from the beginning of one slot to the
654    /// next slot in the same stripe--this is the striped equivalent of
655    /// `static_memory_bound`. Recall that between slots of the same stripe we
656    /// will see a slot from every other stripe.
657    ///
658    /// For example, in a 3-stripe pool, this function measures the distance
659    /// from the beginning of slot 1 to slot 4, which are of the same stripe:
660    ///
661    /// ```text
662    ///  ◄────────────────────►
663    /// ┌────────┬──────┬──────┬────────┬───┐
664    /// │*slot 1*│slot 2│slot 3│*slot 4*│...|
665    /// └────────┴──────┴──────┴────────┴───┘
666    /// ```
667    fn bytes_to_next_stripe_slot(&self) -> HostAlignedByteCount {
668        self.slot_bytes
669            .checked_mul(self.num_stripes)
670            .expect("constructor checks that self.slot_bytes * self.num_stripes is in bounds")
671    }
672}
673
674fn calculate(constraints: &SlabConstraints) -> Result<SlabLayout> {
675    let SlabConstraints {
676        max_memory_bytes,
677        num_slots,
678        expected_slot_bytes,
679        num_pkeys_available,
680        guard_bytes,
681        guard_before_slots,
682    } = *constraints;
683
684    // If the user specifies a guard region, we always need to allocate a
685    // `PROT_NONE` region for it before any memory slots. Recall that we can
686    // avoid bounds checks for loads and stores with immediates up to
687    // `guard_bytes`, but we rely on Wasmtime to emit bounds checks for any
688    // accesses greater than this.
689    let pre_slab_guard_bytes = if guard_before_slots {
690        guard_bytes
691    } else {
692        HostAlignedByteCount::ZERO
693    };
694
695    // To calculate the slot size, we start with the default configured size and
696    // attempt to chip away at this via MPK protection. Note here how we begin
697    // to define a slot as "all of the memory and guard region."
698    let faulting_region_bytes = expected_slot_bytes
699        .max(max_memory_bytes)
700        .checked_add(guard_bytes)
701        .context("faulting region is too large")?;
702
703    let (num_stripes, slot_bytes) = if guard_bytes == 0 || max_memory_bytes == 0 || num_slots == 0 {
704        // In the uncommon case where the memory/guard regions are empty or we don't need any slots , we
705        // will not need any stripes: we just lay out the slots back-to-back
706        // using a single stripe.
707        (1, faulting_region_bytes.byte_count())
708    } else if num_pkeys_available < 2 {
709        // If we do not have enough protection keys to stripe the memory, we do
710        // the same. We can't elide any of the guard bytes because we aren't
711        // overlapping guard regions with other stripes...
712        (1, faulting_region_bytes.byte_count())
713    } else {
714        // ...but if we can create at least two stripes, we can use another
715        // stripe (i.e., a different pkey) as this slot's guard region--this
716        // reduces the guard bytes each slot has to allocate. We must make
717        // sure, though, that if the size of that other stripe(s) does not
718        // fully cover `guard_bytes`, we keep those around to prevent OOB
719        // access.
720
721        // We first calculate the number of stripes we need: we want to
722        // minimize this so that there is less chance of a single store
723        // running out of slots with its stripe--we need at least two,
724        // though. But this is not just an optimization; we need to handle
725        // the case when there are fewer slots than stripes. E.g., if our
726        // pool is configured with only three slots (`num_memory_slots =
727        // 3`), we will run into failures if we attempt to set up more than
728        // three stripes.
729        let needed_num_stripes = faulting_region_bytes
730            .checked_div(max_memory_bytes)
731            .expect("if condition above implies max_memory_bytes is non-zero")
732            + usize::from(
733                faulting_region_bytes
734                    .checked_rem(max_memory_bytes)
735                    .expect("if condition above implies max_memory_bytes is non-zero")
736                    != 0,
737            );
738        assert!(needed_num_stripes > 0);
739        let num_stripes = num_pkeys_available.min(needed_num_stripes).min(num_slots);
740
741        // Next, we try to reduce the slot size by "overlapping" the stripes: we
742        // can make slot `n` smaller since we know that slot `n+1` and following
743        // are in different stripes and will look just like `PROT_NONE` memory.
744        // Recall that codegen expects a guarantee that at least
745        // `faulting_region_bytes` will catch OOB accesses via segfaults.
746        let needed_slot_bytes = faulting_region_bytes
747            .byte_count()
748            .checked_div(num_stripes)
749            .unwrap_or(faulting_region_bytes.byte_count())
750            .max(max_memory_bytes.byte_count());
751        assert!(needed_slot_bytes >= max_memory_bytes.byte_count());
752
753        (num_stripes, needed_slot_bytes)
754    };
755
756    // The page-aligned slot size; equivalent to `memory_and_guard_size`.
757    let slot_bytes =
758        HostAlignedByteCount::new_rounded_up(slot_bytes).context("slot size is too large")?;
759
760    // We may need another guard region (like `pre_slab_guard_bytes`) at the end
761    // of our slab to maintain our `faulting_region_bytes` guarantee. We could
762    // be conservative and just create it as large as `faulting_region_bytes`,
763    // but because we know that the last slot's `slot_bytes` make up the first
764    // part of that region, we reduce the final guard region by that much.
765    let post_slab_guard_bytes = faulting_region_bytes.saturating_sub(slot_bytes);
766
767    // Check that we haven't exceeded the slab we can calculate given the limits
768    // of `usize`.
769    let layout = SlabLayout {
770        num_slots,
771        slot_bytes,
772        max_memory_bytes,
773        pre_slab_guard_bytes,
774        post_slab_guard_bytes,
775        num_stripes,
776    };
777    match layout.total_slab_bytes() {
778        Ok(_) => Ok(layout),
779        Err(e) => Err(e),
780    }
781}
782
783#[cfg(test)]
784mod tests {
785    use super::*;
786    use proptest::prelude::*;
787
788    const WASM_PAGE_SIZE: u32 = wasmtime_environ::Memory::DEFAULT_PAGE_SIZE;
789
790    #[cfg(target_pointer_width = "64")]
791    #[test]
792    fn test_memory_pool() -> Result<()> {
793        let pool = MemoryPool::new(
794            &PoolingInstanceAllocatorConfig {
795                limits: InstanceLimits {
796                    total_memories: 5,
797                    max_tables_per_module: 0,
798                    max_memories_per_module: 3,
799                    table_elements: 0,
800                    max_memory_size: WASM_PAGE_SIZE as usize,
801                    ..Default::default()
802                },
803                ..Default::default()
804            },
805            &Tunables {
806                memory_reservation: WASM_PAGE_SIZE as u64,
807                memory_guard_size: 0,
808                ..Tunables::default_host()
809            },
810        )?;
811
812        assert_eq!(pool.layout.slot_bytes, WASM_PAGE_SIZE as usize);
813        assert_eq!(pool.layout.num_slots, 5);
814        assert_eq!(pool.layout.max_memory_bytes, WASM_PAGE_SIZE as usize);
815
816        let base = pool.mapping.as_ptr() as usize;
817
818        for i in 0..5 {
819            let index = MemoryAllocationIndex(i);
820            let ptr = pool.get_base(index).as_mut_ptr();
821            assert_eq!(
822                ptr as usize - base,
823                i as usize * pool.layout.slot_bytes.byte_count()
824            );
825        }
826
827        Ok(())
828    }
829
830    #[test]
831    #[cfg_attr(miri, ignore)]
832    fn test_pooling_allocator_striping() {
833        if !mpk::is_supported() {
834            println!("skipping `test_pooling_allocator_striping` test; mpk is not supported");
835            return;
836        }
837
838        // Force the use of MPK.
839        let config = PoolingInstanceAllocatorConfig {
840            memory_protection_keys: MpkEnabled::Enable,
841            ..PoolingInstanceAllocatorConfig::default()
842        };
843        let pool = MemoryPool::new(&config, &Tunables::default_host()).unwrap();
844        assert!(pool.stripes.len() >= 2);
845
846        let max_memory_slots = config.limits.total_memories;
847        dbg!(pool.stripes[0].allocator.num_empty_slots());
848        dbg!(pool.stripes[1].allocator.num_empty_slots());
849        let available_memory_slots: usize = pool
850            .stripes
851            .iter()
852            .map(|s| s.allocator.num_empty_slots())
853            .sum();
854        assert_eq!(
855            max_memory_slots,
856            u32::try_from(available_memory_slots).unwrap()
857        );
858    }
859
860    #[test]
861    fn check_known_layout_calculations() {
862        for num_pkeys_available in 0..16 {
863            for num_memory_slots in [0, 1, 10, 64] {
864                for expected_slot_bytes in [0, 1 << 30 /* 1GB */, 4 << 30 /* 4GB */] {
865                    let expected_slot_bytes =
866                        HostAlignedByteCount::new(expected_slot_bytes).unwrap();
867                    for max_memory_bytes in
868                        [0, 1 * WASM_PAGE_SIZE as usize, 10 * WASM_PAGE_SIZE as usize]
869                    {
870                        // Note new rather than new_rounded_up here -- for now,
871                        // WASM_PAGE_SIZE is 64KiB, which is a multiple of the
872                        // host page size on all platforms.
873                        let max_memory_bytes = HostAlignedByteCount::new(max_memory_bytes).unwrap();
874                        for guard_bytes in [0, 2 << 30 /* 2GB */] {
875                            let guard_bytes = HostAlignedByteCount::new(guard_bytes).unwrap();
876                            for guard_before_slots in [true, false] {
877                                let constraints = SlabConstraints {
878                                    max_memory_bytes,
879                                    num_slots: num_memory_slots,
880                                    expected_slot_bytes,
881                                    num_pkeys_available,
882                                    guard_bytes,
883                                    guard_before_slots,
884                                };
885                                match calculate(&constraints) {
886                                    Ok(layout) => {
887                                        assert_slab_layout_invariants(constraints, layout)
888                                    }
889                                    Err(e) => {
890                                        // Only allow failure on 32-bit
891                                        // platforms where the calculation
892                                        // exceeded the size of the address
893                                        // space
894                                        assert!(
895                                            cfg!(target_pointer_width = "32")
896                                                && e.to_string()
897                                                    .contains("exceeds addressable memory"),
898                                            "bad error: {e:?}"
899                                        );
900                                    }
901                                }
902                            }
903                        }
904                    }
905                }
906            }
907        }
908    }
909
910    proptest! {
911        #[test]
912        #[cfg_attr(miri, ignore)]
913        fn check_random_layout_calculations(c in constraints()) {
914            if let Ok(l) = calculate(&c) {
915                assert_slab_layout_invariants(c, l);
916            }
917        }
918    }
919
920    fn constraints() -> impl Strategy<Value = SlabConstraints> {
921        (
922            any::<HostAlignedByteCount>(),
923            any::<usize>(),
924            any::<HostAlignedByteCount>(),
925            any::<usize>(),
926            any::<HostAlignedByteCount>(),
927            any::<bool>(),
928        )
929            .prop_map(
930                |(
931                    max_memory_bytes,
932                    num_memory_slots,
933                    expected_slot_bytes,
934                    num_pkeys_available,
935                    guard_bytes,
936                    guard_before_slots,
937                )| {
938                    SlabConstraints {
939                        max_memory_bytes,
940                        num_slots: num_memory_slots,
941                        expected_slot_bytes,
942                        num_pkeys_available,
943                        guard_bytes,
944                        guard_before_slots,
945                    }
946                },
947            )
948    }
949
950    fn assert_slab_layout_invariants(c: SlabConstraints, s: SlabLayout) {
951        // Check that all the sizes add up.
952        assert_eq!(
953            s.total_slab_bytes().unwrap(),
954            s.pre_slab_guard_bytes
955                .checked_add(s.slot_bytes.checked_mul(c.num_slots).unwrap())
956                .and_then(|c| c.checked_add(s.post_slab_guard_bytes))
957                .unwrap(),
958            "the slab size does not add up: {c:?} => {s:?}"
959        );
960        assert!(
961            s.slot_bytes >= s.max_memory_bytes,
962            "slot is not big enough: {c:?} => {s:?}"
963        );
964
965        // The HostAlignedByteCount newtype wrapper ensures that the various
966        // byte values are page-aligned.
967
968        // Check that we use no more or less stripes than needed.
969        assert!(s.num_stripes >= 1, "not enough stripes: {c:?} => {s:?}");
970        if c.num_pkeys_available == 0 || c.num_slots == 0 {
971            assert_eq!(
972                s.num_stripes, 1,
973                "expected at least one stripe: {c:?} => {s:?}"
974            );
975        } else {
976            assert!(
977                s.num_stripes <= c.num_pkeys_available,
978                "layout has more stripes than available pkeys: {c:?} => {s:?}"
979            );
980            assert!(
981                s.num_stripes <= c.num_slots,
982                "layout has more stripes than memory slots: {c:?} => {s:?}"
983            );
984        }
985
986        // Check that we use the minimum number of stripes/protection keys.
987        // - if the next MPK-protected slot is bigger or the same as the
988        //   required guard region, we only need two stripes
989        // - if the next slot is smaller than the guard region, we only need
990        //   enough stripes to add up to at least that guard region size.
991        if c.num_pkeys_available > 1 && !c.max_memory_bytes.is_zero() {
992            assert!(
993                s.num_stripes <= (c.guard_bytes.checked_div(c.max_memory_bytes).unwrap() + 2),
994                "calculated more stripes than needed: {c:?} => {s:?}"
995            );
996        }
997
998        // Check that the memory-striping will not allow OOB access.
999        // - we may have reduced the slot size from `expected_slot_bytes` to
1000        //   `slot_bytes` assuming MPK striping; we check that our guaranteed
1001        //   "faulting region" is respected
1002        // - the last slot won't have MPK striping after it; we check that the
1003        //   `post_slab_guard_bytes` accounts for this
1004        assert!(
1005            s.bytes_to_next_stripe_slot()
1006                >= c.expected_slot_bytes
1007                    .max(c.max_memory_bytes)
1008                    .checked_add(c.guard_bytes)
1009                    .unwrap(),
1010            "faulting region not large enough: {c:?} => {s:?}"
1011        );
1012        assert!(
1013            s.slot_bytes.checked_add(s.post_slab_guard_bytes).unwrap() >= c.expected_slot_bytes,
1014            "last slot may allow OOB access: {c:?} => {s:?}"
1015        );
1016    }
1017}