const-eval: full support for pointer fragments

This commit is contained in:
Ralf Jung
2025-07-17 20:00:19 +02:00
parent 051d0e8a95
commit ba5b6b9ec4
36 changed files with 543 additions and 359 deletions

View File

@@ -306,8 +306,6 @@ pub enum AllocError {
ScalarSizeMismatch(ScalarSizeMismatch),
/// Encountered a pointer where we needed raw bytes.
ReadPointerAsInt(Option<BadBytesAccess>),
/// Partially overwriting a pointer.
OverwritePartialPointer(Size),
/// Partially copying a pointer.
ReadPartialPointer(Size),
/// Using uninitialized data where it is not allowed.
@@ -331,9 +329,6 @@ impl AllocError {
ReadPointerAsInt(info) => InterpErrorKind::Unsupported(
UnsupportedOpInfo::ReadPointerAsInt(info.map(|b| (alloc_id, b))),
),
OverwritePartialPointer(offset) => InterpErrorKind::Unsupported(
UnsupportedOpInfo::OverwritePartialPointer(Pointer::new(alloc_id, offset)),
),
ReadPartialPointer(offset) => InterpErrorKind::Unsupported(
UnsupportedOpInfo::ReadPartialPointer(Pointer::new(alloc_id, offset)),
),
@@ -633,11 +628,11 @@ impl<Prov: Provenance, Extra, Bytes: AllocBytes> Allocation<Prov, Extra, Bytes>
&mut self,
cx: &impl HasDataLayout,
range: AllocRange,
) -> AllocResult<&mut [u8]> {
) -> &mut [u8] {
self.mark_init(range, true);
self.provenance.clear(range, cx)?;
self.provenance.clear(range, cx);
Ok(&mut self.bytes[range.start.bytes_usize()..range.end().bytes_usize()])
&mut self.bytes[range.start.bytes_usize()..range.end().bytes_usize()]
}
/// A raw pointer variant of `get_bytes_unchecked_for_overwrite` that avoids invalidating existing immutable aliases
@@ -646,15 +641,15 @@ impl<Prov: Provenance, Extra, Bytes: AllocBytes> Allocation<Prov, Extra, Bytes>
&mut self,
cx: &impl HasDataLayout,
range: AllocRange,
) -> AllocResult<*mut [u8]> {
) -> *mut [u8] {
self.mark_init(range, true);
self.provenance.clear(range, cx)?;
self.provenance.clear(range, cx);
assert!(range.end().bytes_usize() <= self.bytes.len()); // need to do our own bounds-check
// Crucially, we go via `AllocBytes::as_mut_ptr`, not `AllocBytes::deref_mut`.
let begin_ptr = self.bytes.as_mut_ptr().wrapping_add(range.start.bytes_usize());
let len = range.end().bytes_usize() - range.start.bytes_usize();
Ok(ptr::slice_from_raw_parts_mut(begin_ptr, len))
ptr::slice_from_raw_parts_mut(begin_ptr, len)
}
/// This gives direct mutable access to the entire buffer, just exposing their internal state
@@ -723,26 +718,45 @@ impl<Prov: Provenance, Extra, Bytes: AllocBytes> Allocation<Prov, Extra, Bytes>
let ptr = Pointer::new(prov, Size::from_bytes(bits));
return Ok(Scalar::from_pointer(ptr, cx));
}
// If we can work on pointers byte-wise, join the byte-wise provenances.
if Prov::OFFSET_IS_ADDR {
let mut prov = self.provenance.get(range.start, cx);
// The other easy case is total absence of provenance.
if self.provenance.range_empty(range, cx) {
return Ok(Scalar::from_uint(bits, range.size));
}
// If we get here, we have to check per-byte provenance, and join them together.
let prov = 'prov: {
// Initialize with first fragment. Must have index 0.
let Some((mut joint_prov, 0)) = self.provenance.get_byte(range.start, cx) else {
break 'prov None;
};
// Update with the remaining fragments.
for offset in Size::from_bytes(1)..range.size {
let this_prov = self.provenance.get(range.start + offset, cx);
prov = Prov::join(prov, this_prov);
// Ensure there is provenance here and it has the right index.
let Some((frag_prov, frag_idx)) =
self.provenance.get_byte(range.start + offset, cx)
else {
break 'prov None;
};
// Wildcard provenance is allowed to come with any index (this is needed
// for Miri's native-lib mode to work).
if u64::from(frag_idx) != offset.bytes() && Some(frag_prov) != Prov::WILDCARD {
break 'prov None;
}
// Merge this byte's provenance with the previous ones.
joint_prov = match Prov::join(joint_prov, frag_prov) {
Some(prov) => prov,
None => break 'prov None,
};
}
// Now use this provenance.
let ptr = Pointer::new(prov, Size::from_bytes(bits));
return Ok(Scalar::from_maybe_pointer(ptr, cx));
} else {
// Without OFFSET_IS_ADDR, the only remaining case we can handle is total absence of
// provenance.
if self.provenance.range_empty(range, cx) {
return Ok(Scalar::from_uint(bits, range.size));
}
// Else we have mixed provenance, that doesn't work.
break 'prov Some(joint_prov);
};
if prov.is_none() && !Prov::OFFSET_IS_ADDR {
// There are some bytes with provenance here but overall the provenance does not add up.
// We need `OFFSET_IS_ADDR` to fall back to no-provenance here; without that option, we must error.
return Err(AllocError::ReadPartialPointer(range.start));
}
// We can use this provenance.
let ptr = Pointer::new(prov, Size::from_bytes(bits));
return Ok(Scalar::from_maybe_pointer(ptr, cx));
} else {
// We are *not* reading a pointer.
// If we can just ignore provenance or there is none, that's easy.
@@ -782,7 +796,7 @@ impl<Prov: Provenance, Extra, Bytes: AllocBytes> Allocation<Prov, Extra, Bytes>
let endian = cx.data_layout().endian;
// Yes we do overwrite all the bytes in `dst`.
let dst = self.get_bytes_unchecked_for_overwrite(cx, range)?;
let dst = self.get_bytes_unchecked_for_overwrite(cx, range);
write_target_uint(endian, dst, bytes).unwrap();
// See if we have to also store some provenance.
@@ -795,10 +809,9 @@ impl<Prov: Provenance, Extra, Bytes: AllocBytes> Allocation<Prov, Extra, Bytes>
}
/// Write "uninit" to the given memory range.
pub fn write_uninit(&mut self, cx: &impl HasDataLayout, range: AllocRange) -> AllocResult {
pub fn write_uninit(&mut self, cx: &impl HasDataLayout, range: AllocRange) {
self.mark_init(range, false);
self.provenance.clear(range, cx)?;
Ok(())
self.provenance.clear(range, cx);
}
/// Mark all bytes in the given range as initialised and reset the provenance
@@ -817,9 +830,12 @@ impl<Prov: Provenance, Extra, Bytes: AllocBytes> Allocation<Prov, Extra, Bytes>
}
/// Remove all provenance in the given memory range.
pub fn clear_provenance(&mut self, cx: &impl HasDataLayout, range: AllocRange) -> AllocResult {
self.provenance.clear(range, cx)?;
return Ok(());
pub fn clear_provenance(&mut self, cx: &impl HasDataLayout, range: AllocRange) {
self.provenance.clear(range, cx);
}
pub fn provenance_merge_bytes(&mut self, cx: &impl HasDataLayout) -> bool {
self.provenance.merge_bytes(cx)
}
/// Applies a previously prepared provenance copy.

View File

@@ -10,7 +10,7 @@ use rustc_macros::HashStable;
use rustc_serialize::{Decodable, Decoder, Encodable, Encoder};
use tracing::trace;
use super::{AllocError, AllocRange, AllocResult, CtfeProvenance, Provenance, alloc_range};
use super::{AllocRange, CtfeProvenance, Provenance, alloc_range};
/// Stores the provenance information of pointers stored in memory.
#[derive(Clone, PartialEq, Eq, Hash, Debug)]
@@ -19,25 +19,25 @@ pub struct ProvenanceMap<Prov = CtfeProvenance> {
/// `Provenance` in this map applies from the given offset for an entire pointer-size worth of
/// bytes. Two entries in this map are always at least a pointer size apart.
ptrs: SortedMap<Size, Prov>,
/// Provenance in this map only applies to the given single byte.
/// This map is disjoint from the previous. It will always be empty when
/// `Prov::OFFSET_IS_ADDR` is false.
bytes: Option<Box<SortedMap<Size, Prov>>>,
/// This stores byte-sized provenance fragments.
/// The `u8` indicates the position of this byte inside its original pointer.
/// If the bytes are re-assembled in their original order, the pointer can be used again.
/// Wildcard provenance is allowed to have index 0 everywhere.
bytes: Option<Box<SortedMap<Size, (Prov, u8)>>>,
}
// These impls are generic over `Prov` since `CtfeProvenance` is only decodable/encodable
// for some particular `D`/`S`.
impl<D: Decoder, Prov: Provenance + Decodable<D>> Decodable<D> for ProvenanceMap<Prov> {
fn decode(d: &mut D) -> Self {
assert!(!Prov::OFFSET_IS_ADDR); // only `CtfeProvenance` is ever serialized
// `bytes` is not in the serialized format
Self { ptrs: Decodable::decode(d), bytes: None }
}
}
impl<S: Encoder, Prov: Provenance + Encodable<S>> Encodable<S> for ProvenanceMap<Prov> {
fn encode(&self, s: &mut S) {
let Self { ptrs, bytes } = self;
assert!(!Prov::OFFSET_IS_ADDR); // only `CtfeProvenance` is ever serialized
debug_assert!(bytes.is_none()); // without `OFFSET_IS_ADDR`, this is always empty
assert!(bytes.is_none()); // interning refuses allocations with pointer fragments
ptrs.encode(s)
}
}
@@ -58,10 +58,10 @@ impl ProvenanceMap {
/// Give access to the ptr-sized provenances (which can also be thought of as relocations, and
/// indeed that is how codegen treats them).
///
/// Only exposed with `CtfeProvenance` provenance, since it panics if there is bytewise provenance.
/// Only use on interned allocations, as other allocations may have per-byte provenance!
#[inline]
pub fn ptrs(&self) -> &SortedMap<Size, CtfeProvenance> {
debug_assert!(self.bytes.is_none()); // `CtfeProvenance::OFFSET_IS_ADDR` is false so this cannot fail
assert!(self.bytes.is_none(), "`ptrs()` called on non-interned allocation");
&self.ptrs
}
}
@@ -88,12 +88,12 @@ impl<Prov: Provenance> ProvenanceMap<Prov> {
}
/// `pm.range_ptrs_is_empty(r, cx)` == `pm.range_ptrs_get(r, cx).is_empty()`, but is faster.
pub(super) fn range_ptrs_is_empty(&self, range: AllocRange, cx: &impl HasDataLayout) -> bool {
fn range_ptrs_is_empty(&self, range: AllocRange, cx: &impl HasDataLayout) -> bool {
self.ptrs.range_is_empty(Self::adjusted_range_ptrs(range, cx))
}
/// Returns all byte-wise provenance in the given range.
fn range_bytes_get(&self, range: AllocRange) -> &[(Size, Prov)] {
fn range_bytes_get(&self, range: AllocRange) -> &[(Size, (Prov, u8))] {
if let Some(bytes) = self.bytes.as_ref() {
bytes.range(range.start..range.end())
} else {
@@ -107,19 +107,47 @@ impl<Prov: Provenance> ProvenanceMap<Prov> {
}
/// Get the provenance of a single byte.
pub fn get(&self, offset: Size, cx: &impl HasDataLayout) -> Option<Prov> {
pub fn get_byte(&self, offset: Size, cx: &impl HasDataLayout) -> Option<(Prov, u8)> {
let prov = self.range_ptrs_get(alloc_range(offset, Size::from_bytes(1)), cx);
debug_assert!(prov.len() <= 1);
if let Some(entry) = prov.first() {
// If it overlaps with this byte, it is on this byte.
debug_assert!(self.bytes.as_ref().is_none_or(|b| !b.contains_key(&offset)));
Some(entry.1)
Some((entry.1, (offset - entry.0).bytes() as u8))
} else {
// Look up per-byte provenance.
self.bytes.as_ref().and_then(|b| b.get(&offset).copied())
}
}
/// Attempt to merge per-byte provenance back into ptr chunks, if the right fragments
/// sit next to each other. Return `false` is that is not possible due to partial pointers.
pub fn merge_bytes(&mut self, cx: &impl HasDataLayout) -> bool {
let Some(bytes) = self.bytes.as_deref_mut() else {
return true;
};
let ptr_size = cx.data_layout().pointer_size();
while let Some((offset, (prov, _))) = bytes.iter().next().copied() {
// Check if this fragment starts a pointer.
let range = offset..offset + ptr_size;
let frags = bytes.range(range.clone());
if frags.len() != ptr_size.bytes_usize() {
return false;
}
for (idx, (_offset, (frag_prov, frag_idx))) in frags.iter().copied().enumerate() {
if frag_prov != prov || frag_idx != idx as u8 {
return false;
}
}
// Looks like a pointer! Move it over to the ptr provenance map.
bytes.remove_range(range);
self.ptrs.insert(offset, prov);
}
// We managed to convert everything into whole pointers.
self.bytes = None;
true
}
/// Check if there is ptr-sized provenance at the given index.
/// Does not mean anything for bytewise provenance! But can be useful as an optimization.
pub fn get_ptr(&self, offset: Size) -> Option<Prov> {
@@ -137,7 +165,7 @@ impl<Prov: Provenance> ProvenanceMap<Prov> {
/// Yields all the provenances stored in this map.
pub fn provenances(&self) -> impl Iterator<Item = Prov> {
let bytes = self.bytes.iter().flat_map(|b| b.values());
let bytes = self.bytes.iter().flat_map(|b| b.values().map(|(p, _i)| p));
self.ptrs.values().chain(bytes).copied()
}
@@ -148,16 +176,12 @@ impl<Prov: Provenance> ProvenanceMap<Prov> {
/// Removes all provenance inside the given range.
/// If there is provenance overlapping with the edges, might result in an error.
pub fn clear(&mut self, range: AllocRange, cx: &impl HasDataLayout) -> AllocResult {
pub fn clear(&mut self, range: AllocRange, cx: &impl HasDataLayout) {
let start = range.start;
let end = range.end();
// Clear the bytewise part -- this is easy.
if Prov::OFFSET_IS_ADDR {
if let Some(bytes) = self.bytes.as_mut() {
bytes.remove_range(start..end);
}
} else {
debug_assert!(self.bytes.is_none());
if let Some(bytes) = self.bytes.as_mut() {
bytes.remove_range(start..end);
}
let pointer_size = cx.data_layout().pointer_size();
@@ -168,7 +192,7 @@ impl<Prov: Provenance> ProvenanceMap<Prov> {
// Find all provenance overlapping the given range.
if self.range_ptrs_is_empty(range, cx) {
// No provenance in this range, we are done. This is the common case.
return Ok(());
return;
}
// This redoes some of the work of `range_get_ptrs_is_empty`, but this path is much
@@ -179,28 +203,20 @@ impl<Prov: Provenance> ProvenanceMap<Prov> {
// We need to handle clearing the provenance from parts of a pointer.
if first < start {
if !Prov::OFFSET_IS_ADDR {
// We can't split up the provenance into less than a pointer.
return Err(AllocError::OverwritePartialPointer(first));
}
// Insert the remaining part in the bytewise provenance.
let prov = self.ptrs[&first];
let bytes = self.bytes.get_or_insert_with(Box::default);
for offset in first..start {
bytes.insert(offset, prov);
bytes.insert(offset, (prov, (offset - first).bytes() as u8));
}
}
if last > end {
let begin_of_last = last - pointer_size;
if !Prov::OFFSET_IS_ADDR {
// We can't split up the provenance into less than a pointer.
return Err(AllocError::OverwritePartialPointer(begin_of_last));
}
// Insert the remaining part in the bytewise provenance.
let prov = self.ptrs[&begin_of_last];
let bytes = self.bytes.get_or_insert_with(Box::default);
for offset in end..last {
bytes.insert(offset, prov);
bytes.insert(offset, (prov, (offset - begin_of_last).bytes() as u8));
}
}
@@ -208,8 +224,6 @@ impl<Prov: Provenance> ProvenanceMap<Prov> {
// Since provenance do not overlap, we know that removing until `last` (exclusive) is fine,
// i.e., this will not remove any other provenance just after the ones we care about.
self.ptrs.remove_range(first..last);
Ok(())
}
/// Overwrites all provenance in the given range with wildcard provenance.
@@ -218,10 +232,6 @@ impl<Prov: Provenance> ProvenanceMap<Prov> {
///
/// Provided for usage in Miri and panics otherwise.
pub fn write_wildcards(&mut self, cx: &impl HasDataLayout, range: AllocRange) {
assert!(
Prov::OFFSET_IS_ADDR,
"writing wildcard provenance is not supported when `OFFSET_IS_ADDR` is false"
);
let wildcard = Prov::WILDCARD.unwrap();
let bytes = self.bytes.get_or_insert_with(Box::default);
@@ -229,21 +239,22 @@ impl<Prov: Provenance> ProvenanceMap<Prov> {
// Remove pointer provenances that overlap with the range, then readd the edge ones bytewise.
let ptr_range = Self::adjusted_range_ptrs(range, cx);
let ptrs = self.ptrs.range(ptr_range.clone());
if let Some((offset, prov)) = ptrs.first() {
for byte_ofs in *offset..range.start {
bytes.insert(byte_ofs, *prov);
if let Some((offset, prov)) = ptrs.first().copied() {
for byte_ofs in offset..range.start {
bytes.insert(byte_ofs, (prov, (byte_ofs - offset).bytes() as u8));
}
}
if let Some((offset, prov)) = ptrs.last() {
for byte_ofs in range.end()..*offset + cx.data_layout().pointer_size() {
bytes.insert(byte_ofs, *prov);
if let Some((offset, prov)) = ptrs.last().copied() {
for byte_ofs in range.end()..offset + cx.data_layout().pointer_size() {
bytes.insert(byte_ofs, (prov, (byte_ofs - offset).bytes() as u8));
}
}
self.ptrs.remove_range(ptr_range);
// Overwrite bytewise provenance.
for offset in range.start..range.end() {
bytes.insert(offset, wildcard);
// The fragment index does not matter for wildcard provenance.
bytes.insert(offset, (wildcard, 0));
}
}
}
@@ -253,7 +264,7 @@ impl<Prov: Provenance> ProvenanceMap<Prov> {
/// Offsets are already adjusted to the destination allocation.
pub struct ProvenanceCopy<Prov> {
dest_ptrs: Option<Box<[(Size, Prov)]>>,
dest_bytes: Option<Box<[(Size, Prov)]>>,
dest_bytes: Option<Box<[(Size, (Prov, u8))]>>,
}
impl<Prov: Provenance> ProvenanceMap<Prov> {
@@ -263,7 +274,7 @@ impl<Prov: Provenance> ProvenanceMap<Prov> {
dest: Size,
count: u64,
cx: &impl HasDataLayout,
) -> AllocResult<ProvenanceCopy<Prov>> {
) -> ProvenanceCopy<Prov> {
let shift_offset = move |idx, offset| {
// compute offset for current repetition
let dest_offset = dest + src.size * idx; // `Size` operations
@@ -301,24 +312,16 @@ impl<Prov: Provenance> ProvenanceMap<Prov> {
let mut dest_bytes_box = None;
let begin_overlap = self.range_ptrs_get(alloc_range(src.start, Size::ZERO), cx).first();
let end_overlap = self.range_ptrs_get(alloc_range(src.end(), Size::ZERO), cx).first();
if !Prov::OFFSET_IS_ADDR {
// There can't be any bytewise provenance, and we cannot split up the begin/end overlap.
if let Some(entry) = begin_overlap {
return Err(AllocError::ReadPartialPointer(entry.0));
}
if let Some(entry) = end_overlap {
return Err(AllocError::ReadPartialPointer(entry.0));
}
debug_assert!(self.bytes.is_none());
} else {
let mut bytes = Vec::new();
// We only need to go here if there is some overlap or some bytewise provenance.
if begin_overlap.is_some() || end_overlap.is_some() || self.bytes.is_some() {
let mut bytes: Vec<(Size, (Prov, u8))> = Vec::new();
// First, if there is a part of a pointer at the start, add that.
if let Some(entry) = begin_overlap {
trace!("start overlapping entry: {entry:?}");
// For really small copies, make sure we don't run off the end of the `src` range.
let entry_end = cmp::min(entry.0 + ptr_size, src.end());
for offset in src.start..entry_end {
bytes.push((offset, entry.1));
bytes.push((offset, (entry.1, (offset - entry.0).bytes() as u8)));
}
} else {
trace!("no start overlapping entry");
@@ -334,8 +337,9 @@ impl<Prov: Provenance> ProvenanceMap<Prov> {
let entry_start = cmp::max(entry.0, src.start);
for offset in entry_start..src.end() {
if bytes.last().is_none_or(|bytes_entry| bytes_entry.0 < offset) {
// The last entry, if it exists, has a lower offset than us.
bytes.push((offset, entry.1));
// The last entry, if it exists, has a lower offset than us, so we
// can add it at the end and remain sorted.
bytes.push((offset, (entry.1, (offset - entry.0).bytes() as u8)));
} else {
// There already is an entry for this offset in there! This can happen when the
// start and end range checks actually end up hitting the same pointer, so we
@@ -358,7 +362,7 @@ impl<Prov: Provenance> ProvenanceMap<Prov> {
dest_bytes_box = Some(dest_bytes.into_boxed_slice());
}
Ok(ProvenanceCopy { dest_ptrs: dest_ptrs_box, dest_bytes: dest_bytes_box })
ProvenanceCopy { dest_ptrs: dest_ptrs_box, dest_bytes: dest_bytes_box }
}
/// Applies a provenance copy.
@@ -368,14 +372,10 @@ impl<Prov: Provenance> ProvenanceMap<Prov> {
if let Some(dest_ptrs) = copy.dest_ptrs {
self.ptrs.insert_presorted(dest_ptrs.into());
}
if Prov::OFFSET_IS_ADDR {
if let Some(dest_bytes) = copy.dest_bytes
&& !dest_bytes.is_empty()
{
self.bytes.get_or_insert_with(Box::default).insert_presorted(dest_bytes.into());
}
} else {
debug_assert!(copy.dest_bytes.is_none());
if let Some(dest_bytes) = copy.dest_bytes
&& !dest_bytes.is_empty()
{
self.bytes.get_or_insert_with(Box::default).insert_presorted(dest_bytes.into());
}
}
}

View File

@@ -577,9 +577,6 @@ pub enum UnsupportedOpInfo {
//
// The variants below are only reachable from CTFE/const prop, miri will never emit them.
//
/// Overwriting parts of a pointer; without knowing absolute addresses, the resulting state
/// cannot be represented by the CTFE interpreter.
OverwritePartialPointer(Pointer<AllocId>),
/// Attempting to read or copy parts of a pointer to somewhere else; without knowing absolute
/// addresses, the resulting state cannot be represented by the CTFE interpreter.
ReadPartialPointer(Pointer<AllocId>),

View File

@@ -56,7 +56,7 @@ impl<T: HasDataLayout> PointerArithmetic for T {}
/// mostly opaque; the `Machine` trait extends it with some more operations that also have access to
/// some global state.
/// The `Debug` rendering is used to display bare provenance, and for the default impl of `fmt`.
pub trait Provenance: Copy + fmt::Debug + 'static {
pub trait Provenance: Copy + PartialEq + fmt::Debug + 'static {
/// Says whether the `offset` field of `Pointer`s with this provenance is the actual physical address.
/// - If `false`, the offset *must* be relative. This means the bytes representing a pointer are
/// different from what the Abstract Machine prescribes, so the interpreter must prevent any
@@ -79,7 +79,7 @@ pub trait Provenance: Copy + fmt::Debug + 'static {
fn get_alloc_id(self) -> Option<AllocId>;
/// Defines the 'join' of provenance: what happens when doing a pointer load and different bytes have different provenance.
fn join(left: Option<Self>, right: Option<Self>) -> Option<Self>;
fn join(left: Self, right: Self) -> Option<Self>;
}
/// The type of provenance in the compile-time interpreter.
@@ -192,8 +192,8 @@ impl Provenance for CtfeProvenance {
Some(self.alloc_id())
}
fn join(_left: Option<Self>, _right: Option<Self>) -> Option<Self> {
panic!("merging provenance is not supported when `OFFSET_IS_ADDR` is false")
fn join(left: Self, right: Self) -> Option<Self> {
if left == right { Some(left) } else { None }
}
}
@@ -224,8 +224,8 @@ impl Provenance for AllocId {
Some(self)
}
fn join(_left: Option<Self>, _right: Option<Self>) -> Option<Self> {
panic!("merging provenance is not supported when `OFFSET_IS_ADDR` is false")
fn join(_left: Self, _right: Self) -> Option<Self> {
unreachable!()
}
}

View File

@@ -1826,7 +1826,7 @@ pub fn write_allocation_bytes<'tcx, Prov: Provenance, Extra, Bytes: AllocBytes>(
ascii.push('╼');
i += ptr_size;
}
} else if let Some(prov) = alloc.provenance().get(i, &tcx) {
} else if let Some((prov, idx)) = alloc.provenance().get_byte(i, &tcx) {
// Memory with provenance must be defined
assert!(
alloc.init_mask().is_range_initialized(alloc_range(i, Size::from_bytes(1))).is_ok()
@@ -1836,7 +1836,7 @@ pub fn write_allocation_bytes<'tcx, Prov: Provenance, Extra, Bytes: AllocBytes>(
// Format is similar to "oversized" above.
let j = i.bytes_usize();
let c = alloc.inspect_with_uninit_and_ptr_outside_interpreter(j..j + 1)[0];
write!(w, "╾{c:02x}{prov:#?} (1 ptr byte)╼")?;
write!(w, "╾{c:02x}{prov:#?} (ptr fragment {idx})╼")?;
i += Size::from_bytes(1);
} else if alloc
.init_mask()