Browse Source

Move the universal geom pool to the new allocator

Panagiotis Christopoulos Charitos 3 years ago
parent
commit
5dae89c610

+ 4 - 6
AnKi/Core/App.cpp

@@ -278,7 +278,7 @@ Error App::initInternal(AllocAlignedCallback allocCb, void* allocCbUserData)
 	// GPU mem
 	//
 	m_unifiedGometryMemPool = newInstance<UnifiedGeometryMemoryPool>(m_mainPool);
-	ANKI_CHECK(m_unifiedGometryMemPool->init(&m_mainPool, m_gr, *m_config));
+	m_unifiedGometryMemPool->init(&m_mainPool, m_gr, *m_config);
 
 	m_stagingMem = newInstance<StagingGpuMemoryPool>(m_mainPool);
 	ANKI_CHECK(m_stagingMem->init(m_gr, *m_config));
@@ -464,6 +464,7 @@ Error App::mainLoop()
 
 			m_gr->swapBuffers();
 			m_stagingMem->endFrame();
+			m_unifiedGometryMemPool->endFrame();
 
 			// Update the trace info with some async loader stats
 			U64 asyncTaskCount = m_resources->getAsyncLoader().getCompletedTaskCount();
@@ -509,14 +510,11 @@ Error App::mainLoop()
 				in.m_cpuFreeCount = m_memStats.m_freeCount.load();
 
 				const GrManagerStats grStats = m_gr->getStats();
-				BuddyAllocatorBuilderStats vertMemStats;
-				m_unifiedGometryMemPool->getMemoryStats(vertMemStats);
+				m_unifiedGometryMemPool->getStats(in.m_unifiedGometryExternalFragmentation,
+												  in.m_unifiedGeometryAllocated, in.m_unifiedGeometryTotal);
 
 				in.m_gpuDeviceMemoryAllocated = grStats.m_deviceMemoryAllocated;
 				in.m_gpuDeviceMemoryInUse = grStats.m_deviceMemoryInUse;
-				in.m_globalVertexAllocated = vertMemStats.m_realAllocatedSize;
-				in.m_globalVertexUsed = vertMemStats.m_userAllocatedSize;
-				in.m_globalVertexExternalFragmentation = vertMemStats.m_externalFragmentation;
 
 				in.m_drawableCount = rqueue.countAllRenderables();
 				in.m_vkCommandBufferCount = grStats.m_commandBufferCount;

+ 7 - 46
AnKi/Core/GpuMemoryPools.cpp

