1//! Inplace iterate-and-collect specialization for `Vec`
2//!
3//! Note: This documents Vec internals, some of the following sections explain implementation
4//! details and are best read together with the source of this module.
5//!
6//! The specialization in this module applies to iterators in the shape of
7//! `source.adapter().adapter().adapter().collect::<Vec<U>>()`
8//! where `source` is an owning iterator obtained from [`Vec<T>`], [`Box<[T]>`][box] (by conversion to `Vec`)
9//! or [`BinaryHeap<T>`], the adapters guarantee to consume enough items per step to make room
10//! for the results (represented by [`InPlaceIterable`]), provide transitive access to `source`
11//! (via [`SourceIter`]) and thus the underlying allocation.
12//! And finally there are alignment and size constraints to consider, this is currently ensured via
13//! const eval instead of trait bounds in the specialized [`SpecFromIter`] implementation.
14//!
15//! [`BinaryHeap<T>`]: crate::collections::BinaryHeap
16//! [box]: crate::boxed::Box
17//!
18//! By extension some other collections which use `collect::<Vec<_>>()` internally in their
19//! `FromIterator` implementation benefit from this too.
20//!
21//! Access to the underlying source goes through a further layer of indirection via the private
22//! trait [`AsVecIntoIter`] to hide the implementation detail that other collections may use
23//! `vec::IntoIter` internally.
24//!
25//! In-place iteration depends on the interaction of several unsafe traits, implementation
26//! details of multiple parts in the iterator pipeline and often requires holistic reasoning
27//! across multiple structs since iterators are executed cooperatively rather than having
28//! a central evaluator/visitor struct executing all iterator components.
29//!
30//! # Reading from and writing to the same allocation
31//!
32//! By its nature collecting in place means that the reader and writer side of the iterator
33//! use the same allocation. Since `try_fold()` (used in [`SpecInPlaceCollect`]) takes a
34//! reference to the iterator for the duration of the iteration that means we can't interleave
35//! the step of reading a value and getting a reference to write to. Instead raw pointers must be
36//! used on the reader and writer side.
37//!
38//! That writes never clobber a yet-to-be-read items is ensured by the [`InPlaceIterable`] requirements.
39//!
40//! # Layout constraints
41//!
42//! When recycling an allocation between different types we must uphold the [`Allocator`] contract
43//! which means that the input and output Layouts have to "fit".
44//!
45//! To complicate things further `InPlaceIterable` supports splitting or merging items into smaller/
46//! larger ones to enable (de)aggregation of arrays.
47//!
48//! Ultimately each step of the iterator must free up enough *bytes* in the source to make room
49//! for the next output item.
50//! If `T` and `U` have the same size no fixup is needed.
51//! If `T`'s size is a multiple of `U`'s we can compensate by multiplying the capacity accordingly.
52//! Otherwise the input capacity (and thus layout) in bytes may not be representable by the output
53//! `Vec<U>`. In that case `alloc.shrink()` is used to update the allocation's layout.
54//!
55//! Alignments of `T` must be the same or larger than `U`. Since alignments are always a power
56//! of two _larger_ implies _is a multiple of_.
57//!
58//! See `in_place_collectible()` for the current conditions.
59//!
60//! Additionally this specialization doesn't make sense for ZSTs as there is no reallocation to
61//! avoid and it would make pointer arithmetic more difficult.
62//!
63//! [`Allocator`]: core::alloc::Allocator
64//!
65//! # Drop- and panic-safety
66//!
67//! Iteration can panic, requiring dropping the already written parts but also the remainder of
68//! the source. Iteration can also leave some source items unconsumed which must be dropped.
69//! All those drops in turn can panic which then must either leak the allocation or abort to avoid
70//! double-drops.
71//!
72//! This is handled by the [`InPlaceDrop`] guard for sink items (`U`) and by
73//! [`vec::IntoIter::forget_allocation_drop_remaining()`] for remaining source items (`T`).
74//!
75//! If dropping any remaining source item (`T`) panics then [`InPlaceDstDataSrcBufDrop`] will handle dropping
76//! the already collected sink items (`U`) and freeing the allocation.
77//!
78//! [`vec::IntoIter::forget_allocation_drop_remaining()`]: super::IntoIter::forget_allocation_drop_remaining()
79//!
80//! # O(1) collect
81//!
82//! The main iteration itself is further specialized when the iterator implements
83//! [`TrustedRandomAccessNoCoerce`] to let the optimizer see that it is a counted loop with a single
84//! [induction variable]. This can turn some iterators into a noop, i.e. it reduces them from O(n) to
85//! O(1). This particular optimization is quite fickle and doesn't always work, see [#79308]
86//!
87//! [#79308]: https://github.com/rust-lang/rust/issues/79308
88//! [induction variable]: https://en.wikipedia.org/wiki/Induction_variable
89//!
90//! Since unchecked accesses through that trait do not advance the read pointer of `IntoIter`
91//! this would interact unsoundly with the requirements about dropping the tail described above.
92//! But since the normal `Drop` implementation of `IntoIter` would suffer from the same problem it
93//! is only correct for `TrustedRandomAccessNoCoerce` to be implemented when the items don't
94//! have a destructor. Thus that implicit requirement also makes the specialization safe to use for
95//! in-place collection.
96//! Note that this safety concern is about the correctness of `impl Drop for IntoIter`,
97//! not the guarantees of `InPlaceIterable`.
98//!
99//! # Adapter implementations
100//!
101//! The invariants for adapters are documented in [`SourceIter`] and [`InPlaceIterable`], but
102//! getting them right can be rather subtle for multiple, sometimes non-local reasons.
103//! For example `InPlaceIterable` would be valid to implement for [`Peekable`], except
104//! that it is stateful, cloneable and `IntoIter`'s clone implementation shortens the underlying
105//! allocation which means if the iterator has been peeked and then gets cloned there no longer is
106//! enough room, thus breaking an invariant ([#85322]).
107//!
108//! [#85322]: https://github.com/rust-lang/rust/issues/85322
109//! [`Peekable`]: core::iter::Peekable
110//!
111//!
112//! # Examples
113//!
114//! Some cases that are optimized by this specialization, more can be found in the `Vec`
115//! benchmarks:
116//!
117//! ```rust
118//! # #[allow(dead_code)]
119//! /// Converts a usize vec into an isize one.
120//! pub fn cast(vec: Vec<usize>) -> Vec<isize> {
121//! // Does not allocate, free or panic. On optlevel>=2 it does not loop.
122//! // Of course this particular case could and should be written with `into_raw_parts` and
123//! // `from_raw_parts` instead.
124//! vec.into_iter().map(|u| u as isize).collect()
125//! }
126//! ```
127//!
128//! ```rust
129//! # #[allow(dead_code)]
130//! /// Drops remaining items in `src` and if the layouts of `T` and `U` match it
131//! /// returns an empty Vec backed by the original allocation. Otherwise it returns a new
132//! /// empty vec.
133//! pub fn recycle_allocation<T, U>(src: Vec<T>) -> Vec<U> {
134//! src.into_iter().filter_map(|_| None).collect()
135//! }
136//! ```
137//!
138//! ```rust
139//! let vec = vec![13usize; 1024];
140//! let _ = vec.into_iter()
141//! .enumerate()
142//! .filter_map(|(idx, val)| if idx % 2 == 0 { Some(val+idx) } else {None})
143//! .collect::<Vec<_>>();
144//!
145//! // is equivalent to the following, but doesn't require bounds checks
146//!
147//! let mut vec = vec![13usize; 1024];
148//! let mut write_idx = 0;
149//! for idx in 0..vec.len() {
150//! if idx % 2 == 0 {
151//! vec[write_idx] = vec[idx] + idx;
152//! write_idx += 1;
153//! }
154//! }
155//! vec.truncate(write_idx);
156//! ```
157use crate::alloc::{handle_alloc_error, Global};
158use core::alloc::Allocator;
159use core::alloc::Layout;
160use core::iter::{InPlaceIterable, SourceIter, TrustedRandomAccessNoCoerce};
161use core::marker::PhantomData;
162use core::mem::{self, ManuallyDrop, SizedTypeProperties};
163use core::num::NonZero;
164use core::ptr;
165
166use super::{InPlaceDrop, InPlaceDstDataSrcBufDrop, SpecFromIter, SpecFromIterNested, Vec};
167
168const fn in_place_collectible<DEST, SRC>(
169 step_merge: Option<NonZero<usize>>,
170 step_expand: Option<NonZero<usize>>,
171) -> bool {
172 // Require matching alignments because an alignment-changing realloc is inefficient on many
173 // system allocators and better implementations would require the unstable Allocator trait.
174 if const { SRC::IS_ZST || DEST::IS_ZST || mem::align_of::<SRC>() != mem::align_of::<DEST>() } {
175 return false;
176 }
177
178 match (step_merge, step_expand) {
179 (Some(step_merge: NonZero), Some(step_expand: NonZero)) => {
180 // At least N merged source items -> at most M expanded destination items
181 // e.g.
182 // - 1 x [u8; 4] -> 4x u8, via flatten
183 // - 4 x u8 -> 1x [u8; 4], via array_chunks
184 mem::size_of::<SRC>() * step_merge.get() >= mem::size_of::<DEST>() * step_expand.get()
185 }
186 // Fall back to other from_iter impls if an overflow occurred in the step merge/expansion
187 // tracking.
188 _ => false,
189 }
190}
191
192const fn needs_realloc<SRC, DEST>(src_cap: usize, dst_cap: usize) -> bool {
193 if const { mem::align_of::<SRC>() != mem::align_of::<DEST>() } {
194 // FIXME: use unreachable! once that works in const
195 panic!("in_place_collectible() prevents this");
196 }
197
198 // If src type size is an integer multiple of the destination type size then
199 // the caller will have calculated a `dst_cap` that is an integer multiple of
200 // `src_cap` without remainder.
201 if const {
202 let src_sz: usize = mem::size_of::<SRC>();
203 let dest_sz: usize = mem::size_of::<DEST>();
204 dest_sz != 0 && src_sz % dest_sz == 0
205 } {
206 return false;
207 }
208
209 // type layouts don't guarantee a fit, so do a runtime check to see if
210 // the allocations happen to match
211 return src_cap > 0 && src_cap * mem::size_of::<SRC>() != dst_cap * mem::size_of::<DEST>();
212}
213
214/// This provides a shorthand for the source type since local type aliases aren't a thing.
215#[rustc_specialization_trait]
216trait InPlaceCollect: SourceIter<Source: AsVecIntoIter> + InPlaceIterable {
217 type Src;
218}
219
220impl<T> InPlaceCollect for T
221where
222 T: SourceIter<Source: AsVecIntoIter> + InPlaceIterable,
223{
224 type Src = <<T as SourceIter>::Source as AsVecIntoIter>::Item;
225}
226
227impl<T, I> SpecFromIter<T, I> for Vec<T>
228where
229 I: Iterator<Item = T> + InPlaceCollect,
230 <I as SourceIter>::Source: AsVecIntoIter,
231{
232 default fn from_iter(iterator: I) -> Self {
233 // Select the implementation in const eval to avoid codegen of the dead branch to improve compile times.
234 let fun: fn(I) -> Vec<T> = const {
235 // See "Layout constraints" section in the module documentation. We use const conditions here
236 // since these conditions currently cannot be expressed as trait bounds
237 if in_place_collectible::<T, I::Src>(I::MERGE_BY, I::EXPAND_BY) {
238 from_iter_in_place
239 } else {
240 // fallback
241 SpecFromIterNested::<T, I>::from_iter
242 }
243 };
244
245 fun(iterator)
246 }
247}
248
249fn from_iter_in_place<I, T>(mut iterator: I) -> Vec<T>
250where
251 I: Iterator<Item = T> + InPlaceCollect,
252 <I as SourceIter>::Source: AsVecIntoIter,
253{
254 let (src_buf, src_ptr, src_cap, mut dst_buf, dst_end, dst_cap) = unsafe {
255 let inner = iterator.as_inner().as_into_iter();
256 (
257 inner.buf,
258 inner.ptr,
259 inner.cap,
260 inner.buf.cast::<T>(),
261 inner.end as *const T,
262 inner.cap * mem::size_of::<I::Src>() / mem::size_of::<T>(),
263 )
264 };
265
266 // SAFETY: `dst_buf` and `dst_end` are the start and end of the buffer.
267 let len = unsafe {
268 SpecInPlaceCollect::collect_in_place(&mut iterator, dst_buf.as_ptr() as *mut T, dst_end)
269 };
270
271 let src = unsafe { iterator.as_inner().as_into_iter() };
272 // check if SourceIter contract was upheld
273 // caveat: if they weren't we might not even make it to this point
274 debug_assert_eq!(src_buf, src.buf);
275 // check InPlaceIterable contract. This is only possible if the iterator advanced the
276 // source pointer at all. If it uses unchecked access via TrustedRandomAccess
277 // then the source pointer will stay in its initial position and we can't use it as reference
278 if src.ptr != src_ptr {
279 debug_assert!(
280 unsafe { dst_buf.add(len).cast() } <= src.ptr,
281 "InPlaceIterable contract violation, write pointer advanced beyond read pointer"
282 );
283 }
284
285 // The ownership of the source allocation and the new `T` values is temporarily moved into `dst_guard`.
286 // This is safe because
287 // * `forget_allocation_drop_remaining` immediately forgets the allocation
288 // before any panic can occur in order to avoid any double free, and then proceeds to drop
289 // any remaining values at the tail of the source.
290 // * the shrink either panics without invalidating the allocation, aborts or
291 // succeeds. In the last case we disarm the guard.
292 //
293 // Note: This access to the source wouldn't be allowed by the TrustedRandomIteratorNoCoerce
294 // contract (used by SpecInPlaceCollect below). But see the "O(1) collect" section in the
295 // module documentation why this is ok anyway.
296 let dst_guard =
297 InPlaceDstDataSrcBufDrop { ptr: dst_buf, len, src_cap, src: PhantomData::<I::Src> };
298 src.forget_allocation_drop_remaining();
299
300 // Adjust the allocation if the source had a capacity in bytes that wasn't a multiple
301 // of the destination type size.
302 // Since the discrepancy should generally be small this should only result in some
303 // bookkeeping updates and no memmove.
304 if needs_realloc::<I::Src, T>(src_cap, dst_cap) {
305 let alloc = Global;
306 debug_assert_ne!(src_cap, 0);
307 debug_assert_ne!(dst_cap, 0);
308 unsafe {
309 // The old allocation exists, therefore it must have a valid layout.
310 let src_align = mem::align_of::<I::Src>();
311 let src_size = mem::size_of::<I::Src>().unchecked_mul(src_cap);
312 let old_layout = Layout::from_size_align_unchecked(src_size, src_align);
313
314 // The allocation must be equal or smaller for in-place iteration to be possible
315 // therefore the new layout must be ≤ the old one and therefore valid.
316 let dst_align = mem::align_of::<T>();
317 let dst_size = mem::size_of::<T>().unchecked_mul(dst_cap);
318 let new_layout = Layout::from_size_align_unchecked(dst_size, dst_align);
319
320 let result = alloc.shrink(dst_buf.cast(), old_layout, new_layout);
321 let Ok(reallocated) = result else { handle_alloc_error(new_layout) };
322 dst_buf = reallocated.cast::<T>();
323 }
324 } else {
325 debug_assert_eq!(src_cap * mem::size_of::<I::Src>(), dst_cap * mem::size_of::<T>());
326 }
327
328 mem::forget(dst_guard);
329
330 let vec = unsafe { Vec::from_nonnull(dst_buf, len, dst_cap) };
331
332 vec
333}
334
335fn write_in_place_with_drop<T>(
336 src_end: *const T,
337) -> impl FnMut(InPlaceDrop<T>, T) -> Result<InPlaceDrop<T>, !> {
338 move |mut sink: InPlaceDrop, item: T| {
339 unsafe {
340 // the InPlaceIterable contract cannot be verified precisely here since
341 // try_fold has an exclusive reference to the source pointer
342 // all we can do is check if it's still in range
343 debug_assert!(sink.dst as *const _ <= src_end, "InPlaceIterable contract violation");
344 ptr::write(sink.dst, src:item);
345 // Since this executes user code which can panic we have to bump the pointer
346 // after each step.
347 sink.dst = sink.dst.add(count:1);
348 }
349 Ok(sink)
350 }
351}
352
353/// Helper trait to hold specialized implementations of the in-place iterate-collect loop
354trait SpecInPlaceCollect<T, I>: Iterator<Item = T> {
355 /// Collects an iterator (`self`) into the destination buffer (`dst`) and returns the number of items
356 /// collected. `end` is the last writable element of the allocation and used for bounds checks.
357 ///
358 /// This method is specialized and one of its implementations makes use of
359 /// `Iterator::__iterator_get_unchecked` calls with a `TrustedRandomAccessNoCoerce` bound
360 /// on `I` which means the caller of this method must take the safety conditions
361 /// of that trait into consideration.
362 unsafe fn collect_in_place(&mut self, dst: *mut T, end: *const T) -> usize;
363}
364
365impl<T, I> SpecInPlaceCollect<T, I> for I
366where
367 I: Iterator<Item = T>,
368{
369 #[inline]
370 default unsafe fn collect_in_place(&mut self, dst_buf: *mut T, end: *const T) -> usize {
371 // use try-fold since
372 // - it vectorizes better for some iterator adapters
373 // - unlike most internal iteration methods, it only takes a &mut self
374 // - it lets us thread the write pointer through its innards and get it back in the end
375 let sink: InPlaceDrop = InPlaceDrop { inner: dst_buf, dst: dst_buf };
376 let sink: InPlaceDrop =
377 self.try_fold::<_, _, Result<_, !>>(init:sink, f:write_in_place_with_drop(src_end:end)).unwrap();
378 // iteration succeeded, don't drop head
379 unsafe { ManuallyDrop::new(sink).dst.sub_ptr(origin:dst_buf) }
380 }
381}
382
383impl<T, I> SpecInPlaceCollect<T, I> for I
384where
385 I: Iterator<Item = T> + TrustedRandomAccessNoCoerce,
386{
387 #[inline]
388 unsafe fn collect_in_place(&mut self, dst_buf: *mut T, end: *const T) -> usize {
389 let len: usize = self.size();
390 let mut drop_guard: InPlaceDrop = InPlaceDrop { inner: dst_buf, dst: dst_buf };
391 for i: usize in 0..len {
392 // Safety: InplaceIterable contract guarantees that for every element we read
393 // one slot in the underlying storage will have been freed up and we can immediately
394 // write back the result.
395 unsafe {
396 let dst: *mut T = dst_buf.add(count:i);
397 debug_assert!(dst as *const _ <= end, "InPlaceIterable contract violation");
398 ptr::write(dst, self.__iterator_get_unchecked(_idx:i));
399 // Since this executes user code which can panic we have to bump the pointer
400 // after each step.
401 drop_guard.dst = dst.add(count:1);
402 }
403 }
404 mem::forget(drop_guard);
405 len
406 }
407}
408
409/// Internal helper trait for in-place iteration specialization.
410///
411/// Currently this is only implemented by [`vec::IntoIter`] - returning a reference to itself - and
412/// [`binary_heap::IntoIter`] which returns a reference to its inner representation.
413///
414/// Since this is an internal trait it hides the implementation detail `binary_heap::IntoIter`
415/// uses `vec::IntoIter` internally.
416///
417/// [`vec::IntoIter`]: super::IntoIter
418/// [`binary_heap::IntoIter`]: crate::collections::binary_heap::IntoIter
419///
420/// # Safety
421///
422/// In-place iteration relies on implementation details of `vec::IntoIter`, most importantly that
423/// it does not create references to the whole allocation during iteration, only raw pointers
424#[rustc_specialization_trait]
425pub(crate) unsafe trait AsVecIntoIter {
426 type Item;
427 fn as_into_iter(&mut self) -> &mut super::IntoIter<Self::Item>;
428}
429