First look at separating main-thread chunk gen impl from World

this is intended to open the gates to having generators that run on the main thread (e.g. flat), or that use other methods of generating chunks entirely (such as requesting them from a chunk server).
This commit is contained in:
Dylan K. Taylor 2025-04-05 22:58:13 +01:00
parent 071c15d7de
commit d338e7624c
No known key found for this signature in database
GPG Key ID: 8927471A91CAFD3D
3 changed files with 481 additions and 327 deletions

View File

@ -0,0 +1,397 @@
<?php
/*
*
* ____ _ _ __ __ _ __ __ ____
* | _ \ ___ ___| | _____| |_| \/ (_)_ __ ___ | \/ | _ \
* | |_) / _ \ / __| |/ / _ \ __| |\/| | | '_ \ / _ \_____| |\/| | |_) |
* | __/ (_) | (__| < __/ |_| | | | | | | | __/_____| | | | __/
* |_| \___/ \___|_|\_\___|\__|_| |_|_|_| |_|\___| |_| |_|_|
*
* This program is free software: you can redistribute it and/or modify
* it under the terms of the GNU Lesser General Public License as published by
* the Free Software Foundation, either version 3 of the License, or
* (at your option) any later version.
*
* @author PocketMine Team
* @link http://www.pocketmine.net/
*
*
*/
declare(strict_types=1);
namespace pocketmine\world;
use pocketmine\event\world\ChunkPopulateEvent;
use pocketmine\promise\Promise;
use pocketmine\promise\PromiseResolver;
use pocketmine\scheduler\AsyncPool;
use pocketmine\utils\AssumptionFailedError;
use pocketmine\world\format\Chunk;
use pocketmine\world\generator\GeneratorRegisterTask;
use pocketmine\world\generator\PopulationTask;
use function array_key_exists;
use function assert;
use function count;
/**
* @phpstan-import-type ChunkPosHash from World
*/
final class AsyncChunkGenerator implements ChunkGenerator{
/**
* @var bool[] chunkHash => isValid
* @phpstan-var array<ChunkPosHash, bool>
*/
private array $activeChunkPopulationTasks = [];
/**
* @var PromiseResolver[] chunkHash => promise
* @phpstan-var array<ChunkPosHash, PromiseResolver<Chunk>>
*/
private array $chunkPopulationRequestMap = [];
/**
* @var \SplQueue (queue of chunkHashes)
* @phpstan-var \SplQueue<ChunkPosHash>
*/
private \SplQueue $chunkPopulationRequestQueue;
/**
* @var true[] chunkHash => dummy
* @phpstan-var array<ChunkPosHash, true>
*/
private array $chunkPopulationRequestQueueIndex = [];
/**
* @var true[]
* @phpstan-var array<int, true>
*/
private array $generatorRegisteredWorkers = [];
/** @phpstan-var \Closure(int) : void */
private \Closure $workerStartHook;
public function __construct(
private readonly AsyncPool $workerPool,
private readonly \Logger $logger,
private readonly int $maxConcurrentChunkPopulationTasks = 2,
){
$this->chunkPopulationRequestQueue = new \SplQueue();
//TODO: don't love the circular reference here, but we need to make sure this gets cleaned up on shutdown
$this->workerStartHook = function(int $workerId) : void{
if(array_key_exists($workerId, $this->generatorRegisteredWorkers)){
$this->logger->debug("Worker $workerId with previously registered generator restarted, flagging as unregistered");
unset($this->generatorRegisteredWorkers[$workerId]);
}
};
$this->workerPool->addWorkerStartHook($this->workerStartHook);
}
private function registerGeneratorToWorker(World $world, int $worker) : void{
$world->getLogger()->debug("Registering generator on worker $worker");
$this->workerPool->submitTaskToWorker(new GeneratorRegisterTask(
$world,
$world->getGeneratorClass(),
$world->getProvider()->getWorldData()->getGeneratorOptions()
), $worker);
$this->generatorRegisteredWorkers[$worker] = true;
}
private function addChunkHashToPopulationRequestQueue(int $chunkHash) : void{
if(!isset($this->chunkPopulationRequestQueueIndex[$chunkHash])){
$this->chunkPopulationRequestQueue->enqueue($chunkHash);
$this->chunkPopulationRequestQueueIndex[$chunkHash] = true;
}
}
/**
* @phpstan-return Promise<Chunk>
*/
private function enqueuePopulationRequest(World $world, int $chunkX, int $chunkZ, ?ChunkLoader $associatedChunkLoader) : Promise{
$chunkHash = World::chunkHash($chunkX, $chunkZ);
$this->addChunkHashToPopulationRequestQueue($chunkHash);
/** @phpstan-var PromiseResolver<Chunk> $resolver */
$resolver = $this->chunkPopulationRequestMap[$chunkHash] = new PromiseResolver();
if($associatedChunkLoader === null){
$temporaryLoader = new class implements ChunkLoader{};
$world->registerChunkLoader($temporaryLoader, $chunkX, $chunkZ);
$resolver->getPromise()->onCompletion(
fn() => $world->unregisterChunkLoader($temporaryLoader, $chunkX, $chunkZ),
static function() : void{}
);
}
return $resolver->getPromise();
}
/**
* Checks if a chunk needs to be populated, and whether it's ready to do so.
* @return bool[]|PromiseResolver[]|null[]
* @phpstan-return array{?PromiseResolver<Chunk>, bool}
*/
private function checkChunkPopulationPreconditions(World $world, int $chunkX, int $chunkZ) : array{
$chunkHash = World::chunkHash($chunkX, $chunkZ);
$resolver = $this->chunkPopulationRequestMap[$chunkHash] ?? null;
if($resolver !== null && isset($this->activeChunkPopulationTasks[$chunkHash])){
//generation is already running
return [$resolver, false];
}
$temporaryChunkLoader = new class implements ChunkLoader{};
$world->registerChunkLoader($temporaryChunkLoader, $chunkX, $chunkZ);
$chunk = $world->loadChunk($chunkX, $chunkZ);
$world->unregisterChunkLoader($temporaryChunkLoader, $chunkX, $chunkZ);
if($chunk !== null && $chunk->isPopulated()){
//chunk is already populated; return a pre-resolved promise that will directly fire callbacks assigned
$resolver ??= new PromiseResolver();
unset($this->chunkPopulationRequestMap[$chunkHash]);
$resolver->resolve($chunk);
return [$resolver, false];
}
return [$resolver, true];
}
private function drainPopulationRequestQueue(World $world) : void{
$failed = [];
while(count($this->activeChunkPopulationTasks) < $this->maxConcurrentChunkPopulationTasks && !$this->chunkPopulationRequestQueue->isEmpty()){
$nextChunkHash = $this->chunkPopulationRequestQueue->dequeue();
unset($this->chunkPopulationRequestQueueIndex[$nextChunkHash]);
World::getXZ($nextChunkHash, $nextChunkX, $nextChunkZ);
if(isset($this->chunkPopulationRequestMap[$nextChunkHash])){
assert(!($this->activeChunkPopulationTasks[$nextChunkHash] ?? false), "Population for chunk $nextChunkX $nextChunkZ already running");
if(
!$this->orderChunkPopulation($world, $nextChunkX, $nextChunkZ, null)->isResolved() &&
!isset($this->activeChunkPopulationTasks[$nextChunkHash])
){
$failed[] = $nextChunkHash;
}
}
}
//these requests failed even though they weren't rate limited; we can't directly re-add them to the back of the
//queue because it would result in an infinite loop
foreach($failed as $hash){
$this->addChunkHashToPopulationRequestQueue($hash);
}
}
/**
* @param Chunk[] $adjacentChunks chunkHash => chunk
* @phpstan-param array<int, Chunk> $adjacentChunks
*/
private function generateChunkCallback(World $world, ChunkLockId $chunkLockId, int $x, int $z, Chunk $chunk, array $adjacentChunks, ChunkLoader $temporaryChunkLoader) : void{
$timings = $world->timings->chunkPopulationCompletion;
$timings->startTiming();
$dirtyChunks = 0;
for($xx = -1; $xx <= 1; ++$xx){
for($zz = -1; $zz <= 1; ++$zz){
$world->unregisterChunkLoader($temporaryChunkLoader, $x + $xx, $z + $zz);
if(!$world->unlockChunk($x + $xx, $z + $zz, $chunkLockId)){
$dirtyChunks++;
}
}
}
$index = World::chunkHash($x, $z);
if(!isset($this->activeChunkPopulationTasks[$index])){
throw new AssumptionFailedError("This should always be set, regardless of whether the task was orphaned or not");
}
if(!$this->activeChunkPopulationTasks[$index]){
$world->getLogger()->debug("Discarding orphaned population result for chunk x=$x,z=$z");
unset($this->activeChunkPopulationTasks[$index]);
}else{
if($dirtyChunks === 0){
$oldChunk = $world->loadChunk($x, $z);
$world->setChunk($x, $z, $chunk);
foreach($adjacentChunks as $relativeChunkHash => $adjacentChunk){
World::getXZ($relativeChunkHash, $relativeX, $relativeZ);
if($relativeX < -1 || $relativeX > 1 || $relativeZ < -1 || $relativeZ > 1){
throw new AssumptionFailedError("Adjacent chunks should be in range -1 ... +1 coordinates");
}
$world->setChunk($x + $relativeX, $z + $relativeZ, $adjacentChunk);
}
if(($oldChunk === null || !$oldChunk->isPopulated()) && $chunk->isPopulated()){
if(ChunkPopulateEvent::hasHandlers()){
(new ChunkPopulateEvent($world, $x, $z, $chunk))->call();
}
foreach($world->getChunkListeners($x, $z) as $listener){
$listener->onChunkPopulated($x, $z, $chunk);
}
}
}else{
$world->getLogger()->debug("Discarding population result for chunk x=$x,z=$z - terrain was modified on the main thread before async population completed");
}
//This needs to be in this specific spot because user code might call back to orderChunkPopulation().
//If it does, and finds the promise, and doesn't find an active task associated with it, it will schedule
//another PopulationTask. We don't want that because we're here processing the results.
//We can't remove the promise from the array before setting the chunks in the world because that would lead
//to the same problem. Therefore, it's necessary that this code be split into two if/else, with this in the
//middle.
unset($this->activeChunkPopulationTasks[$index]);
if($dirtyChunks === 0){
$promise = $this->chunkPopulationRequestMap[$index] ?? null;
if($promise !== null){
unset($this->chunkPopulationRequestMap[$index]);
$promise->resolve($chunk);
}else{
//Handlers of ChunkPopulateEvent, ChunkLoadEvent, or just ChunkListeners can cause this
$world->getLogger()->debug("Unable to resolve population promise for chunk x=$x,z=$z - populated chunk was forcibly unloaded while setting modified chunks");
}
}else{
//request failed, stick it back on the queue
//we didn't resolve the promise or touch it in any way, so any fake chunk loaders are still valid and
//don't need to be added a second time.
$this->addChunkHashToPopulationRequestQueue($index);
}
$this->drainPopulationRequestQueue($world);
}
$timings->stopTiming();
}
/**
* @phpstan-param PromiseResolver<Chunk>|null $resolver
* @phpstan-return Promise<Chunk>
*/
private function internalOrderChunkPopulation(World $world, int $chunkX, int $chunkZ, ?ChunkLoader $associatedChunkLoader, ?PromiseResolver $resolver) : Promise{
$chunkHash = World::chunkHash($chunkX, $chunkZ);
$timings = $world->timings->chunkPopulationOrder;
$timings->startTiming();
try{
for($xx = -1; $xx <= 1; ++$xx){
for($zz = -1; $zz <= 1; ++$zz){
if($world->isChunkLocked($chunkX + $xx, $chunkZ + $zz)){
//chunk is already in use by another generation request; queue the request for later
return $resolver?->getPromise() ?? $this->enqueuePopulationRequest($world, $chunkX, $chunkZ, $associatedChunkLoader);
}
}
}
$this->activeChunkPopulationTasks[$chunkHash] = true;
if($resolver === null){
$resolver = new PromiseResolver();
$this->chunkPopulationRequestMap[$chunkHash] = $resolver;
}
$chunkPopulationLockId = new ChunkLockId();
$temporaryChunkLoader = new class implements ChunkLoader{
};
for($xx = -1; $xx <= 1; ++$xx){
for($zz = -1; $zz <= 1; ++$zz){
$world->lockChunk($chunkX + $xx, $chunkZ + $zz, $chunkPopulationLockId);
$world->registerChunkLoader($temporaryChunkLoader, $chunkX + $xx, $chunkZ + $zz);
}
}
$centerChunk = $world->loadChunk($chunkX, $chunkZ);
$adjacentChunks = $world->getAdjacentChunks($chunkX, $chunkZ);
$task = new PopulationTask(
$world->getId(),
$chunkX,
$chunkZ,
$centerChunk,
$adjacentChunks,
function(Chunk $centerChunk, array $adjacentChunks) use ($world, $chunkPopulationLockId, $chunkX, $chunkZ, $temporaryChunkLoader) : void{
if(!$world->isLoaded()){
return;
}
$this->generateChunkCallback($world, $chunkPopulationLockId, $chunkX, $chunkZ, $centerChunk, $adjacentChunks, $temporaryChunkLoader);
}
);
$workerId = $this->workerPool->selectWorker();
if(!isset($this->workerPool->getRunningWorkers()[$workerId]) && isset($this->generatorRegisteredWorkers[$workerId])){
$world->getLogger()->debug("Selected worker $workerId previously had generator registered, but is now offline");
unset($this->generatorRegisteredWorkers[$workerId]);
}
if(!isset($this->generatorRegisteredWorkers[$workerId])){
$this->registerGeneratorToWorker($world, $workerId);
}
$this->workerPool->submitTaskToWorker($task, $workerId);
return $resolver->getPromise();
}finally{
$timings->stopTiming();
}
}
/**
* Attempts to initiate asynchronous generation/population of the target chunk, if it's currently reasonable to do
* so (and if it isn't already generated/populated).
* If the generator is busy, the request will be put into a queue and delayed until a better time.
*
* A ChunkLoader can be associated with the generation request to ensure that the generation request is cancelled if
* no loaders are attached to the target chunk. If no loader is provided, one will be assigned (and automatically
* removed when the generation request completes).
*
* @phpstan-return Promise<Chunk>
*/
public function requestChunkPopulation(World $world, int $chunkX, int $chunkZ, ?ChunkLoader $associatedChunkLoader) : Promise{
[$resolver, $proceedWithPopulation] = $this->checkChunkPopulationPreconditions($world, $chunkX, $chunkZ);
if(!$proceedWithPopulation){
return $resolver?->getPromise() ?? $this->enqueuePopulationRequest($world, $chunkX, $chunkZ, $associatedChunkLoader);
}
if(count($this->activeChunkPopulationTasks) >= $this->maxConcurrentChunkPopulationTasks){
//too many chunks are already generating; delay resolution of the request until later
return $resolver?->getPromise() ?? $this->enqueuePopulationRequest($world, $chunkX, $chunkZ, $associatedChunkLoader);
}
return $this->internalOrderChunkPopulation($world, $chunkX, $chunkZ, $associatedChunkLoader, $resolver);
}
/**
* Initiates asynchronous generation/population of the target chunk, if it's not already generated/populated.
* If generation has already been requested for the target chunk, the promise for the already active request will be
* returned directly.
*
* If the chunk is currently locked (for example due to another chunk using it for async generation), the request
* will be queued and executed at the earliest opportunity.
*
* @phpstan-return Promise<Chunk>
*/
public function orderChunkPopulation(World $world, int $chunkX, int $chunkZ, ?ChunkLoader $associatedChunkLoader) : Promise{
[$resolver, $proceedWithPopulation] = $this->checkChunkPopulationPreconditions($world, $chunkX, $chunkZ);
if(!$proceedWithPopulation){
return $resolver?->getPromise() ?? $this->enqueuePopulationRequest($world, $chunkX, $chunkZ, $associatedChunkLoader);
}
return $this->internalOrderChunkPopulation($world, $chunkX, $chunkZ, $associatedChunkLoader, $resolver);
}
public function cancelChunkPopulation(World $world, int $chunkX, int $chunkZ) : void{
$chunkHash = World::chunkHash($chunkX, $chunkZ);
if(array_key_exists($chunkHash, $this->chunkPopulationRequestMap)){
$this->logger->debug("Rejecting population promise for chunk $chunkX $chunkZ");
$this->chunkPopulationRequestMap[$chunkHash]->reject();
unset($this->chunkPopulationRequestMap[$chunkHash]);
if(isset($this->activeChunkPopulationTasks[$chunkHash])){
$this->logger->debug("Marking population task for chunk $chunkX $chunkZ as orphaned");
$this->activeChunkPopulationTasks[$chunkHash] = false;
}
}
}
public function shutdown(World $world) : void{
$world->getLogger()->debug("Cancelling unfulfilled generation requests");
foreach($this->chunkPopulationRequestMap as $chunkHash => $promise){
$promise->reject();
unset($this->chunkPopulationRequestMap[$chunkHash]);
}
if(count($this->chunkPopulationRequestMap) !== 0){
//TODO: this might actually get hit because generation rejection callbacks might try to schedule new
//requests, and we can't prevent that right now because there's no way to detect "unloading" state
throw new AssumptionFailedError("New generation requests scheduled during unload");
}
$this->workerPool->removeWorkerStartHook($this->workerStartHook);
}
}

