Procházet zdrojové kódy

Added support to split up large islands (#458)

* Increases performance of scenes with large piles of overlapping bodies by a factor of 2-4
* Added Pyramid and RagdollSinglePile performance test
* Added configuration option to RigPileTest to select the number of piles and number of ragdolls per pile
Jorrit Rouwe před 2 roky
rodič
revize
71e7988e96

+ 2 - 2
.github/workflows/determinism_check.yml

@@ -1,8 +1,8 @@
 name: Determinism Check
 
 env:
-    CONVEX_VS_MESH_HASH: '0x4a775b24d5f4699'
-    RAGDOLL_HASH: '0x7f60101bb58d2677'
+    CONVEX_VS_MESH_HASH: '0x3c55fb5709fda28c'
+    RAGDOLL_HASH: '0x22cf464f3c2d6b14'
 
 on:
   push:

+ 1 - 0
Jolt/Core/Atomics.h

@@ -16,6 +16,7 @@ using std::memory_order;
 using std::memory_order_relaxed;
 using std::memory_order_acquire;
 using std::memory_order_release;
+using std::memory_order_acq_rel;
 using std::memory_order_seq_cst;
 
 /// Atomically compute the min(ioAtomic, inValue) and store it in ioAtomic, returns true if value was updated

+ 2 - 0
Jolt/Jolt.cmake

@@ -340,6 +340,8 @@ set(JOLT_PHYSICS_SRC_FILES
 	${JOLT_PHYSICS_ROOT}/Physics/EActivation.h
 	${JOLT_PHYSICS_ROOT}/Physics/IslandBuilder.cpp
 	${JOLT_PHYSICS_ROOT}/Physics/IslandBuilder.h
+	${JOLT_PHYSICS_ROOT}/Physics/LargeIslandSplitter.cpp
+	${JOLT_PHYSICS_ROOT}/Physics/LargeIslandSplitter.h
 	${JOLT_PHYSICS_ROOT}/Physics/PhysicsLock.cpp
 	${JOLT_PHYSICS_ROOT}/Physics/PhysicsLock.h
 	${JOLT_PHYSICS_ROOT}/Physics/PhysicsScene.cpp

+ 4 - 0
Jolt/Physics/Constraints/Constraint.h

@@ -12,6 +12,7 @@
 JPH_NAMESPACE_BEGIN
 
 class IslandBuilder;
+class LargeIslandSplitter;
 class BodyManager;
 class StateRecorder;
 class StreamIn;
@@ -144,6 +145,9 @@ public:
 	/// Link bodies that are connected by this constraint in the island builder
 	virtual void				BuildIslands(uint32 inConstraintIndex, IslandBuilder &ioBuilder, BodyManager &inBodyManager) = 0;
 
+	/// Link bodies that are connected by this constraint in the same split. Returns the split index.
+	virtual uint				BuildIslandSplits(LargeIslandSplitter &ioSplitter) const = 0;
+
 #ifdef JPH_DEBUG_RENDERER
 	// Drawing interface
 	virtual void				DrawConstraint(DebugRenderer *inRenderer) const = 0;

+ 11 - 0
Jolt/Physics/Constraints/ConstraintManager.cpp

@@ -125,6 +125,17 @@ void ConstraintManager::sSetupVelocityConstraints(Constraint **inActiveConstrain
 	}
 }
 
+void ConstraintManager::sWarmStartVelocityConstraints(Constraint **inActiveConstraints, const uint32 *inConstraintIdxBegin, const uint32 *inConstraintIdxEnd, float inWarmStartImpulseRatio)
+{
+	JPH_PROFILE_FUNCTION();
+
+	for (const uint32 *constraint_idx = inConstraintIdxBegin; constraint_idx < inConstraintIdxEnd; ++constraint_idx)
+	{
+		Constraint *c = inActiveConstraints[*constraint_idx];
+		c->WarmStartVelocityConstraint(inWarmStartImpulseRatio);
+	}
+}
+
 void ConstraintManager::sWarmStartVelocityConstraints(Constraint **inActiveConstraints, const uint32 *inConstraintIdxBegin, const uint32 *inConstraintIdxEnd, float inWarmStartImpulseRatio, int &ioNumVelocitySteps)
 {
 	JPH_PROFILE_FUNCTION();

+ 3 - 0
Jolt/Physics/Constraints/ConstraintManager.h

@@ -60,6 +60,9 @@ public:
 	static void				sSetupVelocityConstraints(Constraint **inActiveConstraints, const uint32 *inConstraintIdxBegin, const uint32 *inConstraintIdxEnd, float inDeltaTime);
 
 	/// Apply last frame's impulses, must be called prior to SolveVelocityConstraints
+	static void				sWarmStartVelocityConstraints(Constraint **inActiveConstraints, const uint32 *inConstraintIdxBegin, const uint32 *inConstraintIdxEnd, float inWarmStartImpulseRatio);
+
+	/// Same as above but also calculates the number of velocity steps
 	static void				sWarmStartVelocityConstraints(Constraint **inActiveConstraints, const uint32 *inConstraintIdxBegin, const uint32 *inConstraintIdxEnd, float inWarmStartImpulseRatio, int &ioNumVelocitySteps);
 
 	/// This function is called multiple times to iteratively come to a solution that meets all velocity constraints

+ 8 - 0
Jolt/Physics/Constraints/ContactConstraintManager.h

@@ -156,6 +156,14 @@ public:
 	/// Sort contact constraints deterministically
 	void						SortContacts(uint32 *inConstraintIdxBegin, uint32 *inConstraintIdxEnd) const;
 
+	/// Get the affected bodies for a given constraint
+	inline void					GetAffectedBodies(uint32 inConstraintIdx, const Body *&outBody1, const Body *&outBody2) const
+	{
+		const ContactConstraint &constraint = mConstraints[inConstraintIdx];
+		outBody1 = constraint.mBody1;
+		outBody2 = constraint.mBody2;
+	}
+
 	/// AddContactConstraint will also setup the velocity constraints for the first sub step. For subsequent sub steps this function must be called prior to warm starting the constraint.
 	void						SetupVelocityConstraints(const uint32 *inConstraintIdxBegin, const uint32 *inConstraintIdxEnd, float inDeltaTime);
 

+ 6 - 0
Jolt/Physics/Constraints/TwoBodyConstraint.cpp

@@ -6,6 +6,7 @@
 
 #include <Jolt/Physics/Constraints/TwoBodyConstraint.h>
 #include <Jolt/Physics/IslandBuilder.h>
+#include <Jolt/Physics/LargeIslandSplitter.h>
 #include <Jolt/Physics/Body/BodyManager.h>
 
 #ifdef JPH_DEBUG_RENDERER
@@ -35,6 +36,11 @@ void TwoBodyConstraint::BuildIslands(uint32 inConstraintIndex, IslandBuilder &io
 	ioBuilder.LinkConstraint(inConstraintIndex, mBody1->GetIndexInActiveBodiesInternal(), mBody2->GetIndexInActiveBodiesInternal()); 
 }
 
+uint TwoBodyConstraint::BuildIslandSplits(LargeIslandSplitter &ioSplitter) const
+{
+	return ioSplitter.AssignSplit(mBody1, mBody2);
+}
+
 #ifdef JPH_DEBUG_RENDERER
 
 void TwoBodyConstraint::DrawConstraintReferenceFrame(DebugRenderer *inRenderer) const

+ 3 - 0
Jolt/Physics/Constraints/TwoBodyConstraint.h

@@ -53,6 +53,9 @@ public:
 	/// Link bodies that are connected by this constraint in the island builder
 	virtual void				BuildIslands(uint32 inConstraintIndex, IslandBuilder &ioBuilder, BodyManager &inBodyManager) override;
 
+	/// Link bodies that are connected by this constraint in the same split. Returns the split index.
+	virtual uint				BuildIslandSplits(LargeIslandSplitter &ioSplitter) const override;
+
 protected:
 	/// The two bodies involved
 	Body *						mBody1;

+ 571 - 0
Jolt/Physics/LargeIslandSplitter.cpp

@@ -0,0 +1,571 @@
+// SPDX-FileCopyrightText: 2023 Jorrit Rouwe
+// SPDX-License-Identifier: MIT
+
+#include <Jolt/Jolt.h>
+
+#include <Jolt/Physics/LargeIslandSplitter.h>
+#include <Jolt/Physics/IslandBuilder.h>
+#include <Jolt/Physics/Constraints/Constraint.h>
+#include <Jolt/Physics/Constraints/ContactConstraintManager.h>
+#include <Jolt/Physics/Body/BodyManager.h>
+#include <Jolt/Core/Profiler.h>
+#include <Jolt/Core/TempAllocator.h>
+
+//#define JPH_LARGE_ISLAND_SPLITTER_DEBUG
+
+JPH_NAMESPACE_BEGIN
+
+LargeIslandSplitter::EStatus LargeIslandSplitter::Splits::FetchNextBatch(uint32 &outConstraintsBegin, uint32 &outConstraintsEnd, uint32 &outContactsBegin, uint32 &outContactsEnd, bool &outFirstIteration)
+{
+	{
+		// First check if we can get a new batch (doing a relaxed read to avoid hammering an atomic with an atomic subtract)
+		// Note this also avoids overflowing the status counter if we're done but there's still one thread processing items
+		uint64 status = mStatus.load(memory_order_relaxed);
+		if (sGetIteration(status) >= mNumIterations)
+			return EStatus::AllBatchesDone;
+
+		// Check for special value that indicates that the splits are still being built
+		// (note we do not check for this condition again below as we reset all splits before kicking off jobs that fetch batches of work)
+		if (status == StatusItemMask)
+			return EStatus::WaitingForBatch;
+
+		uint item = sGetItem(status);
+		uint split_index = sGetSplit(status);
+		if (split_index == cNonParallelSplitIdx)
+		{
+			// Non parallel split needs to be taken as a single batch, only the thread that takes element 0 will do it
+			if (item != 0)
+				return EStatus::WaitingForBatch;
+		}
+		else
+		{
+			// Parallel split is split into batches
+			JPH_ASSERT(split_index < mNumSplits);
+			const Split &split = mSplits[split_index];
+			if (item >= split.GetNumItems())
+				return EStatus::WaitingForBatch;
+		}
+	}
+
+	// Then try to actually get the batch
+	uint64 status = mStatus.fetch_add(cBatchSize, memory_order_acquire);
+	int iteration = sGetIteration(status);
+	if (iteration >= mNumIterations)
+		return EStatus::AllBatchesDone;
+
+	uint split_index = sGetSplit(status);
+	JPH_ASSERT(split_index < mNumSplits || split_index == cNonParallelSplitIdx);
+	const Split &split = mSplits[split_index];
+	uint item_begin = sGetItem(status);
+	if (split_index == cNonParallelSplitIdx)
+	{
+		if (item_begin == 0)
+		{
+			// Non-parallel split always goes as a single batch
+			outConstraintsBegin = split.mConstraintBufferBegin;
+			outConstraintsEnd = split.mConstraintBufferEnd;
+			outContactsBegin = split.mContactBufferBegin;
+			outContactsEnd = split.mContactBufferEnd;
+			outFirstIteration = iteration == 0;
+			return EStatus::BatchRetrieved;
+		}
+		else
+		{
+			// Otherwise we're done with this split
+			return EStatus::WaitingForBatch;
+		}
+	}
+
+	// Parallel split is split into batches
+	uint num_constraints = split.GetNumConstraints();
+	uint num_contacts = split.GetNumContacts();
+	uint num_items = num_constraints + num_contacts;
+	if (item_begin >= num_items)
+		return EStatus::WaitingForBatch;
+
+	uint item_end = min(item_begin + cBatchSize, num_items);
+	if (item_end >= num_constraints)
+	{
+		if (item_begin < num_constraints)
+		{
+			// Partially from constraints and partially from contacts
+			outConstraintsBegin = split.mConstraintBufferBegin + item_begin;
+			outConstraintsEnd = split.mConstraintBufferEnd;
+		}
+		else
+		{
+			// Only contacts
+			outConstraintsBegin = 0;
+			outConstraintsEnd = 0;
+		}
+
+		outContactsBegin = split.mContactBufferBegin + (max(item_begin, num_constraints) - num_constraints);
+		outContactsEnd = split.mContactBufferBegin + (item_end - num_constraints);
+	}
+	else
+	{
+		// Only constraints
+		outConstraintsBegin = split.mConstraintBufferBegin + item_begin;
+		outConstraintsEnd = split.mConstraintBufferBegin + item_end;
+
+		outContactsBegin = 0;
+		outContactsEnd = 0;
+	}
+
+	outFirstIteration = iteration == 0;
+	return EStatus::BatchRetrieved;
+}
+
+void LargeIslandSplitter::Splits::MarkBatchProcessed(uint inNumProcessed, bool &outLastIteration, bool &outFinalBatch)
+{
+	// We fetched this batch, nobody should change the split and or iteration until we mark the last batch as processed so we can safely get the current status
+	uint64 status = mStatus.load(memory_order_relaxed);
+	uint split_index = sGetSplit(status);
+	JPH_ASSERT(split_index < mNumSplits || split_index == cNonParallelSplitIdx);
+	const Split &split = mSplits[split_index];
+	uint num_items_in_split = split.GetNumItems();
+
+	// Determine if this is the last iteration before possibly incrementing it
+	int iteration = sGetIteration(status);
+	outLastIteration = iteration == mNumIterations - 1;
+
+	// Add the number of items we processed to the total number of items processed
+	// Note: This needs to happen after we read the status as other threads may update the status after we mark items as processed
+	JPH_ASSERT(inNumProcessed > 0); // Logic will break if we mark a block of 0 items as processed
+	uint total_items_processed = mItemsProcessed.fetch_add(inNumProcessed, memory_order_acq_rel) + inNumProcessed;
+
+	// Check if we're at the end of the split
+	if (total_items_processed >= num_items_in_split)
+	{
+		JPH_ASSERT(total_items_processed == num_items_in_split); // Should not overflow, that means we're retiring more items than we should process
+
+		// Set items processed back to 0 for the next split/iteration
+		mItemsProcessed.store(0, memory_order_release);
+
+		// Determine next split
+		do
+		{
+			if (split_index == cNonParallelSplitIdx)
+			{
+				// At start of next iteration
+				split_index = 0;
+				++iteration;
+			}
+			else
+			{
+				// At start of next split
+				++split_index;
+			}
+		
+			// If we're beyond the end of splits, go to the non-parallel split
+			if (split_index >= mNumSplits)
+				split_index = cNonParallelSplitIdx;
+		}
+		while (iteration < mNumIterations
+			&& mSplits[split_index].GetNumItems() == 0); // We don't support processing empty splits, skip to the next split in this case
+
+		mStatus.store((uint64(iteration) << StatusIterationShift) | (uint64(split_index) << StatusSplitShift), memory_order_release);
+	}
+
+	// Track if this is the final batch
+	outFinalBatch = iteration >= mNumIterations;
+}
+
+LargeIslandSplitter::~LargeIslandSplitter()
+{
+	JPH_ASSERT(mSplitMasks == nullptr);
+	JPH_ASSERT(mContactAndConstaintsSplitIdx == nullptr);
+	JPH_ASSERT(mContactAndConstraintIndices == nullptr);
+	JPH_ASSERT(mSplitIslands == nullptr);
+}
+
+void LargeIslandSplitter::Prepare(const IslandBuilder &inIslandBuilder, uint32 inNumActiveBodies, TempAllocator *inTempAllocator)
+{
+	JPH_PROFILE_FUNCTION();
+
+	// Count the total number of constraints and contacts that we will be putting in splits
+	mContactAndConstraintsSize = 0;
+	for (uint32 island = 0; island < inIslandBuilder.GetNumIslands(); ++island)
+	{
+		// Get the contacts in this island
+		uint32 *contacts_start, *contacts_end;
+		inIslandBuilder.GetContactsInIsland(island, contacts_start, contacts_end);
+		uint num_contacts_in_island = uint(contacts_end - contacts_start);
+
+		// Get the constraints in this island
+		uint32 *constraints_start, *constraints_end;
+		inIslandBuilder.GetConstraintsInIsland(island, constraints_start, constraints_end);
+		uint num_constraints_in_island = uint(constraints_end - constraints_start);
+
+		uint island_size = num_contacts_in_island + num_constraints_in_island;
+		if (island_size >= cLargeIslandTreshold)
+		{
+			mNumSplitIslands++;
+			mContactAndConstraintsSize += island_size;
+		}
+		else
+			break; // If this island doesn't have enough constraints, the next islands won't either since they're sorted from big to small
+	}
+
+	if (mContactAndConstraintsSize > 0)
+	{
+		mNumActiveBodies = inNumActiveBodies;
+
+		// Allocate split mask buffer
+		mSplitMasks = (SplitMask *)inTempAllocator->Allocate(mNumActiveBodies * sizeof(SplitMask));
+
+		// Allocate contact and constraint buffer
+		uint contact_and_constraint_indices_size = mContactAndConstraintsSize * sizeof(uint32);
+		mContactAndConstaintsSplitIdx = (uint32 *)inTempAllocator->Allocate(contact_and_constraint_indices_size);
+		mContactAndConstraintIndices = (uint32 *)inTempAllocator->Allocate(contact_and_constraint_indices_size);
+
+		// Allocate island split buffer
+		mSplitIslands = (Splits *)inTempAllocator->Allocate(mNumSplitIslands * sizeof(Splits));
+
+		// Prevent any of the splits from being picked up as work
+		for (uint i = 0; i < mNumSplitIslands; ++i)
+			mSplitIslands[i].ResetStatus();
+	}
+}
+
+uint LargeIslandSplitter::AssignSplit(const Body *inBody1, const Body *inBody2)
+{
+	uint32 idx1 = inBody1->GetIndexInActiveBodiesInternal();
+	uint32 idx2 = inBody2->GetIndexInActiveBodiesInternal();
+
+	// Test if either index is negative
+	if (idx1 == Body::cInactiveIndex || !inBody1->IsDynamic())
+	{
+		// Body 1 is not active or a kinematic body, so we only need to set 1 body
+		JPH_ASSERT(idx2 < mNumActiveBodies);
+		SplitMask &mask = mSplitMasks[idx2];
+		uint split = min(CountTrailingZeros(~uint32(mask)), cNonParallelSplitIdx);
+		mask |= SplitMask(1U << split);
+		return split;
+	}
+	else if (idx2 == Body::cInactiveIndex || !inBody2->IsDynamic())
+	{
+		// Body 2 is not active or a kinematic body, so we only need to set 1 body
+		JPH_ASSERT(idx1 < mNumActiveBodies);
+		SplitMask &mask = mSplitMasks[idx1];
+		uint split = min(CountTrailingZeros(~uint32(mask)), cNonParallelSplitIdx);
+		mask |= SplitMask(1U << split);
+		return split;
+	}
+	else
+	{
+		// If both bodies are active, we need to set 2 bodies
+		JPH_ASSERT(idx1 < mNumActiveBodies);
+		JPH_ASSERT(idx2 < mNumActiveBodies);
+		SplitMask &mask1 = mSplitMasks[idx1];
+		SplitMask &mask2 = mSplitMasks[idx2];
+		uint split = min(CountTrailingZeros((~uint32(mask1)) & (~uint32(mask2))), cNonParallelSplitIdx);
+		SplitMask mask = SplitMask(1U << split);
+		mask1 |= mask;
+		mask2 |= mask;
+		return split;
+	}
+}
+
+uint LargeIslandSplitter::AssignToNonParallelSplit(const Body *inBody)
+{
+	uint32 idx = inBody->GetIndexInActiveBodiesInternal();
+	if (idx != Body::cInactiveIndex)
+	{
+		JPH_ASSERT(idx < mNumActiveBodies);
+		mSplitMasks[idx] |= 1U << cNonParallelSplitIdx;
+	}
+
+	return cNonParallelSplitIdx;
+}
+
+bool LargeIslandSplitter::SplitIsland(uint32 inIslandIndex, const IslandBuilder &inIslandBuilder, const BodyManager &inBodyManager, const ContactConstraintManager &inContactManager, Constraint **inActiveConstraints, int inNumVelocitySteps, int inNumPositionSteps)
+{
+	JPH_PROFILE_FUNCTION();
+
+	// Get the contacts in this island
+	uint32 *contacts_start, *contacts_end;
+	inIslandBuilder.GetContactsInIsland(inIslandIndex, contacts_start, contacts_end);
+	uint num_contacts_in_island = uint(contacts_end - contacts_start);
+
+	// Get the constraints in this island
+	uint32 *constraints_start, *constraints_end;
+	inIslandBuilder.GetConstraintsInIsland(inIslandIndex, constraints_start, constraints_end);
+	uint num_constraints_in_island = uint(constraints_end - constraints_start);
+
+	// Check if it exceeds the treshold
+	uint island_size = num_contacts_in_island + num_constraints_in_island;
+	if (island_size < cLargeIslandTreshold)
+		return false;
+
+	// Get bodies in this island
+	BodyID *bodies_start, *bodies_end;
+	inIslandBuilder.GetBodiesInIsland(inIslandIndex, bodies_start, bodies_end);
+
+	// Reset the split mask for all bodies in this island
+	Body const * const *bodies = inBodyManager.GetBodies().data();
+	for (const BodyID *b = bodies_start; b < bodies_end; ++b)
+		mSplitMasks[bodies[b->GetIndex()]->GetIndexInActiveBodiesInternal()] = 0;
+
+	// Count the number of contacts and constraints per split
+	uint num_contacts_in_split[cNumSplits] = { };
+	uint num_constraints_in_split[cNumSplits] = { };
+
+	// Get space to store split indices
+	uint offset = mContactAndConstraintsNextFree.fetch_add(island_size, memory_order_relaxed);
+	uint32 *contact_split_idx = mContactAndConstaintsSplitIdx + offset;
+	uint32 *constraint_split_idx = contact_split_idx + num_contacts_in_island;
+
+	// Assign the contacts to a split
+	uint32 *cur_contact_split_idx = contact_split_idx;
+	for (const uint32 *c = contacts_start; c < contacts_end; ++c)
+	{
+		const Body *body1, *body2;
+		inContactManager.GetAffectedBodies(*c, body1, body2);
+		uint split = AssignSplit(body1, body2);
+		num_contacts_in_split[split]++;
+		*cur_contact_split_idx++ = split;
+	}
+
+	// Assign the constraints to a split
+	uint32 *cur_constraint_split_idx = constraint_split_idx;
+	for (const uint32 *c = constraints_start; c < constraints_end; ++c)
+	{
+		const Constraint *constraint = inActiveConstraints[*c];
+		uint split = constraint->BuildIslandSplits(*this);
+		inNumVelocitySteps = max(inNumVelocitySteps, constraint->GetNumVelocityStepsOverride());
+		inNumPositionSteps = max(inNumPositionSteps, constraint->GetNumPositionStepsOverride());
+		num_constraints_in_split[split]++;
+		*cur_constraint_split_idx++ = split;
+	}
+
+	// Start with 0 splits
+	uint split_remap_table[cNumSplits];
+	uint new_split_idx = mNextSplitIsland.fetch_add(1, memory_order_relaxed);
+	JPH_ASSERT(new_split_idx < mNumSplitIslands);
+	Splits &splits = mSplitIslands[new_split_idx];
+	splits.mIslandIndex = inIslandIndex;
+	splits.mNumSplits = 0;
+	splits.mNumIterations = inNumVelocitySteps + 1; // Iteration 0 is used for warm starting
+	splits.mNumVelocitySteps = inNumVelocitySteps;
+	splits.mNumPositionSteps = inNumPositionSteps;
+	splits.mItemsProcessed.store(0, memory_order_release);
+
+	// Allocate space to store the sorted constraint and contact indices per split
+	uint32 *constraint_buffer_cur[cNumSplits], *contact_buffer_cur[cNumSplits];
+	for (uint s = 0; s < cNumSplits; ++s)
+	{
+		// If this split doesn't contain enough constraints and contacts, we will combine it with the non parallel split
+		if (num_constraints_in_split[s] + num_contacts_in_split[s] < cSplitCombineTreshold
+			&& s < cNonParallelSplitIdx) // The non-parallel split cannot merge into itself
+		{
+			// Remap it
+			split_remap_table[s] = cNonParallelSplitIdx;
+
+			// Add the counts to the non parallel split
+			num_contacts_in_split[cNonParallelSplitIdx] += num_contacts_in_split[s];
+			num_constraints_in_split[cNonParallelSplitIdx] += num_constraints_in_split[s];
+		}
+		else
+		{
+			// This split is valid, map it to the next empty slot
+			uint target_split;
+			if (s < cNonParallelSplitIdx)
+				target_split = splits.mNumSplits++;
+			else
+				target_split = cNonParallelSplitIdx;
+			Split &split = splits.mSplits[target_split];
+			split_remap_table[s] = target_split;
+
+			// Allocate space for contacts
+			split.mContactBufferBegin = offset;
+			split.mContactBufferEnd = split.mContactBufferBegin + num_contacts_in_split[s];
+
+			// Allocate space for constraints
+			split.mConstraintBufferBegin = split.mContactBufferEnd;
+			split.mConstraintBufferEnd = split.mConstraintBufferBegin + num_constraints_in_split[s];
+
+			// Store start for each split
+			contact_buffer_cur[target_split] = mContactAndConstraintIndices + split.mContactBufferBegin;
+			constraint_buffer_cur[target_split] = mContactAndConstraintIndices + split.mConstraintBufferBegin;
+
+			// Update offset
+			offset = split.mConstraintBufferEnd;
+		}
+	}
+
+	// Split the contacts
+	for (uint c = 0; c < num_contacts_in_island; ++c)
+	{
+		uint split = split_remap_table[contact_split_idx[c]];
+		*contact_buffer_cur[split]++ = contacts_start[c];
+	}
+
+	// Split the constraints
+	for (uint c = 0; c < num_constraints_in_island; ++c)
+	{
+		uint split = split_remap_table[constraint_split_idx[c]];
+		*constraint_buffer_cur[split]++ = constraints_start[c];
+	}
+
+#ifdef JPH_LARGE_ISLAND_SPLITTER_DEBUG
+	// Trace the size of all splits
+	uint sum = 0;
+	String stats;
+	for (uint s = 0; s < cNumSplits; ++s)
+	{
+		// If we've processed all splits, jump to the non-parallel split
+		if (s >= splits.GetNumSplits())
+			s = cNonParallelSplitIdx;
+
+		const Split &split = splits.mSplits[s];
+		stats += StringFormat("g:%d:%d:%d, ", s, split.GetNumContacts(), split.GetNumConstraints());
+		sum += split.GetNumItems();
+	}
+	stats += StringFormat("sum: %d", sum);
+	Trace(stats.c_str());
+#endif // JPH_LARGE_ISLAND_SPLITTER_DEBUG
+
+#ifdef JPH_ENABLE_ASSERTS
+	for (uint s = 0; s < cNumSplits; ++s)
+	{
+		// If there are no more splits, process the non-parallel split
+		if (s >= splits.mNumSplits)
+			s = cNonParallelSplitIdx;
+
+		// Check that we wrote all elements
+		Split &split = splits.mSplits[s];
+		JPH_ASSERT(contact_buffer_cur[s] == mContactAndConstraintIndices + split.mContactBufferEnd);
+		JPH_ASSERT(constraint_buffer_cur[s] == mContactAndConstraintIndices + split.mConstraintBufferEnd);
+	}
+
+#ifdef _DEBUG
+	// Validate that the splits are indeed not touching the same body
+	for (uint s = 0; s < splits.mNumSplits; ++s)
+	{
+		Array<bool> body_used(mNumActiveBodies, false);
+
+		// Validate contacts
+		uint32 split_contacts_begin, split_contacts_end;
+		splits.GetContactsInSplit(s, split_contacts_begin, split_contacts_end);
+		for (uint32 *c = mContactAndConstraintIndices + split_contacts_begin; c < mContactAndConstraintIndices + split_contacts_end; ++c)
+		{
+			const Body *body1, *body2;
+			inContactManager.GetAffectedBodies(*c, body1, body2);
+
+			uint32 idx1 = body1->GetIndexInActiveBodiesInternal();
+			if (idx1 != Body::cInactiveIndex && !body1->IsDynamic())
+			{
+				JPH_ASSERT(!body_used[idx1]);
+				body_used[idx1] = true;
+			}
+
+			uint32 idx2 = body2->GetIndexInActiveBodiesInternal();
+			if (idx2 != Body::cInactiveIndex && !body2->IsDynamic())
+			{
+				JPH_ASSERT(!body_used[idx2]);
+				body_used[idx2] = true;
+			}
+		}
+	}
+#endif // _DEBUG
+#endif // JPH_ENABLE_ASSERTS
+
+	// Allow other threads to pick up this split island now
+	splits.StartFirstBatch();
+	return true;
+}
+
+LargeIslandSplitter::EStatus LargeIslandSplitter::FetchNextBatch(uint &outSplitIslandIndex, uint32 *&outConstraintsBegin, uint32 *&outConstraintsEnd, uint32 *&outContactsBegin, uint32 *&outContactsEnd, bool &outFirstIteration)
+{
+	// We can't be done when all islands haven't been submitted yet
+	uint num_splits_created = mNextSplitIsland.load(memory_order_acquire);
+	bool all_done = num_splits_created == mNumSplitIslands;
+
+	// Loop over all split islands to find work
+	uint32 constraints_begin, constraints_end, contacts_begin, contacts_end;
+	for (Splits *s = mSplitIslands; s < mSplitIslands + num_splits_created; ++s)
+		switch (s->FetchNextBatch(constraints_begin, constraints_end, contacts_begin, contacts_end, outFirstIteration))
+		{
+		case EStatus::AllBatchesDone:
+			break;
+
+		case EStatus::WaitingForBatch:
+			all_done = false;
+			break;
+
+		case EStatus::BatchRetrieved:
+			outSplitIslandIndex = uint(s - mSplitIslands);
+			outConstraintsBegin = mContactAndConstraintIndices + constraints_begin;
+			outConstraintsEnd = mContactAndConstraintIndices + constraints_end;
+			outContactsBegin = mContactAndConstraintIndices + contacts_begin;
+			outContactsEnd = mContactAndConstraintIndices + contacts_end;
+			return EStatus::BatchRetrieved;
+		}
+
+	return all_done? EStatus::AllBatchesDone : EStatus::WaitingForBatch;
+}
+
+void LargeIslandSplitter::MarkBatchProcessed(uint inSplitIslandIndex, const uint32 *inConstraintsBegin, const uint32 *inConstraintsEnd, const uint32 *inContactsBegin, const uint32 *inContactsEnd, bool &outLastIteration, bool &outFinalBatch)
+{
+	uint num_items_processed = uint(inConstraintsEnd - inConstraintsBegin) + uint(inContactsEnd - inContactsBegin);
+
+	JPH_ASSERT(inSplitIslandIndex < mNextSplitIsland.load(memory_order_relaxed));
+	Splits &splits = mSplitIslands[inSplitIslandIndex];
+	splits.MarkBatchProcessed(num_items_processed, outLastIteration, outFinalBatch);
+}
+
+void LargeIslandSplitter::PrepareForSolvePositions()
+{
+	for (Splits *s = mSplitIslands, *s_end = mSplitIslands + mNumSplitIslands; s < s_end; ++s)
+	{
+		// Set the number of iterations to the number of position steps
+		s->mNumIterations = s->mNumPositionSteps;
+
+		// We can start again from the first batch
+		s->StartFirstBatch();
+	}
+}
+
+void LargeIslandSplitter::Reset(TempAllocator *inTempAllocator)
+{
+	JPH_PROFILE_FUNCTION();
+
+	// Everything should have been used
+	JPH_ASSERT(mContactAndConstraintsNextFree.load(memory_order_relaxed) == mContactAndConstraintsSize);
+	JPH_ASSERT(mNextSplitIsland.load(memory_order_relaxed) == mNumSplitIslands);
+
+	// Free split islands
+	if (mNumSplitIslands > 0)
+	{
+		inTempAllocator->Free(mSplitIslands, mNumSplitIslands * sizeof(Splits));
+		mSplitIslands = nullptr;
+
+		mNumSplitIslands = 0;
+		mNextSplitIsland.store(0, memory_order_relaxed);
+	}
+
+	// Free contact and constraint buffers
+	if (mContactAndConstraintsSize > 0)
+	{
+		inTempAllocator->Free(mContactAndConstraintIndices, mContactAndConstraintsSize * sizeof(uint32));
+		mContactAndConstraintIndices = nullptr;
+
+		inTempAllocator->Free(mContactAndConstaintsSplitIdx, mContactAndConstraintsSize * sizeof(uint32));
+		mContactAndConstaintsSplitIdx = nullptr;
+
+		mContactAndConstraintsSize = 0;
+		mContactAndConstraintsNextFree.store(0, memory_order_relaxed);
+	}
+
+	// Free split masks
+	if (mSplitMasks != nullptr)
+	{
+		inTempAllocator->Free(mSplitMasks, mNumActiveBodies * sizeof(SplitMask));
+		mSplitMasks = nullptr;
+
+		mNumActiveBodies = 0;
+	}
+}
+
+JPH_NAMESPACE_END

+ 181 - 0
Jolt/Physics/LargeIslandSplitter.h

@@ -0,0 +1,181 @@
+// SPDX-FileCopyrightText: 2023 Jorrit Rouwe
+// SPDX-License-Identifier: MIT
+
+#pragma once
+
+#include <Jolt/Core/NonCopyable.h>
+#include <Jolt/Core/Atomics.h>
+
+JPH_NAMESPACE_BEGIN
+
+class Body;
+class BodyID;
+class IslandBuilder;
+class TempAllocator;
+class Constraint;
+class BodyManager;
+class ContactConstraintManager;
+
+/// Assigns bodies in large islands to multiple groups that can run in parallel
+class LargeIslandSplitter : public NonCopyable
+{
+private:
+	using					SplitMask = uint32;
+
+public:
+	static constexpr uint	cNumSplits = sizeof(SplitMask) * 8;
+	static constexpr uint	cNonParallelSplitIdx = cNumSplits - 1;
+	static constexpr uint	cLargeIslandTreshold = 128;							///< If the number of constraints + contacts in an island is larger than this, we will try to split the island
+
+	/// Status code for retrieving a batch
+	enum class EStatus
+	{
+		WaitingForBatch,														///< Work is expected to be available later
+		BatchRetrieved,															///< Work is being returned
+		AllBatchesDone,															///< No further work is expected from this
+	};
+
+	/// Describes a split of constraints and contacts
+	struct Split
+	{
+		inline uint			GetNumContacts() const								{ return mContactBufferEnd - mContactBufferBegin; }
+		inline uint 		GetNumConstraints() const							{ return mConstraintBufferEnd - mConstraintBufferBegin; }
+		inline uint			GetNumItems() const									{ return GetNumContacts() + GetNumConstraints(); }
+
+		uint32				mContactBufferBegin;								///< Begin of the contact buffer (offset relative to mContactAndConstraintIndices)
+		uint32				mContactBufferEnd;									///< End of the contact buffer
+
+		uint32				mConstraintBufferBegin;								///< Begin of the constraint buffer (offset relative to mContactAndConstraintIndices)
+		uint32				mConstraintBufferEnd;								///< End of the constraint buffer
+	};
+
+	/// Structure that describes the resulting splits from the large island splitter
+	class Splits
+	{
+	public:
+		inline uint			GetNumSplits() const
+		{
+			return mNumSplits;
+		}
+
+		inline void			GetConstraintsInSplit(uint inSplitIndex, uint32 &outConstraintsBegin, uint32 &outConstraintsEnd) const
+		{
+			const Split &split = mSplits[inSplitIndex];
+			outConstraintsBegin = split.mConstraintBufferBegin;
+			outConstraintsEnd = split.mConstraintBufferEnd;
+		}
+
+		inline void			GetContactsInSplit(uint inSplitIndex, uint32 &outContactsBegin, uint32 &outContactsEnd) const
+		{
+			const Split &split = mSplits[inSplitIndex];
+			outContactsBegin = split.mContactBufferBegin;
+			outContactsEnd = split.mContactBufferEnd;
+		}
+
+		/// Reset current status so that no work can be picked up from this split
+		inline void			ResetStatus()
+		{
+			mStatus.store(StatusItemMask, memory_order_relaxed);
+		}
+
+		/// Make the first batch available to other threads
+		inline void			StartFirstBatch()
+		{
+			uint split_index = mNumSplits > 0? 0 : cNonParallelSplitIdx;
+			mStatus.store(uint64(split_index) << StatusSplitShift, memory_order_release);
+		}
+
+		/// Fetch the next batch to process
+		EStatus				FetchNextBatch(uint32 &outConstraintsBegin, uint32 &outConstraintsEnd, uint32 &outContactsBegin, uint32 &outContactsEnd, bool &outFirstIteration);
+
+		/// Mark a batch as processed
+		void				MarkBatchProcessed(uint inNumProcessed, bool &outLastIteration, bool &outFinalBatch);
+
+		enum EIterationStatus : uint64
+		{
+			StatusIterationMask		= 0xffff000000000000,
+			StatusIterationShift	= 48,
+			StatusSplitMask			= 0x0000ffff00000000,
+			StatusSplitShift		= 32,
+			StatusItemMask			= 0x00000000ffffffff,
+		};
+
+		static inline int	sGetIteration(uint64 inStatus)
+		{
+			return int((inStatus & StatusIterationMask) >> StatusIterationShift);
+		}
+
+		static inline uint	sGetSplit(uint64 inStatus)
+		{
+			return uint((inStatus & StatusSplitMask) >> StatusSplitShift);
+		}
+
+		static inline uint	sGetItem(uint64 inStatus)
+		{
+			return uint(inStatus & StatusItemMask);
+		}
+
+		Split				mSplits[cNumSplits];								///< Data per split
+		uint32				mIslandIndex;										///< Index of the island that was split
+		uint				mNumSplits;											///< Number of splits that were created (excluding the non-parallel split)
+		int					mNumIterations;										///< Number of iterations to do
+		int					mNumVelocitySteps;									///< Number of velocity steps to do (cached for 2nd sub step)
+		int					mNumPositionSteps;									///< Number of position steps to do
+		atomic<uint64>		mStatus;											///< Status of the split, see EIterationStatus
+		atomic<uint>		mItemsProcessed;									///< Number of items that have been marked as processed
+	};
+
+public:
+	/// Destructor
+							~LargeIslandSplitter();
+
+	/// Prepare the island splitter by allocating memory
+	void					Prepare(const IslandBuilder &inIslandBuilder, uint32 inNumActiveBodies, TempAllocator *inTempAllocator);
+
+	/// Assign two bodies to a split. Returns the split index.
+	uint					AssignSplit(const Body *inBody1, const Body *inBody2);
+
+	/// Force a body to be in a non parallel split. Returns the split index.
+	uint					AssignToNonParallelSplit(const Body *inBody);
+
+	/// Splits up an island, the created splits will be added to the list of batches and can be fetched with FetchNextBatch. Returns false if the island did not need splitting.
+	bool					SplitIsland(uint32 inIslandIndex, const IslandBuilder &inIslandBuilder, const BodyManager &inBodyManager, const ContactConstraintManager &inContactManager, Constraint **inActiveConstraints, int inNumVelocitySteps, int inNumPositionSteps);
+
+	/// Fetch the next batch to process, returns a handle in outSplitIslandIndex that must be provided to MarkBatchProcessed when complete
+	EStatus					FetchNextBatch(uint &outSplitIslandIndex, uint32 *&outConstraintsBegin, uint32 *&outConstraintsEnd, uint32 *&outContactsBegin, uint32 *&outContactsEnd, bool &outFirstIteration);
+
+	/// Mark a batch as processed
+	void					MarkBatchProcessed(uint inSplitIslandIndex, const uint32 *inConstraintsBegin, const uint32 *inConstraintsEnd, const uint32 *inContactsBegin, const uint32 *inContactsEnd, bool &outLastIteration, bool &outFinalBatch);
+
+	/// Get the island index of the island that was split for a particular split island index
+	inline uint32			GetIslandIndex(uint inSplitIslandIndex) const
+	{
+		JPH_ASSERT(inSplitIslandIndex < mNumSplitIslands);
+		return mSplitIslands[inSplitIslandIndex].mIslandIndex;
+	}
+
+	/// Prepare the island splitter for iterating over the split islands again for position solving. Marks all batches as startable.
+	void					PrepareForSolvePositions();
+
+	/// Reset the island splitter
+	void					Reset(TempAllocator *inTempAllocator);
+
+private:
+	static constexpr uint	cSplitCombineTreshold = 32;							///< If the number of constraints + contacts in a split is lower than this, we will merge this split into the 'non-parallel split'
+	static constexpr uint	cBatchSize = 16;									///< Number of items to process in a constraint batch
+
+	uint32					mNumActiveBodies = 0;								///< Cached number of active bodies
+
+	SplitMask *				mSplitMasks = nullptr;								///< Bits that indicate for each body in the BodyManager::mActiveBodies list which split they already belong to
+
+	uint32 *				mContactAndConstaintsSplitIdx = nullptr;			///< Buffer to store the split index per constraint or contact
+	uint32 *				mContactAndConstraintIndices = nullptr;				///< Buffer to store the ordered constraint indices per split
+	uint					mContactAndConstraintsSize = 0;						///< Total size of mContactAndConstraintsSplitIdx and mContactAndConstraintIndices
+	atomic<uint>			mContactAndConstraintsNextFree { 0 };				///< Next element that is free in both buffers
+
+	uint					mNumSplitIslands = 0;								///< Total number of islands that required splitting
+	Splits *				mSplitIslands = nullptr;							///< List of islands that required splitting
+	atomic<uint>			mNextSplitIsland = 0;								///< Next split island to pick from mSplitIslands
+};
+
+JPH_NAMESPACE_END

+ 3 - 0
Jolt/Physics/PhysicsSettings.h

@@ -101,6 +101,9 @@ struct PhysicsSettings
 	/// Whether or not to reduce manifolds with similar contact normals into one contact manifold
 	bool		mUseManifoldReduction = true;
 
+	/// If we split up large islands into smaller parallel batches of work (to improve performance)
+	bool		mUseLargeIslandSplitter = true;
+
 	/// If objects can go to sleep or not
 	bool		mAllowSleeping = true;
 

+ 325 - 171
Jolt/Physics/PhysicsSystem.cpp

@@ -157,6 +157,7 @@ void PhysicsSystem::Update(float inDeltaTime, int inCollisionSteps, int inIntegr
 	context.mStepDeltaTime = inDeltaTime / inCollisionSteps;
 	context.mSubStepDeltaTime = sub_step_delta_time;
 	context.mWarmStartImpulseRatio = warm_start_impulse_ratio;
+	context.mUseLargeIslandSplitter = mPhysicsSettings.mUseLargeIslandSplitter && inIntegrationSubSteps == 1; // Only use large island splitter if we don't have sub steps, not yet supported
 	context.mSteps.resize(inCollisionSteps);
 
 	// Allocate space for body pairs
@@ -379,8 +380,11 @@ void PhysicsSystem::Update(float inDeltaTime, int inCollisionSteps, int inIntegr
 						// Store the number of active bodies at the start of the step
 						next_step->mNumActiveBodiesAtStepStart = mBodyManager.GetNumActiveBodies();
 
-						// Clear the island builder
+						// Clear the large island splitter
 						TempAllocator *temp_allocator = next_step->mContext->mTempAllocator;
+						mLargeIslandSplitter.Reset(temp_allocator);
+
+						// Clear the island builder
 						mIslandBuilder.ResetIslands(temp_allocator);
 
 						// Setup island builder
@@ -584,7 +588,10 @@ void PhysicsSystem::Update(float inDeltaTime, int inCollisionSteps, int inIntegr
 	// Validate that the cached bounds are correct
 	mBodyManager.ValidateActiveBodyBounds();
 #endif // _DEBUG
-	
+
+	// Clear the large island splitter
+	mLargeIslandSplitter.Reset(inTempAllocator);
+
 	// Clear the island builder
 	mIslandBuilder.ResetIslands(inTempAllocator);
 
@@ -1229,6 +1236,10 @@ void PhysicsSystem::JobFinalizeIslands(PhysicsUpdateContext *ioContext)
 
 	// Finish collecting the islands, at this point the active body list doesn't change so it's safe to access
 	mIslandBuilder.Finalize(mBodyManager.GetActiveBodiesUnsafe(), mBodyManager.GetNumActiveBodies(), mContactManager.GetNumConstraints(), ioContext->mTempAllocator);
+
+	// Prepare the large island splitter
+	if (ioContext->mUseLargeIslandSplitter)
+		mLargeIslandSplitter.Prepare(mIslandBuilder, mBodyManager.GetNumActiveBodies(), ioContext->mTempAllocator);
 }
 
 void PhysicsSystem::JobBodySetIslandIndex()
@@ -1264,93 +1275,157 @@ void PhysicsSystem::JobSolveVelocityConstraints(PhysicsUpdateContext *ioContext,
 	// Only the first sub step of the first step needs to correct for the delta time difference in the previous update
 	float warm_start_impulse_ratio = ioSubStep->mIsFirstOfAll? ioContext->mWarmStartImpulseRatio : 1.0f; 
 
-	for (;;)
+	bool check_islands = true, check_split_islands = ioContext->mUseLargeIslandSplitter;
+	do
 	{
-		// Next island
-		uint32 island_idx = ioSubStep->mSolveVelocityConstraintsNextIsland++;
-		if (island_idx >= mIslandBuilder.GetNumIslands())
-			break;
-
-		JPH_PROFILE("Island");
-
-		// Get iterators
-		uint32 *constraints_begin, *constraints_end;
-		bool has_constraints = mIslandBuilder.GetConstraintsInIsland(island_idx, constraints_begin, constraints_end);
-		uint32 *contacts_begin, *contacts_end;
-		bool has_contacts = mIslandBuilder.GetContactsInIsland(island_idx, contacts_begin, contacts_end);
-		
-		if (first_sub_step)
+		// First try to get work from large islands
+		if (check_split_islands)
 		{
-			// If we don't have any contacts or constraints, we know that none of the following islands have any contacts or constraints
-			// (because they're sorted by most constraints first). This means we're done.
-			if (!has_contacts && !has_constraints)
+			bool first_iteration;
+			uint split_island_index;
+			uint32 *constraints_begin, *constraints_end, *contacts_begin, *contacts_end;
+			switch (mLargeIslandSplitter.FetchNextBatch(split_island_index, constraints_begin, constraints_end, contacts_begin, contacts_end, first_iteration))
 			{
-			#ifdef JPH_ENABLE_ASSERTS
-				// Validate our assumption that the next islands don't have any constraints or contacts
-				for (; island_idx < mIslandBuilder.GetNumIslands(); ++island_idx)
+			case LargeIslandSplitter::EStatus::BatchRetrieved:
 				{
-					JPH_ASSERT(!mIslandBuilder.GetConstraintsInIsland(island_idx, constraints_begin, constraints_end));
-					JPH_ASSERT(!mIslandBuilder.GetContactsInIsland(island_idx, contacts_begin, contacts_end));
-				}
-			#endif // JPH_ENABLE_ASSERTS
-				return;
-			}
+					if (first_iteration)
+					{
+						// Iteration 0 is used to warm start the batch (we added 1 to the number of iterations in LargeIslandSplitter::SplitIsland)
+						ConstraintManager::sWarmStartVelocityConstraints(active_constraints, constraints_begin, constraints_end, warm_start_impulse_ratio);
+						mContactManager.WarmStartVelocityConstraints(contacts_begin, contacts_end, warm_start_impulse_ratio);
+					}
+					else
+					{
+						// Solve velocity constraints
+						ConstraintManager::sSolveVelocityConstraints(active_constraints, constraints_begin, constraints_end, delta_time);
+						mContactManager.SolveVelocityConstraints(contacts_begin, contacts_end);
+					}
 
-			// Sort constraints to give a deterministic simulation
-			ConstraintManager::sSortConstraints(active_constraints, constraints_begin, constraints_end);
+					// Mark the batch as processed
+					bool last_iteration, final_batch;
+					mLargeIslandSplitter.MarkBatchProcessed(split_island_index, constraints_begin, constraints_end, contacts_begin, contacts_end, last_iteration, final_batch);
 
-			// Sort contacts to give a deterministic simulation
-			mContactManager.SortContacts(contacts_begin, contacts_end);
+					// Save back the lambdas in the contact cache for the warm start of the next physics update
+					if (last_sub_step && last_iteration)
+						mContactManager.StoreAppliedImpulses(contacts_begin, contacts_end);
+
+					// We processed work, loop again
+					continue;
+				}
+			case LargeIslandSplitter::EStatus::WaitingForBatch:
+				break;
+			case LargeIslandSplitter::EStatus::AllBatchesDone:
+				check_split_islands = false;
+				break;
+			}
 		}
-		else
+
+		// If that didn't succeed try to process an island
+		if (check_islands)
 		{
+			// Next island
+			uint32 island_idx = ioSubStep->mSolveVelocityConstraintsNextIsland++;
+			if (island_idx >= mIslandBuilder.GetNumIslands())
+			{
+				// We processed all islands, stop checking islands
+				check_islands = false;
+				continue;
+			}
+
+			JPH_PROFILE("Island");
+
+			// Get iterators for this island
+			uint32 *constraints_begin, *constraints_end, *contacts_begin, *contacts_end;
+			bool has_constraints = mIslandBuilder.GetConstraintsInIsland(island_idx, constraints_begin, constraints_end);
+			bool has_contacts = mIslandBuilder.GetContactsInIsland(island_idx, contacts_begin, contacts_end);
+
+			if (first_sub_step)
 			{
-				JPH_PROFILE("Apply Gravity");
+				// If we don't have any contacts or constraints, we know that none of the following islands have any contacts or constraints
+				// (because they're sorted by most constraints first). This means we're done.
+				if (!has_contacts && !has_constraints)
+				{
+				#ifdef JPH_ENABLE_ASSERTS
+					// Validate our assumption that the next islands don't have any constraints or contacts
+					for (; island_idx < mIslandBuilder.GetNumIslands(); ++island_idx)
+					{
+						JPH_ASSERT(!mIslandBuilder.GetConstraintsInIsland(island_idx, constraints_begin, constraints_end));
+						JPH_ASSERT(!mIslandBuilder.GetContactsInIsland(island_idx, contacts_begin, contacts_end));
+					}
+				#endif // JPH_ENABLE_ASSERTS
 
-				// Get bodies in this island
-				BodyID *bodies_begin, *bodies_end;
-				mIslandBuilder.GetBodiesInIsland(island_idx, bodies_begin, bodies_end);
+					check_islands = false;
+					continue;
+				}
 
-				// Apply gravity. In the first step this is done in a separate job.
-				for (const BodyID *body_id = bodies_begin; body_id < bodies_end; ++body_id)
+				// Sort constraints to give a deterministic simulation
+				ConstraintManager::sSortConstraints(active_constraints, constraints_begin, constraints_end);
+
+				// Sort contacts to give a deterministic simulation
+				mContactManager.SortContacts(contacts_begin, contacts_end);
+			}
+			else
+			{
 				{
-					Body &body = mBodyManager.GetBody(*body_id);
-					if (body.IsDynamic())
-						body.GetMotionProperties()->ApplyForceTorqueAndDragInternal(body.GetRotation(), mGravity, delta_time);
+					JPH_PROFILE("Apply Gravity");
+
+					// Get bodies in this island
+					BodyID *bodies_begin, *bodies_end;
+					mIslandBuilder.GetBodiesInIsland(island_idx, bodies_begin, bodies_end);
+
+					// Apply gravity. In the first step this is done in a separate job.
+					for (const BodyID *body_id = bodies_begin; body_id < bodies_end; ++body_id)
+					{
+						Body &body = mBodyManager.GetBody(*body_id);
+						if (body.IsDynamic())
+							body.GetMotionProperties()->ApplyForceTorqueAndDragInternal(body.GetRotation(), mGravity, delta_time);
+					}
 				}
+
+				// If we don't have any contacts or constraints, we don't need to run the solver, but we do need to process
+				// the next island in order to apply gravity
+				if (!has_contacts && !has_constraints)
+					continue;
+
+				// Prepare velocity constraints. In the first step this is done when adding the contact constraints.
+				ConstraintManager::sSetupVelocityConstraints(active_constraints, constraints_begin, constraints_end, delta_time);
+				mContactManager.SetupVelocityConstraints(contacts_begin, contacts_end, delta_time);
 			}
 
-			// If we don't have any contacts or constraints, we don't need to run the solver, but we do need to process
-			// the next island in order to apply gravity
-			if (!has_contacts && !has_constraints)
-				continue;
+			// Split up large islands
+			int num_velocity_steps = mPhysicsSettings.mNumVelocitySteps;
+			if (ioContext->mUseLargeIslandSplitter
+				&& mLargeIslandSplitter.SplitIsland(island_idx, mIslandBuilder, mBodyManager, mContactManager, active_constraints, num_velocity_steps, mPhysicsSettings.mNumPositionSteps))
+				continue; // Loop again to try to fetch the newly split island
 
-			// Prepare velocity constraints. In the first step this is done when adding the contact constraints.
-			ConstraintManager::sSetupVelocityConstraints(active_constraints, constraints_begin, constraints_end, delta_time);
-			mContactManager.SetupVelocityConstraints(contacts_begin, contacts_end, delta_time);
-		}
+			// We didn't create a split, just run the solver now for this entire island. Begin by warm starting.
+			ConstraintManager::sWarmStartVelocityConstraints(active_constraints, constraints_begin, constraints_end, warm_start_impulse_ratio, num_velocity_steps);
+			mContactManager.WarmStartVelocityConstraints(contacts_begin, contacts_end, warm_start_impulse_ratio);
+
+			// Solve velocity constraints
+			for (int velocity_step = 0; velocity_step < num_velocity_steps; ++velocity_step)
+			{
+				bool applied_impulse = ConstraintManager::sSolveVelocityConstraints(active_constraints, constraints_begin, constraints_end, delta_time);
+				applied_impulse |= mContactManager.SolveVelocityConstraints(contacts_begin, contacts_end);
+				if (!applied_impulse)
+					break;
+			}
 
-		// Warm start
-		int num_velocity_steps = mPhysicsSettings.mNumVelocitySteps;
-		ConstraintManager::sWarmStartVelocityConstraints(active_constraints, constraints_begin, constraints_end, warm_start_impulse_ratio, num_velocity_steps);
-		mContactManager.WarmStartVelocityConstraints(contacts_begin, contacts_end, warm_start_impulse_ratio);
+			// Save back the lambdas in the contact cache for the warm start of the next physics update
+			if (last_sub_step)
+				mContactManager.StoreAppliedImpulses(contacts_begin, contacts_end);
 
-		// Solve
-		for (int velocity_step = 0; velocity_step < num_velocity_steps; ++velocity_step)
-		{
-			bool constraint_impulse = ConstraintManager::sSolveVelocityConstraints(active_constraints, constraints_begin, constraints_end, delta_time);
-			bool contact_impulse = mContactManager.SolveVelocityConstraints(contacts_begin, contacts_end);
-			if (!constraint_impulse && !contact_impulse)
-				break;
+			// We processed work, loop again
+			continue;
 		}
 
-		// Save back the lambdas in the contact cache for the warm start of the next physics update
-		if (last_sub_step)
-			mContactManager.StoreAppliedImpulses(contacts_begin, contacts_end);
+		// If we didn't find any work, give up a time slice
+		std::this_thread::yield();
 	}
+	while (check_islands || check_split_islands);
 }
 
-void PhysicsSystem::JobPreIntegrateVelocity(PhysicsUpdateContext *ioContext, PhysicsUpdateContext::SubStep *ioSubStep) const
+void PhysicsSystem::JobPreIntegrateVelocity(PhysicsUpdateContext *ioContext, PhysicsUpdateContext::SubStep *ioSubStep)
 {
 	// Reserve enough space for all bodies that may need a cast
 	TempAllocator *temp_allocator = ioContext->mTempAllocator;
@@ -1362,6 +1437,9 @@ void PhysicsSystem::JobPreIntegrateVelocity(PhysicsUpdateContext *ioContext, Phy
 	JPH_ASSERT(ioSubStep->mActiveBodyToCCDBody == nullptr);
 	ioSubStep->mNumActiveBodyToCCDBody = mBodyManager.GetNumActiveBodies();
 	ioSubStep->mActiveBodyToCCDBody = (int *)temp_allocator->Allocate(ioSubStep->mNumActiveBodyToCCDBody * sizeof(int));
+
+	// Prepare the split island builder for solving the position constraints
+	mLargeIslandSplitter.PrepareForSolvePositions();
 }
 
 void PhysicsSystem::JobIntegrateVelocity(const PhysicsUpdateContext *ioContext, PhysicsUpdateContext::SubStep *ioSubStep)
@@ -2060,6 +2138,107 @@ void PhysicsSystem::JobContactRemovedCallbacks(const PhysicsUpdateContext::Step
 	mContactManager.FinalizeContactCacheAndCallContactPointRemovedCallbacks(ioStep->mNumBodyPairs, ioStep->mNumManifolds);
 }
 
+class PhysicsSystem::BodiesToSleep : public NonCopyable
+{
+public:
+	static constexpr int	cBodiesToSleepSize = 512;
+	static constexpr int	cMaxBodiesToPutInBuffer = 128;
+
+	inline					BodiesToSleep(BodyManager &inBodyManager, BodyID *inBodiesToSleepBuffer) : mBodyManager(inBodyManager), mBodiesToSleepBuffer(inBodiesToSleepBuffer), mBodiesToSleepCur(inBodiesToSleepBuffer) { }
+
+	inline					~BodiesToSleep()
+	{		
+		// Flush the bodies to sleep buffer
+		int num_bodies_in_buffer = int(mBodiesToSleepCur - mBodiesToSleepBuffer);
+		if (num_bodies_in_buffer > 0)
+			mBodyManager.DeactivateBodies(mBodiesToSleepBuffer, num_bodies_in_buffer);
+	}
+
+	inline void				PutToSleep(const BodyID *inBegin, const BodyID *inEnd)
+	{
+		int num_bodies_to_sleep = int(inEnd - inBegin);
+		if (num_bodies_to_sleep > cMaxBodiesToPutInBuffer)
+		{
+			// Too many bodies, deactivate immediately
+			mBodyManager.DeactivateBodies(inBegin, num_bodies_to_sleep);
+		}
+		else
+		{
+			// Check if there's enough space in the bodies to sleep buffer
+			int num_bodies_in_buffer = int(mBodiesToSleepCur - mBodiesToSleepBuffer);
+			if (num_bodies_in_buffer + num_bodies_to_sleep > cBodiesToSleepSize)
+			{
+				// Flush the bodies to sleep buffer
+				mBodyManager.DeactivateBodies(mBodiesToSleepBuffer, num_bodies_in_buffer);
+				mBodiesToSleepCur = mBodiesToSleepBuffer;
+			}
+
+			// Copy the bodies in the buffer
+			memcpy(mBodiesToSleepCur, inBegin, num_bodies_to_sleep * sizeof(BodyID));
+			mBodiesToSleepCur += num_bodies_to_sleep;
+		}
+	}
+
+private:
+	BodyManager &			mBodyManager;
+	BodyID *				mBodiesToSleepBuffer;
+	BodyID *				mBodiesToSleepCur;
+};
+
+void PhysicsSystem::CheckSleepAndUpdateBounds(uint32 inIslandIndex, const PhysicsUpdateContext *ioContext, const PhysicsUpdateContext::SubStep *ioSubStep, BodiesToSleep &ioBodiesToSleep)
+{
+	// Get the bodies that belong to this island
+	BodyID *bodies_begin, *bodies_end;
+	mIslandBuilder.GetBodiesInIsland(inIslandIndex, bodies_begin, bodies_end);
+
+	// Only check sleeping in the last sub step of the last step
+	// Also resets force and torque used during the apply gravity phase
+	if (ioSubStep->mIsLastOfAll)
+	{
+		JPH_PROFILE("Check Sleeping");
+
+		static_assert(int(Body::ECanSleep::CannotSleep) == 0 && int(Body::ECanSleep::CanSleep) == 1, "Loop below makes this assumption");
+		int all_can_sleep = mPhysicsSettings.mAllowSleeping? int(Body::ECanSleep::CanSleep) : int(Body::ECanSleep::CannotSleep);
+
+		float time_before_sleep = mPhysicsSettings.mTimeBeforeSleep;
+		float max_movement = mPhysicsSettings.mPointVelocitySleepThreshold * time_before_sleep;
+
+		for (const BodyID *body_id = bodies_begin; body_id < bodies_end; ++body_id)
+		{
+			Body &body = mBodyManager.GetBody(*body_id);
+
+			// Update bounding box
+			body.CalculateWorldSpaceBoundsInternal();
+
+			// Update sleeping
+			all_can_sleep &= int(body.UpdateSleepStateInternal(ioContext->mSubStepDeltaTime, max_movement, time_before_sleep));
+
+			// Reset force and torque
+			body.GetMotionProperties()->ResetForceAndTorqueInternal();
+		}
+
+		// If all bodies indicate they can sleep we can deactivate them
+		if (all_can_sleep == int(Body::ECanSleep::CanSleep))
+			ioBodiesToSleep.PutToSleep(bodies_begin, bodies_end);
+	}
+	else
+	{
+		JPH_PROFILE("Update Bounds");
+
+		// Update bounding box only for all other sub steps
+		for (const BodyID *body_id = bodies_begin; body_id < bodies_end; ++body_id)
+		{
+			Body &body = mBodyManager.GetBody(*body_id);
+			body.CalculateWorldSpaceBoundsInternal();
+		}
+	}
+
+	// Notify broadphase of changed objects (find ccd contacts can do linear casts in the next step, so
+	// we need to do this every sub step)
+	// Note: Shuffles the BodyID's around!!!
+	mBroadPhase->NotifyBodiesAABBChanged(bodies_begin, int(bodies_end - bodies_begin), false);
+}
+
 void PhysicsSystem::JobSolvePositionConstraints(PhysicsUpdateContext *ioContext, PhysicsUpdateContext::SubStep *ioSubStep)
 {
 #ifdef JPH_ENABLE_ASSERTS
@@ -2071,141 +2250,116 @@ void PhysicsSystem::JobSolvePositionConstraints(PhysicsUpdateContext *ioContext,
 #endif
 
 	float delta_time = ioContext->mSubStepDeltaTime;
+	float baumgarte = mPhysicsSettings.mBaumgarte;
 	Constraint **active_constraints = ioContext->mActiveConstraints;
 
 	// Keep a buffer of bodies that need to go to sleep in order to not constantly lock the active bodies mutex and create contention between all solving threads
-	constexpr int cBodiesToSleepSize = 512;
-	constexpr int cMaxBodiesToPutInBuffer = 64;
-	BodyID *bodies_to_sleep = (BodyID *)JPH_STACK_ALLOC(cBodiesToSleepSize * sizeof(BodyID));
-	BodyID *bodies_to_sleep_cur = bodies_to_sleep;
+	BodiesToSleep bodies_to_sleep(mBodyManager, (BodyID *)JPH_STACK_ALLOC(BodiesToSleep::cBodiesToSleepSize * sizeof(BodyID)));
 
-	for (;;)
+	bool check_islands = true, check_split_islands = ioContext->mUseLargeIslandSplitter;
+	do
 	{
-		// Next island
-		uint32 island_idx = ioSubStep->mSolvePositionConstraintsNextIsland++;
-		if (island_idx >= mIslandBuilder.GetNumIslands())
-			break;
-
-		JPH_PROFILE("Island");
-
-		// Get iterators for this island
-		BodyID *bodies_begin, *bodies_end;
-		mIslandBuilder.GetBodiesInIsland(island_idx, bodies_begin, bodies_end);
-		uint32 *constraints_begin, *constraints_end;
-		bool has_constraints = mIslandBuilder.GetConstraintsInIsland(island_idx, constraints_begin, constraints_end);
-		uint32 *contacts_begin, *contacts_end;
-		bool has_contacts = mIslandBuilder.GetContactsInIsland(island_idx, contacts_begin, contacts_end);
-
-		// Correct positions
-		if (has_contacts || has_constraints)
+		// First try to get work from large islands
+		if (check_split_islands)
 		{
-			float baumgarte = mPhysicsSettings.mBaumgarte;
-
-			// First iteration
-			int num_position_steps = mPhysicsSettings.mNumPositionSteps;
-			if (num_position_steps > 0)
+			bool first_iteration;
+			uint split_island_index;
+			uint32 *constraints_begin, *constraints_end, *contacts_begin, *contacts_end;
+			switch (mLargeIslandSplitter.FetchNextBatch(split_island_index, constraints_begin, constraints_end, contacts_begin, contacts_end, first_iteration))
 			{
-				// In the first iteration also calculate the number of position steps (this way we avoid pulling all constraints into the cache twice)
-				bool constraint_impulse = ConstraintManager::sSolvePositionConstraints(active_constraints, constraints_begin, constraints_end, delta_time, baumgarte, num_position_steps);
-				bool contact_impulse = mContactManager.SolvePositionConstraints(contacts_begin, contacts_end);
+			case LargeIslandSplitter::EStatus::BatchRetrieved:
+				// Solve the batch
+				ConstraintManager::sSolvePositionConstraints(active_constraints, constraints_begin, constraints_end, delta_time, baumgarte);
+				mContactManager.SolvePositionConstraints(contacts_begin, contacts_end);
 
-				// If no impulses were applied we can stop, otherwise we already did 1 iteration
-				if (!constraint_impulse && !contact_impulse)
-					num_position_steps = 0;
-				else
-					--num_position_steps;
-			}
-			else
-			{
-				// Iterate the constraints to see if they override the amount of position steps
-				for (const uint32 *c = constraints_begin; c < constraints_end; ++c)
-					num_position_steps = max(num_position_steps, active_constraints[*c]->GetNumPositionStepsOverride());
-			}
+				// Mark the batch as processed
+				bool last_iteration, final_batch;
+				mLargeIslandSplitter.MarkBatchProcessed(split_island_index, constraints_begin, constraints_end, contacts_begin, contacts_end, last_iteration, final_batch);
 
-			// Further iterations
-			for (int position_step = 0; position_step < num_position_steps; ++position_step)
-			{
-				bool constraint_impulse = ConstraintManager::sSolvePositionConstraints(active_constraints, constraints_begin, constraints_end, delta_time, baumgarte);
-				bool contact_impulse = mContactManager.SolvePositionConstraints(contacts_begin, contacts_end);
-				if (!constraint_impulse && !contact_impulse)
-					break;
+				// The final batch will update all bounds and check sleeping
+				if (final_batch)
+					CheckSleepAndUpdateBounds(mLargeIslandSplitter.GetIslandIndex(split_island_index), ioContext, ioSubStep, bodies_to_sleep);
+
+				// We processed work, loop again
+				continue;
+			case LargeIslandSplitter::EStatus::WaitingForBatch:
+				break;
+			case LargeIslandSplitter::EStatus::AllBatchesDone:
+				check_split_islands = false;
+				break;
 			}
 		}
 
-		// Only check sleeping in the last sub step of the last step
-		// Also resets force and torque used during the apply gravity phase
-		if (ioSubStep->mIsLastOfAll)
+		// If that didn't succeed try to process an island
+		if (check_islands)
 		{
-			JPH_PROFILE("Check Sleeping");
-
-			static_assert(int(Body::ECanSleep::CannotSleep) == 0 && int(Body::ECanSleep::CanSleep) == 1, "Loop below makes this assumption");
-			int all_can_sleep = mPhysicsSettings.mAllowSleeping? int(Body::ECanSleep::CanSleep) : int(Body::ECanSleep::CannotSleep);
-
-			float time_before_sleep = mPhysicsSettings.mTimeBeforeSleep;
-			float max_movement = mPhysicsSettings.mPointVelocitySleepThreshold * time_before_sleep;
-
-			for (const BodyID *body_id = bodies_begin; body_id < bodies_end; ++body_id)
+			// Next island
+			uint32 island_idx = ioSubStep->mSolvePositionConstraintsNextIsland++;
+			if (island_idx >= mIslandBuilder.GetNumIslands())
 			{
-				Body &body = mBodyManager.GetBody(*body_id);
+				// We processed all islands, stop checking islands
+				check_islands = false;
+				continue;
+			}
 
-				// Update bounding box
-				body.CalculateWorldSpaceBoundsInternal();
+			JPH_PROFILE("Island");
 
-				// Update sleeping
-				all_can_sleep &= int(body.UpdateSleepStateInternal(ioContext->mSubStepDeltaTime, max_movement, time_before_sleep));
+			// Get iterators for this island
+			uint32 *constraints_begin, *constraints_end, *contacts_begin, *contacts_end;
+			mIslandBuilder.GetConstraintsInIsland(island_idx, constraints_begin, constraints_end);
+			mIslandBuilder.GetContactsInIsland(island_idx, contacts_begin, contacts_end);
 
-				// Reset force and torque
-				body.GetMotionProperties()->ResetForceAndTorqueInternal();
-			}
+			// If this island is a large island, it will be picked up as a batch and we don't need to do anything here
+			uint num_items = uint(constraints_end - constraints_begin) + uint(contacts_end - contacts_begin);
+			if (ioContext->mUseLargeIslandSplitter
+				&& num_items >= LargeIslandSplitter::cLargeIslandTreshold)
+				continue;
 
-			// If all bodies indicate they can sleep we can deactivate them
-			if (all_can_sleep == int(Body::ECanSleep::CanSleep))
+			// Check if this island needs solving
+			if (num_items > 0)
 			{
-				int num_bodies_to_sleep = int(bodies_end - bodies_begin);
-				if (num_bodies_to_sleep > cMaxBodiesToPutInBuffer)
+				// First iteration
+				int num_position_steps = mPhysicsSettings.mNumPositionSteps;
+				if (num_position_steps > 0)
 				{
-					// Too many bodies, deactivate immediately
-					mBodyManager.DeactivateBodies(bodies_begin, num_bodies_to_sleep);
+					// In the first iteration also calculate the number of position steps (this way we avoid pulling all constraints into the cache twice)
+					bool applied_impulse = ConstraintManager::sSolvePositionConstraints(active_constraints, constraints_begin, constraints_end, delta_time, baumgarte, num_position_steps);
+					applied_impulse |= mContactManager.SolvePositionConstraints(contacts_begin, contacts_end);
+
+					// If no impulses were applied we can stop, otherwise we already did 1 iteration
+					if (!applied_impulse)
+						num_position_steps = 0;
+					else
+						--num_position_steps;
 				}
 				else
 				{
-					// Check if there's enough space in the bodies to sleep buffer
-					int num_bodies_in_buffer = int(bodies_to_sleep_cur - bodies_to_sleep);
-					if (num_bodies_in_buffer + num_bodies_to_sleep > cBodiesToSleepSize)
-					{
-						// Flush the bodies to sleep buffer
-						mBodyManager.DeactivateBodies(bodies_to_sleep, num_bodies_in_buffer);
-						bodies_to_sleep_cur = bodies_to_sleep;
-					}
+					// Iterate the constraints to see if they override the number of position steps
+					for (const uint32 *c = constraints_begin; c < constraints_end; ++c)
+						num_position_steps = max(num_position_steps, active_constraints[*c]->GetNumPositionStepsOverride());
+				}
 
-					// Copy the bodies in the buffer
-					memcpy(bodies_to_sleep_cur, bodies_begin, num_bodies_to_sleep * sizeof(BodyID));
-					bodies_to_sleep_cur += num_bodies_to_sleep;
+				// Further iterations
+				for (int position_step = 0; position_step < num_position_steps; ++position_step)
+				{
+					bool applied_impulse = ConstraintManager::sSolvePositionConstraints(active_constraints, constraints_begin, constraints_end, delta_time, baumgarte);
+					applied_impulse |= mContactManager.SolvePositionConstraints(contacts_begin, contacts_end);
+					if (!applied_impulse)
+						break;
 				}
 			}
-		}
-		else
-		{
-			JPH_PROFILE("Update Bounds");
 
-			// Update bounding box only for all other sub steps
-			for (const BodyID *body_id = bodies_begin; body_id < bodies_end; ++body_id)
-			{
-				Body &body = mBodyManager.GetBody(*body_id);
-				body.CalculateWorldSpaceBoundsInternal();
-			}
+			// After solving we will update all bounds and check sleeping
+			CheckSleepAndUpdateBounds(island_idx, ioContext, ioSubStep, bodies_to_sleep);
+
+			// We processed work, loop again
+			continue;
 		}
 
-		// Notify broadphase of changed objects (find ccd contacts can do linear casts in the next step, so
-		// we need to do this every sub step)
-		// Note: Shuffles the BodyID's around!!!
-		mBroadPhase->NotifyBodiesAABBChanged(bodies_begin, int(bodies_end - bodies_begin), false);
+		// If we didn't find any work, give up a time slice
+		std::this_thread::yield();
 	}
-
-	// Flush the bodies to sleep buffer
-	int num_bodies_in_buffer = int(bodies_to_sleep_cur - bodies_to_sleep);
-	if (num_bodies_in_buffer > 0)
-		mBodyManager.DeactivateBodies(bodies_to_sleep, num_bodies_in_buffer);
+	while (check_islands || check_split_islands);
 }
 
 void PhysicsSystem::SaveState(StateRecorder &inStream) const

+ 11 - 1
Jolt/Physics/PhysicsSystem.h

@@ -10,6 +10,7 @@
 #include <Jolt/Physics/Constraints/ContactConstraintManager.h>
 #include <Jolt/Physics/Constraints/ConstraintManager.h>
 #include <Jolt/Physics/IslandBuilder.h>
+#include <Jolt/Physics/LargeIslandSplitter.h>
 #include <Jolt/Physics/PhysicsUpdateContext.h>
 #include <Jolt/Physics/PhysicsSettings.h>
 
@@ -191,7 +192,7 @@ private:
 	void						JobFinalizeIslands(PhysicsUpdateContext *ioContext);
 	void						JobBodySetIslandIndex();
 	void						JobSolveVelocityConstraints(PhysicsUpdateContext *ioContext, PhysicsUpdateContext::SubStep *ioSubStep);
-	void						JobPreIntegrateVelocity(PhysicsUpdateContext *ioContext, PhysicsUpdateContext::SubStep *ioSubStep) const;
+	void						JobPreIntegrateVelocity(PhysicsUpdateContext *ioContext, PhysicsUpdateContext::SubStep *ioSubStep);
 	void						JobIntegrateVelocity(const PhysicsUpdateContext *ioContext, PhysicsUpdateContext::SubStep *ioSubStep);
 	void						JobPostIntegrateVelocity(PhysicsUpdateContext *ioContext, PhysicsUpdateContext::SubStep *ioSubStep) const;
 	void						JobFindCCDContacts(const PhysicsUpdateContext *ioContext, PhysicsUpdateContext::SubStep *ioSubStep);
@@ -207,6 +208,12 @@ private:
 	/// Process narrow phase for a single body pair
 	void						ProcessBodyPair(ContactAllocator &ioContactAllocator, const BodyPair &inBodyPair);
 
+	/// This helper batches up bodies that need to put to sleep to avoid contention on the activation mutex
+	class BodiesToSleep;
+
+	/// Called at the end of JobSolveVelocityConstraints to check if bodies need to go to sleep and to update their bounding box in the broadphase
+	void						CheckSleepAndUpdateBounds(uint32 inIslandIndex, const PhysicsUpdateContext *ioContext, const PhysicsUpdateContext::SubStep *ioSubStep, BodiesToSleep &ioBodiesToSleep);
+
 	/// Number of constraints to process at once in JobDetermineActiveConstraints
 	static constexpr int		cDetermineActiveConstraintsBatchSize = 64;
 
@@ -261,6 +268,9 @@ private:
 	/// Keeps track of connected bodies and builds islands for multithreaded velocity/position update
 	IslandBuilder				mIslandBuilder;
 
+	/// Will split large islands into smaller groups of bodies that can be processed in parallel
+	LargeIslandSplitter			mLargeIslandSplitter;
+
 	/// Mutex protecting mStepListeners
 	Mutex						mStepListenersMutex;
 

+ 1 - 0
Jolt/Physics/PhysicsUpdateContext.h

@@ -160,6 +160,7 @@ public:
 	float					mStepDeltaTime;											///< Delta time for a simulation step (collision step)
 	float					mSubStepDeltaTime;										///< Delta time for a simulation sub step (integration step)
 	float					mWarmStartImpulseRatio;									///< Ratio of this step delta time vs last step
+	bool					mUseLargeIslandSplitter;								///< If true, use large island splitting
 
 	Constraint **			mActiveConstraints = nullptr;							///< Constraints that were active at the start of the physics update step (activating bodies can activate constraints and we need a consistent snapshot). Only these constraints will be resolved.
 

+ 5 - 0
Jolt/Physics/Vehicle/VehicleConstraint.cpp

@@ -280,6 +280,11 @@ void VehicleConstraint::BuildIslands(uint32 inConstraintIndex, IslandBuilder &io
 	ioBuilder.LinkConstraint(inConstraintIndex, mBody->GetIndexInActiveBodiesInternal(), min_active_index); 
 }
 
+uint VehicleConstraint::BuildIslandSplits(LargeIslandSplitter &ioSplitter) const
+{
+	return ioSplitter.AssignToNonParallelSplit(mBody);
+}
+
 void VehicleConstraint::CalculateWheelContactPoint(RMat44Arg inBodyTransform, const Wheel &inWheel, Vec3 &outR1PlusU, Vec3 &outR2) const
 {
 	RVec3 contact_pos = inBodyTransform * (inWheel.mSettings->mPosition + inWheel.mSettings->mDirection * inWheel.mContactLength);

+ 1 - 0
Jolt/Physics/Vehicle/VehicleConstraint.h

@@ -122,6 +122,7 @@ public:
 	virtual bool				SolveVelocityConstraint(float inDeltaTime) override;
 	virtual bool				SolvePositionConstraint(float inDeltaTime, float inBaumgarte) override;
 	virtual void				BuildIslands(uint32 inConstraintIndex, IslandBuilder &ioBuilder, BodyManager &inBodyManager) override;
+	virtual uint				BuildIslandSplits(LargeIslandSplitter &ioSplitter) const override;
 #ifdef JPH_DEBUG_RENDERER
 	virtual void				DrawConstraint(DebugRenderer *inRenderer) const override;
 	virtual void				DrawConstraintLimits(DebugRenderer *inRenderer) const override;

+ 1 - 0
PerformanceTest/PerformanceTest.cmake

@@ -3,6 +3,7 @@ set(PERFORMANCE_TEST_ROOT ${PHYSICS_REPO_ROOT}/PerformanceTest)
 
 # Source files
 set(PERFORMANCE_TEST_SRC_FILES
+	${PERFORMANCE_TEST_ROOT}/PyramidScene.h
 	${PERFORMANCE_TEST_ROOT}/PerformanceTest.cpp
 	${PERFORMANCE_TEST_ROOT}/PerformanceTest.cmake
 	${PERFORMANCE_TEST_ROOT}/PerformanceTestScene.h

+ 13 - 5
PerformanceTest/PerformanceTest.cpp

@@ -42,6 +42,7 @@ JPH_SUPPRESS_WARNINGS
 // Local includes
 #include "RagdollScene.h"
 #include "ConvexVsMeshScene.h"
+#include "PyramidScene.h"
 
 // Time step for physics
 constexpr float cDeltaTime = 1.0f / 60.0f;
@@ -72,6 +73,9 @@ int main(int argc, char** argv)
 	// Register allocation hook
 	RegisterDefaultAllocator();
 
+	// Helper function that creates the default scene
+	auto create_ragdoll_scene = []{ return unique_ptr<PerformanceTestScene>(new RagdollScene(JPH_IF_DEBUG(2) JPH_IF_NOT_DEBUG(4), JPH_IF_DEBUG(5) JPH_IF_NOT_DEBUG(10), 0.6f)); };
+
 	// Parse command line parameters
 	int specified_quality = -1;
 	int specified_threads = -1;
@@ -95,9 +99,13 @@ int main(int argc, char** argv)
 		{
 			// Parse scene
 			if (strcmp(arg + 3, "Ragdoll") == 0)
-				scene = unique_ptr<PerformanceTestScene>(new RagdollScene);
+				scene = create_ragdoll_scene();
+			else if (strcmp(arg + 3, "RagdollSinglePile") == 0)
+				scene = unique_ptr<PerformanceTestScene>(new RagdollScene(1, JPH_IF_DEBUG(20) JPH_IF_NOT_DEBUG(160), 0.4f));
 			else if (strcmp(arg + 3, "ConvexVsMesh") == 0)
 				scene = unique_ptr<PerformanceTestScene>(new ConvexVsMeshScene);
+			else if (strcmp(arg + 3, "Pyramid") == 0)
+				scene = unique_ptr<PerformanceTestScene>(new PyramidScene);
 			else
 			{
 				Trace("Invalid scene");
@@ -171,7 +179,7 @@ int main(int argc, char** argv)
 		{
 			// Print usage
 			Trace("Usage:\n"
-				  "-s=<scene>: Select scene (Ragdoll, ConvexVsMesh)\n"
+				  "-s=<scene>: Select scene (Ragdoll, RagdollSinglePile, ConvexVsMesh, Pyramid)\n"
 				  "-i=<num physics steps>: Number of physics steps to simulate (default 500)\n"
 				  "-q=<quality>: Test only with specified quality (Discrete, LinearCast)\n"
 				  "-t=<num threads>: Test only with N threads (default is to iterate over 1 .. num hardware threads)\n"
@@ -195,11 +203,11 @@ int main(int argc, char** argv)
 	RegisterTypes();
 
 	// Create temp allocator
-	TempAllocatorImpl temp_allocator(10 * 1024 * 1024);
+	TempAllocatorImpl temp_allocator(32 * 1024 * 1024);
 
 	// Load the scene
 	if (scene == nullptr)
-		scene = unique_ptr<PerformanceTestScene>(new RagdollScene);
+		scene = create_ragdoll_scene();
 	if (!scene->Load())
 		return 1;
 
@@ -254,7 +262,7 @@ int main(int argc, char** argv)
 
 				// Create physics system
 				PhysicsSystem physics_system;
-				physics_system.Init(10240, 0, 65536, 10240, broad_phase_layer_interface, object_vs_broadphase_layer_filter, object_vs_object_layer_filter);
+				physics_system.Init(10240, 0, 65536, 20480, broad_phase_layer_interface, object_vs_broadphase_layer_filter, object_vs_object_layer_filter);
 
 				// Start test scene
 				scene->StartTest(physics_system, motion_quality);

+ 48 - 0
PerformanceTest/PyramidScene.h

@@ -0,0 +1,48 @@
+// Jolt Physics Library (https://github.com/jrouwe/JoltPhysics)
+// SPDX-FileCopyrightText: 2023 Jorrit Rouwe
+// SPDX-License-Identifier: MIT
+
+#pragma once
+
+// Jolt includes
+#include <Jolt/Physics/Collision/Shape/BoxShape.h>
+
+// Local includes
+#include "PerformanceTestScene.h"
+#include "Layers.h"
+
+// A scene that creates a pyramid of boxes to create a very large island
+class PyramidScene : public PerformanceTestScene
+{
+public:
+	virtual const char *	GetName() const override
+	{
+		return "Pyramid";
+	}
+
+	virtual void			StartTest(PhysicsSystem &inPhysicsSystem, EMotionQuality inMotionQuality) override
+	{
+		BodyInterface &bi = inPhysicsSystem.GetBodyInterface();
+
+		// Floor
+		bi.CreateAndAddBody(BodyCreationSettings(new BoxShape(Vec3(50.0f, 1.0f, 50.0f), 0.0f), RVec3(Vec3(0.0f, -1.0f, 0.0f)), Quat::sIdentity(), EMotionType::Static, Layers::NON_MOVING), EActivation::DontActivate);
+
+		const float cBoxSize = 2.0f;
+		const float cBoxSeparation = 0.5f;
+		const float cHalfBoxSize = 0.5f * cBoxSize;
+		const int cPyramidHeight = 15;
+
+		RefConst<Shape> box_shape = new BoxShape(Vec3::sReplicate(cHalfBoxSize), 0.0f); // No convex radius to force more collisions
+
+		// Pyramid
+		for (int i = 0; i < cPyramidHeight; ++i)
+			for (int j = i / 2; j < cPyramidHeight - (i + 1) / 2; ++j)
+				for (int k = i / 2; k < cPyramidHeight - (i + 1) / 2; ++k)
+				{
+					RVec3 position(-cPyramidHeight + cBoxSize * j + (i & 1? cHalfBoxSize : 0.0f), 1.0f + (cBoxSize + cBoxSeparation) * i, -cPyramidHeight + cBoxSize * k + (i & 1? cHalfBoxSize : 0.0f));
+					BodyCreationSettings settings(box_shape, position, Quat::sIdentity(), EMotionType::Dynamic, Layers::MOVING);
+					settings.mAllowSleeping = false; // No sleeping to force the large island to stay awake
+					bi.CreateAndAddBody(settings, EActivation::Activate);
+				}
+	}
+};

+ 11 - 16
PerformanceTest/RagdollScene.h

@@ -19,9 +19,11 @@
 class RagdollScene : public PerformanceTestScene
 {
 public:
+							RagdollScene(int inNumPilesPerAxis, int inPileSize, float inVerticalSeparation) : mNumPilesPerAxis(inNumPilesPerAxis), mPileSize(inPileSize), mVerticalSeparation(inVerticalSeparation) { }
+
 	virtual const char *	GetName() const override
 	{
-		return "Ragdoll";
+		return mNumPilesPerAxis == 1? "RagdollSinglePile" : "Ragdoll";
 	}
 
 	virtual bool			Load() override
@@ -69,16 +71,6 @@ public:
 	{
 		// Test configuration
 		const Real cHorizontalSeparation = 4.0_r;
-		const float cVerticalSeparation = 0.6f;
-		#ifdef _DEBUG
-			const int cPileSize = 5;
-			const int cNumRows = 2;
-			const int cNumCols = 2;
-		#else
-			const int cPileSize = 10;
-			const int cNumRows = 4;
-			const int cNumCols = 4;
-		#endif
 
 		// Set motion quality on ragdoll
 		for (BodyCreationSettings &body : mRagdollSettings->mParts)
@@ -91,11 +83,11 @@ public:
 		mt19937 random;
 		uniform_real_distribution<float> angle(0.0f, JPH_PI);
 		CollisionGroup::GroupID group_id = 1;
-		for (int row = 0; row < cNumRows; ++row)
-			for (int col = 0; col < cNumCols; ++col)
+		for (int row = 0; row < mNumPilesPerAxis; ++row)
+			for (int col = 0; col < mNumPilesPerAxis; ++col)
 			{
 				// Determine start location of ray
-				RVec3 start(cHorizontalSeparation * (col - (cNumCols - 1) / 2.0_r), 100, cHorizontalSeparation * (row - (cNumRows - 1) / 2.0_r));
+				RVec3 start(cHorizontalSeparation * (col - (mNumPilesPerAxis - 1) / 2.0_r), 100, cHorizontalSeparation * (row - (mNumPilesPerAxis - 1) / 2.0_r));
 
 				// Cast ray down to terrain
 				RayCastResult hit;
@@ -104,7 +96,7 @@ public:
 				if (inPhysicsSystem.GetNarrowPhaseQuery().CastRay(ray, hit, SpecifiedBroadPhaseLayerFilter(BroadPhaseLayers::NON_MOVING), SpecifiedObjectLayerFilter(Layers::NON_MOVING)))
 					start = ray.GetPointOnRay(hit.mFraction);
 
-				for (int i = 0; i < cPileSize; ++i)
+				for (int i = 0; i < mPileSize; ++i)
 				{
 					// Create ragdoll
 					Ref<Ragdoll> ragdoll = mRagdollSettings->CreateRagdoll(group_id++, 0, &inPhysicsSystem);
@@ -113,7 +105,7 @@ public:
 					SkeletonPose pose_copy = mPose;
 					pose_copy.SetRootOffset(start);
 					SkeletonPose::JointState &root = pose_copy.GetJoint(0);
-					root.mTranslation = Vec3(0, cVerticalSeparation * (i + 1), 0);
+					root.mTranslation = Vec3(0, mVerticalSeparation * (i + 1), 0);
 					root.mRotation = Quat::sRotation(Vec3::sAxisY(), angle(random)) * root.mRotation;
 					pose_copy.CalculateJointMatrices();
 
@@ -137,6 +129,9 @@ public:
 	}
 
 private:
+	int 					mNumPilesPerAxis;
+	int 					mPileSize;
+	float 					mVerticalSeparation;
 	Ref<RagdollSettings>	mRagdollSettings;
 	Ref<SkeletalAnimation>	mAnimation;
 	SkeletonPose			mPose;

+ 2 - 0
Samples/Samples.cmake

@@ -117,6 +117,8 @@ set(SAMPLES_SRC_FILES
 	${SAMPLES_ROOT}/Tests/General/ManifoldReductionTest.h
 	${SAMPLES_ROOT}/Tests/General/MultithreadedTest.cpp
 	${SAMPLES_ROOT}/Tests/General/MultithreadedTest.h
+	${SAMPLES_ROOT}/Tests/General/PyramidTest.cpp
+	${SAMPLES_ROOT}/Tests/General/PyramidTest.h
 	${SAMPLES_ROOT}/Tests/General/RestitutionTest.cpp
 	${SAMPLES_ROOT}/Tests/General/RestitutionTest.h
 	${SAMPLES_ROOT}/Tests/General/SensorTest.cpp

+ 5 - 2
Samples/SamplesApp.cpp

@@ -65,6 +65,7 @@ struct TestCategory
 JPH_DECLARE_RTTI_FOR_FACTORY(SimpleTest)
 JPH_DECLARE_RTTI_FOR_FACTORY(StackTest)
 JPH_DECLARE_RTTI_FOR_FACTORY(WallTest)
+JPH_DECLARE_RTTI_FOR_FACTORY(PyramidTest)
 JPH_DECLARE_RTTI_FOR_FACTORY(IslandTest)
 JPH_DECLARE_RTTI_FOR_FACTORY(FunnelTest)
 JPH_DECLARE_RTTI_FOR_FACTORY(FrictionTest)
@@ -98,6 +99,7 @@ static TestNameAndRTTI sGeneralTests[] =
 	{ "Simple",								JPH_RTTI(SimpleTest) },
 	{ "Stack",								JPH_RTTI(StackTest) },
 	{ "Wall",								JPH_RTTI(WallTest) },
+	{ "Pyramid",							JPH_RTTI(PyramidTest) },
 	{ "Island",								JPH_RTTI(IslandTest) },
 	{ "Funnel",								JPH_RTTI(FunnelTest) },
 	{ "2D Funnel",							JPH_RTTI(TwoDFunnelTest) },
@@ -334,7 +336,7 @@ static TestCategory sAllCategories[] =
 static constexpr uint cNumBodies = 10240;
 static constexpr uint cNumBodyMutexes = 0; // Autodetect
 static constexpr uint cMaxBodyPairs = 65536;
-static constexpr uint cMaxContactConstraints = 10240;
+static constexpr uint cMaxContactConstraints = 20480;
 
 SamplesApp::SamplesApp()
 {
@@ -342,7 +344,7 @@ SamplesApp::SamplesApp()
 #ifdef JPH_DISABLE_TEMP_ALLOCATOR
 	mTempAllocator = new TempAllocatorMalloc();
 #else
-	mTempAllocator = new TempAllocatorImpl(16 * 1024 * 1024);
+	mTempAllocator = new TempAllocatorImpl(32 * 1024 * 1024);
 #endif
 
 	// Create job system
@@ -402,6 +404,7 @@ SamplesApp::SamplesApp()
 			mDebugUI->CreateCheckBox(phys_settings, "Constraint Warm Starting", mPhysicsSettings.mConstraintWarmStart, [this](UICheckBox::EState inState) { mPhysicsSettings.mConstraintWarmStart = inState == UICheckBox::STATE_CHECKED; mPhysicsSystem->SetPhysicsSettings(mPhysicsSettings); });
 			mDebugUI->CreateCheckBox(phys_settings, "Use Body Pair Contact Cache", mPhysicsSettings.mUseBodyPairContactCache, [this](UICheckBox::EState inState) { mPhysicsSettings.mUseBodyPairContactCache = inState == UICheckBox::STATE_CHECKED; mPhysicsSystem->SetPhysicsSettings(mPhysicsSettings); });
 			mDebugUI->CreateCheckBox(phys_settings, "Contact Manifold Reduction", mPhysicsSettings.mUseManifoldReduction, [this](UICheckBox::EState inState) { mPhysicsSettings.mUseManifoldReduction = inState == UICheckBox::STATE_CHECKED; mPhysicsSystem->SetPhysicsSettings(mPhysicsSettings); });
+			mDebugUI->CreateCheckBox(phys_settings, "Use Large Island Splitter", mPhysicsSettings.mUseLargeIslandSplitter, [this](UICheckBox::EState inState) { mPhysicsSettings.mUseLargeIslandSplitter = inState == UICheckBox::STATE_CHECKED; mPhysicsSystem->SetPhysicsSettings(mPhysicsSettings); });
 			mDebugUI->CreateCheckBox(phys_settings, "Allow Sleeping", mPhysicsSettings.mAllowSleeping, [this](UICheckBox::EState inState) { mPhysicsSettings.mAllowSleeping = inState == UICheckBox::STATE_CHECKED; mPhysicsSystem->SetPhysicsSettings(mPhysicsSettings); });
 			mDebugUI->CreateCheckBox(phys_settings, "Check Active Triangle Edges", mPhysicsSettings.mCheckActiveEdges, [this](UICheckBox::EState inState) { mPhysicsSettings.mCheckActiveEdges = inState == UICheckBox::STATE_CHECKED; mPhysicsSystem->SetPhysicsSettings(mPhysicsSettings); });
 			mDebugUI->CreateCheckBox(phys_settings, "Record State For Playback", mRecordState, [this](UICheckBox::EState inState) { mRecordState = inState == UICheckBox::STATE_CHECKED; });

+ 37 - 0
Samples/Tests/General/PyramidTest.cpp

@@ -0,0 +1,37 @@
+// Jolt Physics Library (https://github.com/jrouwe/JoltPhysics)
+// SPDX-FileCopyrightText: 2023 Jorrit Rouwe
+// SPDX-License-Identifier: MIT
+
+#include <TestFramework.h>
+
+#include <Tests/General/PyramidTest.h>
+#include <Jolt/Physics/Collision/Shape/BoxShape.h>
+#include <Jolt/Physics/Body/BodyCreationSettings.h>
+#include <Layers.h>
+
+JPH_IMPLEMENT_RTTI_VIRTUAL(PyramidTest) 
+{ 
+	JPH_ADD_BASE_CLASS(PyramidTest, Test) 
+}
+
+void PyramidTest::Initialize() 
+{
+	// Floor
+	CreateFloor();
+
+	const float cBoxSize = 2.0f;
+	const float cBoxSeparation = 0.5f;
+	const float cHalfBoxSize = 0.5f * cBoxSize;
+	const int cPyramidHeight = 15;
+
+	RefConst<Shape> box_shape = new BoxShape(Vec3::sReplicate(cHalfBoxSize));
+
+	// Pyramid
+	for (int i = 0; i < cPyramidHeight; ++i)
+		for (int j = i / 2; j < cPyramidHeight - (i + 1) / 2; ++j)
+			for (int k = i / 2; k < cPyramidHeight - (i + 1) / 2; ++k)
+			{
+				RVec3 position(-cPyramidHeight + cBoxSize * j + (i & 1? cHalfBoxSize : 0.0f), 1.0f + (cBoxSize + cBoxSeparation) * i, -cPyramidHeight + cBoxSize * k + (i & 1? cHalfBoxSize : 0.0f));
+				mBodyInterface->CreateAndAddBody(BodyCreationSettings(box_shape, position, Quat::sIdentity(), EMotionType::Dynamic, Layers::MOVING), EActivation::Activate);
+			}
+}

+ 17 - 0
Samples/Tests/General/PyramidTest.h

@@ -0,0 +1,17 @@
+// Jolt Physics Library (https://github.com/jrouwe/JoltPhysics)
+// SPDX-FileCopyrightText: 2023 Jorrit Rouwe
+// SPDX-License-Identifier: MIT
+
+#pragma once
+
+#include <Tests/Test.h>
+
+// This test tests a large pyramid of boxes to check stacking and performance behavior.
+class PyramidTest : public Test
+{
+public:
+	JPH_DECLARE_RTTI_VIRTUAL(PyramidTest)
+
+	// See: Test
+	virtual void		Initialize() override;
+};

+ 14 - 14
Samples/Tests/Rig/RigPileTest.cpp

@@ -34,8 +34,12 @@ const char *RigPileTest::sScenes[] =
 
 #ifdef _DEBUG
 	const char *RigPileTest::sSceneName = "PerlinMesh";
+	int RigPileTest::sPileSize = 5;
+	int RigPileTest::sNumPilesPerAxis = 2;
 #else
 	const char *RigPileTest::sSceneName = "Terrain1";
+	int RigPileTest::sPileSize = 10;
+	int RigPileTest::sNumPilesPerAxis = 4;
 #endif
 
 RigPileTest::~RigPileTest()
@@ -82,25 +86,19 @@ void RigPileTest::Initialize()
 
 	const float cHorizontalSeparation = 4.0f;
 	const float cVerticalSeparation = 0.6f;
-#ifdef _DEBUG
-	const int cPileSize = 5;
-	const int cNumRows = 2;
-	const int cNumCols = 2;
-#else
-	const int cPileSize = 10;
-	const int cNumRows = 4;
-	const int cNumCols = 4;
-#endif
+
+	// Limit the size of the piles so we don't go over 160 ragdolls
+	int pile_size = min(sPileSize, 160 / Square(sNumPilesPerAxis));
 
 	// Create piles
 	default_random_engine random;
 	uniform_real_distribution<float> angle(0.0f, JPH_PI);
 	CollisionGroup::GroupID group_id = 1;
-	for (int row = 0; row < cNumRows; ++row)
-		for (int col = 0; col < cNumCols; ++col)
+	for (int row = 0; row < sNumPilesPerAxis; ++row)
+		for (int col = 0; col < sNumPilesPerAxis; ++col)
 		{
 			// Determine start location of ray
-			RVec3 start = RVec3(cHorizontalSeparation * (col - (cNumCols - 1) / 2.0f), 100, cHorizontalSeparation * (row - (cNumRows - 1) / 2.0f));
+			RVec3 start = RVec3(cHorizontalSeparation * (col - (sNumPilesPerAxis - 1) / 2.0f), 100, cHorizontalSeparation * (row - (sNumPilesPerAxis - 1) / 2.0f));
 
 			// Cast ray down to terrain
 			RayCastResult hit;
@@ -109,7 +107,7 @@ void RigPileTest::Initialize()
 			if (mPhysicsSystem->GetNarrowPhaseQuery().CastRay(ray, hit, SpecifiedBroadPhaseLayerFilter(BroadPhaseLayers::NON_MOVING), SpecifiedObjectLayerFilter(Layers::NON_MOVING)))
 				start = ray.GetPointOnRay(hit.mFraction);
 
-			for (int i = 0; i < cPileSize; ++i)
+			for (int i = 0; i < pile_size; ++i)
 			{
 				// Create ragdoll
 				Ref<Ragdoll> ragdoll = settings->CreateRagdoll(group_id++, 0, mPhysicsSystem);
@@ -144,5 +142,7 @@ void RigPileTest::CreateSettingsMenu(DebugUI *inUI, UIElement *inSubMenu)
 			inUI->CreateTextButton(scene_name, sScenes[i], [this, i]() { sSceneName = sScenes[i]; RestartTest(); });
 		inUI->ShowMenu(scene_name);
 	});
-}
 
+	inUI->CreateSlider(inSubMenu, "Num Ragdolls Per Pile", float(sPileSize), 1, 160, 1, [](float inValue) { sPileSize = (int)inValue; });
+	inUI->CreateSlider(inSubMenu, "Num Piles Per Axis", float(sNumPilesPerAxis), 1, 4, 1, [](float inValue) { sNumPilesPerAxis = (int)inValue; });
+}

+ 6 - 0
Samples/Tests/Rig/RigPileTest.h

@@ -32,6 +32,12 @@ private:
 	// Filename of animation to load for this test
 	static const char *		sSceneName;
 
+	// Number of ragdolls per pile
+	static int				sPileSize;
+
+	// Number of piles per axis
+	static int				sNumPilesPerAxis;
+
 	// All active ragdolls
 	Array<Ref<Ragdoll>>		mRagdolls;
 };