@@ -10,61 +10,22 @@
 
 namespace anki {
 
-UnifiedGeometryMemoryPool::~UnifiedGeometryMemoryPool()
-{
-	// Do nothing
-}
-
-Error UnifiedGeometryMemoryPool::init(HeapMemoryPool* pool, GrManager* gr, const ConfigSet& cfg)
+void UnifiedGeometryMemoryPool::init(HeapMemoryPool* pool, GrManager* gr, const ConfigSet& cfg)
 {
 	ANKI_ASSERT(pool && gr);
-	m_gr = gr;
-
-	// Create the GPU buffer.
-	BufferInitInfo bufferInit("Global vertex & index");
-	bufferInit.m_size = cfg.getCoreGlobalVertexMemorySize();
-	if(!isPowerOfTwo(bufferInit.m_size))
-	{
-		ANKI_CORE_LOGE("core_globalVertexMemorySize should be a power of two (because of the buddy allocator");
-		return Error::kUserData;
-	}
-
-	bufferInit.m_usage = BufferUsageBit::kVertex | BufferUsageBit::kIndex | BufferUsageBit::kTransferDestination;
-	if(gr->getDeviceCapabilities().m_rayTracingEnabled)
-	{
-		bufferInit.m_usage |= BufferUsageBit::kAccelerationStructureBuild;
-	}
 
-	m_vertBuffer = gr->newBuffer(bufferInit);
+	const PtrSize poolSize = cfg.getCoreGlobalVertexMemorySize();
 
-	// Init the rest
-	m_buddyAllocator.init(pool, __builtin_ctzll(bufferInit.m_size));
+	const Array classes = {1_KB, 8_KB, 32_KB, 128_KB, 512_KB, 4_MB, 8_MB, 16_MB, poolSize};
 
-	return Error::kNone;
-}
+	BufferUsageBit buffUsage = BufferUsageBit::kVertex | BufferUsageBit::kIndex | BufferUsageBit::kTransferDestination;
 
-Error UnifiedGeometryMemoryPool::allocate(PtrSize size, U32 alignment, PtrSize& offset)
-{
-	U32 offset32;
-	const Bool success = m_buddyAllocator.allocate(size, alignment, offset32);
-	if(ANKI_UNLIKELY(!success))
+	if(gr->getDeviceCapabilities().m_rayTracingEnabled)
 	{
-		BuddyAllocatorBuilderStats stats;
-		m_buddyAllocator.getStats(stats);
-		ANKI_CORE_LOGE("Failed to allocate vertex memory of size %zu. The allocator has %zu (user requested %zu) out "
-					   "%zu allocated",
-					   size, stats.m_realAllocatedSize, stats.m_userAllocatedSize, m_vertBuffer->getSize());
-		return Error::kOutOfMemory;
+		buffUsage |= BufferUsageBit::kAccelerationStructureBuild;
 	}
 
-	offset = offset32;
-
-	return Error::kNone;
-}
-
-void UnifiedGeometryMemoryPool::free(PtrSize size, U32 alignment, PtrSize offset)
-{
-	m_buddyAllocator.free(U32(offset), size, alignment);
+	m_alloc.init(gr, pool, buffUsage, classes, poolSize, "UnifiedGeometry", false);
 }
 
 StagingGpuMemoryPool::~StagingGpuMemoryPool()

+ 20 - 13
AnKi/Core/GpuMemoryPools.h

@@ -8,7 +8,7 @@
 #include <AnKi/Core/Common.h>
 #include <AnKi/Gr/Buffer.h>
 #include <AnKi/Gr/Utils/FrameGpuAllocator.h>
-#include <AnKi/Util/BuddyAllocatorBuilder.h>
+#include <AnKi/Gr/Utils/SegregatedListsGpuAllocator.h>
 
 namespace anki {
 
@@ -26,30 +26,37 @@ public:
 
 	UnifiedGeometryMemoryPool(const UnifiedGeometryMemoryPool&) = delete; // Non-copyable
 
-	~UnifiedGeometryMemoryPool();
-
 	UnifiedGeometryMemoryPool& operator=(const UnifiedGeometryMemoryPool&) = delete; // Non-copyable
 
-	Error init(HeapMemoryPool* pool, GrManager* gr, const ConfigSet& cfg);
+	void init(HeapMemoryPool* pool, GrManager* gr, const ConfigSet& cfg);
 
-	Error allocate(PtrSize size, U32 alignment, PtrSize& offset);
+	void allocate(PtrSize size, U32 alignment, SegregatedListsGpuAllocatorToken& token)
+	{
+		m_alloc.allocate(size, alignment, token);
+	}
 
-	void free(PtrSize size, U32 alignment, PtrSize offset);
+	void free(const SegregatedListsGpuAllocatorToken& token)
+	{
+		m_alloc.free(token);
+	}
 
-	const BufferPtr& getVertexBuffer() const
+	void endFrame()
 	{
-		return m_vertBuffer;
+		m_alloc.endFrame();
 	}
 
-	void getMemoryStats(BuddyAllocatorBuilderStats& stats) const
+	const BufferPtr& getBuffer() const
 	{
-		m_buddyAllocator.getStats(stats);
+		return m_alloc.getGpuBuffer();
+	}
+
+	void getStats(F32& externalFragmentation, PtrSize& userAllocatedSize, PtrSize& totalSize) const
+	{
+		m_alloc.getStats(externalFragmentation, userAllocatedSize, totalSize);
 	}
 
 private:
-	GrManager* m_gr = nullptr;
-	BufferPtr m_vertBuffer;
-	BuddyAllocatorBuilder<32, Mutex> m_buddyAllocator;
+	SegregatedListsGpuAllocator m_alloc;
 };
 
 enum class StagingGpuMemoryType : U8

+ 3 - 3
AnKi/Core/StatsUi.defs.h

@@ -24,9 +24,9 @@ ANKI_STATS_UI_VALUE(PtrSize, cpuFreeCount, "Number of frees", ValueFlag::kNone)
 ANKI_STATS_UI_BEGIN_GROUP("GPU memory")
 ANKI_STATS_UI_VALUE(PtrSize, gpuDeviceMemoryAllocated, "Really allocated", ValueFlag::kNone | ValueFlag::kBytes)
 ANKI_STATS_UI_VALUE(PtrSize, gpuDeviceMemoryInUse, "Used", ValueFlag::kNone | ValueFlag::kBytes)
-ANKI_STATS_UI_VALUE(PtrSize, globalVertexAllocated, "Vertex really allocated", ValueFlag::kNone | ValueFlag::kBytes)
-ANKI_STATS_UI_VALUE(PtrSize, globalVertexUsed, "Vertex used", ValueFlag::kNone | ValueFlag::kBytes)
-ANKI_STATS_UI_VALUE(F32, globalVertexExternalFragmentation, "Vertex external fragmentation", ValueFlag::kNone)
+ANKI_STATS_UI_VALUE(PtrSize, unifiedGeometryAllocated, "Unified geom allocated", ValueFlag::kNone | ValueFlag::kBytes)
+ANKI_STATS_UI_VALUE(PtrSize, unifiedGeometryTotal, "Unified geom total", ValueFlag::kNone | ValueFlag::kBytes)
+ANKI_STATS_UI_VALUE(F32, unifiedGometryExternalFragmentation, "Unified geom ext fragmentation", ValueFlag::kNone)
 
 ANKI_STATS_UI_BEGIN_GROUP("Other")
 ANKI_STATS_UI_VALUE(U32, drawableCount, "Render queue drawbles", ValueFlag::kNone)

+ 4 - 2
AnKi/Gr/CMakeLists.txt

@@ -4,7 +4,8 @@ set(common_sources
 	RenderGraph.cpp
 	ShaderProgram.cpp
 	Utils/FrameGpuAllocator.cpp
-	Utils/Functions.cpp)
+	Utils/Functions.cpp
+	Utils/SegregatedListsGpuAllocator.cpp)
 
 set(common_headers
 	AccelerationStructure.h
@@ -30,7 +31,8 @@ set(common_headers
 	GrUpscaler.h
 	Utils/FrameGpuAllocator.h
 	Utils/Functions.h
-	Utils/InstantiationMacros.h)
+	Utils/InstantiationMacros.h
+	Utils/SegregatedListsGpuAllocator.h)
 
 if(VULKAN)
 	set(backend_sources

+ 1 - 0
AnKi/Gr/Common.h

@@ -36,6 +36,7 @@ class GrUpscalerInitInfo;
 #define ANKI_GR_LOGE(...) ANKI_LOG("GR", kError, __VA_ARGS__)
 #define ANKI_GR_LOGW(...) ANKI_LOG("GR", kWarning, __VA_ARGS__)
 #define ANKI_GR_LOGF(...) ANKI_LOG("GR", kFatal, __VA_ARGS__)
+#define ANKI_GR_LOGV(...) ANKI_LOG("GR", kVerbose, __VA_ARGS__)
 
 // Some constants
 constexpr U32 kMaxVertexAttributes = 8;

+ 269 - 0
AnKi/Gr/Utils/SegregatedListsGpuAllocator.cpp

@@ -0,0 +1,269 @@
+// Copyright (C) 2009-2022, Panagiotis Christopoulos Charitos and contributors.
+// All rights reserved.
+// Code licensed under the BSD License.
+// http://www.anki3d.org/LICENSE
+
+#include <AnKi/Gr/Utils/SegregatedListsGpuAllocator.h>
+#include <AnKi/Gr/GrManager.h>
+#include <AnKi/Gr/CommandBuffer.h>
+
+namespace anki {
+
+class SegregatedListsGpuAllocator::Chunk : public SegregatedListsAllocatorBuilderChunkBase
+{
+public:
+	PtrSize m_offsetInGpuBuffer;
+};
+
+class SegregatedListsGpuAllocator::BuilderInterface
+{
+public:
+	SegregatedListsGpuAllocator* m_parent = nullptr;
+
+	/// @name Interface methods
+	/// @{
+	U32 getClassCount() const
+	{
+		return m_parent->m_classes.getSize();
+	}
+
+	void getClassInfo(U32 idx, PtrSize& size) const
+	{
+		size = m_parent->m_classes[idx];
+	}
+
+	Error allocateChunk(Chunk*& newChunk, PtrSize& chunkSize)
+	{
+		return m_parent->allocateChunk(newChunk, chunkSize);
+	}
+
+	void deleteChunk(Chunk* chunk)
+	{
+		m_parent->deleteChunk(chunk);
+	}
+
+	BaseMemoryPool& getMemoryPool() const
+	{
+		return *m_parent->m_pool;
+	}
+
+	static constexpr PtrSize getMinSizeAlignment()
+	{
+		return 4;
+	}
+	/// @}
+};
+
+void SegregatedListsGpuAllocator::init(GrManager* gr, BaseMemoryPool* pool, BufferUsageBit gpuBufferUsage,
+									   ConstWeakArray<PtrSize> classUpperSizes, PtrSize initialGpuBufferSize,
+									   CString bufferName, Bool allowCoWs)
+{
+	ANKI_ASSERT(!isInitialized());
+	ANKI_ASSERT(gr && pool);
+	m_gr = gr;
+	m_pool = pool;
+
+	ANKI_ASSERT(gpuBufferUsage != BufferUsageBit::kNone);
+	m_bufferUsage = gpuBufferUsage;
+
+	ANKI_ASSERT(classUpperSizes.getSize() > 0);
+	m_classes.create(*m_pool, classUpperSizes.getSize());
+	for(U32 i = 0; i < m_classes.getSize(); ++i)
+	{
+		m_classes[i] = classUpperSizes[i];
+	}
+
+	ANKI_ASSERT(initialGpuBufferSize > 0);
+	m_initialBufferSize = initialGpuBufferSize;
+
+	m_bufferName.create(*m_pool, bufferName);
+
+	m_builder = newInstance<Builder>(*m_pool);
+	m_builder->getInterface().m_parent = this;
+
+	m_frame = 0;
+	m_allocatedSize = 0;
+	m_allowCoWs = allowCoWs;
+}
+
+void SegregatedListsGpuAllocator::destroy()
+{
+	if(!isInitialized())
+	{
+		return;
+	}
+
+	m_gr->finish();
+	m_gr = nullptr;
+
+	for(DynamicArray<SegregatedListsGpuAllocatorToken>& arr : m_garbage)
+	{
+		for(const SegregatedListsGpuAllocatorToken& token : arr)
+		{
+			m_builder->free(static_cast<Chunk*>(token.m_chunk), token.m_chunkOffset, token.m_size);
+		}
+
+		arr.destroy(*m_pool);
+	}
+
+	m_classes.destroy(*m_pool);
+	deleteInstance(*m_pool, m_builder);
+	m_gpuBuffer.reset(nullptr);
+	m_bufferName.destroy(*m_pool);
+
+	for(Chunk* chunk : m_deletedChunks)
+	{
+		deleteInstance(*m_pool, chunk);
+	}
+	m_deletedChunks.destroy(*m_pool);
+}
+
+Error SegregatedListsGpuAllocator::allocateChunk(Chunk*& newChunk, PtrSize& chunkSize)
+{
+	ANKI_ASSERT(isInitialized());
+
+	if(!m_gpuBuffer.isCreated())
+	{
+		// First chunk, our job is easy, create the buffer
+
+		BufferInitInfo buffInit(m_bufferName);
+		buffInit.m_size = m_initialBufferSize;
+		buffInit.m_usage = m_bufferUsage | BufferUsageBit::kAllTransfer;
+		m_gpuBuffer = m_gr->newBuffer(buffInit);
+
+		newChunk = newInstance<Chunk>(*m_pool);
+		newChunk->m_offsetInGpuBuffer = 0;
+	}
+	else if(m_deletedChunks.getSize() > 0)
+	{
+		// We already have a deleted chunk, use that
+
+		newChunk = m_deletedChunks.getBack();
+		m_deletedChunks.popBack(*m_pool);
+	}
+	else if(m_allowCoWs)
+	{
+		// Current buffer is not enough. Need to grow it which we can't grow GPU buffers do a CoW
+
+		ANKI_GR_LOGV("Will grow the %s buffer and perform a copy-on-write", m_bufferName.cstr());
+
+		// Create the new buffer
+		BufferInitInfo buffInit(m_bufferName);
+		buffInit.m_size = m_gpuBuffer->getSize() * 2;
+		buffInit.m_usage = m_bufferUsage | BufferUsageBit::kAllTransfer;
+		BufferPtr newBuffer = m_gr->newBuffer(buffInit);
+
+		// Do the copy
+		CommandBufferInitInfo cmdbInit("SegregatedListsGpuAllocator CoW");
+		cmdbInit.m_flags = CommandBufferFlag::kSmallBatch;
+		CommandBufferPtr cmdb = m_gr->newCommandBuffer(cmdbInit);
+
+		Array<BufferBarrierInfo, 2> barriers;
+		barriers[0].m_buffer = m_gpuBuffer.get();
+		barriers[0].m_previousUsage = m_bufferUsage;
+		barriers[0].m_nextUsage = BufferUsageBit::kTransferSource;
+		barriers[1].m_buffer = newBuffer.get();
+		barriers[1].m_previousUsage = BufferUsageBit::kNone;
+		barriers[1].m_nextUsage = BufferUsageBit::kTransferDestination;
+		cmdb->setPipelineBarrier({}, barriers, {});
+
+		cmdb->copyBufferToBuffer(m_gpuBuffer, 0, newBuffer, 0, m_gpuBuffer->getSize());
+
+		barriers[1].m_previousUsage = BufferUsageBit::kTransferDestination;
+		barriers[1].m_nextUsage = m_bufferUsage;
+		cmdb->setPipelineBarrier({}, ConstWeakArray<BufferBarrierInfo>{&barriers[1], 1}, {});
+
+		cmdb->flush();
+
+		// Create the new chunk
+		newChunk = newInstance<Chunk>(*m_pool);
+		newChunk->m_offsetInGpuBuffer = m_gpuBuffer->getSize();
+
+		// Switch the buffers
+		m_gpuBuffer = newBuffer;
+	}
+	else
+	{
+		ANKI_GR_LOGE("Out of memory and can't copy-on-write");
+		return Error::kOutOfMemory;
+	}
+
+	chunkSize = m_initialBufferSize;
+
+	return Error::kNone;
+}
+
+void SegregatedListsGpuAllocator::deleteChunk(Chunk* chunk)
+{
+	m_deletedChunks.emplaceBack(*m_pool, chunk);
+}
+
+void SegregatedListsGpuAllocator::allocate(PtrSize size, U32 alignment, SegregatedListsGpuAllocatorToken& token)
+{
+	ANKI_ASSERT(isInitialized());
+	ANKI_ASSERT(size > 0 && alignment > 0 && isPowerOfTwo(alignment));
+
+	LockGuard lock(m_lock);
+
+	Chunk* chunk;
+	PtrSize offset;
+	const Error err = m_builder->allocate(size, alignment, chunk, offset);
+	if(err)
+	{
+		ANKI_GR_LOGF("Failed to allocate memory");
+	}
+
+	token.m_chunk = chunk;
+	token.m_chunkOffset = offset;
+	token.m_offset = offset + chunk->m_offsetInGpuBuffer;
+	token.m_size = size;
+
+	m_allocatedSize += size;
+}
+
+void SegregatedListsGpuAllocator::free(const SegregatedListsGpuAllocatorToken& token)
+{
+	ANKI_ASSERT(isInitialized());
+
+	if(token.m_chunk == nullptr)
+	{
+		return;
+	}
+
+	LockGuard lock(m_lock);
+	m_garbage[m_frame].emplaceBack(*m_pool, token);
+}
+
+void SegregatedListsGpuAllocator::endFrame()
+{
+	ANKI_ASSERT(isInitialized());
+
+	LockGuard lock(m_lock);
+
+	m_frame = (m_frame + 1) % kMaxFramesInFlight;
+
+	// Throw out the garbage
+	for(SegregatedListsGpuAllocatorToken& token : m_garbage[m_frame])
+	{
+		m_builder->free(static_cast<Chunk*>(token.m_chunk), token.m_chunkOffset, token.m_size);
+
+		ANKI_ASSERT(m_allocatedSize >= token.m_size);
+		m_allocatedSize -= token.m_size;
+	}
+
+	m_garbage[m_frame].destroy(*m_pool);
+}
+
+void SegregatedListsGpuAllocator::getStats(F32& externalFragmentation, PtrSize& userAllocatedSize,
+										   PtrSize& totalSize) const
+{
+	ANKI_ASSERT(isInitialized());
+
+	LockGuard lock(m_lock);
+
+	externalFragmentation = m_builder->computeExternalFragmentation();
+	userAllocatedSize = m_allocatedSize;
+	totalSize = m_gpuBuffer->getSize();
+}
+
+} // end namespace anki

+ 108 - 0
AnKi/Gr/Utils/SegregatedListsGpuAllocator.h

@@ -0,0 +1,108 @@
+// Copyright (C) 2009-2022, Panagiotis Christopoulos Charitos and contributors.
+// All rights reserved.
+// Code licensed under the BSD License.
+// http://www.anki3d.org/LICENSE
+
+#include <AnKi/Util/SegregatedListsAllocatorBuilder.h>
+#include <AnKi/Gr/Buffer.h>
+
+namespace anki {
+
+/// @addtogroup graphics
+/// @{
+
+/// The result of an allocation of SegregatedListsGpuAllocator.
+/// @memberof SegregatedListsGpuAllocator
+class SegregatedListsGpuAllocatorToken
+{
+	friend class SegregatedListsGpuAllocator;
+
+public:
+	/// The offset in the SegregatedListsGpuAllocatorToken::getBuffer() buffer.
+	PtrSize m_offset = kMaxPtrSize;
+
+private:
+	void* m_chunk = nullptr;
+	PtrSize m_chunkOffset = kMaxPtrSize;
+	PtrSize m_size = kMaxPtrSize;
+};
+
+/// GPU memory allocator based on segregated lists. It allocates a GPU buffer with some initial size. If there is a need
+/// to grow it allocates a bigger buffer and copies contents of the old one to the new (CoW).
+class SegregatedListsGpuAllocator
+{
+public:
+	SegregatedListsGpuAllocator() = default;
+
+	~SegregatedListsGpuAllocator()
+	{
+		destroy();
+	}
+
+	SegregatedListsGpuAllocator(const SegregatedListsGpuAllocator&) = delete;
+
+	SegregatedListsGpuAllocator& operator=(const SegregatedListsGpuAllocator&) = delete;
+
+	void init(GrManager* gr, BaseMemoryPool* pool, BufferUsageBit gpuBufferUsage,
+			  ConstWeakArray<PtrSize> classUpperSizes, PtrSize initialGpuBufferSize, CString bufferName,
+			  Bool allowCoWs);
+
+	void destroy();
+
+	/// Allocate memory.
+	/// @note It's thread-safe.
+	void allocate(PtrSize size, U32 alignment, SegregatedListsGpuAllocatorToken& token);
+
+	/// Free memory.
+	/// @note It's thread-safe.
+	void free(const SegregatedListsGpuAllocatorToken& token);
+
+	/// @note It's thread-safe.
+	void endFrame();
+
+	/// Need to be checking this constantly to get the updated buffer in case of CoWs.
+	/// @note It's not thread-safe.
+	const BufferPtr& getGpuBuffer() const
+	{
+		ANKI_ASSERT(m_gpuBuffer.isCreated() && "The buffer hasn't been created yet");
+		return m_gpuBuffer;
+	}
+
+	/// @note It's thread-safe.
+	void getStats(F32& externalFragmentation, PtrSize& userAllocatedSize, PtrSize& totalSize) const;
+
+private:
+	class BuilderInterface;
+	class Chunk;
+	using Builder = SegregatedListsAllocatorBuilder<Chunk, BuilderInterface, DummyMutex>;
+
+	GrManager* m_gr = nullptr;
+	mutable BaseMemoryPool* m_pool = nullptr;
+	BufferUsageBit m_bufferUsage = BufferUsageBit::kNone;
+	DynamicArray<PtrSize> m_classes;
+	PtrSize m_initialBufferSize = 0;
+	String m_bufferName;
+
+	mutable Mutex m_lock;
+
+	Builder* m_builder = nullptr;
+	BufferPtr m_gpuBuffer;
+	PtrSize m_allocatedSize = 0;
+
+	DynamicArray<Chunk*> m_deletedChunks;
+
+	Array<DynamicArray<SegregatedListsGpuAllocatorToken>, kMaxFramesInFlight> m_garbage;
+	U8 m_frame = 0;
+	Bool m_allowCoWs = true;
+
+	Error allocateChunk(Chunk*& newChunk, PtrSize& chunkSize);
+	void deleteChunk(Chunk* chunk);
+
+	Bool isInitialized() const
+	{
+		return m_gr != nullptr;
+	}
+};
+/// @}
+
+} // end namespace anki

+ 20 - 31
AnKi/Resource/MeshResource.cpp

@@ -59,23 +59,11 @@ MeshResource::~MeshResource()
 
 	for(Lod& lod : m_lods)
 	{
-		if(lod.m_unifiedGeometryIndexBufferOffset != kMaxPtrSize)
-		{
-			const U32 alignment = getIndexSize(m_indexType);
-			const PtrSize size = lod.m_indexCount * PtrSize(alignment);
-			getManager().getUnifiedGeometryMemoryPool().free(size, alignment, lod.m_unifiedGeometryIndexBufferOffset);
-		}
+		getManager().getUnifiedGeometryMemoryPool().free(lod.m_indexBufferAllocationToken);
 
 		for(VertexStreamId stream : EnumIterable(VertexStreamId::kMeshRelatedFirst, VertexStreamId::kMeshRelatedCount))
 		{
-			if(lod.m_unifiedGeometryVertBufferOffsets[stream] != kMaxPtrSize)
-			{
-				const U32 alignment = getFormatInfo(kMeshRelatedVertexStreamFormats[stream]).m_texelSize;
-				const PtrSize size = PtrSize(alignment) * lod.m_vertexCount;
-
-				getManager().getUnifiedGeometryMemoryPool().free(size, alignment,
-																 lod.m_unifiedGeometryVertBufferOffsets[stream]);
-			}
+			getManager().getUnifiedGeometryMemoryPool().free(lod.m_vertexBuffersAllocationToken[stream]);
 		}
 	}
 
@@ -92,7 +80,6 @@ Error MeshResource::load(const ResourceFilename& filename, Bool async)
 	getFilepathFilename(filename, basename);
 
 	const Bool rayTracingEnabled = getManager().getGrManager().getDeviceCapabilities().m_rayTracingEnabled;
-	BufferPtr unifiedGeometryBuffer = getManager().getUnifiedGeometryMemoryPool().getVertexBuffer();
 
 	if(async)
 	{
@@ -137,8 +124,8 @@ Error MeshResource::load(const ResourceFilename& filename, Bool async)
 		lod.m_indexCount = header.m_totalIndexCounts[l];
 		ANKI_ASSERT((lod.m_indexCount % 3) == 0 && "Expecting triangles");
 		const PtrSize indexBufferSize = PtrSize(lod.m_indexCount) * getIndexSize(m_indexType);
-		ANKI_CHECK(getManager().getUnifiedGeometryMemoryPool().allocate(indexBufferSize, getIndexSize(m_indexType),
-																		lod.m_unifiedGeometryIndexBufferOffset));
+		getManager().getUnifiedGeometryMemoryPool().allocate(indexBufferSize, getIndexSize(m_indexType),
+															 lod.m_indexBufferAllocationToken);
 
 		// Vertex stuff
 		lod.m_vertexCount = header.m_totalVertexCounts[l];
@@ -146,7 +133,6 @@ Error MeshResource::load(const ResourceFilename& filename, Bool async)
 		{
 			if(header.m_vertexAttributes[stream].m_format == Format::kNone)
 			{
-				lod.m_unifiedGeometryVertBufferOffsets[stream] = kMaxPtrSize;
 				continue;
 			}
 
@@ -155,8 +141,8 @@ Error MeshResource::load(const ResourceFilename& filename, Bool async)
 			const U32 texelSize = getFormatInfo(kMeshRelatedVertexStreamFormats[stream]).m_texelSize;
 			const PtrSize vertexBufferSize = PtrSize(lod.m_vertexCount) * texelSize;
 			const U32 alignment = 4;
-			ANKI_CHECK(getManager().getUnifiedGeometryMemoryPool().allocate(
-				vertexBufferSize, alignment, lod.m_unifiedGeometryVertBufferOffsets[stream]));
+			getManager().getUnifiedGeometryMemoryPool().allocate(vertexBufferSize, alignment,
+																 lod.m_vertexBuffersAllocationToken[stream]);
 		}
 
 		// BLAS
@@ -166,13 +152,13 @@ Error MeshResource::load(const ResourceFilename& filename, Bool async)
 				StringRaii(&getTempMemoryPool()).sprintf("%s_%s", "Blas", basename.cstr()));
 			inf.m_type = AccelerationStructureType::kBottomLevel;
 
-			inf.m_bottomLevel.m_indexBuffer = unifiedGeometryBuffer;
-			inf.m_bottomLevel.m_indexBufferOffset = lod.m_unifiedGeometryIndexBufferOffset;
+			inf.m_bottomLevel.m_indexBuffer = getManager().getUnifiedGeometryMemoryPool().getBuffer();
+			inf.m_bottomLevel.m_indexBufferOffset = lod.m_indexBufferAllocationToken.m_offset;
 			inf.m_bottomLevel.m_indexCount = lod.m_indexCount;
 			inf.m_bottomLevel.m_indexType = m_indexType;
-			inf.m_bottomLevel.m_positionBuffer = unifiedGeometryBuffer;
+			inf.m_bottomLevel.m_positionBuffer = getManager().getUnifiedGeometryMemoryPool().getBuffer();
 			inf.m_bottomLevel.m_positionBufferOffset =
-				lod.m_unifiedGeometryVertBufferOffsets[VertexStreamId::kPosition];
+				lod.m_vertexBuffersAllocationToken[VertexStreamId::kPosition].m_offset;
 			inf.m_bottomLevel.m_positionStride =
 				getFormatInfo(kMeshRelatedVertexStreamFormats[VertexStreamId::kPosition]).m_texelSize;
 			inf.m_bottomLevel.m_positionsFormat = kMeshRelatedVertexStreamFormats[VertexStreamId::kPosition];
@@ -191,7 +177,8 @@ Error MeshResource::load(const ResourceFilename& filename, Bool async)
 
 		for(const Lod& lod : m_lods)
 		{
-			cmdb->fillBuffer(unifiedGeometryBuffer, lod.m_unifiedGeometryIndexBufferOffset,
+			cmdb->fillBuffer(getManager().getUnifiedGeometryMemoryPool().getBuffer(),
+							 lod.m_indexBufferAllocationToken.m_offset,
 							 PtrSize(lod.m_indexCount) * getIndexSize(m_indexType), 0);
 
 			for(VertexStreamId stream :
@@ -199,7 +186,8 @@ Error MeshResource::load(const ResourceFilename& filename, Bool async)
 			{
 				if(header.m_vertexAttributes[stream].m_format != Format::kNone)
 				{
-					cmdb->fillBuffer(unifiedGeometryBuffer, lod.m_unifiedGeometryVertBufferOffsets[stream],
+					cmdb->fillBuffer(getManager().getUnifiedGeometryMemoryPool().getBuffer(),
+									 lod.m_vertexBuffersAllocationToken[stream].m_offset,
 									 PtrSize(lod.m_vertexCount)
 										 * getFormatInfo(kMeshRelatedVertexStreamFormats[stream]).m_texelSize,
 									 0);
@@ -207,8 +195,9 @@ Error MeshResource::load(const ResourceFilename& filename, Bool async)
 			}
 		}
 
-		const BufferBarrierInfo barrier = {unifiedGeometryBuffer.get(), BufferUsageBit::kTransferDestination,
-										   BufferUsageBit::kVertex, 0, kMaxPtrSize};
+		const BufferBarrierInfo barrier = {getManager().getUnifiedGeometryMemoryPool().getBuffer().get(),
+										   BufferUsageBit::kTransferDestination, BufferUsageBit::kVertex, 0,
+										   kMaxPtrSize};
 
 		cmdb->setPipelineBarrier({}, {&barrier, 1}, {});
 
@@ -238,7 +227,7 @@ Error MeshResource::loadAsync(MeshBinaryLoader& loader) const
 	Array<TransferGpuAllocatorHandle, kMaxLodCount*(U32(VertexStreamId::kMeshRelatedCount) + 1)> handles;
 	U32 handleCount = 0;
 
-	BufferPtr unifiedGeometryBuffer = getManager().getUnifiedGeometryMemoryPool().getVertexBuffer();
+	BufferPtr unifiedGeometryBuffer = getManager().getUnifiedGeometryMemoryPool().getBuffer();
 	const BufferUsageBit unifiedGeometryBufferNonTransferUsage =
 		unifiedGeometryBuffer->getBufferUsage() ^ BufferUsageBit::kTransferDestination;
 
@@ -268,7 +257,7 @@ Error MeshResource::loadAsync(MeshBinaryLoader& loader) const
 			ANKI_CHECK(loader.storeIndexBuffer(lodIdx, data, indexBufferSize));
 
 			cmdb->copyBufferToBuffer(handle.getBuffer(), handle.getOffset(), unifiedGeometryBuffer,
-									 lod.m_unifiedGeometryIndexBufferOffset, handle.getRange());
+									 lod.m_indexBufferAllocationToken.m_offset, handle.getRange());
 		}
 
 		// Upload vert buffers
@@ -292,7 +281,7 @@ Error MeshResource::loadAsync(MeshBinaryLoader& loader) const
 
 			// Copy
 			cmdb->copyBufferToBuffer(handle.getBuffer(), handle.getOffset(), unifiedGeometryBuffer,
-									 lod.m_unifiedGeometryVertBufferOffsets[stream], handle.getRange());
+									 lod.m_vertexBuffersAllocationToken[stream].m_offset, handle.getRange());
 		}
 	}
 