View File

@ -0,0 +1,65 @@
<?php
/*
*
* ____ _ _ __ __ _ __ __ ____
* | _ \ ___ ___| | _____| |_| \/ (_)_ __ ___ | \/ | _ \
* | |_) / _ \ / __| |/ / _ \ __| |\/| | | '_ \ / _ \_____| |\/| | |_) |
* | __/ (_) | (__| < __/ |_| | | | | | | | __/_____| | | | __/
* |_| \___/ \___|_|\_\___|\__|_| |_|_|_| |_|\___| |_| |_|_|
*
* This program is free software: you can redistribute it and/or modify
* it under the terms of the GNU Lesser General Public License as published by
* the Free Software Foundation, either version 3 of the License, or
* (at your option) any later version.
*
* @author PocketMine Team
* @link http://www.pocketmine.net/
*
*
*/
declare(strict_types=1);
namespace pocketmine\world;
use pocketmine\promise\Promise;
use pocketmine\world\format\Chunk;
/**
* @phpstan-import-type ChunkPosHash from World
*/
interface ChunkGenerator{
/**
* Attempts to initiate asynchronous generation/population of the target chunk, if it's currently reasonable to do
* so (and if it isn't already generated/populated).
* If the generator is busy, the request will be put into a queue and delayed until a better time.
*
* A ChunkLoader can be associated with the generation request to ensure that the generation request is cancelled if
* no loaders are attached to the target chunk. If no loader is provided, one will be assigned (and automatically
* removed when the generation request completes).
*
* @phpstan-return Promise<Chunk>
*/
public function requestChunkPopulation(World $world, int $chunkX, int $chunkZ, ?ChunkLoader $associatedChunkLoader) : Promise;
/**
* Initiates asynchronous generation/population of the target chunk, if it's not already generated/populated.
* If generation has already been requested for the target chunk, the promise for the already active request will be
* returned directly.
*
* If the chunk is currently locked (for example due to another chunk using it for async generation), the request
* will be queued and executed at the earliest opportunity.
*
* @phpstan-return Promise<Chunk>
*/
public function orderChunkPopulation(World $world, int $chunkX, int $chunkZ, ?ChunkLoader $associatedChunkLoader) : Promise;
/**
* Called when the World needs to cancel a previously-requested population request.
* This is typically due to the chunk being unloaded.
*/
public function cancelChunkPopulation(World $world, int $chunkX, int $chunkZ) : void;
public function shutdown(World $world) : void;
}

