Refactor command application for more consistency (#13249)

# Objective

- Prevent the case where a hook/observer is triggered but the source
entity/component no longer exists

## Solution

- Re-order command application such that all hooks/observers that are
notified will run before any have a chance to invalidate the result.

## Testing
Updated relevant tests in `bevy_ecs`, all other tests pass.

---------

Co-authored-by: Carter Anderson <mcanders1@gmail.com>
Co-authored-by: Mike Hsu <mike.hsu@gmail.com>
Co-authored-by: Alice Cecile <alice.i.cecile@gmail.com>
This commit is contained in:
James O'Brien 2024-05-28 05:17:57 -07:00 committed by GitHub
parent cdc605cc48
commit bc102d41de
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
6 changed files with 329 additions and 149 deletions

View file

@ -1205,14 +1205,14 @@ mod tests {
.register_component_hooks::<A>()
.on_add(|mut world, entity, _| {
world.resource_mut::<R>().assert_order(0);
world.commands().entity(entity).insert(B).insert(D);
world.commands().entity(entity).insert(B).insert(C);
});
world
.register_component_hooks::<B>()
.on_add(|mut world, entity, _| {
world.resource_mut::<R>().assert_order(1);
world.commands().entity(entity).insert(C);
world.commands().entity(entity).insert(D);
});
world

View file

@ -7,9 +7,9 @@ use crate::{
component::ComponentId,
entity::{Entities, Entity},
system::{RunSystemWithInput, SystemId},
world::command_queue::RawCommandQueue,
world::{Command, CommandQueue, EntityWorldMut, FromWorld, World},
};
use bevy_ecs_macros::SystemParam;
use bevy_utils::tracing::{error, info};
pub use parallel_scope::*;
use std::marker::PhantomData;
@ -65,12 +65,84 @@ use std::marker::PhantomData;
/// ```
///
/// [`apply_deferred`]: crate::schedule::apply_deferred
#[derive(SystemParam)]
pub struct Commands<'w, 's> {
queue: Deferred<'s, CommandQueue>,
queue: InternalQueue<'s>,
entities: &'w Entities,
}
const _: () = {
type __StructFieldsAlias<'w, 's> = (Deferred<'s, CommandQueue>, &'w Entities);
#[doc(hidden)]
pub struct FetchState {
state: <__StructFieldsAlias<'static, 'static> as bevy_ecs::system::SystemParam>::State,
}
// SAFETY: Only reads Entities
unsafe impl bevy_ecs::system::SystemParam for Commands<'_, '_> {
type State = FetchState;
type Item<'w, 's> = Commands<'w, 's>;
fn init_state(
world: &mut bevy_ecs::world::World,
system_meta: &mut bevy_ecs::system::SystemMeta,
) -> Self::State {
FetchState {
state: <__StructFieldsAlias<'_, '_> as bevy_ecs::system::SystemParam>::init_state(
world,
system_meta,
),
}
}
unsafe fn new_archetype(
state: &mut Self::State,
archetype: &bevy_ecs::archetype::Archetype,
system_meta: &mut bevy_ecs::system::SystemMeta,
) {
// SAFETY: Caller guarantees the archetype is from the world used in `init_state`
unsafe {
<__StructFieldsAlias<'_, '_> as bevy_ecs::system::SystemParam>::new_archetype(
&mut state.state,
archetype,
system_meta,
);
};
}
fn apply(
state: &mut Self::State,
system_meta: &bevy_ecs::system::SystemMeta,
world: &mut bevy_ecs::world::World,
) {
<__StructFieldsAlias<'_, '_> as bevy_ecs::system::SystemParam>::apply(
&mut state.state,
system_meta,
world,
);
}
unsafe fn get_param<'w, 's>(
state: &'s mut Self::State,
system_meta: &bevy_ecs::system::SystemMeta,
world: bevy_ecs::world::unsafe_world_cell::UnsafeWorldCell<'w>,
change_tick: bevy_ecs::component::Tick,
) -> Self::Item<'w, 's> {
let(f0,f1,) = <(Deferred<'s,CommandQueue> , &'w Entities,)as bevy_ecs::system::SystemParam> ::get_param(&mut state.state,system_meta,world,change_tick);
Commands {
queue: InternalQueue::CommandQueue(f0),
entities: f1,
}
}
}
// SAFETY: Only reads Entities
unsafe impl<'w, 's> bevy_ecs::system::ReadOnlySystemParam for Commands<'w, 's>
where
Deferred<'s, CommandQueue>: bevy_ecs::system::ReadOnlySystemParam,
&'w Entities: bevy_ecs::system::ReadOnlySystemParam,
{
}
};
enum InternalQueue<'s> {
CommandQueue(Deferred<'s, CommandQueue>),
RawCommandQueue(RawCommandQueue),
}
impl<'w, 's> Commands<'w, 's> {
/// Returns a new `Commands` instance from a [`CommandQueue`] and a [`World`].
///
@ -88,7 +160,24 @@ impl<'w, 's> Commands<'w, 's> {
/// [system parameter]: crate::system::SystemParam
pub fn new_from_entities(queue: &'s mut CommandQueue, entities: &'w Entities) -> Self {
Self {
queue: Deferred(queue),
queue: InternalQueue::CommandQueue(Deferred(queue)),
entities,
}
}
/// Returns a new `Commands` instance from a [`RawCommandQueue`] and an [`Entities`] reference.
///
/// This is used when constructing [`Commands`] from a [`DeferredWorld`](crate::world::DeferredWorld).
///
/// # Safety
///
/// * Caller ensures that `queue` must outlive 'w
pub(crate) unsafe fn new_raw_from_entities(
queue: RawCommandQueue,
entities: &'w Entities,
) -> Self {
Self {
queue: InternalQueue::RawCommandQueue(queue),
entities,
}
}
@ -113,14 +202,25 @@ impl<'w, 's> Commands<'w, 's> {
/// ```
pub fn reborrow(&mut self) -> Commands<'w, '_> {
Commands {
queue: self.queue.reborrow(),
queue: match &mut self.queue {
InternalQueue::CommandQueue(queue) => InternalQueue::CommandQueue(queue.reborrow()),
InternalQueue::RawCommandQueue(queue) => {
InternalQueue::RawCommandQueue(queue.clone())
}
},
entities: self.entities,
}
}
/// Take all commands from `other` and append them to `self`, leaving `other` empty
pub fn append(&mut self, other: &mut CommandQueue) {
self.queue.append(other);
match &mut self.queue {
InternalQueue::CommandQueue(queue) => queue.bytes.append(&mut other.bytes),
InternalQueue::RawCommandQueue(queue) => {
// SAFETY: Pointers in `RawCommandQueue` are never null
unsafe { queue.bytes.as_mut() }.append(&mut other.bytes);
}
}
}
/// Pushes a [`Command`] to the queue for creating a new empty [`Entity`],
@ -381,7 +481,23 @@ impl<'w, 's> Commands<'w, 's> {
I: IntoIterator + Send + Sync + 'static,
I::Item: Bundle,
{
self.queue.push(spawn_batch(bundles_iter));
self.push(spawn_batch(bundles_iter));
}
/// Push a [`Command`] onto the queue.
pub fn push<C: Command>(&mut self, command: C) {
match &mut self.queue {
InternalQueue::CommandQueue(queue) => {
queue.push(command);
}
InternalQueue::RawCommandQueue(queue) => {
// SAFETY: `RawCommandQueue` is only every constructed in `Commands::new_raw_from_entities`
// where the caller of that has ensured that `queue` outlives `self`
unsafe {
queue.push(command);
}
}
}
}
/// Pushes a [`Command`] to the queue for creating entities, if needed,
@ -410,7 +526,7 @@ impl<'w, 's> Commands<'w, 's> {
I: IntoIterator<Item = (Entity, B)> + Send + Sync + 'static,
B: Bundle,
{
self.queue.push(insert_or_spawn_batch(bundles_iter));
self.push(insert_or_spawn_batch(bundles_iter));
}
/// Pushes a [`Command`] to the queue for inserting a [`Resource`] in the [`World`] with an inferred value.
@ -438,7 +554,7 @@ impl<'w, 's> Commands<'w, 's> {
/// # bevy_ecs::system::assert_is_system(initialise_scoreboard);
/// ```
pub fn init_resource<R: Resource + FromWorld>(&mut self) {
self.queue.push(init_resource::<R>);
self.push(init_resource::<R>);
}
/// Pushes a [`Command`] to the queue for inserting a [`Resource`] in the [`World`] with a specific value.
@ -467,7 +583,7 @@ impl<'w, 's> Commands<'w, 's> {
/// # bevy_ecs::system::assert_is_system(system);
/// ```
pub fn insert_resource<R: Resource>(&mut self, resource: R) {
self.queue.push(insert_resource(resource));
self.push(insert_resource(resource));
}
/// Pushes a [`Command`] to the queue for removing a [`Resource`] from the [`World`].
@ -491,7 +607,7 @@ impl<'w, 's> Commands<'w, 's> {
/// # bevy_ecs::system::assert_is_system(system);
/// ```
pub fn remove_resource<R: Resource>(&mut self) {
self.queue.push(remove_resource::<R>);
self.push(remove_resource::<R>);
}
/// Runs the system corresponding to the given [`SystemId`].
@ -517,8 +633,7 @@ impl<'w, 's> Commands<'w, 's> {
/// execution of the system happens later. To get the output of a system, use
/// [`World::run_system`] or [`World::run_system_with_input`] instead of running the system as a command.
pub fn run_system_with_input<I: 'static + Send>(&mut self, id: SystemId<I>, input: I) {
self.queue
.push(RunSystemWithInput::new_with_input(id, input));
self.push(RunSystemWithInput::new_with_input(id, input));
}
/// Registers a system and returns a [`SystemId`] so it can later be called by [`World::run_system`].
@ -580,7 +695,7 @@ impl<'w, 's> Commands<'w, 's> {
system: S,
) -> SystemId<I, O> {
let entity = self.spawn_empty().id();
self.queue.push(RegisterSystem::new(system, entity));
self.push(RegisterSystem::new(system, entity));
SystemId::from_entity(entity)
}
@ -618,7 +733,7 @@ impl<'w, 's> Commands<'w, 's> {
/// # bevy_ecs::system::assert_is_system(add_twenty_five_to_counter_system);
/// ```
pub fn add<C: Command>(&mut self, command: C) {
self.queue.push(command);
self.push(command);
}
}

View file

@ -1,6 +1,10 @@
use crate::system::{SystemBuffer, SystemMeta};
use std::{fmt::Debug, mem::MaybeUninit};
use std::{
fmt::Debug,
mem::MaybeUninit,
ptr::{addr_of_mut, NonNull},
};
use bevy_ptr::{OwningPtr, Unaligned};
use bevy_utils::tracing::warn;
@ -13,9 +17,12 @@ struct CommandMeta {
///
/// `world` is optional to allow this one function pointer to perform double-duty as a drop.
///
/// Returns the size of `T` in bytes.
consume_command_and_get_size:
unsafe fn(value: OwningPtr<Unaligned>, world: Option<&mut World>) -> usize,
/// Advances `cursor` by the size of `T` in bytes.
consume_command_and_get_size: unsafe fn(
value: OwningPtr<Unaligned>,
world: Option<NonNull<World>>,
cursor: NonNull<usize>,
),
}
/// Densely and efficiently stores a queue of heterogenous types implementing [`Command`].
@ -32,7 +39,16 @@ pub struct CommandQueue {
// For each command, one `CommandMeta` is stored, followed by zero or more bytes
// to store the command itself. To interpret these bytes, a pointer must
// be passed to the corresponding `CommandMeta.apply_command_and_get_size` fn pointer.
bytes: Vec<MaybeUninit<u8>>,
pub(crate) bytes: Vec<MaybeUninit<u8>>,
pub(crate) cursor: usize,
}
/// Wraps pointers to a [`CommandQueue`], used internally to avoid stacked borrow rules when
/// partially applying the world's command queue recursively
#[derive(Clone)]
pub(crate) struct RawCommandQueue {
pub(crate) bytes: NonNull<Vec<MaybeUninit<u8>>>,
pub(crate) cursor: NonNull<usize>,
}
// CommandQueue needs to implement Debug manually, rather than deriving it, because the derived impl just prints
@ -58,6 +74,83 @@ impl CommandQueue {
/// Push a [`Command`] onto the queue.
#[inline]
pub fn push<C>(&mut self, command: C)
where
C: Command,
{
// SAFETY: self is guaranteed to live for the lifetime of this method
unsafe {
self.get_raw().push(command);
}
}
/// Execute the queued [`Command`]s in the world after applying any commands in the world's internal queue.
/// This clears the queue.
#[inline]
pub fn apply(&mut self, world: &mut World) {
// flush the previously queued entities
world.flush_entities();
// flush the world's internal queue
world.flush_commands();
// SAFETY: A reference is always a valid pointer
unsafe {
self.get_raw().apply_or_drop_queued(Some(world.into()));
}
}
/// Take all commands from `other` and append them to `self`, leaving `other` empty
pub fn append(&mut self, other: &mut CommandQueue) {
self.bytes.append(&mut other.bytes);
}
/// Returns false if there are any commands in the queue
#[inline]
pub fn is_empty(&self) -> bool {
self.cursor >= self.bytes.len()
}
/// Returns a [`RawCommandQueue`] instance sharing the underlying command queue.
pub(crate) fn get_raw(&mut self) -> RawCommandQueue {
// SAFETY: self is always valid memory
unsafe {
RawCommandQueue {
bytes: NonNull::new_unchecked(addr_of_mut!(self.bytes)),
cursor: NonNull::new_unchecked(addr_of_mut!(self.cursor)),
}
}
}
}
impl RawCommandQueue {
/// Returns a new `RawCommandQueue` instance, this must be manually dropped.
pub(crate) fn new() -> Self {
// SAFETY: Pointers returned by `Box::into_raw` are guaranteed to be non null
unsafe {
Self {
bytes: NonNull::new_unchecked(Box::into_raw(Box::default())),
cursor: NonNull::new_unchecked(Box::into_raw(Box::new(0usize))),
}
}
}
/// Returns true if the queue is empty.
///
/// # Safety
///
/// * Caller ensures that `bytes` and `cursor` point to valid memory
pub unsafe fn is_empty(&self) -> bool {
// SAFETY: Pointers are guaranteed to be valid by requirements on `.clone_unsafe`
(unsafe { *self.cursor.as_ref() }) >= (unsafe { self.bytes.as_ref() }).len()
}
/// Push a [`Command`] onto the queue.
///
/// # Safety
///
/// * Caller ensures that `self` has not outlived the underlying queue
#[inline]
pub unsafe fn push<C>(&mut self, command: C)
where
C: Command,
{
@ -71,28 +164,34 @@ impl CommandQueue {
}
let meta = CommandMeta {
consume_command_and_get_size: |command, world| {
consume_command_and_get_size: |command, world, mut cursor| {
// SAFETY: Pointer is assured to be valid in `CommandQueue.apply_or_drop_queued`
unsafe { *cursor.as_mut() += std::mem::size_of::<C>() }
// SAFETY: According to the invariants of `CommandMeta.consume_command_and_get_size`,
// `command` must point to a value of type `C`.
let command: C = unsafe { command.read_unaligned() };
match world {
// Apply command to the provided world...
Some(world) => command.apply(world),
// SAFETY: Calller ensures pointer is not null
Some(mut world) => command.apply(unsafe { world.as_mut() }),
// ...or discard it.
None => drop(command),
}
std::mem::size_of::<C>()
},
};
let old_len = self.bytes.len();
// SAFETY: There are no outstanding references to self.bytes
let bytes = unsafe { self.bytes.as_mut() };
let old_len = bytes.len();
// Reserve enough bytes for both the metadata and the command itself.
self.bytes.reserve(std::mem::size_of::<Packed<C>>());
bytes.reserve(std::mem::size_of::<Packed<C>>());
// Pointer to the bytes at the end of the buffer.
// SAFETY: We know it is within bounds of the allocation, due to the call to `.reserve()`.
let ptr = unsafe { self.bytes.as_mut_ptr().add(old_len) };
let ptr = unsafe { bytes.as_mut_ptr().add(old_len) };
// Write the metadata into the buffer, followed by the command.
// We are using a packed struct to write them both as one operation.
@ -108,121 +207,66 @@ impl CommandQueue {
// SAFETY: The new length is guaranteed to fit in the vector's capacity,
// due to the call to `.reserve()` above.
unsafe {
self.bytes
.set_len(old_len + std::mem::size_of::<Packed<C>>());
bytes.set_len(old_len + std::mem::size_of::<Packed<C>>());
}
}
/// Execute the queued [`Command`]s in the world after applying any commands in the world's internal queue.
/// This clears the queue.
#[inline]
pub fn apply(&mut self, world: &mut World) {
// flush the previously queued entities
world.flush_entities();
self.apply_or_drop_queued(Some(world));
}
/// If `world` is [`Some`], this will apply the queued [commands](`Command`).
/// If `world` is [`None`], this will drop the queued [commands](`Command`) (without applying them).
/// This clears the queue.
///
/// # Safety
///
/// * Caller ensures that `self` has not outlived the underlying queue
#[inline]
fn apply_or_drop_queued(&mut self, mut world: Option<&mut World>) {
// The range of pointers of the filled portion of `self.bytes`.
let bytes_range = self.bytes.as_mut_ptr_range();
pub(crate) unsafe fn apply_or_drop_queued(&mut self, world: Option<NonNull<World>>) {
// SAFETY: If this is the command queue on world, world will not be dropped as we have a mutable reference
// If this is not the command queue on world we have exclusive ownership and self will not be mutated
while *self.cursor.as_ref() < self.bytes.as_ref().len() {
// SAFETY: The cursor is either at the start of the buffer, or just after the previous command.
// Since we know that the cursor is in bounds, it must point to the start of a new command.
let meta = unsafe {
self.bytes
.as_mut()
.as_mut_ptr()
.add(*self.cursor.as_ref())
.cast::<CommandMeta>()
.read_unaligned()
};
// Pointer that will iterate over the entries of the buffer.
let cursor = bytes_range.start;
let end = bytes_range.end;
// Advance to the bytes just after `meta`, which represent a type-erased command.
// SAFETY: For most types of `Command`, the pointer immediately following the metadata
// is guaranteed to be in bounds. If the command is a zero-sized type (ZST), then the cursor
// might be 1 byte past the end of the buffer, which is safe.
unsafe { *self.cursor.as_mut() += std::mem::size_of::<CommandMeta>() };
// Construct an owned pointer to the command.
// SAFETY: It is safe to transfer ownership out of `self.bytes`, since the increment of `cursor` above
// guarantees that nothing stored in the buffer will get observed after this function ends.
// `cmd` points to a valid address of a stored command, so it must be non-null.
let cmd = unsafe {
OwningPtr::<Unaligned>::new(std::ptr::NonNull::new_unchecked(
self.bytes
.as_mut()
.as_mut_ptr()
.add(*self.cursor.as_ref())
.cast(),
))
};
// SAFETY: The data underneath the cursor must correspond to the type erased in metadata,
// since they were stored next to each other by `.push()`.
// For ZSTs, the type doesn't matter as long as the pointer is non-null.
// This also advances the cursor past the command. For ZSTs, the cursor will not move.
// At this point, it will either point to the next `CommandMeta`,
// or the cursor will be out of bounds and the loop will end.
unsafe { (meta.consume_command_and_get_size)(cmd, world, self.cursor) };
}
// Reset the buffer, so it can be reused after this function ends.
// In the loop below, ownership of each command will be transferred into user code.
// SAFETY: `set_len(0)` is always valid.
unsafe { self.bytes.set_len(0) };
// Create a stack for the command queue's we will be applying as commands may queue additional commands.
// This is preferred over recursion to avoid stack overflows.
let mut resolving_commands = vec![(cursor, end)];
// Take ownership of any additional buffers so they are not free'd uintil they are iterated.
let mut buffers = Vec::new();
// Add any commands in the world's internal queue to the top of the stack.
if let Some(world) = &mut world {
if !world.command_queue.is_empty() {
let mut bytes = std::mem::take(&mut world.command_queue.bytes);
let bytes_range = bytes.as_mut_ptr_range();
resolving_commands.push((bytes_range.start, bytes_range.end));
buffers.push(bytes);
}
}
while let Some((mut cursor, mut end)) = resolving_commands.pop() {
while cursor < end {
// SAFETY: The cursor is either at the start of the buffer, or just after the previous command.
// Since we know that the cursor is in bounds, it must point to the start of a new command.
let meta = unsafe { cursor.cast::<CommandMeta>().read_unaligned() };
// Advance to the bytes just after `meta`, which represent a type-erased command.
// SAFETY: For most types of `Command`, the pointer immediately following the metadata
// is guaranteed to be in bounds. If the command is a zero-sized type (ZST), then the cursor
// might be 1 byte past the end of the buffer, which is safe.
cursor = unsafe { cursor.add(std::mem::size_of::<CommandMeta>()) };
// Construct an owned pointer to the command.
// SAFETY: It is safe to transfer ownership out of `self.bytes`, since the call to `set_len(0)` above
// guarantees that nothing stored in the buffer will get observed after this function ends.
// `cmd` points to a valid address of a stored command, so it must be non-null.
let cmd = unsafe {
OwningPtr::<Unaligned>::new(std::ptr::NonNull::new_unchecked(cursor.cast()))
};
// SAFETY: The data underneath the cursor must correspond to the type erased in metadata,
// since they were stored next to each other by `.push()`.
// For ZSTs, the type doesn't matter as long as the pointer is non-null.
let size =
unsafe { (meta.consume_command_and_get_size)(cmd, world.as_deref_mut()) };
// Advance the cursor past the command. For ZSTs, the cursor will not move.
// At this point, it will either point to the next `CommandMeta`,
// or the cursor will be out of bounds and the loop will end.
// SAFETY: The address just past the command is either within the buffer,
// or 1 byte past the end, so this addition will not overflow the pointer's allocation.
cursor = unsafe { cursor.add(size) };
if let Some(world) = &mut world {
// If the command we just applied generated more commands we must apply those first
if !world.command_queue.is_empty() {
// If our current list of commands isn't complete push it to the `resolving_commands` stack to be applied after
if cursor < end {
resolving_commands.push((cursor, end));
}
let mut bytes = std::mem::take(&mut world.command_queue.bytes);
// Start applying the new queue
let bytes_range = bytes.as_mut_ptr_range();
cursor = bytes_range.start;
end = bytes_range.end;
// Store our buffer so it is not dropped;
buffers.push(bytes);
}
}
}
// Re-use last buffer to avoid re-allocation
if let (Some(world), Some(buffer)) = (&mut world, buffers.pop()) {
world.command_queue.bytes = buffer;
// SAFETY: `set_len(0)` is always valid.
unsafe { world.command_queue.bytes.set_len(0) };
}
}
}
/// Take all commands from `other` and append them to `self`, leaving `other` empty
pub fn append(&mut self, other: &mut CommandQueue) {
self.bytes.append(&mut other.bytes);
}
/// Returns false if there are any commands in the queue
#[inline]
pub fn is_empty(&self) -> bool {
self.bytes.is_empty()
unsafe {
self.bytes.as_mut().set_len(0);
*self.cursor.as_mut() = 0;
};
}
}
@ -231,7 +275,8 @@ impl Drop for CommandQueue {
if !self.bytes.is_empty() {
warn!("CommandQueue has un-applied commands being dropped.");
}
self.apply_or_drop_queued(None);
// SAFETY: A reference is always a valid pointer
unsafe { self.get_raw().apply_or_drop_queued(None) };
}
}
@ -368,15 +413,15 @@ mod test {
}));
// even though the first command panicking.
// the `bytes`/`metas` vectors were cleared.
assert_eq!(queue.bytes.len(), 0);
// the cursor was incremented.
assert!(queue.cursor > 0);
// Even though the first command panicked, it's still ok to push
// more commands.
queue.push(SpawnCommand);
queue.push(SpawnCommand);
queue.apply(&mut world);
assert_eq!(world.entities().len(), 2);
assert_eq!(world.entities().len(), 3);
}
// NOTE: `CommandQueue` is `Send` because `Command` is send.

View file

@ -56,8 +56,9 @@ impl<'w> DeferredWorld<'w> {
#[inline]
pub fn commands(&mut self) -> Commands {
// SAFETY: &mut self ensure that there are no outstanding accesses to the queue
let queue = unsafe { self.world.get_command_queue() };
Commands::new_from_entities(queue, self.world.entities())
let command_queue = unsafe { self.world.get_raw_command_queue() };
// SAFETY: command_queue is stored on world and always valid while the world exists
unsafe { Commands::new_raw_from_entities(command_queue, self.world.entities()) }
}
/// Retrieves a mutable reference to the given `entity`'s [`Component`] of the given type.

View file

@ -1,6 +1,6 @@
//! Defines the [`World`] and APIs for accessing it directly.
mod command_queue;
pub(crate) mod command_queue;
mod deferred_world;
mod entity_ref;
pub mod error;
@ -31,6 +31,7 @@ use crate::{
schedule::{Schedule, ScheduleLabel, Schedules},
storage::{ResourceData, Storages},
system::{Commands, Res, Resource},
world::command_queue::RawCommandQueue,
world::error::TryRunScheduleError,
};
use bevy_ptr::{OwningPtr, Ptr};
@ -112,7 +113,7 @@ pub struct World {
pub(crate) change_tick: AtomicU32,
pub(crate) last_change_tick: Tick,
pub(crate) last_check_tick: Tick,
pub(crate) command_queue: CommandQueue,
pub(crate) command_queue: RawCommandQueue,
}
impl Default for World {
@ -130,11 +131,22 @@ impl Default for World {
change_tick: AtomicU32::new(1),
last_change_tick: Tick::new(0),
last_check_tick: Tick::new(0),
command_queue: CommandQueue::default(),
command_queue: RawCommandQueue::new(),
}
}
}
impl Drop for World {
fn drop(&mut self) {
// SAFETY: Not passing a pointer so the argument is always valid
unsafe { self.command_queue.apply_or_drop_queued(None) };
// SAFETY: Pointers in internal command queue are only invalidated here
drop(unsafe { Box::from_raw(self.command_queue.bytes.as_ptr()) });
// SAFETY: Pointers in internal command queue are only invalidated here
drop(unsafe { Box::from_raw(self.command_queue.cursor.as_ptr()) });
}
}
impl World {
/// Creates a new empty [`World`].
///
@ -216,7 +228,8 @@ impl World {
/// Use [`World::flush_commands`] to apply all queued commands
#[inline]
pub fn commands(&mut self) -> Commands {
Commands::new_from_entities(&mut self.command_queue, &self.entities)
// SAFETY: command_queue is stored on world and always valid while the world exists
unsafe { Commands::new_raw_from_entities(self.command_queue.clone(), &self.entities) }
}
/// Initializes a new [`Component`] type and returns the [`ComponentId`] created for it.
@ -1871,9 +1884,14 @@ impl World {
/// This does not apply commands from any systems, only those stored in the world.
#[inline]
pub fn flush_commands(&mut self) {
if !self.command_queue.is_empty() {
// `CommandQueue` application always applies commands from the world queue first so this will apply all stored commands
CommandQueue::default().apply(self);
// SAFETY: `self.command_queue` is only de-allocated in `World`'s `Drop`
if !unsafe { self.command_queue.is_empty() } {
// SAFETY: `self.command_queue` is only de-allocated in `World`'s `Drop`
unsafe {
self.command_queue
.clone()
.apply_or_drop_queued(Some(self.into()));
};
}
}

View file

@ -2,7 +2,7 @@
#![warn(unsafe_op_in_unsafe_fn)]
use super::{command_queue::CommandQueue, Mut, Ref, World, WorldId};
use super::{Mut, Ref, World, WorldId};
use crate::{
archetype::{Archetype, Archetypes},
bundle::Bundles,
@ -13,9 +13,10 @@ use crate::{
removal_detection::RemovedComponentEvents,
storage::{Column, ComponentSparseSet, Storages},
system::{Res, Resource},
world::RawCommandQueue,
};
use bevy_ptr::Ptr;
use std::{any::TypeId, cell::UnsafeCell, fmt::Debug, marker::PhantomData, ptr, ptr::addr_of_mut};
use std::{any::TypeId, cell::UnsafeCell, fmt::Debug, marker::PhantomData, ptr};
/// Variant of the [`World`] where resource and component accesses take `&self`, and the responsibility to avoid
/// aliasing violations are given to the caller instead of being checked at compile-time by rust's unique XOR shared rule.
@ -564,11 +565,11 @@ impl<'w> UnsafeWorldCell<'w> {
/// It is the callers responsibility to ensure that
/// - the [`UnsafeWorldCell`] has permission to access the queue mutably
/// - no mutable references to the queue exist at the same time
pub(crate) unsafe fn get_command_queue(self) -> &'w mut CommandQueue {
pub(crate) unsafe fn get_raw_command_queue(self) -> RawCommandQueue {
// SAFETY:
// - caller ensures there are no existing mutable references
// - caller ensures that we have permission to access the queue
unsafe { &mut *addr_of_mut!((*self.0).command_queue) }
unsafe { (*self.0).command_queue.clone() }
}
}