+ 5 - 10
AnKi/Resource/MeshResource.h

@@ -10,6 +10,7 @@
 #include <AnKi/Gr.h>
 #include <AnKi/Collision/Aabb.h>
 #include <AnKi/Shaders/Include/MeshTypes.h>
+#include <AnKi/Core/GpuMemoryPools.h>
 
 namespace anki {
 
@@ -54,7 +55,7 @@ public:
 	/// Get all info around vertex indices.
 	void getIndexBufferInfo(U32 lod, PtrSize& buffOffset, U32& indexCount, IndexType& indexType) const
 	{
-		buffOffset = m_lods[lod].m_unifiedGeometryIndexBufferOffset;
+		buffOffset = m_lods[lod].m_indexBufferAllocationToken.m_offset;
 		ANKI_ASSERT(isAligned(getIndexSize(m_indexType), buffOffset));
 		indexCount = m_lods[lod].m_indexCount;
 		indexType = m_indexType;
@@ -63,7 +64,7 @@ public:
 	/// Get vertex buffer info.
 	void getVertexStreamInfo(U32 lod, VertexStreamId stream, PtrSize& bufferOffset, U32& vertexCount) const
 	{
-		bufferOffset = m_lods[lod].m_unifiedGeometryVertBufferOffsets[stream];
+		bufferOffset = m_lods[lod].m_vertexBuffersAllocationToken[stream].m_offset;
 		vertexCount = m_lods[lod].m_vertexCount;
 	}
 
@@ -101,19 +102,13 @@ private:
 	class Lod
 	{
 	public:
-		PtrSize m_unifiedGeometryIndexBufferOffset = kMaxPtrSize;
-		Array<PtrSize, U32(VertexStreamId::kMeshRelatedCount)> m_unifiedGeometryVertBufferOffsets;
+		SegregatedListsGpuAllocatorToken m_indexBufferAllocationToken;
+		Array<SegregatedListsGpuAllocatorToken, U32(VertexStreamId::kMeshRelatedCount)> m_vertexBuffersAllocationToken;
 
 		U32 m_indexCount = 0;
 		U32 m_vertexCount = 0;
 
 		AccelerationStructurePtr m_blas;
-
-		Lod()
-		{
-			m_unifiedGeometryVertBufferOffsets.fill(m_unifiedGeometryVertBufferOffsets.getBegin(),
-													m_unifiedGeometryVertBufferOffsets.getEnd(), kMaxPtrSize);
-		}
 	};
 
 	class SubMesh

+ 2 - 2
AnKi/Scene/ModelNode.cpp

@@ -307,12 +307,12 @@ void ModelNode::draw(RenderQueueDrawContext& ctx, ConstWeakArray<void*> userData
 
 			cmdb->setVertexAttribute(attribLocation, bufferBinding, fmt, relativeOffset);
 
-			cmdb->bindVertexBuffer(bufferBinding, getUnifiedGeometryMemoryPool().getVertexBuffer(),
+			cmdb->bindVertexBuffer(bufferBinding, getUnifiedGeometryMemoryPool().getBuffer(),
 								   modelInf.m_vertexBufferOffsets[streamId], vertexStride, VertexStepRate::kVertex);
 		}
 
 		// Bind index buffer
-		cmdb->bindIndexBuffer(getUnifiedGeometryMemoryPool().getVertexBuffer(), modelInf.m_indexBufferOffset,
+		cmdb->bindIndexBuffer(getUnifiedGeometryMemoryPool().getBuffer(), modelInf.m_indexBufferOffset,
 							  IndexType::kU16);
 
 		// Draw

+ 15 - 4
AnKi/Util/SegregatedListsAllocatorBuilder.h

@@ -30,14 +30,13 @@ public:
 
 /// The base class for all user memory chunks of SegregatedListsAllocatorBuilder.
 /// @memberof SegregatedListsAllocatorBuilder
-template<typename TChunk, U32 kClassCount>
 class SegregatedListsAllocatorBuilderChunkBase
 {
-	template<typename TChunk_, typename TInterface, typename TLock>
+	template<typename TChunk, typename TInterface, typename TLock>
 	friend class SegregatedListsAllocatorBuilder;
 
 private:
-	Array<DynamicArray<detail::SegregatedListsAllocatorBuilderFreeBlock>, kClassCount> m_freeLists;
+	DynamicArray<DynamicArray<detail::SegregatedListsAllocatorBuilderFreeBlock>> m_freeLists;
 	PtrSize m_totalSize = 0;
 	PtrSize m_freeSize = 0;
 };
@@ -48,7 +47,7 @@ private:
 /// @tparam TInterface The interface that contains the following members:
 ///                    @code
 ///                    /// The number of classes
-///                    static constexpr U32 getClassCount();
+///                    U32 getClassCount() const;
 ///                    /// Max size for each class.
 ///                    void getClassInfo(U32 idx, PtrSize& size) const;
 ///                    /// Allocates a new user defined chunk of memory.
@@ -57,6 +56,8 @@ private:
 ///                    void deleteChunk(TChunk* chunk);
 ///                    /// Get an allocator for internal allocations of the builder.
 ///                    SomeMemoryPool& getMemoryPool() const;
+///                    /// Get the min alignment that will be required.
+///                    PtrSize getMinSizeAlignment() const;
 ///                    @endcode
 /// @tparam TLock User defined lock (eg Mutex).
 template<typename TChunk, typename TInterface, typename TLock>
@@ -96,6 +97,16 @@ public:
 	/// Adam Sawicki metric. 0.0 is no fragmentation, 1.0 is totally fragmented.
 	[[nodiscard]] F32 computeExternalFragmentationSawicki(PtrSize baseSize = 1) const;
 
+	TLock& getLock() const
+	{
+		return m_lock;
+	}
+
+	TInterface& getInterface()
+	{
+		return m_interface;
+	}
+
 private:
 	using FreeBlock = detail::SegregatedListsAllocatorBuilderFreeBlock;
 	using ChunksIterator = typename DynamicArray<TChunk*>::Iterator;

+ 15 - 10
AnKi/Util/SegregatedListsAllocatorBuilder.inl.h

@@ -18,7 +18,7 @@ U32 SegregatedListsAllocatorBuilder<TChunk, TInterface, TLock>::findClass(PtrSiz
 {
 	ANKI_ASSERT(size > 0 && alignment > 0);
 
-	for(U32 i = 0; i < TInterface::getClassCount(); ++i)
+	for(U32 i = 0; i < m_interface.getClassCount(); ++i)
 	{
 		PtrSize maxSize;
 		m_interface.getClassInfo(i, maxSize);
@@ -113,7 +113,7 @@ void SegregatedListsAllocatorBuilder<TChunk, TInterface, TLock>::placeFreeBlock(
 	U32 leftClass = kMaxU32;
 	U32 rightBlock = kMaxU32;
 	U32 rightClass = kMaxU32;
-	for(U32 classIdx = 0; classIdx < TInterface::getClassCount(); ++classIdx)
+	for(U32 classIdx = 0; classIdx < m_interface.getClassCount(); ++classIdx)
 	{
 		const DynamicArray<FreeBlock>& freeLists = chunk.m_freeLists[classIdx];
 
@@ -225,6 +225,7 @@ void SegregatedListsAllocatorBuilder<TChunk, TInterface, TLock>::placeFreeBlock(
 
 	// Store the new block
 	const U32 newClassIdx = findClass(newBlock.m_size, 1);
+	ANKI_ASSERT(newClassIdx != kMaxU32);
 	chunk.m_freeLists[newClassIdx].emplaceBack(m_interface.getMemoryPool(), newBlock);
 
 	std::sort(chunk.m_freeLists[newClassIdx].getBegin(), chunk.m_freeLists[newClassIdx].getEnd(),
@@ -240,12 +241,14 @@ void SegregatedListsAllocatorBuilder<TChunk, TInterface, TLock>::placeFreeBlock(
 		// Chunk completely free, delete it
 
 		U32 blockCount = 0;
-		for(U32 classIdx = 0; classIdx < TInterface::getClassCount(); ++classIdx)
+		for(U32 classIdx = 0; classIdx < m_interface.getClassCount(); ++classIdx)
 		{
 			blockCount += chunk.m_freeLists[classIdx].getSize();
 			chunk.m_freeLists[classIdx].destroy(m_interface.getMemoryPool());
 		}
 
+		chunk.m_freeLists.destroy(m_interface.getMemoryPool());
+
 		ANKI_ASSERT(blockCount == 1);
 
 		m_chunks.erase(m_interface.getMemoryPool(), chunkIt);
@@ -284,7 +287,7 @@ Error SegregatedListsAllocatorBuilder<TChunk, TInterface, TLock>::allocate(PtrSi
 	{
 		classIdx = startingClassIdx;
 
-		while(classIdx < TInterface::getClassCount())
+		while(classIdx < m_interface.getClassCount())
 		{
 			// Find the best fit
 			for(FreeBlock& block : (*chunkIt)->m_freeLists[classIdx])
@@ -322,10 +325,12 @@ Error SegregatedListsAllocatorBuilder<TChunk, TInterface, TLock>::allocate(PtrSi
 		PtrSize chunkSize;
 		TChunk* chunk;
 		ANKI_CHECK(m_interface.allocateChunk(chunk, chunkSize));
+		chunk->m_freeLists.resize(m_interface.getMemoryPool(), m_interface.getClassCount());
 
 		if(chunkSize < size)
 		{
 			ANKI_UTIL_LOGE("Chunk allocated can't fit the current allocation of %zu", origSize);
+			chunk->m_freeLists.destroy(m_interface.getMemoryPool());
 			m_interface.deleteChunk(chunk);
 			return Error::kOutOfMemory;
 		}
@@ -428,7 +433,7 @@ Error SegregatedListsAllocatorBuilder<TChunk, TInterface, TLock>::validate() con
 						 chunk->m_totalSize);
 
 		PtrSize freeSize = 0;
-		for(U32 c = 0; c < TInterface::getClassCount(); ++c)
+		for(U32 c = 0; c < m_interface.getClassCount(); ++c)
 		{
 			for(U32 i = 0; i < chunk->m_freeLists[c].getSize(); ++i)
 			{
@@ -461,13 +466,13 @@ Error SegregatedListsAllocatorBuilder<TChunk, TInterface, TLock>::validate() con
 	chunkIdx = 0;
 	for(const TChunk* chunk : m_chunks)
 	{
-		for(U32 c = 0; c < TInterface::getClassCount(); ++c)
+		for(U32 c = 0; c < m_interface.getClassCount(); ++c)
 		{
 			for(U32 i = 0; i < chunk->m_freeLists[c].getSize(); ++i)
 			{
 				const FreeBlock& crnt = chunk->m_freeLists[c][i];
 
-				for(U32 c2 = 0; c2 < TInterface::getClassCount(); ++c2)
+				for(U32 c2 = 0; c2 < m_interface.getClassCount(); ++c2)
 				{
 					for(U32 j = 0; j < chunk->m_freeLists[c2].getSize(); ++j)
 					{
@@ -524,7 +529,7 @@ void SegregatedListsAllocatorBuilder<TChunk, TInterface, TLock>::printFreeBlocks
 		strList.pushBackSprintf("Chunk #%u, total size %zu, free size %zu\n", chunkCount, chunk->m_totalSize,
 								chunk->m_freeSize);
 
-		for(U32 c = 0; c < TInterface::getClassCount(); ++c)
+		for(U32 c = 0; c < m_interface.getClassCount(); ++c)
 		{
 			if(chunk->m_freeLists[c].getSize())
 			{
@@ -560,7 +565,7 @@ F32 SegregatedListsAllocatorBuilder<TChunk, TInterface, TLock>::computeExternalF
 	{
 		PtrSize largestFreeBlockSize = 0;
 
-		for(U32 c = 0; c < TInterface::getClassCount(); ++c)
+		for(U32 c = 0; c < m_interface.getClassCount(); ++c)
 		{
 			for(const FreeBlock& block : chunk->m_freeLists[c])
 			{
@@ -590,7 +595,7 @@ F32 SegregatedListsAllocatorBuilder<TChunk, TInterface, TLock>::computeExternalF
 	{
 		F64 quality = 0.0;
 
-		for(U32 c = 0; c < TInterface::getClassCount(); ++c)
+		for(U32 c = 0; c < m_interface.getClassCount(); ++c)
 		{
 			for(const FreeBlock& block : chunk->m_freeLists[c])
 			{

+ 4 - 5
Tests/Util/SegregatedListsAllocatorBuilder.cpp

@@ -11,8 +11,7 @@ using namespace anki;
 
 static constexpr U32 kClassCount = 6;
 
-class SegregatedListsAllocatorBuilderChunk :
-	public SegregatedListsAllocatorBuilderChunkBase<SegregatedListsAllocatorBuilderChunk, kClassCount>
+class SegregatedListsAllocatorBuilderChunk : public SegregatedListsAllocatorBuilderChunkBase
 {
 };
 
@@ -22,14 +21,14 @@ public:
 	HeapMemoryPool m_pool = {allocAligned, nullptr};
 	static constexpr PtrSize kChunkSize = 100_MB;
 
-	static constexpr U32 getClassCount()
+	U32 getClassCount() const
 	{
 		return kClassCount;
 	}
 
 	void getClassInfo(U32 idx, PtrSize& size) const
 	{
-		static const Array<PtrSize, getClassCount()> classes = {512_KB, 1_MB, 5_MB, 10_MB, 30_MB, kChunkSize};
+		static const Array<PtrSize, kClassCount> classes = {512_KB, 1_MB, 5_MB, 10_MB, 30_MB, kChunkSize};
 		size = classes[idx];
 	}
 
@@ -57,7 +56,7 @@ public:
 };
 
 using SLAlloc = SegregatedListsAllocatorBuilder<SegregatedListsAllocatorBuilderChunk,
-												SegregatedListsAllocatorBuilderInterface, DummyMutex>;
+												SegregatedListsAllocatorBuilderInterface, Mutex>;
 
 template<typename TAlloc>
 static void printAllocatorBuilder(const TAlloc& sl)