View File

@ -49,7 +49,6 @@ use pocketmine\event\block\BlockPlaceEvent;
use pocketmine\event\block\BlockUpdateEvent; use pocketmine\event\block\BlockUpdateEvent;
use pocketmine\event\player\PlayerInteractEvent; use pocketmine\event\player\PlayerInteractEvent;
use pocketmine\event\world\ChunkLoadEvent; use pocketmine\event\world\ChunkLoadEvent;
use pocketmine\event\world\ChunkPopulateEvent;
use pocketmine\event\world\ChunkUnloadEvent; use pocketmine\event\world\ChunkUnloadEvent;
use pocketmine\event\world\SpawnChangeEvent; use pocketmine\event\world\SpawnChangeEvent;
use pocketmine\event\world\WorldDifficultyChangeEvent; use pocketmine\event\world\WorldDifficultyChangeEvent;
@ -94,9 +93,6 @@ use pocketmine\world\format\io\WritableWorldProvider;
use pocketmine\world\format\LightArray; use pocketmine\world\format\LightArray;
use pocketmine\world\format\SubChunk; use pocketmine\world\format\SubChunk;
use pocketmine\world\generator\GeneratorManager; use pocketmine\world\generator\GeneratorManager;
use pocketmine\world\generator\GeneratorRegisterTask;
use pocketmine\world\generator\GeneratorUnregisterTask;
use pocketmine\world\generator\PopulationTask;
use pocketmine\world\light\BlockLightUpdate; use pocketmine\world\light\BlockLightUpdate;
use pocketmine\world\light\LightPopulationTask; use pocketmine\world\light\LightPopulationTask;
use pocketmine\world\light\SkyLightUpdate; use pocketmine\world\light\SkyLightUpdate;
@ -309,38 +305,13 @@ class World implements ChunkManager{
*/ */
private array $neighbourBlockUpdateQueueIndex = []; private array $neighbourBlockUpdateQueueIndex = [];
/** private readonly ChunkGenerator $chunkGenerator;
* @var bool[] chunkHash => isValid
* @phpstan-var array<ChunkPosHash, bool>
*/
private array $activeChunkPopulationTasks = [];
/** /**
* @var ChunkLockId[] * @var ChunkLockId[]
* @phpstan-var array<ChunkPosHash, ChunkLockId> * @phpstan-var array<ChunkPosHash, ChunkLockId>
*/ */
private array $chunkLock = []; private array $chunkLock = [];
private int $maxConcurrentChunkPopulationTasks = 2;
/**
* @var PromiseResolver[] chunkHash => promise
* @phpstan-var array<ChunkPosHash, PromiseResolver<Chunk>>
*/
private array $chunkPopulationRequestMap = [];
/**
* @var \SplQueue (queue of chunkHashes)
* @phpstan-var \SplQueue<ChunkPosHash>
*/
private \SplQueue $chunkPopulationRequestQueue;
/**
* @var true[] chunkHash => dummy
* @phpstan-var array<ChunkPosHash, true>
*/
private array $chunkPopulationRequestQueueIndex = [];
/**
* @var true[]
* @phpstan-var array<int, true>
*/
private array $generatorRegisteredWorkers = [];
private bool $autoSave = true; private bool $autoSave = true;
@ -499,19 +470,15 @@ class World implements ChunkManager{
throw new AssumptionFailedError("WorldManager should already have checked that the generator exists"); throw new AssumptionFailedError("WorldManager should already have checked that the generator exists");
$generator->validateGeneratorOptions($this->provider->getWorldData()->getGeneratorOptions()); $generator->validateGeneratorOptions($this->provider->getWorldData()->getGeneratorOptions());
$this->generator = $generator->getGeneratorClass(); $this->generator = $generator->getGeneratorClass();
$this->chunkPopulationRequestQueue = new \SplQueue();
$this->addOnUnloadCallback(function() : void{
$this->logger->debug("Cancelling unfulfilled generation requests");
foreach($this->chunkPopulationRequestMap as $chunkHash => $promise){ $cfg = $this->server->getConfigGroup();
$promise->reject(); $this->chunkGenerator = new AsyncChunkGenerator(
unset($this->chunkPopulationRequestMap[$chunkHash]); $this->workerPool,
} $this->logger,
if(count($this->chunkPopulationRequestMap) !== 0){ $cfg->getPropertyInt(YmlServerProperties::CHUNK_GENERATION_POPULATION_QUEUE_SIZE, 2)
//TODO: this might actually get hit because generation rejection callbacks might try to schedule new );
//requests, and we can't prevent that right now because there's no way to detect "unloading" state $this->addOnUnloadCallback(function() : void{
throw new AssumptionFailedError("New generation requests scheduled during unload"); $this->chunkGenerator->shutdown($this);
}
}); });
$this->scheduledBlockUpdateQueue = new ReversePriorityQueue(); $this->scheduledBlockUpdateQueue = new ReversePriorityQueue();
@ -521,7 +488,6 @@ class World implements ChunkManager{
$this->time = $this->provider->getWorldData()->getTime(); $this->time = $this->provider->getWorldData()->getTime();
$cfg = $this->server->getConfigGroup();
$this->chunkTickRadius = min($this->server->getViewDistance(), max(0, $cfg->getPropertyInt(YmlServerProperties::CHUNK_TICKING_TICK_RADIUS, 4))); $this->chunkTickRadius = min($this->server->getViewDistance(), max(0, $cfg->getPropertyInt(YmlServerProperties::CHUNK_TICKING_TICK_RADIUS, 4)));
if($cfg->getPropertyInt("chunk-ticking.per-tick", 40) <= 0){ if($cfg->getPropertyInt("chunk-ticking.per-tick", 40) <= 0){
//TODO: this needs l10n //TODO: this needs l10n
@ -529,22 +495,9 @@ class World implements ChunkManager{
$this->chunkTickRadius = 0; $this->chunkTickRadius = 0;
} }
$this->tickedBlocksPerSubchunkPerTick = $cfg->getPropertyInt(YmlServerProperties::CHUNK_TICKING_BLOCKS_PER_SUBCHUNK_PER_TICK, self::DEFAULT_TICKED_BLOCKS_PER_SUBCHUNK_PER_TICK); $this->tickedBlocksPerSubchunkPerTick = $cfg->getPropertyInt(YmlServerProperties::CHUNK_TICKING_BLOCKS_PER_SUBCHUNK_PER_TICK, self::DEFAULT_TICKED_BLOCKS_PER_SUBCHUNK_PER_TICK);
$this->maxConcurrentChunkPopulationTasks = $cfg->getPropertyInt(YmlServerProperties::CHUNK_GENERATION_POPULATION_QUEUE_SIZE, 2);
$this->initRandomTickBlocksFromConfig($cfg); $this->initRandomTickBlocksFromConfig($cfg);
$this->timings = new WorldTimings($this); $this->timings = new WorldTimings($this);
$this->workerPool->addWorkerStartHook($workerStartHook = function(int $workerId) : void{
if(array_key_exists($workerId, $this->generatorRegisteredWorkers)){
$this->logger->debug("Worker $workerId with previously registered generator restarted, flagging as unregistered");
unset($this->generatorRegisteredWorkers[$workerId]);
}
});
$workerPool = $this->workerPool;
$this->addOnUnloadCallback(static function() use ($workerPool, $workerStartHook) : void{
$workerPool->removeWorkerStartHook($workerStartHook);
});
} }
private function initRandomTickBlocksFromConfig(ServerConfigGroup $cfg) : void{ private function initRandomTickBlocksFromConfig(ServerConfigGroup $cfg) : void{
@ -585,19 +538,11 @@ class World implements ChunkManager{
return $this->tickRateTime; return $this->tickRateTime;
} }
public function registerGeneratorToWorker(int $worker) : void{ /**
$this->logger->debug("Registering generator on worker $worker"); * @phpstan-return class-string<covariant \pocketmine\world\generator\Generator>
$this->workerPool->submitTaskToWorker(new GeneratorRegisterTask($this, $this->generator, $this->provider->getWorldData()->getGeneratorOptions()), $worker); */
$this->generatorRegisteredWorkers[$worker] = true; public function getGeneratorClass() : string{
} return $this->generator;
public function unregisterGenerator() : void{
foreach($this->workerPool->getRunningWorkers() as $i){
if(isset($this->generatorRegisteredWorkers[$i])){
$this->workerPool->submitTaskToWorker(new GeneratorUnregisterTask($this), $i);
}
}
$this->generatorRegisteredWorkers = [];
} }
public function getServer() : Server{ public function getServer() : Server{
@ -657,8 +602,6 @@ class World implements ChunkManager{
$this->save(); $this->save();
$this->unregisterGenerator();
$this->provider->close(); $this->provider->close();
$this->blockCache = []; $this->blockCache = [];
$this->blockCacheSize = 0; $this->blockCacheSize = 0;
@ -837,10 +780,7 @@ class World implements ChunkManager{
if(count($this->chunkLoaders[$chunkHash]) === 1){ if(count($this->chunkLoaders[$chunkHash]) === 1){
unset($this->chunkLoaders[$chunkHash]); unset($this->chunkLoaders[$chunkHash]);
$this->unloadChunkRequest($chunkX, $chunkZ, true); $this->unloadChunkRequest($chunkX, $chunkZ, true);
if(isset($this->chunkPopulationRequestMap[$chunkHash]) && !isset($this->activeChunkPopulationTasks[$chunkHash])){ $this->chunkGenerator->cancelChunkPopulation($this, $chunkX, $chunkZ);
$this->chunkPopulationRequestMap[$chunkHash]->reject();
unset($this->chunkPopulationRequestMap[$chunkHash]);
}
}else{ }else{
unset($this->chunkLoaders[$chunkHash][$loaderId]); unset($this->chunkLoaders[$chunkHash][$loaderId]);
} }
@ -3117,15 +3057,7 @@ class World implements ChunkManager{
unset($this->registeredTickingChunks[$chunkHash]); unset($this->registeredTickingChunks[$chunkHash]);
$this->markTickingChunkForRecheck($x, $z); $this->markTickingChunkForRecheck($x, $z);
if(array_key_exists($chunkHash, $this->chunkPopulationRequestMap)){ $this->chunkGenerator->cancelChunkPopulation($this, $x, $z);
$this->logger->debug("Rejecting population promise for chunk $x $z");
$this->chunkPopulationRequestMap[$chunkHash]->reject();
unset($this->chunkPopulationRequestMap[$chunkHash]);
if(isset($this->activeChunkPopulationTasks[$chunkHash])){
$this->logger->debug("Marking population task for chunk $x $z as orphaned");
$this->activeChunkPopulationTasks[$chunkHash] = false;
}
}
$this->timings->doChunkUnload->stopTiming(); $this->timings->doChunkUnload->stopTiming();
@ -3310,83 +3242,6 @@ class World implements ChunkManager{
} }
} }
private function addChunkHashToPopulationRequestQueue(int $chunkHash) : void{
if(!isset($this->chunkPopulationRequestQueueIndex[$chunkHash])){
$this->chunkPopulationRequestQueue->enqueue($chunkHash);
$this->chunkPopulationRequestQueueIndex[$chunkHash] = true;
}
}
/**
* @phpstan-return Promise<Chunk>
*/
private function enqueuePopulationRequest(int $chunkX, int $chunkZ, ?ChunkLoader $associatedChunkLoader) : Promise{
$chunkHash = World::chunkHash($chunkX, $chunkZ);
$this->addChunkHashToPopulationRequestQueue($chunkHash);
/** @phpstan-var PromiseResolver<Chunk> $resolver */
$resolver = $this->chunkPopulationRequestMap[$chunkHash] = new PromiseResolver();
if($associatedChunkLoader === null){
$temporaryLoader = new class implements ChunkLoader{};
$this->registerChunkLoader($temporaryLoader, $chunkX, $chunkZ);
$resolver->getPromise()->onCompletion(
fn() => $this->unregisterChunkLoader($temporaryLoader, $chunkX, $chunkZ),
static function() : void{}
);
}
return $resolver->getPromise();
}
private function drainPopulationRequestQueue() : void{
$failed = [];
while(count($this->activeChunkPopulationTasks) < $this->maxConcurrentChunkPopulationTasks && !$this->chunkPopulationRequestQueue->isEmpty()){
$nextChunkHash = $this->chunkPopulationRequestQueue->dequeue();
unset($this->chunkPopulationRequestQueueIndex[$nextChunkHash]);
World::getXZ($nextChunkHash, $nextChunkX, $nextChunkZ);
if(isset($this->chunkPopulationRequestMap[$nextChunkHash])){
assert(!($this->activeChunkPopulationTasks[$nextChunkHash] ?? false), "Population for chunk $nextChunkX $nextChunkZ already running");
if(
!$this->orderChunkPopulation($nextChunkX, $nextChunkZ, null)->isResolved() &&
!isset($this->activeChunkPopulationTasks[$nextChunkHash])
){
$failed[] = $nextChunkHash;
}
}
}
//these requests failed even though they weren't rate limited; we can't directly re-add them to the back of the
//queue because it would result in an infinite loop
foreach($failed as $hash){
$this->addChunkHashToPopulationRequestQueue($hash);
}
}
/**
* Checks if a chunk needs to be populated, and whether it's ready to do so.
* @return bool[]|PromiseResolver[]|null[]
* @phpstan-return array{?PromiseResolver<Chunk>, bool}
*/
private function checkChunkPopulationPreconditions(int $chunkX, int $chunkZ) : array{
$chunkHash = World::chunkHash($chunkX, $chunkZ);
$resolver = $this->chunkPopulationRequestMap[$chunkHash] ?? null;
if($resolver !== null && isset($this->activeChunkPopulationTasks[$chunkHash])){
//generation is already running
return [$resolver, false];
}
$temporaryChunkLoader = new class implements ChunkLoader{};
$this->registerChunkLoader($temporaryChunkLoader, $chunkX, $chunkZ);
$chunk = $this->loadChunk($chunkX, $chunkZ);
$this->unregisterChunkLoader($temporaryChunkLoader, $chunkX, $chunkZ);
if($chunk !== null && $chunk->isPopulated()){
//chunk is already populated; return a pre-resolved promise that will directly fire callbacks assigned
$resolver ??= new PromiseResolver();
unset($this->chunkPopulationRequestMap[$chunkHash]);
$resolver->resolve($chunk);
return [$resolver, false];
}
return [$resolver, true];
}
/** /**
* Attempts to initiate asynchronous generation/population of the target chunk, if it's currently reasonable to do * Attempts to initiate asynchronous generation/population of the target chunk, if it's currently reasonable to do
* so (and if it isn't already generated/populated). * so (and if it isn't already generated/populated).
@ -3399,16 +3254,7 @@ class World implements ChunkManager{
* @phpstan-return Promise<Chunk> * @phpstan-return Promise<Chunk>
*/ */
public function requestChunkPopulation(int $chunkX, int $chunkZ, ?ChunkLoader $associatedChunkLoader) : Promise{ public function requestChunkPopulation(int $chunkX, int $chunkZ, ?ChunkLoader $associatedChunkLoader) : Promise{
[$resolver, $proceedWithPopulation] = $this->checkChunkPopulationPreconditions($chunkX, $chunkZ); return $this->chunkGenerator->requestChunkPopulation($this, $chunkX, $chunkZ, $associatedChunkLoader);
if(!$proceedWithPopulation){
return $resolver?->getPromise() ?? $this->enqueuePopulationRequest($chunkX, $chunkZ, $associatedChunkLoader);
}
if(count($this->activeChunkPopulationTasks) >= $this->maxConcurrentChunkPopulationTasks){
//too many chunks are already generating; delay resolution of the request until later
return $resolver?->getPromise() ?? $this->enqueuePopulationRequest($chunkX, $chunkZ, $associatedChunkLoader);
}
return $this->internalOrderChunkPopulation($chunkX, $chunkZ, $associatedChunkLoader, $resolver);
} }
/** /**
@ -3422,161 +3268,7 @@ class World implements ChunkManager{
* @phpstan-return Promise<Chunk> * @phpstan-return Promise<Chunk>
*/ */
public function orderChunkPopulation(int $chunkX, int $chunkZ, ?ChunkLoader $associatedChunkLoader) : Promise{ public function orderChunkPopulation(int $chunkX, int $chunkZ, ?ChunkLoader $associatedChunkLoader) : Promise{
[$resolver, $proceedWithPopulation] = $this->checkChunkPopulationPreconditions($chunkX, $chunkZ); return $this->chunkGenerator->orderChunkPopulation($this, $chunkX, $chunkZ, $associatedChunkLoader);
if(!$proceedWithPopulation){
return $resolver?->getPromise() ?? $this->enqueuePopulationRequest($chunkX, $chunkZ, $associatedChunkLoader);
}
return $this->internalOrderChunkPopulation($chunkX, $chunkZ, $associatedChunkLoader, $resolver);
}
/**
* @phpstan-param PromiseResolver<Chunk>|null $resolver
* @phpstan-return Promise<Chunk>
*/
private function internalOrderChunkPopulation(int $chunkX, int $chunkZ, ?ChunkLoader $associatedChunkLoader, ?PromiseResolver $resolver) : Promise{
$chunkHash = World::chunkHash($chunkX, $chunkZ);
$timings = $this->timings->chunkPopulationOrder;
$timings->startTiming();
try{
for($xx = -1; $xx <= 1; ++$xx){
for($zz = -1; $zz <= 1; ++$zz){
if($this->isChunkLocked($chunkX + $xx, $chunkZ + $zz)){
//chunk is already in use by another generation request; queue the request for later
return $resolver?->getPromise() ?? $this->enqueuePopulationRequest($chunkX, $chunkZ, $associatedChunkLoader);
}
}
}
$this->activeChunkPopulationTasks[$chunkHash] = true;
if($resolver === null){
$resolver = new PromiseResolver();
$this->chunkPopulationRequestMap[$chunkHash] = $resolver;
}
$chunkPopulationLockId = new ChunkLockId();
$temporaryChunkLoader = new class implements ChunkLoader{
};
for($xx = -1; $xx <= 1; ++$xx){
for($zz = -1; $zz <= 1; ++$zz){
$this->lockChunk($chunkX + $xx, $chunkZ + $zz, $chunkPopulationLockId);
$this->registerChunkLoader($temporaryChunkLoader, $chunkX + $xx, $chunkZ + $zz);
}
}
$centerChunk = $this->loadChunk($chunkX, $chunkZ);
$adjacentChunks = $this->getAdjacentChunks($chunkX, $chunkZ);
$task = new PopulationTask(
$this->worldId,
$chunkX,
$chunkZ,
$centerChunk,
$adjacentChunks,
function(Chunk $centerChunk, array $adjacentChunks) use ($chunkPopulationLockId, $chunkX, $chunkZ, $temporaryChunkLoader) : void{
if(!$this->isLoaded()){
return;
}
$this->generateChunkCallback($chunkPopulationLockId, $chunkX, $chunkZ, $centerChunk, $adjacentChunks, $temporaryChunkLoader);
}
);
$workerId = $this->workerPool->selectWorker();
if(!isset($this->workerPool->getRunningWorkers()[$workerId]) && isset($this->generatorRegisteredWorkers[$workerId])){
$this->logger->debug("Selected worker $workerId previously had generator registered, but is now offline");
unset($this->generatorRegisteredWorkers[$workerId]);
}
if(!isset($this->generatorRegisteredWorkers[$workerId])){
$this->registerGeneratorToWorker($workerId);
}
$this->workerPool->submitTaskToWorker($task, $workerId);
return $resolver->getPromise();
}finally{
$timings->stopTiming();
}
}
/**
* @param Chunk[] $adjacentChunks chunkHash => chunk
* @phpstan-param array<int, Chunk> $adjacentChunks
*/
private function generateChunkCallback(ChunkLockId $chunkLockId, int $x, int $z, Chunk $chunk, array $adjacentChunks, ChunkLoader $temporaryChunkLoader) : void{
$timings = $this->timings->chunkPopulationCompletion;
$timings->startTiming();
$dirtyChunks = 0;
for($xx = -1; $xx <= 1; ++$xx){
for($zz = -1; $zz <= 1; ++$zz){
$this->unregisterChunkLoader($temporaryChunkLoader, $x + $xx, $z + $zz);
if(!$this->unlockChunk($x + $xx, $z + $zz, $chunkLockId)){
$dirtyChunks++;
}
}
}
$index = World::chunkHash($x, $z);
if(!isset($this->activeChunkPopulationTasks[$index])){
throw new AssumptionFailedError("This should always be set, regardless of whether the task was orphaned or not");
}
if(!$this->activeChunkPopulationTasks[$index]){
$this->logger->debug("Discarding orphaned population result for chunk x=$x,z=$z");
unset($this->activeChunkPopulationTasks[$index]);
}else{
if($dirtyChunks === 0){
$oldChunk = $this->loadChunk($x, $z);
$this->setChunk($x, $z, $chunk);
foreach($adjacentChunks as $relativeChunkHash => $adjacentChunk){
World::getXZ($relativeChunkHash, $relativeX, $relativeZ);
if($relativeX < -1 || $relativeX > 1 || $relativeZ < -1 || $relativeZ > 1){
throw new AssumptionFailedError("Adjacent chunks should be in range -1 ... +1 coordinates");
}
$this->setChunk($x + $relativeX, $z + $relativeZ, $adjacentChunk);
}
if(($oldChunk === null || !$oldChunk->isPopulated()) && $chunk->isPopulated()){
if(ChunkPopulateEvent::hasHandlers()){
(new ChunkPopulateEvent($this, $x, $z, $chunk))->call();
}
foreach($this->getChunkListeners($x, $z) as $listener){
$listener->onChunkPopulated($x, $z, $chunk);
}
}
}else{
$this->logger->debug("Discarding population result for chunk x=$x,z=$z - terrain was modified on the main thread before async population completed");
}
//This needs to be in this specific spot because user code might call back to orderChunkPopulation().
//If it does, and finds the promise, and doesn't find an active task associated with it, it will schedule
//another PopulationTask. We don't want that because we're here processing the results.
//We can't remove the promise from the array before setting the chunks in the world because that would lead
//to the same problem. Therefore, it's necessary that this code be split into two if/else, with this in the
//middle.
unset($this->activeChunkPopulationTasks[$index]);
if($dirtyChunks === 0){
$promise = $this->chunkPopulationRequestMap[$index] ?? null;
if($promise !== null){
unset($this->chunkPopulationRequestMap[$index]);
$promise->resolve($chunk);
}else{
//Handlers of ChunkPopulateEvent, ChunkLoadEvent, or just ChunkListeners can cause this
$this->logger->debug("Unable to resolve population promise for chunk x=$x,z=$z - populated chunk was forcibly unloaded while setting modified chunks");
}
}else{
//request failed, stick it back on the queue
//we didn't resolve the promise or touch it in any way, so any fake chunk loaders are still valid and
//don't need to be added a second time.
$this->addChunkHashToPopulationRequestQueue($index);
}
$this->drainPopulationRequestQueue();
}
$timings->stopTiming();
} }
public function doChunkGarbageCollection() : void{ public function doChunkGarbageCollection() : void{