geforkt von Mirrors/Paper
4162 Zeilen
187 KiB
Diff
4162 Zeilen
187 KiB
Diff
From 8bd81e609943d7056ee1bcd881521a4e54acb8b4 Mon Sep 17 00:00:00 2001
|
|
From: Spottedleaf <Spottedleaf@users.noreply.github.com>
|
|
Date: Sat, 13 Jul 2019 09:23:10 -0700
|
|
Subject: [PATCH] Asynchronous chunk IO and loading
|
|
|
|
This patch re-adds a file IO thread as well as shoving de-serializing
|
|
chunk NBT data onto worker threads. This patch also will shove
|
|
chunk data serialization onto the same worker threads when the chunk
|
|
is unloaded - this cannot be done for regular saves since that's unsafe.
|
|
|
|
The file IO Thread
|
|
|
|
Unlike 1.13 and below, the file IO thread is prioritized - IO tasks can
|
|
be reoredered, however they are "stuck" to a world & coordinate.
|
|
|
|
Scheduling IO tasks works as follows, given a world & coordinate - location:
|
|
|
|
The IO thread has been designed to ensure that reads and writes appear to
|
|
occur synchronously for a given location, however the implementation also
|
|
has the unfortunate side-effect of making every write appear as if
|
|
they occur without failure.
|
|
|
|
The IO thread has also been designed to accomodate Mojang's decision to
|
|
store chunk data and POI data separately. It can independently schedule
|
|
tasks for each.
|
|
|
|
However threads can wait for writes to complete and check if:
|
|
- The write was overwriten by another scheduler
|
|
- The write failed (however it does not indicate whether it was overwritten by another scheduler)
|
|
|
|
Scheduling reads:
|
|
|
|
- If a write task is in progress, the task is not scheduled and returns the in-progress write data
|
|
This means that readers cannot modify the NBTTagCompound returned and must clone if it they wish to write
|
|
- If a write task is not in progress but a read task is in progress, then the read task is simply chained
|
|
This means that again, readers cannot modify the NBTTagCompound returned
|
|
|
|
Scheduling writes:
|
|
|
|
- If a read task is in progress, ignore the read task and schedule the write
|
|
We cannot complete the read task since we assume it wants old data - not current
|
|
- If a write task is pending, overwrite the write data
|
|
The file IO thread does correctly handle cases where the data is overwritten when it
|
|
is writing data (before completing a task it will check if the data was overwritten and
|
|
will retry).
|
|
|
|
When the file IO thread executes a task for a location, the it will
|
|
execute the read task first (if it exists), then it will execute the
|
|
write task. This ensures that, even when scheduling at different
|
|
priorities, that reads/writes for a location act synchronously.
|
|
|
|
The downside of the file IO thread is that write failure can only be
|
|
indicated to the scheduling thread if:
|
|
|
|
- No other thread decides to schedule another write for the location
|
|
concurrently
|
|
- The scheduling thread blocks on the write to complete (however the
|
|
current implementation can be modified to indicate success
|
|
asynchronously)
|
|
|
|
The file io thread can be modified easily to provide indications
|
|
of write failure and write overwriting if needed.
|
|
|
|
The upside of the file IO thread is that if a write failures, then
|
|
chunk data is not lost until server restart. This leaves more room
|
|
for spurious failure.
|
|
|
|
Finally, the io thread will indicate to the console when reads
|
|
or writes fail - with relevant detail.
|
|
|
|
Asynchronous chunk data serialization for unloading chunks
|
|
|
|
When chunks unload they make a call to PlayerChunkMap#saveChunk(IChunkAccess).
|
|
Even if I make the IO asynchronous for this call, the data serialization
|
|
still hits pretty hard. And given that now the chunk system will
|
|
aggressively unload chunks more often (queued immediately at
|
|
ticket level 45 or higher), unloads occur more often, and
|
|
combined with our changes to the unload queue to make it
|
|
significantly more aggresive - chunk unloads can hit pretty hard.
|
|
Especially players running around with elytras and fireworks.
|
|
|
|
For serializing chunk data off main, there are some tasks which cannot be
|
|
done asynchronously. Lighting data must be saved beforehand as well as
|
|
potentially some tick lists. These are completed before scheduling the
|
|
asynchronous save.
|
|
|
|
However serializing chunk data off of the main thread is still risky.
|
|
Even though this patch schedules the save to occur after ALL references
|
|
of the chunk are removed from the world, plugins can still technically
|
|
access entities inside the chunks. For this, if the serialization task
|
|
fails for any reason, it will be re-scheduled to be serialized on the
|
|
main thread - with the hopes that the reason it failed was due to a plugin
|
|
and not an error with the save code itself. Like vanilla code - if the
|
|
serialization fails, the chunk data is lost.
|
|
|
|
Asynchronous chunk io/loading
|
|
|
|
Mojang's current implementation for loading chunk data off disk is
|
|
to return a CompletableFuture that will be completed by scheduling a
|
|
task to be executed on the world's chunk queue (which is only drained
|
|
on the main thread). This task will read the IO off disk and it will
|
|
apply data conversions & deserialization synchronously. Obviously
|
|
all 3 of these operations are expensive however all can be completed
|
|
asynchronously instead.
|
|
|
|
The solution this patch uses is as follows:
|
|
|
|
0. If an asynchronous chunk save is in progress (see above), wait
|
|
for that task to complete. It will use the serialized NBTTagCompound
|
|
created by the task. If the task fails to complete, then we would continue
|
|
with step 1. If it does not, we skip step 1. (Note: We actually load
|
|
POI data no matter what in this case).
|
|
1. Schedule an IO task to read chunk & poi data off disk.
|
|
2. The IO task will schedule a chunk load task.
|
|
3. The chunk load task executes on the async chunk loader threads
|
|
and will apply datafixers & de-serialize the chunk into a ProtoChunk
|
|
or ProtoChunkExtension.
|
|
4. The in progress chunk is then passed on to the world's chunk queue
|
|
to complete the ComletableFuture and execute any of the synchronous
|
|
tasks required to be executed by the chunk load task (i.e lighting
|
|
and some poi tasks).
|
|
|
|
diff --git a/src/main/java/co/aikar/timings/WorldTimingsHandler.java b/src/main/java/co/aikar/timings/WorldTimingsHandler.java
|
|
index 27ce4a828..30bafb214 100644
|
|
--- a/src/main/java/co/aikar/timings/WorldTimingsHandler.java
|
|
+++ b/src/main/java/co/aikar/timings/WorldTimingsHandler.java
|
|
@@ -61,6 +61,17 @@ public class WorldTimingsHandler {
|
|
|
|
public final Timing miscMobSpawning;
|
|
|
|
+ public final Timing poiUnload;
|
|
+ public final Timing chunkUnload;
|
|
+ public final Timing poiSaveDataSerialization;
|
|
+ public final Timing chunkSave;
|
|
+ public final Timing chunkSaveOverwriteCheck;
|
|
+ public final Timing chunkSaveDataSerialization;
|
|
+ public final Timing chunkSaveIOWait;
|
|
+ public final Timing chunkUnloadPrepareSave;
|
|
+ public final Timing chunkUnloadPOISerialization;
|
|
+ public final Timing chunkUnloadDataSave;
|
|
+
|
|
public WorldTimingsHandler(World server) {
|
|
String name = server.worldData.getName() +" - ";
|
|
|
|
@@ -118,6 +129,17 @@ public class WorldTimingsHandler {
|
|
|
|
|
|
miscMobSpawning = Timings.ofSafe(name + "Mob spawning - Misc");
|
|
+
|
|
+ poiUnload = Timings.ofSafe(name + "Chunk unload - POI");
|
|
+ chunkUnload = Timings.ofSafe(name + "Chunk unload - Chunk");
|
|
+ poiSaveDataSerialization = Timings.ofSafe(name + "Chunk save - POI Data serialization");
|
|
+ chunkSave = Timings.ofSafe(name + "Chunk save - Chunk");
|
|
+ chunkSaveOverwriteCheck = Timings.ofSafe(name + "Chunk save - Chunk Overwrite Check");
|
|
+ chunkSaveDataSerialization = Timings.ofSafe(name + "Chunk save - Chunk Data serialization");
|
|
+ chunkSaveIOWait = Timings.ofSafe(name + "Chunk save - Chunk IO Wait");
|
|
+ chunkUnloadPrepareSave = Timings.ofSafe(name + "Chunk unload - Async Save Prepare");
|
|
+ chunkUnloadPOISerialization = Timings.ofSafe(name + "Chunk unload - POI Data Serialization");
|
|
+ chunkUnloadDataSave = Timings.ofSafe(name + "Chunk unload - Data Serialization");
|
|
}
|
|
|
|
public static Timing getTickList(WorldServer worldserver, String timingsType) {
|
|
diff --git a/src/main/java/com/destroystokyo/paper/PaperConfig.java b/src/main/java/com/destroystokyo/paper/PaperConfig.java
|
|
index dbd143997..6916ed30c 100644
|
|
--- a/src/main/java/com/destroystokyo/paper/PaperConfig.java
|
|
+++ b/src/main/java/com/destroystokyo/paper/PaperConfig.java
|
|
@@ -1,5 +1,6 @@
|
|
package com.destroystokyo.paper;
|
|
|
|
+import com.destroystokyo.paper.io.chunk.ChunkTaskManager;
|
|
import com.google.common.base.Strings;
|
|
import com.google.common.base.Throwables;
|
|
|
|
@@ -367,4 +368,64 @@ public class PaperConfig {
|
|
maxBookPageSize = getInt("settings.book-size.page-max", maxBookPageSize);
|
|
maxBookTotalSizeMultiplier = getDouble("settings.book-size.total-multiplier", maxBookTotalSizeMultiplier);
|
|
}
|
|
+
|
|
+ public static boolean asyncChunks = false;
|
|
+ //public static boolean asyncChunkGeneration = true; // Leave out for now until we can control this
|
|
+ //public static boolean asyncChunkGenThreadPerWorld = true; // Leave out for now until we can control this
|
|
+ public static int asyncChunkLoadThreads = -1;
|
|
+ private static void asyncChunks() {
|
|
+ if (version < 15) {
|
|
+ boolean enabled = config.getBoolean("settings.async-chunks", true);
|
|
+ ConfigurationSection section = config.createSection("settings.async-chunks");
|
|
+ section.set("enable", enabled);
|
|
+ section.set("load-threads", -1);
|
|
+ section.set("generation", true);
|
|
+ section.set("thread-per-world-generation", true);
|
|
+ }
|
|
+
|
|
+ // TODO load threads now control async chunk save for unloading chunks, look into renaming this?
|
|
+
|
|
+ asyncChunks = getBoolean("settings.async-chunks.enable", true);
|
|
+ //asyncChunkGeneration = getBoolean("settings.async-chunks.generation", true); // Leave out for now until we can control this
|
|
+ //asyncChunkGenThreadPerWorld = getBoolean("settings.async-chunks.thread-per-world-generation", true); // Leave out for now until we can control this
|
|
+ asyncChunkLoadThreads = getInt("settings.async-chunks.load-threads", -1);
|
|
+ if (asyncChunkLoadThreads <= 0) {
|
|
+ asyncChunkLoadThreads = (int) Math.min(Integer.getInteger("paper.maxChunkThreads", 8), Math.max(1, Runtime.getRuntime().availableProcessors() - 1));
|
|
+ }
|
|
+
|
|
+ // Let Shared Host set some limits
|
|
+ String sharedHostEnvGen = System.getenv("PAPER_ASYNC_CHUNKS_SHARED_HOST_GEN");
|
|
+ String sharedHostEnvLoad = System.getenv("PAPER_ASYNC_CHUNKS_SHARED_HOST_LOAD");
|
|
+ /* Ignore temporarily - we cannot control the gen threads (for now)
|
|
+ if ("1".equals(sharedHostEnvGen)) {
|
|
+ log("Async Chunks - Generation: Your host has requested to use a single thread world generation");
|
|
+ asyncChunkGenThreadPerWorld = false;
|
|
+ } else if ("2".equals(sharedHostEnvGen)) {
|
|
+ log("Async Chunks - Generation: Your host has disabled async world generation - You will experience lag from world generation");
|
|
+ asyncChunkGeneration = false;
|
|
+ }
|
|
+ */
|
|
+
|
|
+ if (sharedHostEnvLoad != null) {
|
|
+ try {
|
|
+ asyncChunkLoadThreads = Math.max(1, Math.min(asyncChunkLoadThreads, Integer.parseInt(sharedHostEnvLoad)));
|
|
+ } catch (NumberFormatException ignored) {}
|
|
+ }
|
|
+
|
|
+ if (!asyncChunks) {
|
|
+ log("Async Chunks: Disabled - Chunks will be managed synchronosuly, and will cause tremendous lag.");
|
|
+ } else {
|
|
+ ChunkTaskManager.initGlobalLoadThreads(asyncChunkLoadThreads);
|
|
+ log("Async Chunks: Enabled - Chunks will be loaded much faster, without lag.");
|
|
+ /* Ignore temporarily - we cannot control the gen threads (for now)
|
|
+ if (!asyncChunkGeneration) {
|
|
+ log("Async Chunks - Generation: Disabled - Chunks will be generated synchronosuly, and will cause tremendous lag.");
|
|
+ } else if (asyncChunkGenThreadPerWorld) {
|
|
+ log("Async Chunks - Generation: Enabled - Chunks will be generated much faster, without lag.");
|
|
+ } else {
|
|
+ log("Async Chunks - Generation: Enabled (Single Thread) - Chunks will be generated much faster, without lag.");
|
|
+ }
|
|
+ */
|
|
+ }
|
|
+ }
|
|
}
|
|
diff --git a/src/main/java/com/destroystokyo/paper/antixray/ChunkPacketBlockControllerAntiXray.java b/src/main/java/com/destroystokyo/paper/antixray/ChunkPacketBlockControllerAntiXray.java
|
|
index 23626bef3..1edcecd2e 100644
|
|
--- a/src/main/java/com/destroystokyo/paper/antixray/ChunkPacketBlockControllerAntiXray.java
|
|
+++ b/src/main/java/com/destroystokyo/paper/antixray/ChunkPacketBlockControllerAntiXray.java
|
|
@@ -9,6 +9,7 @@ import java.util.concurrent.Executors;
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
import java.util.function.Supplier;
|
|
|
|
+import com.destroystokyo.paper.io.PrioritizedTaskQueue;
|
|
import net.minecraft.server.*;
|
|
import org.bukkit.Bukkit;
|
|
import org.bukkit.World.Environment;
|
|
@@ -150,6 +151,12 @@ public class ChunkPacketBlockControllerAntiXray extends ChunkPacketBlockControll
|
|
|
|
private final AtomicInteger xrayRequests = new AtomicInteger();
|
|
|
|
+ // Paper start - async chunk api
|
|
+ private Integer nextTicketHold() {
|
|
+ return Integer.valueOf(this.xrayRequests.getAndIncrement());
|
|
+ }
|
|
+ // Paper end
|
|
+
|
|
private Integer addXrayTickets(final int x, final int z, final ChunkProviderServer chunkProvider) {
|
|
final Integer hold = Integer.valueOf(this.xrayRequests.getAndIncrement());
|
|
|
|
@@ -181,6 +188,35 @@ public class ChunkPacketBlockControllerAntiXray extends ChunkPacketBlockControll
|
|
chunk.world.getChunkAt(locX, locZ + 1);
|
|
}
|
|
|
|
+ // Paper start - async chunk api
|
|
+ private void loadNeighbourAsync(ChunkProviderServer chunkProvider, WorldServer world, int chunkX, int chunkZ, int[] counter, java.util.function.Consumer<Chunk> onNeighourLoad, Runnable onAllNeighboursLoad) {
|
|
+ chunkProvider.getChunkAtAsynchronously(chunkX, chunkZ, true, (Chunk neighbour) -> {
|
|
+ onNeighourLoad.accept(neighbour);
|
|
+ if (++counter[0] == 4) {
|
|
+ onAllNeighboursLoad.run();
|
|
+ }
|
|
+ });
|
|
+ world.asyncChunkTaskManager.raisePriority(chunkX, chunkZ, PrioritizedTaskQueue.HIGHER_PRIORITY);
|
|
+ }
|
|
+
|
|
+ private void loadNeighboursAsync(Chunk chunk, java.util.function.Consumer<Chunk> onNeighourLoad, Runnable onAllNeighboursLoad) {
|
|
+ int[] loaded = new int[1];
|
|
+
|
|
+ int locX = chunk.getPos().x;
|
|
+ int locZ = chunk.getPos().z;
|
|
+ WorldServer world = ((WorldServer)chunk.world);
|
|
+
|
|
+ onNeighourLoad.accept(chunk);
|
|
+
|
|
+ ChunkProviderServer chunkProvider = world.getChunkProvider();
|
|
+
|
|
+ this.loadNeighbourAsync(chunkProvider, world, locX - 1, locZ, loaded, onNeighourLoad, onAllNeighboursLoad);
|
|
+ this.loadNeighbourAsync(chunkProvider, world, locX + 1, locZ, loaded, onNeighourLoad, onAllNeighboursLoad);
|
|
+ this.loadNeighbourAsync(chunkProvider, world, locX, locZ - 1, loaded, onNeighourLoad, onAllNeighboursLoad);
|
|
+ this.loadNeighbourAsync(chunkProvider, world, locX, locZ + 1, loaded, onNeighourLoad, onAllNeighboursLoad);
|
|
+ }
|
|
+ // Paper end
|
|
+
|
|
@Override
|
|
public boolean onChunkPacketCreate(Chunk chunk, int chunkSectionSelector, boolean force) {
|
|
int locX = chunk.getPos().x;
|
|
@@ -256,11 +292,15 @@ public class ChunkPacketBlockControllerAntiXray extends ChunkPacketBlockControll
|
|
|
|
if (chunks[0] == null || chunks[1] == null || chunks[2] == null || chunks[3] == null) {
|
|
// we need to load
|
|
- MinecraftServer.getServer().scheduleOnMain(() -> {
|
|
- Integer ticketHold = this.addXrayTickets(locX, locZ, world.getChunkProvider());
|
|
- this.loadNeighbours(chunk);
|
|
+ // Paper start - async chunk api
|
|
+ Integer ticketHold = this.nextTicketHold();
|
|
+ this.loadNeighboursAsync(chunk, (Chunk neighbour) -> { // when a neighbour is loaded
|
|
+ ((WorldServer)neighbour.world).getChunkProvider().addTicket(TicketType.ANTIXRAY, neighbour.getPos(), 0, ticketHold);
|
|
+ },
|
|
+ () -> { // once neighbours get loaded
|
|
this.modifyBlocks(packetPlayOutMapChunk, chunkPacketInfo, false, ticketHold);
|
|
});
|
|
+ // Paper end
|
|
return;
|
|
}
|
|
|
|
diff --git a/src/main/java/com/destroystokyo/paper/io/IOUtil.java b/src/main/java/com/destroystokyo/paper/io/IOUtil.java
|
|
new file mode 100644
|
|
index 000000000..5af0ac3d9
|
|
--- /dev/null
|
|
+++ b/src/main/java/com/destroystokyo/paper/io/IOUtil.java
|
|
@@ -0,0 +1,62 @@
|
|
+package com.destroystokyo.paper.io;
|
|
+
|
|
+import org.bukkit.Bukkit;
|
|
+
|
|
+public final class IOUtil {
|
|
+
|
|
+ /* Copied from concrete or concurrentutil */
|
|
+
|
|
+ public static long getCoordinateKey(final int x, final int z) {
|
|
+ return ((long)z << 32) | (x & 0xFFFFFFFFL);
|
|
+ }
|
|
+
|
|
+ public static int getCoordinateX(final long key) {
|
|
+ return (int)key;
|
|
+ }
|
|
+
|
|
+ public static int getCoordinateZ(final long key) {
|
|
+ return (int)(key >>> 32);
|
|
+ }
|
|
+
|
|
+ public static int getRegionCoordinate(final int chunkCoordinate) {
|
|
+ return chunkCoordinate >> 5;
|
|
+ }
|
|
+
|
|
+ public static int getChunkInRegion(final int chunkCoordinate) {
|
|
+ return chunkCoordinate & 31;
|
|
+ }
|
|
+
|
|
+ public static String genericToString(final Object object) {
|
|
+ return object == null ? "null" : object.getClass().getName() + ":" + object.toString();
|
|
+ }
|
|
+
|
|
+ public static <T> T notNull(final T obj) {
|
|
+ if (obj == null) {
|
|
+ throw new NullPointerException();
|
|
+ }
|
|
+ return obj;
|
|
+ }
|
|
+
|
|
+ public static <T> T notNull(final T obj, final String msgIfNull) {
|
|
+ if (obj == null) {
|
|
+ throw new NullPointerException(msgIfNull);
|
|
+ }
|
|
+ return obj;
|
|
+ }
|
|
+
|
|
+ public static void arrayBounds(final int off, final int len, final int arrayLength, final String msgPrefix) {
|
|
+ if (off < 0 || len < 0 || (arrayLength - off) < len) {
|
|
+ throw new ArrayIndexOutOfBoundsException(msgPrefix + ": off: " + off + ", len: " + len + ", array length: " + arrayLength);
|
|
+ }
|
|
+ }
|
|
+
|
|
+ public static int getPriorityForCurrentThread() {
|
|
+ return Bukkit.isPrimaryThread() ? PrioritizedTaskQueue.HIGHEST_PRIORITY : PrioritizedTaskQueue.NORMAL_PRIORITY;
|
|
+ }
|
|
+
|
|
+ @SuppressWarnings("unchecked")
|
|
+ public static <T extends Throwable> void rethrow(final Throwable throwable) throws T {
|
|
+ throw (T)throwable;
|
|
+ }
|
|
+
|
|
+}
|
|
diff --git a/src/main/java/com/destroystokyo/paper/io/PaperFileIOThread.java b/src/main/java/com/destroystokyo/paper/io/PaperFileIOThread.java
|
|
new file mode 100644
|
|
index 000000000..4f10a8311
|
|
--- /dev/null
|
|
+++ b/src/main/java/com/destroystokyo/paper/io/PaperFileIOThread.java
|
|
@@ -0,0 +1,661 @@
|
|
+package com.destroystokyo.paper.io;
|
|
+
|
|
+import net.minecraft.server.ChunkCoordIntPair;
|
|
+import net.minecraft.server.ExceptionWorldConflict;
|
|
+import net.minecraft.server.MinecraftServer;
|
|
+import net.minecraft.server.NBTTagCompound;
|
|
+import net.minecraft.server.RegionFile;
|
|
+import net.minecraft.server.WorldServer;
|
|
+import org.apache.logging.log4j.Logger;
|
|
+
|
|
+import java.io.IOException;
|
|
+import java.util.concurrent.CompletableFuture;
|
|
+import java.util.concurrent.ConcurrentHashMap;
|
|
+import java.util.concurrent.atomic.AtomicLong;
|
|
+import java.util.function.Consumer;
|
|
+import java.util.function.Function;
|
|
+
|
|
+/**
|
|
+ * Prioritized singleton thread responsible for all chunk IO that occurs in a minecraft server.
|
|
+ *
|
|
+ * <p>
|
|
+ * Singleton access: {@link Holder#INSTANCE}
|
|
+ * </p>
|
|
+ *
|
|
+ * <p>
|
|
+ * All functions provided are MT-Safe, however certain ordering constraints are (but not enforced):
|
|
+ * <li>
|
|
+ * Chunk saves may not occur for unloaded chunks.
|
|
+ * </li>
|
|
+ * <li>
|
|
+ * Tasks must be scheduled on the main thread.
|
|
+ * </li>
|
|
+ * </p>
|
|
+ *
|
|
+ * @see Holder#INSTANCE
|
|
+ * @see #scheduleSave(WorldServer, int, int, NBTTagCompound, NBTTagCompound, int)
|
|
+ * @see #loadChunkDataAsync(WorldServer, int, int, int, Consumer, boolean, boolean, boolean)
|
|
+ */
|
|
+public final class PaperFileIOThread extends QueueExecutorThread {
|
|
+
|
|
+ public static final Logger LOGGER = MinecraftServer.LOGGER;
|
|
+ public static final NBTTagCompound FAILURE_VALUE = new NBTTagCompound();
|
|
+
|
|
+ public static final class Holder {
|
|
+
|
|
+ public static final PaperFileIOThread INSTANCE = new PaperFileIOThread();
|
|
+
|
|
+ static {
|
|
+ INSTANCE.start();
|
|
+ }
|
|
+ }
|
|
+
|
|
+ private final AtomicLong writeCounter = new AtomicLong();
|
|
+
|
|
+ private PaperFileIOThread() {
|
|
+ super(new PrioritizedTaskQueue<>(), (int)(1.0e6)); // 1.0ms spinwait time
|
|
+ this.setName("Paper RegionFile IO Thread");
|
|
+ this.setPriority(Thread.NORM_PRIORITY - 1); // we keep priority close to normal because threads can wait on us
|
|
+ this.setUncaughtExceptionHandler((final Thread unused, final Throwable thr) -> {
|
|
+ LOGGER.fatal("Uncaught exception thrown from IO thread, report this!", thr);
|
|
+ });
|
|
+ }
|
|
+
|
|
+ /* run() is implemented by superclass */
|
|
+
|
|
+ /*
|
|
+ *
|
|
+ * IO thread will perform reads before writes
|
|
+ *
|
|
+ * How reads/writes are scheduled:
|
|
+ *
|
|
+ * If read in progress while scheduling write, ignore read and schedule write
|
|
+ * If read in progress while scheduling read (no write in progress), chain the read task
|
|
+ *
|
|
+ *
|
|
+ * If write in progress while scheduling read, use the pending write data and ret immediately
|
|
+ * If write in progress while scheduling write (ignore read in progress), overwrite the write in progress data
|
|
+ *
|
|
+ * This allows the reads and writes to act as if they occur synchronously to the thread scheduling them, however
|
|
+ * it fails to properly propagate write failures. When writes fail the data is kept so future reads will actually
|
|
+ * read the failed write data. This should hopefully act as a way to prevent data loss for spurious fails for writing data.
|
|
+ *
|
|
+ */
|
|
+
|
|
+ /**
|
|
+ * Attempts to bump the priority of all IO tasks for the given chunk coordinates. This has no effect if no tasks are queued.
|
|
+ * @param world Chunk's world
|
|
+ * @param chunkX Chunk's x coordinate
|
|
+ * @param chunkZ Chunk's z coordinate
|
|
+ * @param priority Priority level to try to bump to
|
|
+ */
|
|
+ public void bumpPriority(final WorldServer world, final int chunkX, final int chunkZ, final int priority) {
|
|
+ if (!PrioritizedTaskQueue.validPriority(priority)) {
|
|
+ throw new IllegalArgumentException("Invalid priority: " + priority);
|
|
+ }
|
|
+
|
|
+ final Long key = Long.valueOf(IOUtil.getCoordinateKey(chunkX, chunkZ));
|
|
+
|
|
+ final ChunkDataTask poiTask = world.poiDataController.tasks.get(key);
|
|
+ final ChunkDataTask chunkTask = world.chunkDataController.tasks.get(key);
|
|
+
|
|
+ if (poiTask != null) {
|
|
+ poiTask.raisePriority(priority);
|
|
+ }
|
|
+ if (chunkTask != null) {
|
|
+ chunkTask.raisePriority(priority);
|
|
+ }
|
|
+ }
|
|
+
|
|
+ // Hack start
|
|
+ /**
|
|
+ * if {@code waitForRead} is true, then this task will wait on an available read task, else it will wait on an available
|
|
+ * write task
|
|
+ * if {@code poiTask} is true, then this task will wait on a poi task, else it will wait on chunk data task
|
|
+ * @deprecated API is garbage and will only work for main thread queueing of tasks (which is vanilla), plugins messing
|
|
+ * around asynchronously will give unexpected results
|
|
+ * @return whether the task succeeded, or {@code null} if there is no task
|
|
+ */
|
|
+ @Deprecated
|
|
+ public Boolean waitForIOToComplete(final WorldServer world, final int chunkX, final int chunkZ, final boolean waitForRead,
|
|
+ final boolean poiTask) {
|
|
+ final ChunkDataTask task;
|
|
+
|
|
+ final Long key = IOUtil.getCoordinateKey(chunkX, chunkZ);
|
|
+ if (poiTask) {
|
|
+ task = world.poiDataController.tasks.get(key);
|
|
+ } else {
|
|
+ task = world.chunkDataController.tasks.get(key);
|
|
+ }
|
|
+
|
|
+ if (task == null) {
|
|
+ return null;
|
|
+ }
|
|
+
|
|
+ if (waitForRead) {
|
|
+ ChunkDataController.InProgressRead read = task.inProgressRead;
|
|
+ if (read == null) {
|
|
+ return null;
|
|
+ }
|
|
+ return Boolean.valueOf(read.readFuture.join() != PaperFileIOThread.FAILURE_VALUE);
|
|
+ }
|
|
+
|
|
+ // wait for write
|
|
+ ChunkDataController.InProgressWrite write = task.inProgressWrite;
|
|
+ if (write == null) {
|
|
+ return null;
|
|
+ }
|
|
+ return Boolean.valueOf(write.wrote.join() != PaperFileIOThread.FAILURE_VALUE);
|
|
+ }
|
|
+ // Hack end
|
|
+
|
|
+ public NBTTagCompound getPendingWrite(final WorldServer world, final int chunkX, final int chunkZ, final boolean poiData) {
|
|
+ final ChunkDataController taskController = poiData ? world.poiDataController : world.chunkDataController;
|
|
+
|
|
+ final ChunkDataTask dataTask = taskController.tasks.get(Long.valueOf(IOUtil.getCoordinateKey(chunkX, chunkZ)));
|
|
+
|
|
+ if (dataTask == null) {
|
|
+ return null;
|
|
+ }
|
|
+
|
|
+ final ChunkDataController.InProgressWrite write = dataTask.inProgressWrite;
|
|
+
|
|
+ if (write == null) {
|
|
+ return null;
|
|
+ }
|
|
+
|
|
+ return write.data;
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Sets the priority of all IO tasks for the given chunk coordinates. This has no effect if no tasks are queued.
|
|
+ * @param world Chunk's world
|
|
+ * @param chunkX Chunk's x coordinate
|
|
+ * @param chunkZ Chunk's z coordinate
|
|
+ * @param priority Priority level to set to
|
|
+ */
|
|
+ public void setPriority(final WorldServer world, final int chunkX, final int chunkZ, final int priority) {
|
|
+ if (!PrioritizedTaskQueue.validPriority(priority)) {
|
|
+ throw new IllegalArgumentException("Invalid priority: " + priority);
|
|
+ }
|
|
+
|
|
+ final Long key = Long.valueOf(IOUtil.getCoordinateKey(chunkX, chunkZ));
|
|
+
|
|
+ final ChunkDataTask poiTask = world.poiDataController.tasks.get(key);
|
|
+ final ChunkDataTask chunkTask = world.chunkDataController.tasks.get(key);
|
|
+
|
|
+ if (poiTask != null) {
|
|
+ poiTask.updatePriority(priority);
|
|
+ }
|
|
+ if (chunkTask != null) {
|
|
+ chunkTask.updatePriority(priority);
|
|
+ }
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Schedules the chunk data to be written asynchronously.
|
|
+ * <p>
|
|
+ * Impl notes:
|
|
+ * </p>
|
|
+ * <li>
|
|
+ * This function presumes a chunk load for the coordinates is not called during this function (anytime after is OK). This means
|
|
+ * saves must be scheduled before a chunk is unloaded.
|
|
+ * </li>
|
|
+ * <li>
|
|
+ * Writes may be called concurrently, although only the "later" write will go through.
|
|
+ * </li>
|
|
+ * @param world Chunk's world
|
|
+ * @param chunkX Chunk's x coordinate
|
|
+ * @param chunkZ Chunk's z coordinate
|
|
+ * @param poiData Chunk point of interest data. If {@code null}, then no poi data is saved.
|
|
+ * @param chunkData Chunk data. If {@code null}, then no chunk data is saved.
|
|
+ * @param priority Priority level for this task. See {@link PrioritizedTaskQueue}
|
|
+ * @throws IllegalArgumentException If both {@code poiData} and {@code chunkData} are {@code null}.
|
|
+ * @throws IllegalStateException If the file io thread has shutdown.
|
|
+ */
|
|
+ public void scheduleSave(final WorldServer world, final int chunkX, final int chunkZ,
|
|
+ final NBTTagCompound poiData, final NBTTagCompound chunkData,
|
|
+ final int priority) throws IllegalArgumentException {
|
|
+ if (!PrioritizedTaskQueue.validPriority(priority)) {
|
|
+ throw new IllegalArgumentException("Invalid priority: " + priority);
|
|
+ }
|
|
+
|
|
+ final long writeCounter = this.writeCounter.getAndIncrement();
|
|
+
|
|
+ if (poiData != null) {
|
|
+ this.scheduleWrite(world.poiDataController, world, chunkX, chunkZ, poiData, priority, writeCounter);
|
|
+ }
|
|
+ if (chunkData != null) {
|
|
+ this.scheduleWrite(world.chunkDataController, world, chunkX, chunkZ, chunkData, priority, writeCounter);
|
|
+ }
|
|
+ }
|
|
+
|
|
+ private void scheduleWrite(final ChunkDataController dataController, final WorldServer world,
|
|
+ final int chunkX, final int chunkZ, final NBTTagCompound data, final int priority, final long writeCounter) {
|
|
+ dataController.tasks.compute(Long.valueOf(IOUtil.getCoordinateKey(chunkX, chunkZ)), (final Long keyInMap, final ChunkDataTask taskRunning) -> {
|
|
+ if (taskRunning == null) {
|
|
+ // no task is scheduled
|
|
+
|
|
+ // create task
|
|
+ final ChunkDataTask newTask = new ChunkDataTask(priority, world, chunkX, chunkZ, dataController);
|
|
+ newTask.inProgressWrite = new ChunkDataController.InProgressWrite();
|
|
+ newTask.inProgressWrite.writeCounter = writeCounter;
|
|
+ newTask.inProgressWrite.data = data;
|
|
+
|
|
+ PaperFileIOThread.this.queueTask(newTask); // schedule
|
|
+ return newTask;
|
|
+ }
|
|
+
|
|
+ taskRunning.raisePriority(priority);
|
|
+
|
|
+ if (taskRunning.inProgressWrite == null) {
|
|
+ taskRunning.inProgressWrite = new ChunkDataController.InProgressWrite();
|
|
+ }
|
|
+
|
|
+ boolean reschedule = taskRunning.inProgressWrite.writeCounter == -1L;
|
|
+
|
|
+ // synchronize for readers
|
|
+ //noinspection SynchronizationOnLocalVariableOrMethodParameter
|
|
+ synchronized (taskRunning) {
|
|
+ taskRunning.inProgressWrite.data = data;
|
|
+ taskRunning.inProgressWrite.writeCounter = writeCounter;
|
|
+ }
|
|
+
|
|
+ if (reschedule) {
|
|
+ // We need to reschedule this task since the previous one is not currently scheduled since it failed
|
|
+ taskRunning.reschedule(priority);
|
|
+ }
|
|
+
|
|
+ return taskRunning;
|
|
+ });
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Same as {@link #loadChunkDataAsync(WorldServer, int, int, int, Consumer, boolean, boolean, boolean)}, except this function returns
|
|
+ * a {@link CompletableFuture} which is potentially completed <b>ASYNCHRONOUSLY ON THE FILE IO THREAD</b> when the load task
|
|
+ * has completed.
|
|
+ * <p>
|
|
+ * Note that if the chunk fails to load the returned future is completed with {@code null}.
|
|
+ * </p>
|
|
+ */
|
|
+ public CompletableFuture<ChunkData> loadChunkDataAsyncFuture(final WorldServer world, final int chunkX, final int chunkZ,
|
|
+ final int priority, final boolean readPoiData, final boolean readChunkData,
|
|
+ final boolean intendingToBlock) {
|
|
+ final CompletableFuture<ChunkData> future = new CompletableFuture<>();
|
|
+ this.loadChunkDataAsync(world, chunkX, chunkZ, priority, future::complete, readPoiData, readChunkData, intendingToBlock);
|
|
+ return future;
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Schedules a load to be executed asynchronously.
|
|
+ * <p>
|
|
+ * Impl notes:
|
|
+ * </p>
|
|
+ * <li>
|
|
+ * If a chunk fails to load, the {@code onComplete} parameter is completed with {@code null}.
|
|
+ * </li>
|
|
+ * <li>
|
|
+ * It is possible for the {@code onComplete} parameter to be given {@link ChunkData} containing data
|
|
+ * this call did not request.
|
|
+ * </li>
|
|
+ * <li>
|
|
+ * The {@code onComplete} parameter may be completed during the execution of this function synchronously or it may
|
|
+ * be completed asynchronously on this file io thread. Interacting with the file IO thread in the completion of
|
|
+ * data is undefined behaviour, and can cause deadlock.
|
|
+ * </li>
|
|
+ * @param world Chunk's world
|
|
+ * @param chunkX Chunk's x coordinate
|
|
+ * @param chunkZ Chunk's z coordinate
|
|
+ * @param priority Priority level for this task. See {@link PrioritizedTaskQueue}
|
|
+ * @param onComplete Consumer to execute once this task has completed
|
|
+ * @param readPoiData Whether to read point of interest data. If {@code false}, the {@code NBTTagCompound} will be {@code null}.
|
|
+ * @param readChunkData Whether to read chunk data. If {@code false}, the {@code NBTTagCompound} will be {@code null}.
|
|
+ * @return The {@link PrioritizedTaskQueue.PrioritizedTask} associated with this task. Note that this task does not support
|
|
+ * cancellation.
|
|
+ */
|
|
+ public void loadChunkDataAsync(final WorldServer world, final int chunkX, final int chunkZ,
|
|
+ final int priority, final Consumer<ChunkData> onComplete,
|
|
+ final boolean readPoiData, final boolean readChunkData,
|
|
+ final boolean intendingToBlock) {
|
|
+ if (!PrioritizedTaskQueue.validPriority(priority)) {
|
|
+ throw new IllegalArgumentException("Invalid priority: " + priority);
|
|
+ }
|
|
+
|
|
+ if (!(readPoiData | readChunkData)) {
|
|
+ throw new IllegalArgumentException("Must read chunk data or poi data");
|
|
+ }
|
|
+
|
|
+ final ChunkData complete = new ChunkData();
|
|
+ final boolean[] requireCompletion = new boolean[] { readPoiData, readChunkData };
|
|
+
|
|
+ if (readPoiData) {
|
|
+ this.scheduleRead(world.poiDataController, world, chunkX, chunkZ, (final NBTTagCompound poiData) -> {
|
|
+ complete.poiData = poiData;
|
|
+
|
|
+ final boolean finished;
|
|
+
|
|
+ // avoid a race condition where the file io thread completes and we complete synchronously
|
|
+ // Note: Synchronization can be elided if both of the accesses are volatile
|
|
+ synchronized (requireCompletion) {
|
|
+ requireCompletion[0] = false; // 0 -> poi data
|
|
+ finished = !requireCompletion[1]; // 1 -> chunk data
|
|
+ }
|
|
+
|
|
+ if (finished) {
|
|
+ onComplete.accept(complete);
|
|
+ }
|
|
+ }, priority, intendingToBlock);
|
|
+ }
|
|
+
|
|
+ if (readChunkData) {
|
|
+ this.scheduleRead(world.chunkDataController, world, chunkX, chunkZ, (final NBTTagCompound chunkData) -> {
|
|
+ complete.chunkData = chunkData;
|
|
+
|
|
+ final boolean finished;
|
|
+
|
|
+ // avoid a race condition where the file io thread completes and we complete synchronously
|
|
+ // Note: Synchronization can be elided if both of the accesses are volatile
|
|
+ synchronized (requireCompletion) {
|
|
+ requireCompletion[1] = false; // 1 -> chunk data
|
|
+ finished = !requireCompletion[0]; // 0 -> poi data
|
|
+ }
|
|
+
|
|
+ if (finished) {
|
|
+ onComplete.accept(complete);
|
|
+ }
|
|
+ }, priority, intendingToBlock);
|
|
+ }
|
|
+
|
|
+ }
|
|
+
|
|
+ // Note: the onComplete may be called asynchronously or synchronously here.
|
|
+ private void scheduleRead(final ChunkDataController dataController, final WorldServer world,
|
|
+ final int chunkX, final int chunkZ, final Consumer<NBTTagCompound> onComplete, final int priority,
|
|
+ final boolean intendingToBlock) {
|
|
+
|
|
+ Function<RegionFile, Boolean> tryLoadFunction = (final RegionFile file) -> {
|
|
+ if (file == null) {
|
|
+ return Boolean.TRUE;
|
|
+ }
|
|
+ return Boolean.valueOf(file.chunkExists(new ChunkCoordIntPair(chunkX, chunkZ)));
|
|
+ };
|
|
+
|
|
+ dataController.tasks.compute(Long.valueOf(IOUtil.getCoordinateKey(chunkX, chunkZ)), (final Long keyInMap, final ChunkDataTask running) -> {
|
|
+ if (running == null) {
|
|
+ // not scheduled
|
|
+
|
|
+ final Boolean shouldSchedule = intendingToBlock ? dataController.computeForRegionFile(chunkX, chunkZ, tryLoadFunction) :
|
|
+ dataController.computeForRegionFileIfLoaded(chunkX, chunkZ, tryLoadFunction);
|
|
+
|
|
+ if (shouldSchedule == Boolean.FALSE) {
|
|
+ // not on disk
|
|
+ onComplete.accept(null);
|
|
+ return null;
|
|
+ }
|
|
+
|
|
+ // set up task
|
|
+ final ChunkDataTask newTask = new ChunkDataTask(priority, world, chunkX, chunkZ, dataController);
|
|
+ newTask.inProgressRead = new ChunkDataController.InProgressRead();
|
|
+ newTask.inProgressRead.readFuture.thenAccept(onComplete);
|
|
+
|
|
+ PaperFileIOThread.this.queueTask(newTask); // schedule task
|
|
+ return newTask;
|
|
+ }
|
|
+
|
|
+ running.raisePriority(priority);
|
|
+
|
|
+ if (running.inProgressWrite == null) {
|
|
+ // chain to the read future
|
|
+ running.inProgressRead.readFuture.thenAccept(onComplete);
|
|
+ return running;
|
|
+ }
|
|
+
|
|
+ // at this stage we have to use the in progress write's data to avoid an order issue
|
|
+ // we don't synchronize since all writes to data occur in the compute() call
|
|
+ onComplete.accept(running.inProgressWrite.data);
|
|
+ return running;
|
|
+ });
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Same as {@link #loadChunkDataAsync(WorldServer, int, int, int, Consumer, boolean, boolean, boolean)}, except this function returns
|
|
+ * the {@link ChunkData} associated with the specified chunk when the task is complete.
|
|
+ * @return The chunk data, or {@code null} if the chunk failed to load.
|
|
+ */
|
|
+ public ChunkData loadChunkData(final WorldServer world, final int chunkX, final int chunkZ, final int priority,
|
|
+ final boolean readPoiData, final boolean readChunkData) {
|
|
+ return this.loadChunkDataAsyncFuture(world, chunkX, chunkZ, priority, readPoiData, readChunkData, true).join();
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Schedules the given task at the specified priority to be executed on the IO thread.
|
|
+ * <p>
|
|
+ * Internal api. Do not use.
|
|
+ * </p>
|
|
+ */
|
|
+ public void runTask(final int priority, final Runnable runnable) {
|
|
+ this.queueTask(new GeneralTask(priority, runnable));
|
|
+ }
|
|
+
|
|
+ static final class GeneralTask extends PrioritizedTaskQueue.PrioritizedTask implements Runnable {
|
|
+
|
|
+ private final Runnable run;
|
|
+
|
|
+ public GeneralTask(final int priority, final Runnable run) {
|
|
+ super(priority);
|
|
+ this.run = IOUtil.notNull(run, "Task may not be null");
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public void run() {
|
|
+ try {
|
|
+ this.run.run();
|
|
+ } catch (final Throwable throwable) {
|
|
+ if (throwable instanceof ThreadDeath) {
|
|
+ throw (ThreadDeath)throwable;
|
|
+ }
|
|
+ LOGGER.fatal("Failed to execute general task on IO thread " + IOUtil.genericToString(this.run), throwable);
|
|
+ }
|
|
+ }
|
|
+ }
|
|
+
|
|
+ public static final class ChunkData {
|
|
+
|
|
+ public NBTTagCompound poiData;
|
|
+ public NBTTagCompound chunkData;
|
|
+
|
|
+ public ChunkData() {}
|
|
+
|
|
+ public ChunkData(final NBTTagCompound poiData, final NBTTagCompound chunkData) {
|
|
+ this.poiData = poiData;
|
|
+ this.chunkData = chunkData;
|
|
+ }
|
|
+ }
|
|
+
|
|
+ public static abstract class ChunkDataController {
|
|
+
|
|
+ // ConcurrentHashMap synchronizes per chain, so reduce the chance of task's hashes colliding.
|
|
+ public final ConcurrentHashMap<Long, ChunkDataTask> tasks = new ConcurrentHashMap<>(64, 0.5f);
|
|
+
|
|
+ public abstract void writeData(final int x, final int z, final NBTTagCompound compound) throws IOException;
|
|
+ public abstract NBTTagCompound readData(final int x, final int z) throws IOException;
|
|
+
|
|
+ public abstract <T> T computeForRegionFile(final int chunkX, final int chunkZ, final Function<RegionFile, T> function);
|
|
+ public abstract <T> T computeForRegionFileIfLoaded(final int chunkX, final int chunkZ, final Function<RegionFile, T> function);
|
|
+
|
|
+ public static final class InProgressWrite {
|
|
+ public long writeCounter;
|
|
+ public NBTTagCompound data;
|
|
+
|
|
+ // Hack start
|
|
+ @Deprecated
|
|
+ public CompletableFuture<NBTTagCompound> wrote = new CompletableFuture<>();
|
|
+ // Hack end
|
|
+ }
|
|
+
|
|
+ public static final class InProgressRead {
|
|
+ public final CompletableFuture<NBTTagCompound> readFuture = new CompletableFuture<>();
|
|
+ }
|
|
+ }
|
|
+
|
|
+ public static final class ChunkDataTask extends PrioritizedTaskQueue.PrioritizedTask implements Runnable {
|
|
+
|
|
+ public ChunkDataController.InProgressWrite inProgressWrite;
|
|
+ public ChunkDataController.InProgressRead inProgressRead;
|
|
+
|
|
+ private final WorldServer world;
|
|
+ private final int x;
|
|
+ private final int z;
|
|
+ private final ChunkDataController taskController;
|
|
+
|
|
+ public ChunkDataTask(final int priority, final WorldServer world, final int x, final int z, final ChunkDataController taskController) {
|
|
+ super(priority);
|
|
+ this.world = world;
|
|
+ this.x = x;
|
|
+ this.z = z;
|
|
+ this.taskController = taskController;
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public String toString() {
|
|
+ return "Task for world: '" + this.world.getWorld().getName() + "' at " + this.x + "," + this.z +
|
|
+ " poi: " + (this.taskController == this.world.poiDataController) + ", hash: " + this.hashCode();
|
|
+ }
|
|
+
|
|
+ /*
|
|
+ *
|
|
+ * IO thread will perform reads before writes
|
|
+ *
|
|
+ * How reads/writes are scheduled:
|
|
+ *
|
|
+ * If read in progress while scheduling write, ignore read and schedule write
|
|
+ * If read in progress while scheduling read (no write in progress), chain the read task
|
|
+ *
|
|
+ *
|
|
+ * If write in progress while scheduling read, use the pending write data and ret immediately
|
|
+ * If write in progress while scheduling write (ignore read in progress), overwrite the write in progress data
|
|
+ *
|
|
+ * This allows the reads and writes to act as if they occur synchronously to the thread scheduling them, however
|
|
+ * it fails to properly propagate write failures
|
|
+ *
|
|
+ */
|
|
+
|
|
+ void reschedule(final int priority) {
|
|
+ // priority is checked before this stage // TODO what
|
|
+ this.queue.lazySet(null);
|
|
+ this.inProgressWrite.wrote = new CompletableFuture<>(); // Hack
|
|
+ this.priority.lazySet(priority);
|
|
+ PaperFileIOThread.Holder.INSTANCE.queueTask(this);
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public void run() {
|
|
+ ChunkDataController.InProgressRead read = this.inProgressRead;
|
|
+ if (read != null) {
|
|
+ NBTTagCompound compound = PaperFileIOThread.FAILURE_VALUE;
|
|
+ try {
|
|
+ compound = this.taskController.readData(this.x, this.z);
|
|
+ } catch (final Throwable thr) {
|
|
+ if (thr instanceof ThreadDeath) {
|
|
+ throw (ThreadDeath)thr;
|
|
+ }
|
|
+ LOGGER.fatal("Failed to read chunk data for task: " + this.toString(), thr);
|
|
+ // fall through to complete with null data
|
|
+ }
|
|
+ read.readFuture.complete(compound);
|
|
+ }
|
|
+
|
|
+ final Long chunkKey = Long.valueOf(IOUtil.getCoordinateKey(this.x, this.z));
|
|
+
|
|
+ ChunkDataController.InProgressWrite write = this.inProgressWrite;
|
|
+
|
|
+ if (write == null) {
|
|
+ // IntelliJ warns this is invalid, however it does not consider that writes to the task map & the inProgress field can occur concurrently.
|
|
+ ChunkDataTask inMap = this.taskController.tasks.compute(chunkKey, (final Long keyInMap, final ChunkDataTask valueInMap) -> {
|
|
+ if (valueInMap == null) {
|
|
+ throw new IllegalStateException("Write completed concurrently, expected this task: " + ChunkDataTask.this.toString() + ", report this!");
|
|
+ }
|
|
+ if (valueInMap != ChunkDataTask.this) {
|
|
+ throw new IllegalStateException("Chunk task mismatch, expected this task: " + ChunkDataTask.this.toString() + ", got: " + valueInMap.toString() + ", report this!");
|
|
+ }
|
|
+ return valueInMap.inProgressWrite == null ? null : valueInMap;
|
|
+ });
|
|
+
|
|
+ if (inMap == null) {
|
|
+ return; // set the task value to null, indicating we're done
|
|
+ }
|
|
+
|
|
+ // not null, which means there was a concurrent write
|
|
+ write = this.inProgressWrite;
|
|
+ }
|
|
+
|
|
+ // check if another process is writing
|
|
+ try {
|
|
+ this.world.checkSession();
|
|
+ } catch (final ExceptionWorldConflict ex) {
|
|
+ LOGGER.fatal("Couldn't save chunk; already in use by another instance of Minecraft?", ex);
|
|
+ // we don't need to set the write counter to -1 as we know at this stage there's no point in re-scheduling
|
|
+ // writes since they'll fail anyways.
|
|
+ write.wrote.complete(PaperFileIOThread.FAILURE_VALUE); // Hack - However we need to fail the write
|
|
+ return;
|
|
+ }
|
|
+
|
|
+ for (;;) {
|
|
+ final long writeCounter;
|
|
+ final NBTTagCompound data;
|
|
+
|
|
+ //noinspection SynchronizationOnLocalVariableOrMethodParameter
|
|
+ synchronized (write) {
|
|
+ writeCounter = write.writeCounter;
|
|
+ data = write.data;
|
|
+ }
|
|
+
|
|
+ boolean failedWrite = false;
|
|
+
|
|
+ try {
|
|
+ this.taskController.writeData(this.x, this.z, data);
|
|
+ } catch (final Throwable thr) {
|
|
+ if (thr instanceof ThreadDeath) {
|
|
+ throw (ThreadDeath)thr;
|
|
+ }
|
|
+ LOGGER.fatal("Failed to write chunk data for task: " + this.toString(), thr);
|
|
+ failedWrite = true;
|
|
+ }
|
|
+
|
|
+ boolean finalFailWrite = failedWrite;
|
|
+
|
|
+ ChunkDataTask inMap = this.taskController.tasks.compute(chunkKey, (final Long keyInMap, final ChunkDataTask valueInMap) -> {
|
|
+ if (valueInMap == null) {
|
|
+ ChunkDataTask.this.inProgressWrite.wrote.complete(PaperFileIOThread.FAILURE_VALUE); // Hack
|
|
+ throw new IllegalStateException("Write completed concurrently, expected this task: " + ChunkDataTask.this.toString() + ", report this!");
|
|
+ }
|
|
+ if (valueInMap != ChunkDataTask.this) {
|
|
+ ChunkDataTask.this.inProgressWrite.wrote.complete(PaperFileIOThread.FAILURE_VALUE); // Hack
|
|
+ throw new IllegalStateException("Chunk task mismatch, expected this task: " + ChunkDataTask.this.toString() + ", got: " + valueInMap.toString() + ", report this!");
|
|
+ }
|
|
+ if (valueInMap.inProgressWrite.writeCounter == writeCounter) {
|
|
+ if (finalFailWrite) {
|
|
+ valueInMap.inProgressWrite.writeCounter = -1L;
|
|
+ valueInMap.inProgressWrite.wrote.complete(PaperFileIOThread.FAILURE_VALUE);
|
|
+ } else {
|
|
+ valueInMap.inProgressWrite.wrote.complete(data);
|
|
+ }
|
|
+
|
|
+ return null;
|
|
+ }
|
|
+ return valueInMap;
|
|
+ // Hack end
|
|
+ });
|
|
+
|
|
+ if (inMap == null) {
|
|
+ // write counter matched, so we wrote the most up-to-date pending data, we're done here
|
|
+ // or we failed to write and successfully set the write counter to -1
|
|
+ return; // we're done here
|
|
+ }
|
|
+
|
|
+ // fetch & write new data
|
|
+ continue;
|
|
+ }
|
|
+ }
|
|
+ }
|
|
+}
|
|
diff --git a/src/main/java/com/destroystokyo/paper/io/PrioritizedTaskQueue.java b/src/main/java/com/destroystokyo/paper/io/PrioritizedTaskQueue.java
|
|
new file mode 100644
|
|
index 000000000..97f2e433c
|
|
--- /dev/null
|
|
+++ b/src/main/java/com/destroystokyo/paper/io/PrioritizedTaskQueue.java
|
|
@@ -0,0 +1,277 @@
|
|
+package com.destroystokyo.paper.io;
|
|
+
|
|
+import java.util.concurrent.ConcurrentLinkedQueue;
|
|
+import java.util.concurrent.atomic.AtomicBoolean;
|
|
+import java.util.concurrent.atomic.AtomicInteger;
|
|
+import java.util.concurrent.atomic.AtomicReference;
|
|
+
|
|
+public class PrioritizedTaskQueue<T extends PrioritizedTaskQueue.PrioritizedTask> {
|
|
+
|
|
+ // lower numbers are a higher priority (except < 0)
|
|
+ // higher priorities are always executed before lower priorities
|
|
+
|
|
+ /**
|
|
+ * Priority value indicating the task has completed or is being completed.
|
|
+ */
|
|
+ public static final int COMPLETING_PRIORITY = -1;
|
|
+
|
|
+ /**
|
|
+ * Highest priority, should only be used for main thread tasks or tasks that are blocking the main thread.
|
|
+ */
|
|
+ public static final int HIGHEST_PRIORITY = 0;
|
|
+
|
|
+ /**
|
|
+ * Should be only used in an IO task so that chunk loads do not wait on other IO tasks.
|
|
+ * This only exists because IO tasks are scheduled before chunk load tasks to decrease IO waiting times.
|
|
+ */
|
|
+ public static final int HIGHER_PRIORITY = 1;
|
|
+
|
|
+ /**
|
|
+ * Should be used for scheduling chunk loads/generation that would increase response times to users.
|
|
+ */
|
|
+ public static final int HIGH_PRIORITY = 2;
|
|
+
|
|
+ /**
|
|
+ * Default priority.
|
|
+ */
|
|
+ public static final int NORMAL_PRIORITY = 3;
|
|
+
|
|
+ /**
|
|
+ * Use for tasks not at all critical and can potentially be delayed.
|
|
+ */
|
|
+ public static final int LOW_PRIORITY = 4;
|
|
+
|
|
+ /**
|
|
+ * Use for tasks that should "eventually" execute.
|
|
+ */
|
|
+ public static final int LOWEST_PRIORITY = 5;
|
|
+
|
|
+ private static final int TOTAL_PRIORITIES = 6;
|
|
+
|
|
+ final ConcurrentLinkedQueue<T>[] queues = (ConcurrentLinkedQueue<T>[])new ConcurrentLinkedQueue[TOTAL_PRIORITIES];
|
|
+
|
|
+ private final AtomicBoolean shutdown = new AtomicBoolean();
|
|
+
|
|
+ {
|
|
+ for (int i = 0; i < TOTAL_PRIORITIES; ++i) {
|
|
+ this.queues[i] = new ConcurrentLinkedQueue<>();
|
|
+ }
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Returns whether the specified priority is valid
|
|
+ */
|
|
+ public static boolean validPriority(final int priority) {
|
|
+ return priority >= 0 && priority < TOTAL_PRIORITIES;
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Queues a task.
|
|
+ * @throws IllegalStateException If the task has already been queued. Use {@link PrioritizedTask#raisePriority(int)} to
|
|
+ * raise a task's priority.
|
|
+ * This can also be thrown if the queue has shutdown.
|
|
+ */
|
|
+ public void add(final T task) throws IllegalStateException {
|
|
+ int priority = task.getPriority();
|
|
+ if (priority != COMPLETING_PRIORITY) {
|
|
+ task.setQueue(this);
|
|
+ this.queues[priority].add(task);
|
|
+ }
|
|
+ if (this.shutdown.get()) {
|
|
+ // note: we're not actually sure at this point if our task will go through
|
|
+ throw new IllegalStateException("Queue has shutdown, refusing to execute task " + IOUtil.genericToString(task));
|
|
+ }
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Polls the highest priority task currently available. {@code null} if none.
|
|
+ */
|
|
+ public T poll() {
|
|
+ T task;
|
|
+ for (int i = 0; i < TOTAL_PRIORITIES; ++i) {
|
|
+ final ConcurrentLinkedQueue<T> queue = this.queues[i];
|
|
+
|
|
+ while ((task = queue.poll()) != null) {
|
|
+ final int prevPriority = task.tryComplete(i);
|
|
+ if (prevPriority != COMPLETING_PRIORITY && prevPriority <= i) {
|
|
+ // if the prev priority was greater-than or equal to our current priority
|
|
+ return task;
|
|
+ }
|
|
+ }
|
|
+ }
|
|
+
|
|
+ return null;
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Returns whether this queue may have tasks queued.
|
|
+ * <p>
|
|
+ * This operation is not atomic, but is MT-Safe.
|
|
+ * </p>
|
|
+ * @return {@code true} if tasks may be queued, {@code false} otherwise
|
|
+ */
|
|
+ public boolean hasTasks() {
|
|
+ for (int i = 0; i < TOTAL_PRIORITIES; ++i) {
|
|
+ final ConcurrentLinkedQueue<T> queue = this.queues[i];
|
|
+
|
|
+ if (queue.peek() != null) {
|
|
+ return true;
|
|
+ }
|
|
+ }
|
|
+ return false;
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Prevent further additions to this queue. Attempts to add after this call has completed (potentially during) will
|
|
+ * result in {@link IllegalStateException} being thrown.
|
|
+ * <p>
|
|
+ * This operation is atomic with respect to other shutdown calls
|
|
+ * </p>
|
|
+ * <p>
|
|
+ * After this call has completed, regardless of return value, this queue will be shutdown.
|
|
+ * </p>
|
|
+ * @return {@code true} if the queue was shutdown, {@code false} if it has shut down already
|
|
+ */
|
|
+ public boolean shutdown() {
|
|
+ return this.shutdown.getAndSet(false);
|
|
+ }
|
|
+
|
|
+ public abstract static class PrioritizedTask {
|
|
+
|
|
+ protected final AtomicReference<PrioritizedTaskQueue> queue = new AtomicReference<>();
|
|
+
|
|
+ protected final AtomicInteger priority;
|
|
+
|
|
+ protected PrioritizedTask() {
|
|
+ this(PrioritizedTaskQueue.NORMAL_PRIORITY);
|
|
+ }
|
|
+
|
|
+ protected PrioritizedTask(final int priority) {
|
|
+ if (!PrioritizedTaskQueue.validPriority(priority)) {
|
|
+ throw new IllegalArgumentException("Invalid priority " + priority);
|
|
+ }
|
|
+ this.priority = new AtomicInteger(priority);
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Returns the current priority. Note that {@link PrioritizedTaskQueue#COMPLETING_PRIORITY} will be returned
|
|
+ * if this task is completing or has completed.
|
|
+ */
|
|
+ public final int getPriority() {
|
|
+ return this.priority.get();
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Returns whether this task is scheduled to execute, or has been already executed.
|
|
+ */
|
|
+ public boolean isScheduled() {
|
|
+ return this.queue.get() != null;
|
|
+ }
|
|
+
|
|
+ final int tryComplete(final int minPriority) {
|
|
+ for (int curr = this.getPriorityVolatile();;) {
|
|
+ if (curr == COMPLETING_PRIORITY) {
|
|
+ return COMPLETING_PRIORITY;
|
|
+ }
|
|
+ if (curr > minPriority) {
|
|
+ // curr is lower priority
|
|
+ return curr;
|
|
+ }
|
|
+
|
|
+ if (curr == (curr = this.compareAndExchangePriorityVolatile(curr, COMPLETING_PRIORITY))) {
|
|
+ return curr;
|
|
+ }
|
|
+ continue;
|
|
+ }
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Forces this task to be completed.
|
|
+ * @return {@code true} if the task was cancelled, {@code false} if the task has already completed or is being completed.
|
|
+ */
|
|
+ public boolean cancel() {
|
|
+ return this.exchangePriorityVolatile(PrioritizedTaskQueue.COMPLETING_PRIORITY) != PrioritizedTaskQueue.COMPLETING_PRIORITY;
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Attempts to raise the priority to the priority level specified.
|
|
+ * @param priority Priority specified
|
|
+ * @return {@code true} if successful, {@code false} otherwise.
|
|
+ */
|
|
+ public boolean raisePriority(final int priority) {
|
|
+ if (!PrioritizedTaskQueue.validPriority(priority)) {
|
|
+ throw new IllegalArgumentException("Invalid priority");
|
|
+ }
|
|
+
|
|
+ for (int curr = this.getPriorityVolatile();;) {
|
|
+ if (curr == COMPLETING_PRIORITY) {
|
|
+ return false;
|
|
+ }
|
|
+ if (priority >= curr) {
|
|
+ return true;
|
|
+ }
|
|
+
|
|
+ if (curr == (curr = this.compareAndExchangePriorityVolatile(curr, priority))) {
|
|
+ PrioritizedTaskQueue queue = this.queue.get();
|
|
+ if (queue != null) {
|
|
+ //noinspection unchecked
|
|
+ queue.queues[priority].add(this); // silently fail on shutdown
|
|
+ }
|
|
+ return true;
|
|
+ }
|
|
+ continue;
|
|
+ }
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Attempts to set this task's priority level to the level specified.
|
|
+ * @param priority Specified priority level.
|
|
+ * @return {@code true} if successful, {@code false} if this task is completing or has completed.
|
|
+ */
|
|
+ public boolean updatePriority(final int priority) {
|
|
+ if (!PrioritizedTaskQueue.validPriority(priority)) {
|
|
+ throw new IllegalArgumentException("Invalid priority");
|
|
+ }
|
|
+
|
|
+ for (int curr = this.getPriorityVolatile();;) {
|
|
+ if (curr == COMPLETING_PRIORITY) {
|
|
+ return false;
|
|
+ }
|
|
+ if (curr == priority) {
|
|
+ return true;
|
|
+ }
|
|
+
|
|
+ if (curr == (curr = this.compareAndExchangePriorityVolatile(curr, priority))) {
|
|
+ PrioritizedTaskQueue queue = this.queue.get();
|
|
+ if (queue != null) {
|
|
+ //noinspection unchecked
|
|
+ queue.queues[priority].add(this); // silently fail on shutdown
|
|
+ }
|
|
+ return true;
|
|
+ }
|
|
+ continue;
|
|
+ }
|
|
+ }
|
|
+
|
|
+ void setQueue(final PrioritizedTaskQueue queue) {
|
|
+ this.queue.set(queue);
|
|
+ }
|
|
+
|
|
+ /* priority */
|
|
+
|
|
+ protected final int getPriorityVolatile() {
|
|
+ return this.priority.get();
|
|
+ }
|
|
+
|
|
+ protected final int compareAndExchangePriorityVolatile(final int expect, final int update) {
|
|
+ if (this.priority.compareAndSet(expect, update)) {
|
|
+ return expect;
|
|
+ }
|
|
+ return this.priority.get();
|
|
+ }
|
|
+
|
|
+ protected final int exchangePriorityVolatile(final int value) {
|
|
+ return this.priority.getAndSet(value);
|
|
+ }
|
|
+ }
|
|
+}
|
|
diff --git a/src/main/java/com/destroystokyo/paper/io/QueueExecutorThread.java b/src/main/java/com/destroystokyo/paper/io/QueueExecutorThread.java
|
|
new file mode 100644
|
|
index 000000000..ee906b594
|
|
--- /dev/null
|
|
+++ b/src/main/java/com/destroystokyo/paper/io/QueueExecutorThread.java
|
|
@@ -0,0 +1,241 @@
|
|
+package com.destroystokyo.paper.io;
|
|
+
|
|
+import net.minecraft.server.MinecraftServer;
|
|
+import org.apache.logging.log4j.Logger;
|
|
+
|
|
+import java.util.concurrent.ConcurrentLinkedQueue;
|
|
+import java.util.concurrent.atomic.AtomicBoolean;
|
|
+import java.util.concurrent.locks.LockSupport;
|
|
+
|
|
+public class QueueExecutorThread<T extends PrioritizedTaskQueue.PrioritizedTask & Runnable> extends Thread {
|
|
+
|
|
+ private static final Logger LOGGER = MinecraftServer.LOGGER;
|
|
+
|
|
+ protected final PrioritizedTaskQueue<T> queue;
|
|
+ protected final long spinWaitTime;
|
|
+
|
|
+ protected volatile boolean closed;
|
|
+
|
|
+ protected final AtomicBoolean parked = new AtomicBoolean();
|
|
+
|
|
+ protected volatile ConcurrentLinkedQueue<Thread> flushQueue = new ConcurrentLinkedQueue<>();
|
|
+ protected volatile long flushCycles;
|
|
+
|
|
+ public QueueExecutorThread(final PrioritizedTaskQueue<T> queue) {
|
|
+ this(queue, (int)(1.e6)); // 1.0ms
|
|
+ }
|
|
+
|
|
+ public QueueExecutorThread(final PrioritizedTaskQueue<T> queue, final long spinWaitTime) { // in ms
|
|
+ this.queue = queue;
|
|
+ this.spinWaitTime = spinWaitTime;
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public void run() {
|
|
+ final long spinWaitTime = this.spinWaitTime;
|
|
+ main_loop:
|
|
+ for (;;) {
|
|
+ this.pollTasks(true);
|
|
+
|
|
+ // spinwait
|
|
+
|
|
+ final long start = System.nanoTime();
|
|
+
|
|
+ for (;;) {
|
|
+ // If we are interrpted for any reason, park() will always return immediately. Clear so that we don't needlessly use cpu in such an event.
|
|
+ Thread.interrupted();
|
|
+ LockSupport.parkNanos("Spinwaiting on tasks", 1000L); // 1us
|
|
+
|
|
+ if (this.pollTasks(true)) {
|
|
+ // restart loop, found tasks
|
|
+ continue main_loop;
|
|
+ }
|
|
+
|
|
+ if (this.handleClose()) {
|
|
+ return; // we're done
|
|
+ }
|
|
+
|
|
+ if ((System.nanoTime() - start) >= spinWaitTime) {
|
|
+ break;
|
|
+ }
|
|
+ }
|
|
+
|
|
+ if (this.handleClose()) {
|
|
+ return;
|
|
+ }
|
|
+
|
|
+ this.parked.set(true);
|
|
+
|
|
+ // We need to parse here to avoid a race condition where a thread queues a task before we set parked to true
|
|
+ // (i.e it will not notify us)
|
|
+ if (this.pollTasks(true)) {
|
|
+ this.parked.set(false);
|
|
+ continue;
|
|
+ }
|
|
+
|
|
+ if (this.handleClose()) {
|
|
+ return;
|
|
+ }
|
|
+
|
|
+ // we don't need to check parked before sleeping, but we do need to check parked in a do-while loop
|
|
+ // LockSupport.park() can fail for any reason
|
|
+ do {
|
|
+ Thread.interrupted();
|
|
+ LockSupport.park("Waiting on tasks");
|
|
+ } while (this.parked.get());
|
|
+ }
|
|
+ }
|
|
+
|
|
+ protected boolean handleClose() {
|
|
+ if (this.closed) {
|
|
+ this.pollTasks(true); // this ensures we've emptied the queue
|
|
+ this.handleFlushThreads(true);
|
|
+ return true;
|
|
+ }
|
|
+ return false;
|
|
+ }
|
|
+
|
|
+ protected boolean pollTasks(boolean flushTasks) {
|
|
+ Runnable task;
|
|
+ boolean ret = false;
|
|
+
|
|
+ while ((task = this.queue.poll()) != null) {
|
|
+ ret = true;
|
|
+ try {
|
|
+ task.run();
|
|
+ } catch (final Throwable throwable) {
|
|
+ if (throwable instanceof ThreadDeath) {
|
|
+ throw (ThreadDeath)throwable;
|
|
+ }
|
|
+ LOGGER.fatal("Exception thrown from prioritized runnable task in thread '" + this.getName() + "': " + IOUtil.genericToString(task), throwable);
|
|
+ }
|
|
+ }
|
|
+
|
|
+ if (flushTasks) {
|
|
+ this.handleFlushThreads(false);
|
|
+ }
|
|
+
|
|
+ return ret;
|
|
+ }
|
|
+
|
|
+ protected void handleFlushThreads(final boolean shutdown) {
|
|
+ Thread parking;
|
|
+ ConcurrentLinkedQueue<Thread> flushQueue = this.flushQueue;
|
|
+ do {
|
|
+ ++flushCycles; // may be plain read opaque write
|
|
+ while ((parking = flushQueue.poll()) != null) {
|
|
+ LockSupport.unpark(parking);
|
|
+ }
|
|
+ } while (this.pollTasks(false));
|
|
+
|
|
+ if (shutdown) {
|
|
+ this.flushQueue = null;
|
|
+
|
|
+ // defend against a race condition where a flush thread double-checks right before we set to null
|
|
+ while ((parking = flushQueue.poll()) != null) {
|
|
+ LockSupport.unpark(parking);
|
|
+ }
|
|
+ }
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Notify's this thread that a task has been added to its queue
|
|
+ * @return {@code true} if this thread was waiting for tasks, {@code false} if it is executing tasks
|
|
+ */
|
|
+ public boolean notifyTasks() {
|
|
+ if (this.parked.get() && this.parked.getAndSet(false)) {
|
|
+ LockSupport.unpark(this);
|
|
+ return true;
|
|
+ }
|
|
+ return false;
|
|
+ }
|
|
+
|
|
+ protected void queueTask(final T task) {
|
|
+ this.queue.add(task);
|
|
+ this.notifyTasks();
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Waits until this thread's queue is empty.
|
|
+ *
|
|
+ * @throws IllegalStateException If the current thread is {@code this} thread.
|
|
+ */
|
|
+ public void flush() {
|
|
+ final Thread currentThread = Thread.currentThread();
|
|
+
|
|
+ if (currentThread == this) {
|
|
+ // avoid deadlock
|
|
+ throw new IllegalStateException("Cannot flush the queue executor thread while on the queue executor thread");
|
|
+ }
|
|
+
|
|
+ // order is important
|
|
+
|
|
+ int successes = 0;
|
|
+ long lastCycle = -1L;
|
|
+
|
|
+ do {
|
|
+ final ConcurrentLinkedQueue<Thread> flushQueue = this.flushQueue;
|
|
+ if (flushQueue == null) {
|
|
+ return;
|
|
+ }
|
|
+
|
|
+ flushQueue.add(currentThread);
|
|
+
|
|
+ // double check flush queue
|
|
+ if (this.flushQueue == null) {
|
|
+ return;
|
|
+ }
|
|
+
|
|
+ final long currentCycle = this.flushCycles; // may be opaque read
|
|
+
|
|
+ if (currentCycle == lastCycle) {
|
|
+ Thread.yield();
|
|
+ continue;
|
|
+ }
|
|
+
|
|
+ // force response
|
|
+ this.parked.set(false);
|
|
+ LockSupport.unpark(this);
|
|
+
|
|
+ LockSupport.park("flushing queue executor thread");
|
|
+
|
|
+ // returns whether there are tasks queued, does not return whether there are tasks executing
|
|
+ // this is why we cycle twice twice through flush (we know a pollTask call is made after a flush cycle)
|
|
+ // we really only need to guarantee that the tasks this thread has queued has gone through, and can leave
|
|
+ // tasks queued concurrently that are unsychronized with this thread as undefined behavior
|
|
+ if (this.queue.hasTasks()) {
|
|
+ successes = 0;
|
|
+ } else {
|
|
+ ++successes;
|
|
+ }
|
|
+
|
|
+ } while (successes != 2);
|
|
+
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Closes this queue executor's queue and optionally waits for it to empty.
|
|
+ * <p>
|
|
+ * If wait is {@code true}, then the queue will be empty by the time this call completes.
|
|
+ * </p>
|
|
+ * <p>
|
|
+ * This function is MT-Safe.
|
|
+ * </p>
|
|
+ * @param wait If this call is to wait until the queue is empty
|
|
+ * @param killQueue Whether to shutdown this thread's queue
|
|
+ * @return whether this thread shut down the queue
|
|
+ */
|
|
+ public boolean close(final boolean wait, final boolean killQueue) {
|
|
+ boolean ret = !killQueue ? false : this.queue.shutdown();
|
|
+ this.closed = true;
|
|
+
|
|
+ // force thread to respond to the shutdown
|
|
+ this.parked.set(false);
|
|
+ LockSupport.unpark(this);
|
|
+
|
|
+ if (wait) {
|
|
+ this.flush();
|
|
+ }
|
|
+ return ret;
|
|
+ }
|
|
+}
|
|
diff --git a/src/main/java/com/destroystokyo/paper/io/chunk/ChunkLoadTask.java b/src/main/java/com/destroystokyo/paper/io/chunk/ChunkLoadTask.java
|
|
new file mode 100644
|
|
index 000000000..305da4786
|
|
--- /dev/null
|
|
+++ b/src/main/java/com/destroystokyo/paper/io/chunk/ChunkLoadTask.java
|
|
@@ -0,0 +1,149 @@
|
|
+package com.destroystokyo.paper.io.chunk;
|
|
+
|
|
+import co.aikar.timings.Timing;
|
|
+import com.destroystokyo.paper.io.PaperFileIOThread;
|
|
+import com.destroystokyo.paper.io.IOUtil;
|
|
+import net.minecraft.server.ChunkCoordIntPair;
|
|
+import net.minecraft.server.ChunkRegionLoader;
|
|
+import net.minecraft.server.PlayerChunkMap;
|
|
+import net.minecraft.server.WorldServer;
|
|
+
|
|
+import java.util.ArrayDeque;
|
|
+import java.util.function.Consumer;
|
|
+
|
|
+public final class ChunkLoadTask extends ChunkTask {
|
|
+
|
|
+ public boolean cancelled;
|
|
+
|
|
+ Consumer<ChunkRegionLoader.InProgressChunkHolder> onComplete;
|
|
+ public PaperFileIOThread.ChunkData chunkData;
|
|
+
|
|
+ private boolean hasCompleted;
|
|
+
|
|
+ public ChunkLoadTask(final WorldServer world, final int chunkX, final int chunkZ, final int priority,
|
|
+ final ChunkTaskManager taskManager,
|
|
+ final Consumer<ChunkRegionLoader.InProgressChunkHolder> onComplete) {
|
|
+ super(world, chunkX, chunkZ, priority, taskManager);
|
|
+ this.onComplete = onComplete;
|
|
+ }
|
|
+
|
|
+ private static final ArrayDeque<Runnable> EMPTY_QUEUE = new ArrayDeque<>();
|
|
+
|
|
+ private static ChunkRegionLoader.InProgressChunkHolder createEmptyHolder() {
|
|
+ return new ChunkRegionLoader.InProgressChunkHolder(null, EMPTY_QUEUE);
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public void run() {
|
|
+ try {
|
|
+ this.executeTask();
|
|
+ } catch (final Throwable ex) {
|
|
+ PaperFileIOThread.LOGGER.error("Failed to execute chunk load task: " + this.toString(), ex);
|
|
+ if (!this.hasCompleted) {
|
|
+ this.complete(ChunkLoadTask.createEmptyHolder());
|
|
+ }
|
|
+ }
|
|
+ }
|
|
+
|
|
+ private boolean checkCancelled() {
|
|
+ if (this.cancelled) {
|
|
+ // IntelliJ does not understand writes may occur to cancelled concurrently.
|
|
+ return this.taskManager.chunkLoadTasks.compute(Long.valueOf(IOUtil.getCoordinateKey(this.chunkX, this.chunkZ)), (final Long keyInMap, final ChunkLoadTask valueInMap) -> {
|
|
+ if (valueInMap != ChunkLoadTask.this) {
|
|
+ throw new IllegalStateException("Expected this task to be scheduled, but another was! Other: " + valueInMap + ", current: " + ChunkLoadTask.this);
|
|
+ }
|
|
+
|
|
+ if (valueInMap.cancelled) {
|
|
+ return null;
|
|
+ }
|
|
+ return valueInMap;
|
|
+ }) == null;
|
|
+ }
|
|
+ return false;
|
|
+ }
|
|
+
|
|
+ public void executeTask() {
|
|
+ if (this.checkCancelled()) {
|
|
+ return;
|
|
+ }
|
|
+
|
|
+ // either executed synchronously or asynchronously
|
|
+ final PaperFileIOThread.ChunkData chunkData = this.chunkData;
|
|
+
|
|
+ if (chunkData.poiData == PaperFileIOThread.FAILURE_VALUE || chunkData.chunkData == PaperFileIOThread.FAILURE_VALUE) {
|
|
+ PaperFileIOThread.LOGGER.error("Could not load chunk for task: " + this.toString() + ", file IO thread has dumped the relevant exception above");
|
|
+ this.complete(ChunkLoadTask.createEmptyHolder());
|
|
+ return;
|
|
+ }
|
|
+
|
|
+ if (chunkData.chunkData == null) {
|
|
+ // not on disk
|
|
+ this.complete(ChunkLoadTask.createEmptyHolder());
|
|
+ return;
|
|
+ }
|
|
+
|
|
+ final ChunkCoordIntPair chunkPos = new ChunkCoordIntPair(this.chunkX, this.chunkZ);
|
|
+
|
|
+ final PlayerChunkMap chunkManager = this.world.getChunkProvider().playerChunkMap;
|
|
+
|
|
+ try (Timing ignored = this.world.timings.chunkIOStage1.startTimingIfSync()) {
|
|
+ final ChunkRegionLoader.InProgressChunkHolder chunkHolder;
|
|
+
|
|
+ // apply fixes
|
|
+
|
|
+ try {
|
|
+ if (chunkData.poiData != null) {
|
|
+ chunkData.poiData = chunkData.poiData.clone(); // clone data for safety, file IO thread does not clone
|
|
+ }
|
|
+ chunkData.chunkData = chunkManager.getChunkData(this.world.getWorldProvider().getDimensionManager(),
|
|
+ chunkManager.getWorldPersistentDataSupplier(), chunkData.chunkData.clone(), chunkPos, this.world); // clone data for safety, file IO thread does not clone
|
|
+ } catch (final Throwable ex) {
|
|
+ PaperFileIOThread.LOGGER.error("Could not apply datafixers for chunk task: " + this.toString(), ex);
|
|
+ this.complete(ChunkLoadTask.createEmptyHolder());
|
|
+ }
|
|
+
|
|
+ if (this.checkCancelled()) {
|
|
+ return;
|
|
+ }
|
|
+
|
|
+ try {
|
|
+ this.world.getChunkProvider().playerChunkMap.updateChunkStatusOnDisk(chunkPos, chunkData.chunkData);
|
|
+ } catch (final Throwable ex) {
|
|
+ PaperFileIOThread.LOGGER.warn("Failed to update chunk status cache for task: " + this.toString(), ex);
|
|
+ // non-fatal, continue
|
|
+ }
|
|
+
|
|
+ try {
|
|
+ chunkHolder = ChunkRegionLoader.loadChunk(this.world,
|
|
+ chunkManager.definedStructureManager, chunkManager.getVillagePlace(), chunkPos,
|
|
+ chunkData.chunkData, true);
|
|
+ } catch (final Throwable ex) {
|
|
+ PaperFileIOThread.LOGGER.error("Could not de-serialize chunk data for task: " + this.toString(), ex);
|
|
+ this.complete(ChunkLoadTask.createEmptyHolder());
|
|
+ return;
|
|
+ }
|
|
+
|
|
+ this.complete(chunkHolder);
|
|
+ }
|
|
+ }
|
|
+
|
|
+ private void complete(final ChunkRegionLoader.InProgressChunkHolder holder) {
|
|
+ this.hasCompleted = true;
|
|
+ holder.poiData = this.chunkData == null ? null : this.chunkData.poiData;
|
|
+
|
|
+ this.taskManager.chunkLoadTasks.compute(Long.valueOf(IOUtil.getCoordinateKey(this.chunkX, this.chunkZ)), (final Long keyInMap, final ChunkLoadTask valueInMap) -> {
|
|
+ if (valueInMap != ChunkLoadTask.this) {
|
|
+ throw new IllegalStateException("Expected this task to be scheduled, but another was! Other: " + valueInMap + ", current: " + ChunkLoadTask.this);
|
|
+ }
|
|
+ if (valueInMap.cancelled) {
|
|
+ return null;
|
|
+ }
|
|
+ try {
|
|
+ ChunkLoadTask.this.onComplete.accept(holder);
|
|
+ } catch (final Throwable thr) {
|
|
+ PaperFileIOThread.LOGGER.error("Failed to complete chunk data for task: " + this.toString(), thr);
|
|
+ }
|
|
+ return null;
|
|
+ });
|
|
+ }
|
|
+}
|
|
diff --git a/src/main/java/com/destroystokyo/paper/io/chunk/ChunkSaveTask.java b/src/main/java/com/destroystokyo/paper/io/chunk/ChunkSaveTask.java
|
|
new file mode 100644
|
|
index 000000000..60312b85f
|
|
--- /dev/null
|
|
+++ b/src/main/java/com/destroystokyo/paper/io/chunk/ChunkSaveTask.java
|
|
@@ -0,0 +1,112 @@
|
|
+package com.destroystokyo.paper.io.chunk;
|
|
+
|
|
+import co.aikar.timings.Timing;
|
|
+import com.destroystokyo.paper.io.PaperFileIOThread;
|
|
+import com.destroystokyo.paper.io.IOUtil;
|
|
+import com.destroystokyo.paper.io.PrioritizedTaskQueue;
|
|
+import net.minecraft.server.ChunkRegionLoader;
|
|
+import net.minecraft.server.IAsyncTaskHandler;
|
|
+import net.minecraft.server.IChunkAccess;
|
|
+import net.minecraft.server.NBTTagCompound;
|
|
+import net.minecraft.server.WorldServer;
|
|
+
|
|
+import java.util.concurrent.CompletableFuture;
|
|
+import java.util.concurrent.atomic.AtomicInteger;
|
|
+
|
|
+public final class ChunkSaveTask extends ChunkTask {
|
|
+
|
|
+ public final ChunkRegionLoader.AsyncSaveData asyncSaveData;
|
|
+ public final IChunkAccess chunk;
|
|
+ public final CompletableFuture<NBTTagCompound> onComplete = new CompletableFuture<>();
|
|
+
|
|
+ private final AtomicInteger attemptedPriority;
|
|
+
|
|
+ public ChunkSaveTask(final WorldServer world, final int chunkX, final int chunkZ, final int priority,
|
|
+ final ChunkTaskManager taskManager, final ChunkRegionLoader.AsyncSaveData asyncSaveData,
|
|
+ final IChunkAccess chunk) {
|
|
+ super(world, chunkX, chunkZ, priority, taskManager);
|
|
+ this.chunk = chunk;
|
|
+ this.asyncSaveData = asyncSaveData;
|
|
+ this.attemptedPriority = new AtomicInteger(priority);
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public void run() {
|
|
+ // can be executed asynchronously or synchronously
|
|
+ final NBTTagCompound compound;
|
|
+
|
|
+ try (Timing ignored = this.world.timings.chunkUnloadDataSave.startTimingIfSync()) {
|
|
+ compound = ChunkRegionLoader.saveChunk(this.world, this.chunk, this.asyncSaveData);
|
|
+ } catch (final Throwable ex) {
|
|
+ // has a plugin modified something it should not have and made us CME?
|
|
+ PaperFileIOThread.LOGGER.error("Failed to serialize unloading chunk data for task: " + this.toString() + ", falling back to a synchronous execution", ex);
|
|
+
|
|
+ // Note: We add to the server thread queue here since this is what the server will drain tasks from
|
|
+ // when waiting for chunks
|
|
+ ChunkTaskManager.queueChunkWaitTask(() -> {
|
|
+ try (Timing ignored = this.world.timings.chunkUnloadDataSave.startTiming()) {
|
|
+ NBTTagCompound data = PaperFileIOThread.FAILURE_VALUE;
|
|
+
|
|
+ try {
|
|
+ data = ChunkRegionLoader.saveChunk(this.world, this.chunk, this.asyncSaveData);
|
|
+ PaperFileIOThread.LOGGER.info("Successfully serialized chunk data for task: " + this.toString() + " synchronously");
|
|
+ } catch (final Throwable ex1) {
|
|
+ PaperFileIOThread.LOGGER.fatal("Failed to synchronously serialize unloading chunk data for task: " + this.toString() + "! Chunk data will be lost", ex1);
|
|
+ }
|
|
+
|
|
+ ChunkSaveTask.this.complete(data);
|
|
+ }
|
|
+ });
|
|
+
|
|
+ return; // the main thread will now complete the data
|
|
+ }
|
|
+
|
|
+ this.complete(compound);
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public boolean raisePriority(final int priority) {
|
|
+ if (!PrioritizedTaskQueue.validPriority(priority)) {
|
|
+ throw new IllegalStateException("Invalid priority: " + priority);
|
|
+ }
|
|
+
|
|
+ // we know priority is valid here
|
|
+ for (int curr = this.attemptedPriority.get();;) {
|
|
+ if (curr <= priority) {
|
|
+ break; // curr is higher/same priority
|
|
+ }
|
|
+ if (this.attemptedPriority.compareAndSet(curr, priority)) {
|
|
+ break;
|
|
+ }
|
|
+ curr = this.attemptedPriority.get();
|
|
+ }
|
|
+
|
|
+ return super.raisePriority(priority);
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public boolean updatePriority(final int priority) {
|
|
+ if (!PrioritizedTaskQueue.validPriority(priority)) {
|
|
+ throw new IllegalStateException("Invalid priority: " + priority);
|
|
+ }
|
|
+ this.attemptedPriority.set(priority);
|
|
+ return super.updatePriority(priority);
|
|
+ }
|
|
+
|
|
+ private void complete(final NBTTagCompound compound) {
|
|
+ try {
|
|
+ this.onComplete.complete(compound);
|
|
+ } catch (final Throwable thr) {
|
|
+ PaperFileIOThread.LOGGER.error("Failed to complete chunk data for task: " + this.toString(), thr);
|
|
+ }
|
|
+ if (compound != PaperFileIOThread.FAILURE_VALUE) {
|
|
+ PaperFileIOThread.Holder.INSTANCE.scheduleSave(this.world, this.chunkX, this.chunkZ, null, compound, this.attemptedPriority.get());
|
|
+ }
|
|
+ this.taskManager.chunkSaveTasks.compute(Long.valueOf(IOUtil.getCoordinateKey(this.chunkX, this.chunkZ)), (final Long keyInMap, final ChunkSaveTask valueInMap) -> {
|
|
+ if (valueInMap != ChunkSaveTask.this) {
|
|
+ throw new IllegalStateException("Expected this task to be scheduled, but another was! Other: " + valueInMap + ", this: " + ChunkSaveTask.this);
|
|
+ }
|
|
+ return null;
|
|
+ });
|
|
+ }
|
|
+}
|
|
diff --git a/src/main/java/com/destroystokyo/paper/io/chunk/ChunkTask.java b/src/main/java/com/destroystokyo/paper/io/chunk/ChunkTask.java
|
|
new file mode 100644
|
|
index 000000000..1dfa8abfd
|
|
--- /dev/null
|
|
+++ b/src/main/java/com/destroystokyo/paper/io/chunk/ChunkTask.java
|
|
@@ -0,0 +1,40 @@
|
|
+package com.destroystokyo.paper.io.chunk;
|
|
+
|
|
+import com.destroystokyo.paper.io.PaperFileIOThread;
|
|
+import com.destroystokyo.paper.io.PrioritizedTaskQueue;
|
|
+import net.minecraft.server.WorldServer;
|
|
+
|
|
+abstract class ChunkTask extends PrioritizedTaskQueue.PrioritizedTask implements Runnable {
|
|
+
|
|
+ public final WorldServer world;
|
|
+ public final int chunkX;
|
|
+ public final int chunkZ;
|
|
+ public final ChunkTaskManager taskManager;
|
|
+
|
|
+ public ChunkTask(final WorldServer world, final int chunkX, final int chunkZ, final int priority,
|
|
+ final ChunkTaskManager taskManager) {
|
|
+ super(priority);
|
|
+ this.world = world;
|
|
+ this.chunkX = chunkX;
|
|
+ this.chunkZ = chunkZ;
|
|
+ this.taskManager = taskManager;
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public String toString() {
|
|
+ return "Chunk task: class:" + this.getClass().getName() + ", for world '" + this.world.getWorld().getName() +
|
|
+ "', (" + this.chunkX + "," + this.chunkZ + "), hashcode:" + this.hashCode() + ", priority: " + this.getPriority();
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public boolean raisePriority(final int priority) {
|
|
+ PaperFileIOThread.Holder.INSTANCE.bumpPriority(this.world, this.chunkX, this.chunkZ, priority);
|
|
+ return super.raisePriority(priority);
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public boolean updatePriority(final int priority) {
|
|
+ PaperFileIOThread.Holder.INSTANCE.setPriority(this.world, this.chunkX, this.chunkZ, priority);
|
|
+ return super.updatePriority(priority);
|
|
+ }
|
|
+}
|
|
diff --git a/src/main/java/com/destroystokyo/paper/io/chunk/ChunkTaskManager.java b/src/main/java/com/destroystokyo/paper/io/chunk/ChunkTaskManager.java
|
|
new file mode 100644
|
|
index 000000000..2b20c159f
|
|
--- /dev/null
|
|
+++ b/src/main/java/com/destroystokyo/paper/io/chunk/ChunkTaskManager.java
|
|
@@ -0,0 +1,492 @@
|
|
+package com.destroystokyo.paper.io.chunk;
|
|
+
|
|
+import com.destroystokyo.paper.io.PaperFileIOThread;
|
|
+import com.destroystokyo.paper.io.IOUtil;
|
|
+import com.destroystokyo.paper.io.PrioritizedTaskQueue;
|
|
+import com.destroystokyo.paper.io.QueueExecutorThread;
|
|
+import net.minecraft.server.ChunkRegionLoader;
|
|
+import net.minecraft.server.IAsyncTaskHandler;
|
|
+import net.minecraft.server.IChunkAccess;
|
|
+import net.minecraft.server.MinecraftServer;
|
|
+import net.minecraft.server.NBTTagCompound;
|
|
+import net.minecraft.server.WorldServer;
|
|
+import org.apache.logging.log4j.Level;
|
|
+import org.bukkit.Bukkit;
|
|
+import org.spigotmc.AsyncCatcher;
|
|
+
|
|
+import java.util.ArrayDeque;
|
|
+import java.util.concurrent.CompletableFuture;
|
|
+import java.util.concurrent.ConcurrentHashMap;
|
|
+import java.util.concurrent.ConcurrentLinkedQueue;
|
|
+import java.util.function.Consumer;
|
|
+
|
|
+public final class ChunkTaskManager {
|
|
+
|
|
+ private final QueueExecutorThread<ChunkTask>[] workers;
|
|
+ private final WorldServer world;
|
|
+
|
|
+ private final PrioritizedTaskQueue<ChunkTask> queue;
|
|
+ private final boolean perWorldQueue;
|
|
+
|
|
+ final ConcurrentHashMap<Long, ChunkLoadTask> chunkLoadTasks = new ConcurrentHashMap<>(64, 0.5f);
|
|
+ final ConcurrentHashMap<Long, ChunkSaveTask> chunkSaveTasks = new ConcurrentHashMap<>(64, 0.5f);
|
|
+
|
|
+ private final PrioritizedTaskQueue<ChunkTask> chunkTasks = new PrioritizedTaskQueue<>(); // used if async chunks are disabled in config
|
|
+
|
|
+ protected static QueueExecutorThread<ChunkTask>[] globalWorkers;
|
|
+ protected static QueueExecutorThread<ChunkTask> globalUrgentWorker;
|
|
+ protected static PrioritizedTaskQueue<ChunkTask> globalQueue;
|
|
+ protected static PrioritizedTaskQueue<ChunkTask> globalUrgentQueue;
|
|
+
|
|
+ protected static final ConcurrentLinkedQueue<Runnable> CHUNK_WAIT_QUEUE = new ConcurrentLinkedQueue<>();
|
|
+
|
|
+ public static final ArrayDeque<ChunkInfo> WAITING_CHUNKS = new ArrayDeque<>(); // stack
|
|
+
|
|
+ private static final class ChunkInfo {
|
|
+
|
|
+ public final int chunkX;
|
|
+ public final int chunkZ;
|
|
+ public final WorldServer world;
|
|
+
|
|
+ public ChunkInfo(final int chunkX, final int chunkZ, final WorldServer world) {
|
|
+ this.chunkX = chunkX;
|
|
+ this.chunkZ = chunkZ;
|
|
+ this.world = world;
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public String toString() {
|
|
+ return "[( " + this.chunkX + "," + this.chunkZ + ") in '" + this.world.getWorld().getName() + "']";
|
|
+ }
|
|
+ }
|
|
+
|
|
+ public static void pushChunkWait(final WorldServer world, final int chunkX, final int chunkZ) {
|
|
+ synchronized (WAITING_CHUNKS) {
|
|
+ WAITING_CHUNKS.push(new ChunkInfo(chunkX, chunkZ, world));
|
|
+ }
|
|
+ }
|
|
+
|
|
+ public static void popChunkWait() {
|
|
+ synchronized (WAITING_CHUNKS) {
|
|
+ WAITING_CHUNKS.pop();
|
|
+ }
|
|
+ }
|
|
+
|
|
+ public static String getChunkWaitInfo() {
|
|
+ synchronized (WAITING_CHUNKS) {
|
|
+ return WAITING_CHUNKS.toString();
|
|
+ }
|
|
+ }
|
|
+
|
|
+ public static void dumpAllChunkLoadInfo() {
|
|
+ synchronized (WAITING_CHUNKS) {
|
|
+ if (WAITING_CHUNKS.isEmpty()) {
|
|
+ return;
|
|
+ }
|
|
+
|
|
+ PaperFileIOThread.LOGGER.log(Level.ERROR, "Chunk wait task info below: ");
|
|
+
|
|
+ for (final ChunkInfo chunkInfo : WAITING_CHUNKS) {
|
|
+ final long key = IOUtil.getCoordinateKey(chunkInfo.chunkX, chunkInfo.chunkZ);
|
|
+ final ChunkLoadTask loadTask = chunkInfo.world.asyncChunkTaskManager.chunkLoadTasks.get(key);
|
|
+ final ChunkSaveTask saveTask = chunkInfo.world.asyncChunkTaskManager.chunkSaveTasks.get(key);
|
|
+
|
|
+ PaperFileIOThread.LOGGER.log(Level.ERROR, chunkInfo.chunkX + "," + chunkInfo.chunkZ + " in '" + chunkInfo.world.getWorld().getName() + ":");
|
|
+ PaperFileIOThread.LOGGER.log(Level.ERROR, "Load Task - " + (loadTask == null ? "none" : loadTask.toString()));
|
|
+ PaperFileIOThread.LOGGER.log(Level.ERROR, "Save Task - " + (saveTask == null ? "none" : saveTask.toString()));
|
|
+ // log current status of chunk to indicate whether we're waiting on generation or loading
|
|
+ net.minecraft.server.PlayerChunk chunkHolder = chunkInfo.world.getChunkProvider().playerChunkMap.getVisibleChunk(key);
|
|
+
|
|
+ if (chunkHolder == null) {
|
|
+ PaperFileIOThread.LOGGER.log(Level.ERROR, "Chunk Holder - null");
|
|
+ } else {
|
|
+ IChunkAccess chunk = chunkHolder.getAvailableChunkNow();
|
|
+ net.minecraft.server.ChunkStatus holderStatus = chunkHolder.getChunkHolderStatus();
|
|
+ PaperFileIOThread.LOGGER.log(Level.ERROR, "Chunk Holder - non-null");
|
|
+ PaperFileIOThread.LOGGER.log(Level.ERROR, "Chunk Status - " + ((chunk == null) ? "null chunk" : chunk.getChunkStatus().toString()));
|
|
+ PaperFileIOThread.LOGGER.log(Level.ERROR, "Chunk Holder Status - " + ((holderStatus == null) ? "null" : holderStatus.toString()));
|
|
+ }
|
|
+
|
|
+ }
|
|
+ }
|
|
+ }
|
|
+
|
|
+ public static void initGlobalLoadThreads(int threads) {
|
|
+ if (threads <= 0 || globalWorkers != null) {
|
|
+ return;
|
|
+ }
|
|
+
|
|
+ globalWorkers = new QueueExecutorThread[threads];
|
|
+ globalQueue = new PrioritizedTaskQueue<>();
|
|
+ globalUrgentQueue = new PrioritizedTaskQueue<>();
|
|
+
|
|
+ for (int i = 0; i < threads; ++i) {
|
|
+ globalWorkers[i] = new QueueExecutorThread<>(globalQueue, (long)0.10e6); //0.1ms
|
|
+ globalWorkers[i].setName("Paper Async Chunk Task Thread #" + i);
|
|
+ globalWorkers[i].setPriority(Thread.NORM_PRIORITY - 1);
|
|
+ globalWorkers[i].setUncaughtExceptionHandler((final Thread thread, final Throwable throwable) -> {
|
|
+ PaperFileIOThread.LOGGER.fatal("Thread '" + thread.getName() + "' threw an uncaught exception!", throwable);
|
|
+ });
|
|
+
|
|
+ globalWorkers[i].start();
|
|
+ }
|
|
+
|
|
+ globalUrgentWorker = new QueueExecutorThread<>(globalUrgentQueue, (long)0.10e6); //0.1ms
|
|
+ globalUrgentWorker.setName("Paper Async Chunk Urgent Task Thread");
|
|
+ globalUrgentWorker.setPriority(Thread.NORM_PRIORITY+1);
|
|
+ globalUrgentWorker.setUncaughtExceptionHandler((final Thread thread, final Throwable throwable) -> {
|
|
+ PaperFileIOThread.LOGGER.fatal("Thread '" + thread.getName() + "' threw an uncaught exception!", throwable);
|
|
+ });
|
|
+
|
|
+ globalUrgentWorker.start();
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Creates this chunk task manager to operate off the specified number of threads. If the specified number of threads is
|
|
+ * less-than or equal to 0, then this chunk task manager will operate off of the world's chunk task queue.
|
|
+ * @param world Specified world.
|
|
+ * @param threads Specified number of threads.
|
|
+ * @see net.minecraft.server.ChunkProviderServer#serverThreadQueue
|
|
+ */
|
|
+ public ChunkTaskManager(final WorldServer world, final int threads) {
|
|
+ this.world = world;
|
|
+ this.workers = threads <= 0 ? null : new QueueExecutorThread[threads];
|
|
+ this.queue = new PrioritizedTaskQueue<>();
|
|
+ this.perWorldQueue = true;
|
|
+
|
|
+ for (int i = 0; i < threads; ++i) {
|
|
+ this.workers[i] = new QueueExecutorThread<>(this.queue, (long)0.10e6); //0.1ms
|
|
+ this.workers[i].setName("Async chunk loader thread #" + i + " for world: " + world.getWorldData().getName());
|
|
+ this.workers[i].setPriority(Thread.NORM_PRIORITY - 1);
|
|
+ this.workers[i].setUncaughtExceptionHandler((final Thread thread, final Throwable throwable) -> {
|
|
+ PaperFileIOThread.LOGGER.fatal("Thread '" + thread.getName() + "' threw an uncaught exception!", throwable);
|
|
+ });
|
|
+
|
|
+ this.workers[i].start();
|
|
+ }
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Creates the chunk task manager to work from the global workers. When {@link #close(boolean)} is invoked,
|
|
+ * the global queue is not shutdown. If the global workers is configured to be disabled or use 0 threads, then
|
|
+ * this chunk task manager will operate off of the world's chunk task queue.
|
|
+ * @param world The world that this task manager is responsible for
|
|
+ * @see net.minecraft.server.ChunkProviderServer#serverThreadQueue
|
|
+ */
|
|
+ public ChunkTaskManager(final WorldServer world) {
|
|
+ this.world = world;
|
|
+ this.workers = globalWorkers;
|
|
+ this.queue = globalQueue;
|
|
+ this.perWorldQueue = false;
|
|
+ }
|
|
+
|
|
+ public boolean pollNextChunkTask() {
|
|
+ final ChunkTask task = this.chunkTasks.poll();
|
|
+
|
|
+ if (task != null) {
|
|
+ task.run();
|
|
+ return true;
|
|
+ }
|
|
+ return false;
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Polls and runs the next available chunk wait queue task. This is to be used when the server is waiting on a chunk queue.
|
|
+ * (per-world can cause issues if all the worker threads are blocked waiting for a response from the main thread)
|
|
+ */
|
|
+ public static boolean pollChunkWaitQueue() {
|
|
+ final Runnable run = CHUNK_WAIT_QUEUE.poll();
|
|
+ if (run != null) {
|
|
+ run.run();
|
|
+ return true;
|
|
+ }
|
|
+ return false;
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Queues a chunk wait task. Note that this will execute out of order with respect to tasks scheduled on a world's
|
|
+ * chunk task queue, since this is the global chunk wait queue.
|
|
+ */
|
|
+ public static void queueChunkWaitTask(final Runnable runnable) {
|
|
+ CHUNK_WAIT_QUEUE.add(runnable);
|
|
+ }
|
|
+
|
|
+ private static void drainChunkWaitQueue() {
|
|
+ Runnable run;
|
|
+ while ((run = CHUNK_WAIT_QUEUE.poll()) != null) {
|
|
+ run.run();
|
|
+ }
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * The exact same as {@link #scheduleChunkLoad(int, int, int, Consumer, boolean)}, except that the chunk data is provided as
|
|
+ * the {@code data} parameter.
|
|
+ */
|
|
+ public ChunkLoadTask scheduleChunkLoad(final int chunkX, final int chunkZ, final int priority,
|
|
+ final Consumer<ChunkRegionLoader.InProgressChunkHolder> onComplete,
|
|
+ final boolean intendingToBlock, final CompletableFuture<NBTTagCompound> dataFuture) {
|
|
+ final WorldServer world = this.world;
|
|
+
|
|
+ return this.chunkLoadTasks.compute(Long.valueOf(IOUtil.getCoordinateKey(chunkX, chunkZ)), (final Long keyInMap, final ChunkLoadTask valueInMap) -> {
|
|
+ if (valueInMap != null) {
|
|
+ if (!valueInMap.cancelled) {
|
|
+ throw new IllegalStateException("Double scheduling chunk load for task: " + valueInMap.toString());
|
|
+ }
|
|
+ valueInMap.cancelled = false;
|
|
+ valueInMap.onComplete = onComplete;
|
|
+ return valueInMap;
|
|
+ }
|
|
+
|
|
+ final ChunkLoadTask ret = new ChunkLoadTask(world, chunkX, chunkZ, priority, ChunkTaskManager.this, onComplete);
|
|
+
|
|
+ dataFuture.thenAccept((final NBTTagCompound data) -> {
|
|
+ final boolean failed = data == PaperFileIOThread.FAILURE_VALUE;
|
|
+ PaperFileIOThread.Holder.INSTANCE.loadChunkDataAsync(world, chunkX, chunkZ, priority, (final PaperFileIOThread.ChunkData chunkData) -> {
|
|
+ ret.chunkData = chunkData;
|
|
+ if (!failed) {
|
|
+ chunkData.chunkData = data;
|
|
+ }
|
|
+ ChunkTaskManager.this.internalSchedule(ret); // only schedule to the worker threads here
|
|
+ }, true, failed, intendingToBlock); // read data off disk if the future fails
|
|
+ });
|
|
+
|
|
+ return ret;
|
|
+ });
|
|
+ }
|
|
+
|
|
+ public void cancelChunkLoad(final int chunkX, final int chunkZ) {
|
|
+ this.chunkLoadTasks.compute(IOUtil.getCoordinateKey(chunkX, chunkZ), (final Long keyInMap, final ChunkLoadTask valueInMap) -> {
|
|
+ if (valueInMap == null) {
|
|
+ return null;
|
|
+ }
|
|
+
|
|
+ if (valueInMap.cancelled) {
|
|
+ PaperFileIOThread.LOGGER.warn("Task " + valueInMap.toString() + " is already cancelled!");
|
|
+ }
|
|
+ valueInMap.cancelled = true;
|
|
+ if (valueInMap.cancel()) {
|
|
+ return null;
|
|
+ }
|
|
+
|
|
+ return valueInMap;
|
|
+ });
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Schedules an asynchronous chunk load for the specified coordinates. The onComplete parameter may be invoked asynchronously
|
|
+ * on a worker thread or on the world's chunk executor queue. As such the code that is executed for the parameter should be
|
|
+ * carefully chosen.
|
|
+ * @param chunkX Chunk's x coordinate
|
|
+ * @param chunkZ Chunk's z coordinate
|
|
+ * @param priority Priority for this task
|
|
+ * @param onComplete The consumer to invoke with the {@link net.minecraft.server.ChunkRegionLoader.InProgressChunkHolder} object once this task is complete
|
|
+ * @param intendingToBlock Whether the caller is intending to block on this task completing (this is a performance tune, and has no adverse side-effects)
|
|
+ * @return The {@link ChunkLoadTask} associated with
|
|
+ */
|
|
+ public ChunkLoadTask scheduleChunkLoad(final int chunkX, final int chunkZ, final int priority,
|
|
+ final Consumer<ChunkRegionLoader.InProgressChunkHolder> onComplete,
|
|
+ final boolean intendingToBlock) {
|
|
+ final WorldServer world = this.world;
|
|
+
|
|
+ return this.chunkLoadTasks.compute(Long.valueOf(IOUtil.getCoordinateKey(chunkX, chunkZ)), (final Long keyInMap, final ChunkLoadTask valueInMap) -> {
|
|
+ if (valueInMap != null) {
|
|
+ if (!valueInMap.cancelled) {
|
|
+ throw new IllegalStateException("Double scheduling chunk load for task: " + valueInMap.toString());
|
|
+ }
|
|
+ valueInMap.cancelled = false;
|
|
+ valueInMap.onComplete = onComplete;
|
|
+ return valueInMap;
|
|
+ }
|
|
+
|
|
+ final ChunkLoadTask ret = new ChunkLoadTask(world, chunkX, chunkZ, priority, ChunkTaskManager.this, onComplete);
|
|
+
|
|
+ PaperFileIOThread.Holder.INSTANCE.loadChunkDataAsync(world, chunkX, chunkZ, priority, (final PaperFileIOThread.ChunkData chunkData) -> {
|
|
+ ret.chunkData = chunkData;
|
|
+ ChunkTaskManager.this.internalSchedule(ret); // only schedule to the worker threads here
|
|
+ }, true, true, intendingToBlock);
|
|
+
|
|
+ return ret;
|
|
+ });
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Schedules an async save for the specified chunk. The chunk, at the beginning of this call, must be completely unloaded
|
|
+ * from the world.
|
|
+ * @param chunkX Chunk's x coordinate
|
|
+ * @param chunkZ Chunk's z coordinate
|
|
+ * @param priority Priority for this task
|
|
+ * @param asyncSaveData Async save data. See {@link ChunkRegionLoader#getAsyncSaveData(WorldServer, IChunkAccess)}
|
|
+ * @param chunk Chunk to save
|
|
+ * @return The {@link ChunkSaveTask} associated with the save task.
|
|
+ */
|
|
+ public ChunkSaveTask scheduleChunkSave(final int chunkX, final int chunkZ, final int priority,
|
|
+ final ChunkRegionLoader.AsyncSaveData asyncSaveData,
|
|
+ final IChunkAccess chunk) {
|
|
+ AsyncCatcher.catchOp("chunk save schedule");
|
|
+
|
|
+ final WorldServer world = this.world;
|
|
+
|
|
+ return this.chunkSaveTasks.compute(Long.valueOf(IOUtil.getCoordinateKey(chunkX, chunkZ)), (final Long keyInMap, final ChunkSaveTask valueInMap) -> {
|
|
+ if (valueInMap != null) {
|
|
+ throw new IllegalStateException("Double scheduling chunk save for task: " + valueInMap.toString());
|
|
+ }
|
|
+
|
|
+ final ChunkSaveTask ret = new ChunkSaveTask(world, chunkX, chunkZ, priority, ChunkTaskManager.this, asyncSaveData, chunk);
|
|
+
|
|
+ ChunkTaskManager.this.internalSchedule(ret);
|
|
+
|
|
+ return ret;
|
|
+ });
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Returns a completable future which will be completed with the <b>un-copied</b> chunk data for an in progress async save.
|
|
+ * Returns {@code null} if no save is in progress.
|
|
+ * @param chunkX Chunk's x coordinate
|
|
+ * @param chunkZ Chunk's z coordinate
|
|
+ */
|
|
+ public CompletableFuture<NBTTagCompound> getChunkSaveFuture(final int chunkX, final int chunkZ) {
|
|
+ final ChunkSaveTask chunkSaveTask = this.chunkSaveTasks.get(Long.valueOf(IOUtil.getCoordinateKey(chunkX, chunkZ)));
|
|
+ if (chunkSaveTask == null) {
|
|
+ return null;
|
|
+ }
|
|
+ return chunkSaveTask.onComplete;
|
|
+ }
|
|
+
|
|
+ /**
|
|
+ * Returns the chunk object being used to serialize data async for an unloaded chunk. Note that modifying this chunk
|
|
+ * is not safe to do as another thread is handling its save. The chunk is also not loaded into the world.
|
|
+ * @param chunkX Chunk's x coordinate
|
|
+ * @param chunkZ Chunk's z coordinate
|
|
+ * @return Chunk object for an in-progress async save, or {@code null} if no save is in progress
|
|
+ */
|
|
+ public IChunkAccess getChunkInSaveProgress(final int chunkX, final int chunkZ) {
|
|
+ final ChunkSaveTask chunkSaveTask = this.chunkSaveTasks.get(Long.valueOf(IOUtil.getCoordinateKey(chunkX, chunkZ)));
|
|
+ if (chunkSaveTask == null) {
|
|
+ return null;
|
|
+ }
|
|
+ return chunkSaveTask.chunk;
|
|
+ }
|
|
+
|
|
+ public void flush() {
|
|
+ // flush here since we schedule tasks on the IO thread that can schedule tasks here
|
|
+ drainChunkWaitQueue();
|
|
+ PaperFileIOThread.Holder.INSTANCE.flush();
|
|
+ drainChunkWaitQueue();
|
|
+
|
|
+ if (this.workers == null) {
|
|
+ if (Bukkit.isPrimaryThread() || MinecraftServer.getServer().hasStopped()) {
|
|
+ ((IAsyncTaskHandler<Runnable>)this.world.getChunkProvider().serverThreadQueue).executeAll();
|
|
+ } else {
|
|
+ CompletableFuture<Void> wait = new CompletableFuture<>();
|
|
+ MinecraftServer.getServer().scheduleOnMain(() -> {
|
|
+ ((IAsyncTaskHandler<Runnable>)this.world.getChunkProvider().serverThreadQueue).executeAll();
|
|
+ });
|
|
+ wait.join();
|
|
+ }
|
|
+ } else {
|
|
+ for (final QueueExecutorThread<ChunkTask> worker : this.workers) {
|
|
+ worker.flush();
|
|
+ }
|
|
+ }
|
|
+ if (globalUrgentWorker != null) globalUrgentWorker.flush();
|
|
+
|
|
+ // flush again since tasks we execute async saves
|
|
+ drainChunkWaitQueue();
|
|
+ PaperFileIOThread.Holder.INSTANCE.flush();
|
|
+ }
|
|
+
|
|
+ public void close(final boolean wait) {
|
|
+ // flush here since we schedule tasks on the IO thread that can schedule tasks to this task manager
|
|
+ // we do this regardless of the wait param since after we invoke close no tasks can be queued
|
|
+ PaperFileIOThread.Holder.INSTANCE.flush();
|
|
+
|
|
+ if (this.workers == null) {
|
|
+ if (wait) {
|
|
+ this.flush();
|
|
+ }
|
|
+ return;
|
|
+ }
|
|
+
|
|
+ if (this.workers != globalWorkers) {
|
|
+ for (final QueueExecutorThread<ChunkTask> worker : this.workers) {
|
|
+ worker.close(false, this.perWorldQueue);
|
|
+ }
|
|
+ }
|
|
+
|
|
+ if (wait) {
|
|
+ this.flush();
|
|
+ }
|
|
+ }
|
|
+
|
|
+ public void raisePriority(final int chunkX, final int chunkZ, final int priority) {
|
|
+ final Long chunkKey = Long.valueOf(IOUtil.getCoordinateKey(chunkX, chunkZ));
|
|
+
|
|
+ ChunkTask chunkSaveTask = this.chunkSaveTasks.get(chunkKey);
|
|
+ if (chunkSaveTask != null) {
|
|
+ // don't bump save into urgent queue
|
|
+ raiseTaskPriority(chunkSaveTask, priority != PrioritizedTaskQueue.HIGHEST_PRIORITY ? priority : PrioritizedTaskQueue.HIGH_PRIORITY);
|
|
+ }
|
|
+
|
|
+ ChunkLoadTask chunkLoadTask = this.chunkLoadTasks.get(chunkKey);
|
|
+ if (chunkLoadTask != null) {
|
|
+ raiseTaskPriority(chunkLoadTask, priority);
|
|
+ }
|
|
+ }
|
|
+
|
|
+ private void raiseTaskPriority(ChunkTask task, int priority) {
|
|
+ final boolean raised = task.raisePriority(priority);
|
|
+ if (task.isScheduled() && raised && this.workers != null) {
|
|
+ // only notify if we're in queue to be executed
|
|
+ if (priority == PrioritizedTaskQueue.HIGHEST_PRIORITY) {
|
|
+ // was in another queue but became urgent later, add to urgent queue and the previous
|
|
+ // queue will just have to ignore this task if it has already been started.
|
|
+ // Ultimately, we now have 2 potential queues that can pull it out whoever gets it first
|
|
+ // but the urgent queue has dedicated thread(s) so it's likely to win....
|
|
+ globalUrgentQueue.add(task);
|
|
+ this.internalScheduleNotifyUrgent();
|
|
+ } else {
|
|
+ this.internalScheduleNotify();
|
|
+ }
|
|
+ }
|
|
+ }
|
|
+
|
|
+ protected void internalSchedule(final ChunkTask task) {
|
|
+ if (this.workers == null) {
|
|
+ this.chunkTasks.add(task);
|
|
+ return;
|
|
+ }
|
|
+
|
|
+ // It's important we order the task to be executed before notifying. Avoid a race condition where the worker thread
|
|
+ // wakes up and goes to sleep before we actually schedule (or it's just about to sleep)
|
|
+ if (task.getPriority() == PrioritizedTaskQueue.HIGHEST_PRIORITY) {
|
|
+ globalUrgentQueue.add(task);
|
|
+ this.internalScheduleNotifyUrgent();
|
|
+ } else {
|
|
+ this.queue.add(task);
|
|
+ this.internalScheduleNotify();
|
|
+ }
|
|
+
|
|
+ }
|
|
+
|
|
+ protected void internalScheduleNotify() {
|
|
+ if (this.workers == null) {
|
|
+ return;
|
|
+ }
|
|
+ for (final QueueExecutorThread<ChunkTask> worker : this.workers) {
|
|
+ if (worker.notifyTasks()) {
|
|
+ // break here since we only want to wake up one worker for scheduling one task
|
|
+ break;
|
|
+ }
|
|
+ }
|
|
+ }
|
|
+
|
|
+
|
|
+ protected void internalScheduleNotifyUrgent() {
|
|
+ if (globalUrgentWorker == null) {
|
|
+ return;
|
|
+ }
|
|
+ globalUrgentWorker.notifyTasks();
|
|
+ }
|
|
+
|
|
+}
|
|
diff --git a/src/main/java/net/minecraft/server/ChunkProviderServer.java b/src/main/java/net/minecraft/server/ChunkProviderServer.java
|
|
index b582171c5..03d7ce829 100644
|
|
--- a/src/main/java/net/minecraft/server/ChunkProviderServer.java
|
|
+++ b/src/main/java/net/minecraft/server/ChunkProviderServer.java
|
|
@@ -218,11 +218,137 @@ public class ChunkProviderServer extends IChunkProvider {
|
|
return playerChunk.getAvailableChunkNow();
|
|
|
|
}
|
|
+
|
|
+ private long asyncLoadSeqCounter;
|
|
+
|
|
+ public void getChunkAtAsynchronously(int x, int z, boolean gen, java.util.function.Consumer<Chunk> onComplete) {
|
|
+ if (Thread.currentThread() != this.serverThread) {
|
|
+ this.serverThreadQueue.execute(() -> {
|
|
+ this.getChunkAtAsynchronously(x, z, gen, onComplete);
|
|
+ });
|
|
+ return;
|
|
+ }
|
|
+
|
|
+ long k = ChunkCoordIntPair.pair(x, z);
|
|
+ ChunkCoordIntPair chunkPos = new ChunkCoordIntPair(x, z);
|
|
+
|
|
+ IChunkAccess ichunkaccess;
|
|
+
|
|
+ // try cache
|
|
+ for (int l = 0; l < 4; ++l) {
|
|
+ if (k == this.cachePos[l] && ChunkStatus.FULL == this.cacheStatus[l]) {
|
|
+ ichunkaccess = this.cacheChunk[l];
|
|
+ if (ichunkaccess != null) { // CraftBukkit - the chunk can become accessible in the meantime TODO for non-null chunks it might also make sense to check that the chunk's state hasn't changed in the meantime
|
|
+
|
|
+ // move to first in cache
|
|
+
|
|
+ for (int i1 = 3; i1 > 0; --i1) {
|
|
+ this.cachePos[i1] = this.cachePos[i1 - 1];
|
|
+ this.cacheStatus[i1] = this.cacheStatus[i1 - 1];
|
|
+ this.cacheChunk[i1] = this.cacheChunk[i1 - 1];
|
|
+ }
|
|
+
|
|
+ this.cachePos[0] = k;
|
|
+ this.cacheStatus[0] = ChunkStatus.FULL;
|
|
+ this.cacheChunk[0] = ichunkaccess;
|
|
+
|
|
+ onComplete.accept((Chunk)ichunkaccess);
|
|
+
|
|
+ return;
|
|
+ }
|
|
+ }
|
|
+ }
|
|
+
|
|
+ if (gen) {
|
|
+ this.bringToFullStatusAsync(x, z, chunkPos, onComplete);
|
|
+ return;
|
|
+ }
|
|
+
|
|
+ IChunkAccess current = this.getChunkAtImmediately(x, z); // we want to bypass ticket restrictions
|
|
+ if (current != null) {
|
|
+ if (!(current instanceof ProtoChunkExtension) && !(current instanceof net.minecraft.server.Chunk)) {
|
|
+ onComplete.accept(null); // the chunk is not gen'd
|
|
+ return;
|
|
+ }
|
|
+ // we know the chunk is at full status here (either in read-only mode or the real thing)
|
|
+ this.bringToFullStatusAsync(x, z, chunkPos, onComplete);
|
|
+ return;
|
|
+ }
|
|
+
|
|
+ ChunkStatus status = world.getChunkProvider().playerChunkMap.getStatusOnDiskNoLoad(x, z);
|
|
+
|
|
+ if (status != null && status != ChunkStatus.FULL) {
|
|
+ // does not exist on disk
|
|
+ onComplete.accept(null);
|
|
+ return;
|
|
+ }
|
|
+
|
|
+ if (status == ChunkStatus.FULL) {
|
|
+ this.bringToFullStatusAsync(x, z, chunkPos, onComplete);
|
|
+ return;
|
|
+ }
|
|
+
|
|
+ // status is null here
|
|
+
|
|
+ // here we don't know what status it is and we're not supposed to generate
|
|
+ // so we asynchronously load empty status
|
|
+
|
|
+ this.bringToStatusAsync(x, z, chunkPos, ChunkStatus.EMPTY, (IChunkAccess chunk) -> {
|
|
+ if (!(chunk instanceof ProtoChunkExtension) && !(chunk instanceof net.minecraft.server.Chunk)) {
|
|
+ // the chunk on disk was not a full status chunk
|
|
+ onComplete.accept(null);
|
|
+ return;
|
|
+ }
|
|
+ this.bringToFullStatusAsync(x, z, chunkPos, onComplete); // bring to full status if required
|
|
+ });
|
|
+ }
|
|
+
|
|
+ private void bringToFullStatusAsync(int x, int z, ChunkCoordIntPair chunkPos, java.util.function.Consumer<Chunk> onComplete) {
|
|
+ this.bringToStatusAsync(x, z, chunkPos, ChunkStatus.FULL, (java.util.function.Consumer)onComplete);
|
|
+ }
|
|
+
|
|
+ private void bringToStatusAsync(int x, int z, ChunkCoordIntPair chunkPos, ChunkStatus status, java.util.function.Consumer<IChunkAccess> onComplete) {
|
|
+ CompletableFuture<Either<IChunkAccess, PlayerChunk.Failure>> future = this.getChunkFutureMainThread(x, z, status, true);
|
|
+ Long identifier = Long.valueOf(this.asyncLoadSeqCounter++);
|
|
+ int ticketLevel = MCUtil.getTicketLevelFor(status);
|
|
+ this.addTicketAtLevel(TicketType.ASYNC_LOAD, chunkPos, ticketLevel, identifier);
|
|
+
|
|
+ future.whenCompleteAsync((Either<IChunkAccess, PlayerChunk.Failure> either, Throwable throwable) -> {
|
|
+ // either left -> success
|
|
+ // either right -> failure
|
|
+
|
|
+ if (throwable != null) {
|
|
+ throw new RuntimeException(throwable);
|
|
+ }
|
|
+
|
|
+ this.removeTicketAtLevel(TicketType.ASYNC_LOAD, chunkPos, ticketLevel, identifier);
|
|
+ this.addTicketAtLevel(TicketType.UNKNOWN, chunkPos, ticketLevel, chunkPos); // allow unloading
|
|
+
|
|
+ Optional<PlayerChunk.Failure> failure = either.right();
|
|
+
|
|
+ if (failure.isPresent()) {
|
|
+ // failure
|
|
+ throw new IllegalStateException("Chunk failed to load: " + failure.get().toString());
|
|
+ }
|
|
+
|
|
+ onComplete.accept(either.left().get());
|
|
+
|
|
+ }, this.serverThreadQueue);
|
|
+ }
|
|
+
|
|
+ public <T> void addTicketAtLevel(TicketType<T> ticketType, ChunkCoordIntPair chunkPos, int ticketLevel, T identifier) {
|
|
+ this.chunkMapDistance.addTicketAtLevel(ticketType, chunkPos, ticketLevel, identifier);
|
|
+ }
|
|
+
|
|
+ public <T> void removeTicketAtLevel(TicketType<T> ticketType, ChunkCoordIntPair chunkPos, int ticketLevel, T identifier) {
|
|
+ this.chunkMapDistance.removeTicketAtLevel(ticketType, chunkPos, ticketLevel, identifier);
|
|
+ }
|
|
// Paper end
|
|
|
|
@Nullable
|
|
@Override
|
|
public IChunkAccess getChunkAt(int i, int j, ChunkStatus chunkstatus, boolean flag) {
|
|
+ final int x = i; final int z = j; // Paper - conflict on variable change
|
|
if (Thread.currentThread() != this.serverThread) {
|
|
return (IChunkAccess) CompletableFuture.supplyAsync(() -> {
|
|
return this.getChunkAt(i, j, chunkstatus, flag);
|
|
@@ -248,8 +374,13 @@ public class ChunkProviderServer extends IChunkProvider {
|
|
CompletableFuture<Either<IChunkAccess, PlayerChunk.Failure>> completablefuture = this.getChunkFutureMainThread(i, j, chunkstatus, flag);
|
|
|
|
if (!completablefuture.isDone()) { // Paper
|
|
+ // Paper start - async chunk io/loading
|
|
+ this.world.asyncChunkTaskManager.raisePriority(x, z, com.destroystokyo.paper.io.PrioritizedTaskQueue.HIGHEST_PRIORITY);
|
|
+ com.destroystokyo.paper.io.chunk.ChunkTaskManager.pushChunkWait(this.world, x, z);
|
|
+ // Paper end
|
|
this.world.timings.chunkAwait.startTiming(); // Paper
|
|
this.serverThreadQueue.awaitTasks(completablefuture::isDone);
|
|
+ com.destroystokyo.paper.io.chunk.ChunkTaskManager.popChunkWait(); // Paper - async chunk debug
|
|
this.world.timings.chunkAwait.stopTiming(); // Paper
|
|
} // Paper
|
|
ichunkaccess = (IChunkAccess) ((Either) completablefuture.join()).map((ichunkaccess1) -> {
|
|
@@ -754,11 +885,12 @@ public class ChunkProviderServer extends IChunkProvider {
|
|
protected boolean executeNext() {
|
|
// CraftBukkit start - process pending Chunk loadCallback() and unloadCallback() after each run task
|
|
try {
|
|
+ boolean execChunkTask = com.destroystokyo.paper.io.chunk.ChunkTaskManager.pollChunkWaitQueue() || ChunkProviderServer.this.world.asyncChunkTaskManager.pollNextChunkTask(); // Paper
|
|
if (ChunkProviderServer.this.tickDistanceManager()) {
|
|
return true;
|
|
} else {
|
|
ChunkProviderServer.this.lightEngine.queueUpdate();
|
|
- return super.executeNext();
|
|
+ return super.executeNext() || execChunkTask; // Paper
|
|
}
|
|
} finally {
|
|
playerChunkMap.callbackExecutor.run();
|
|
diff --git a/src/main/java/net/minecraft/server/ChunkRegionLoader.java b/src/main/java/net/minecraft/server/ChunkRegionLoader.java
|
|
index 8816c90e2..1298a07dc 100644
|
|
--- a/src/main/java/net/minecraft/server/ChunkRegionLoader.java
|
|
+++ b/src/main/java/net/minecraft/server/ChunkRegionLoader.java
|
|
@@ -6,6 +6,7 @@ import it.unimi.dsi.fastutil.longs.LongOpenHashSet;
|
|
import it.unimi.dsi.fastutil.longs.LongSet;
|
|
import it.unimi.dsi.fastutil.shorts.ShortList;
|
|
import it.unimi.dsi.fastutil.shorts.ShortListIterator;
|
|
+import java.util.ArrayDeque; // Paper
|
|
import java.util.Arrays;
|
|
import java.util.BitSet;
|
|
import java.util.EnumSet;
|
|
@@ -22,7 +23,29 @@ public class ChunkRegionLoader {
|
|
|
|
private static final Logger LOGGER = LogManager.getLogger();
|
|
|
|
+ // Paper start
|
|
+ public static final class InProgressChunkHolder {
|
|
+
|
|
+ public final ProtoChunk protoChunk;
|
|
+ public final ArrayDeque<Runnable> tasks;
|
|
+
|
|
+ public NBTTagCompound poiData;
|
|
+
|
|
+ public InProgressChunkHolder(final ProtoChunk protoChunk, final ArrayDeque<Runnable> tasks) {
|
|
+ this.protoChunk = protoChunk;
|
|
+ this.tasks = tasks;
|
|
+ }
|
|
+ }
|
|
+
|
|
public static ProtoChunk loadChunk(WorldServer worldserver, DefinedStructureManager definedstructuremanager, VillagePlace villageplace, ChunkCoordIntPair chunkcoordintpair, NBTTagCompound nbttagcompound) {
|
|
+ InProgressChunkHolder holder = loadChunk(worldserver, definedstructuremanager, villageplace, chunkcoordintpair, nbttagcompound, true);
|
|
+ holder.tasks.forEach(Runnable::run);
|
|
+ return holder.protoChunk;
|
|
+ }
|
|
+
|
|
+ public static InProgressChunkHolder loadChunk(WorldServer worldserver, DefinedStructureManager definedstructuremanager, VillagePlace villageplace, ChunkCoordIntPair chunkcoordintpair, NBTTagCompound nbttagcompound, boolean distinguish) {
|
|
+ ArrayDeque<Runnable> tasksToExecuteOnMain = new ArrayDeque<>();
|
|
+ // Paper end
|
|
ChunkGenerator<?> chunkgenerator = worldserver.getChunkProvider().getChunkGenerator();
|
|
WorldChunkManager worldchunkmanager = chunkgenerator.getWorldChunkManager();
|
|
NBTTagCompound nbttagcompound1 = nbttagcompound.getCompound("Level");
|
|
@@ -49,7 +72,9 @@ public class ChunkRegionLoader {
|
|
LightEngine lightengine = chunkproviderserver.getLightEngine();
|
|
|
|
if (flag) {
|
|
- lightengine.b(chunkcoordintpair, true);
|
|
+ tasksToExecuteOnMain.add(() -> { // Paper - delay this task since we're executing off-main
|
|
+ lightengine.b(chunkcoordintpair, true);
|
|
+ }); // Paper - delay this task since we're executing off-main
|
|
}
|
|
|
|
for (int i = 0; i < nbttaglist.size(); ++i) {
|
|
@@ -65,16 +90,30 @@ public class ChunkRegionLoader {
|
|
achunksection[b0] = chunksection;
|
|
}
|
|
|
|
- villageplace.a(chunkcoordintpair, chunksection);
|
|
+ tasksToExecuteOnMain.add(() -> { // Paper - delay this task since we're executing off-main
|
|
+ villageplace.a(chunkcoordintpair, chunksection);
|
|
+ }); // Paper - delay this task since we're executing off-main
|
|
}
|
|
|
|
if (flag) {
|
|
if (nbttagcompound2.hasKeyOfType("BlockLight", 7)) {
|
|
- lightengine.a(EnumSkyBlock.BLOCK, SectionPosition.a(chunkcoordintpair, b0), new NibbleArray(nbttagcompound2.getByteArray("BlockLight")));
|
|
+ // Paper start - delay this task since we're executing off-main
|
|
+ NibbleArray blockLight = new NibbleArray(nbttagcompound2.getByteArray("BlockLight"));
|
|
+ // Note: We move the block light nibble array creation here for perf & in case the compound is modified
|
|
+ tasksToExecuteOnMain.add(() -> {
|
|
+ lightengine.a(EnumSkyBlock.BLOCK, SectionPosition.a(chunkcoordintpair, b0), blockLight);
|
|
+ });
|
|
+ // Paper end
|
|
}
|
|
|
|
if (flag2 && nbttagcompound2.hasKeyOfType("SkyLight", 7)) {
|
|
- lightengine.a(EnumSkyBlock.SKY, SectionPosition.a(chunkcoordintpair, b0), new NibbleArray(nbttagcompound2.getByteArray("SkyLight")));
|
|
+ // Paper start - delay this task since we're executing off-main
|
|
+ NibbleArray skyLight = new NibbleArray(nbttagcompound2.getByteArray("SkyLight"));
|
|
+ // Note: We move the block light nibble array creation here for perf & in case the compound is modified
|
|
+ tasksToExecuteOnMain.add(() -> {
|
|
+ lightengine.a(EnumSkyBlock.SKY, SectionPosition.a(chunkcoordintpair, b0), skyLight);
|
|
+ });
|
|
+ // Paper end
|
|
}
|
|
}
|
|
}
|
|
@@ -177,7 +216,7 @@ public class ChunkRegionLoader {
|
|
}
|
|
|
|
if (chunkstatus_type == ChunkStatus.Type.LEVELCHUNK) {
|
|
- return new ProtoChunkExtension((Chunk) object);
|
|
+ return new InProgressChunkHolder(new ProtoChunkExtension((Chunk) object), tasksToExecuteOnMain); // Paper - Async chunk loading
|
|
} else {
|
|
ProtoChunk protochunk1 = (ProtoChunk) object;
|
|
|
|
@@ -216,11 +255,83 @@ public class ChunkRegionLoader {
|
|
protochunk1.a(worldgenstage_features, BitSet.valueOf(nbttagcompound5.getByteArray(s1)));
|
|
}
|
|
|
|
- return protochunk1;
|
|
+ return new InProgressChunkHolder(protochunk1, tasksToExecuteOnMain); // Paper - Async chunk loading
|
|
}
|
|
}
|
|
|
|
+ // Paper start - async chunk save for unload
|
|
+ public static final class AsyncSaveData {
|
|
+ public final NibbleArray[] blockLight; // null or size of 17 (for indices -1 through 15)
|
|
+ public final NibbleArray[] skyLight;
|
|
+
|
|
+ public final NBTTagList blockTickList; // non-null if we had to go to the server's tick list
|
|
+ public final NBTTagList fluidTickList; // non-null if we had to go to the server's tick list
|
|
+
|
|
+ public final long worldTime;
|
|
+
|
|
+ public AsyncSaveData(NibbleArray[] blockLight, NibbleArray[] skyLight, NBTTagList blockTickList, NBTTagList fluidTickList,
|
|
+ long worldTime) {
|
|
+ this.blockLight = blockLight;
|
|
+ this.skyLight = skyLight;
|
|
+ this.blockTickList = blockTickList;
|
|
+ this.fluidTickList = fluidTickList;
|
|
+ this.worldTime = worldTime;
|
|
+ }
|
|
+ }
|
|
+
|
|
+ // must be called sync
|
|
+ public static AsyncSaveData getAsyncSaveData(WorldServer world, IChunkAccess chunk) {
|
|
+ org.spigotmc.AsyncCatcher.catchOp("preparation of chunk data for async save");
|
|
+ ChunkCoordIntPair chunkPos = chunk.getPos();
|
|
+
|
|
+ LightEngineThreaded lightenginethreaded = world.getChunkProvider().getLightEngine();
|
|
+
|
|
+ NibbleArray[] blockLight = new NibbleArray[17 - (-1)];
|
|
+ NibbleArray[] skyLight = new NibbleArray[17 - (-1)];
|
|
+
|
|
+ for (int i = -1; i < 17; ++i) {
|
|
+ NibbleArray blockArray = lightenginethreaded.a(EnumSkyBlock.BLOCK).a(SectionPosition.a(chunkPos, i));
|
|
+ NibbleArray skyArray = lightenginethreaded.a(EnumSkyBlock.SKY).a(SectionPosition.a(chunkPos, i));
|
|
+
|
|
+ // copy data for safety
|
|
+ if (blockArray != null) {
|
|
+ blockArray = blockArray.copy();
|
|
+ }
|
|
+ if (skyArray != null) {
|
|
+ skyArray = skyArray.copy();
|
|
+ }
|
|
+
|
|
+ // apply offset of 1 for -1 starting index
|
|
+ blockLight[i + 1] = blockArray;
|
|
+ skyLight[i + 1] = skyArray;
|
|
+ }
|
|
+
|
|
+ TickList<Block> blockTickList = chunk.n();
|
|
+
|
|
+ NBTTagList blockTickListSerialized;
|
|
+ if (blockTickList instanceof ProtoChunkTickList || blockTickList instanceof TickListChunk) {
|
|
+ blockTickListSerialized = null;
|
|
+ } else {
|
|
+ blockTickListSerialized = world.getBlockTickList().a(chunkPos);
|
|
+ }
|
|
+
|
|
+ TickList<FluidType> fluidTickList = chunk.o();
|
|
+
|
|
+ NBTTagList fluidTickListSerialized;
|
|
+ if (fluidTickList instanceof ProtoChunkTickList || fluidTickList instanceof TickListChunk) {
|
|
+ fluidTickListSerialized = null;
|
|
+ } else {
|
|
+ fluidTickListSerialized = world.getFluidTickList().a(chunkPos);
|
|
+ }
|
|
+
|
|
+ return new AsyncSaveData(blockLight, skyLight, blockTickListSerialized, fluidTickListSerialized, world.getTime());
|
|
+ }
|
|
+
|
|
public static NBTTagCompound saveChunk(WorldServer worldserver, IChunkAccess ichunkaccess) {
|
|
+ return saveChunk(worldserver, ichunkaccess, null);
|
|
+ }
|
|
+ public static NBTTagCompound saveChunk(WorldServer worldserver, IChunkAccess ichunkaccess, AsyncSaveData asyncsavedata) {
|
|
+ // Paper end
|
|
ChunkCoordIntPair chunkcoordintpair = ichunkaccess.getPos();
|
|
NBTTagCompound nbttagcompound = new NBTTagCompound();
|
|
NBTTagCompound nbttagcompound1 = new NBTTagCompound();
|
|
@@ -229,7 +340,7 @@ public class ChunkRegionLoader {
|
|
nbttagcompound.set("Level", nbttagcompound1);
|
|
nbttagcompound1.setInt("xPos", chunkcoordintpair.x);
|
|
nbttagcompound1.setInt("zPos", chunkcoordintpair.z);
|
|
- nbttagcompound1.setLong("LastUpdate", worldserver.getTime());
|
|
+ nbttagcompound1.setLong("LastUpdate", asyncsavedata != null ? asyncsavedata.worldTime : worldserver.getTime()); // Paper - async chunk unloading
|
|
nbttagcompound1.setLong("InhabitedTime", ichunkaccess.getInhabitedTime());
|
|
nbttagcompound1.setString("Status", ichunkaccess.getChunkStatus().d());
|
|
ChunkConverter chunkconverter = ichunkaccess.p();
|
|
@@ -245,14 +356,22 @@ public class ChunkRegionLoader {
|
|
|
|
NBTTagCompound nbttagcompound2;
|
|
|
|
- for (int i = -1; i < 17; ++i) {
|
|
+ for (int i = -1; i < 17; ++i) { // Paper - conflict on loop parameter change
|
|
int finalI = i;
|
|
ChunkSection chunksection = (ChunkSection) Arrays.stream(achunksection).filter((chunksection1) -> {
|
|
return chunksection1 != null && chunksection1.getYPosition() >> 4 == finalI;
|
|
}).findFirst().orElse(Chunk.a);
|
|
- NibbleArray nibblearray = lightenginethreaded.a(EnumSkyBlock.BLOCK).a(SectionPosition.a(chunkcoordintpair, i));
|
|
- NibbleArray nibblearray1 = lightenginethreaded.a(EnumSkyBlock.SKY).a(SectionPosition.a(chunkcoordintpair, i));
|
|
-
|
|
+ // Paper start - async chunk save for unload
|
|
+ NibbleArray nibblearray; // block light
|
|
+ NibbleArray nibblearray1; // sky light
|
|
+ if (asyncsavedata == null) {
|
|
+ nibblearray = lightenginethreaded.a(EnumSkyBlock.BLOCK).a(SectionPosition.a(chunkcoordintpair, i)); /// Paper - diff on method change (see getAsyncSaveData)
|
|
+ nibblearray1 = lightenginethreaded.a(EnumSkyBlock.SKY).a(SectionPosition.a(chunkcoordintpair, i)); // Paper - diff on method change (see getAsyncSaveData)
|
|
+ } else {
|
|
+ nibblearray = asyncsavedata.blockLight[i + 1]; // +1 to offset the -1 starting index
|
|
+ nibblearray1 = asyncsavedata.skyLight[i + 1]; // +1 to offset the -1 starting index
|
|
+ }
|
|
+ // Paper end
|
|
if (chunksection != Chunk.a || nibblearray != null || nibblearray1 != null) {
|
|
nbttagcompound2 = new NBTTagCompound();
|
|
nbttagcompound2.setByte("Y", (byte) (i & 255));
|
|
@@ -313,7 +432,7 @@ public class ChunkRegionLoader {
|
|
Entity entity = (Entity) iterator1.next();
|
|
NBTTagCompound nbttagcompound4 = new NBTTagCompound();
|
|
// Paper start
|
|
- if ((int) Math.floor(entity.locX()) >> 4 != chunk.getPos().x || (int) Math.floor(entity.locZ()) >> 4 != chunk.getPos().z) {
|
|
+ if (asyncsavedata == null && !entity.dead && (int) Math.floor(entity.locX()) >> 4 != chunk.getPos().x || (int) Math.floor(entity.locZ()) >> 4 != chunk.getPos().z) {
|
|
toUpdate.add(entity);
|
|
continue;
|
|
}
|
|
@@ -353,24 +472,32 @@ public class ChunkRegionLoader {
|
|
}
|
|
|
|
nbttagcompound1.set("Entities", nbttaglist2);
|
|
- TickList<Block> ticklist = ichunkaccess.n();
|
|
+ TickList<Block> ticklist = ichunkaccess.n(); // Paper - diff on method change (see getAsyncSaveData)
|
|
|
|
if (ticklist instanceof ProtoChunkTickList) {
|
|
nbttagcompound1.set("ToBeTicked", ((ProtoChunkTickList) ticklist).b());
|
|
} else if (ticklist instanceof TickListChunk) {
|
|
- nbttagcompound1.set("TileTicks", ((TickListChunk) ticklist).a(worldserver.getTime()));
|
|
+ nbttagcompound1.set("TileTicks", ((TickListChunk) ticklist).a(asyncsavedata != null ? asyncsavedata.worldTime : worldserver.getTime())); // Paper - async chunk unloading
|
|
+ // Paper start - async chunk save for unload
|
|
+ } else if (asyncsavedata != null) {
|
|
+ nbttagcompound1.set("TileTicks", asyncsavedata.blockTickList);
|
|
+ // Paper end
|
|
} else {
|
|
- nbttagcompound1.set("TileTicks", worldserver.getBlockTickList().a(chunkcoordintpair));
|
|
+ nbttagcompound1.set("TileTicks", worldserver.getBlockTickList().a(chunkcoordintpair)); // Paper - diff on method change (see getAsyncSaveData)
|
|
}
|
|
|
|
- TickList<FluidType> ticklist1 = ichunkaccess.o();
|
|
+ TickList<FluidType> ticklist1 = ichunkaccess.o(); // Paper - diff on method change (see getAsyncSaveData)
|
|
|
|
if (ticklist1 instanceof ProtoChunkTickList) {
|
|
nbttagcompound1.set("LiquidsToBeTicked", ((ProtoChunkTickList) ticklist1).b());
|
|
} else if (ticklist1 instanceof TickListChunk) {
|
|
- nbttagcompound1.set("LiquidTicks", ((TickListChunk) ticklist1).a(worldserver.getTime()));
|
|
+ nbttagcompound1.set("LiquidTicks", ((TickListChunk) ticklist1).a(asyncsavedata != null ? asyncsavedata.worldTime : worldserver.getTime())); // Paper - async chunk unloading
|
|
+ // Paper start - async chunk save for unload
|
|
+ } else if (asyncsavedata != null) {
|
|
+ nbttagcompound1.set("LiquidTicks", asyncsavedata.fluidTickList);
|
|
+ // Paper end
|
|
} else {
|
|
- nbttagcompound1.set("LiquidTicks", worldserver.getFluidTickList().a(chunkcoordintpair));
|
|
+ nbttagcompound1.set("LiquidTicks", worldserver.getFluidTickList().a(chunkcoordintpair)); // Paper - diff on method change (see getAsyncSaveData)
|
|
}
|
|
|
|
nbttagcompound1.set("PostProcessing", a(ichunkaccess.l()));
|
|
diff --git a/src/main/java/net/minecraft/server/ChunkStatus.java b/src/main/java/net/minecraft/server/ChunkStatus.java
|
|
index 134a4f0b7..40ce30cdc 100644
|
|
--- a/src/main/java/net/minecraft/server/ChunkStatus.java
|
|
+++ b/src/main/java/net/minecraft/server/ChunkStatus.java
|
|
@@ -153,6 +153,7 @@ public class ChunkStatus {
|
|
return ChunkStatus.q.size();
|
|
}
|
|
|
|
+ public static int getTicketLevelOffset(ChunkStatus status) { return ChunkStatus.a(status); } // Paper - OBFHELPER
|
|
public static int a(ChunkStatus chunkstatus) {
|
|
return ChunkStatus.r.getInt(chunkstatus.c());
|
|
}
|
|
@@ -168,6 +169,7 @@ public class ChunkStatus {
|
|
this.t = chunkstatus == null ? 0 : chunkstatus.c() + 1;
|
|
}
|
|
|
|
+ public int getStatusIndex() { return c(); } // Paper - OBFHELPER
|
|
public int c() {
|
|
return this.t;
|
|
}
|
|
@@ -189,6 +191,7 @@ public class ChunkStatus {
|
|
return this.w.doWork(this, worldserver, definedstructuremanager, lightenginethreaded, function, ichunkaccess);
|
|
}
|
|
|
|
+ public int getNeighborRadius() { return this.f(); } // Paper - OBFHELPER
|
|
public int f() {
|
|
return this.x;
|
|
}
|
|
@@ -216,6 +219,7 @@ public class ChunkStatus {
|
|
return this.z;
|
|
}
|
|
|
|
+ public boolean isAtLeastStatus(ChunkStatus chunkstatus) { return b(chunkstatus); } // Paper - OBFHELPER
|
|
public boolean b(ChunkStatus chunkstatus) {
|
|
return this.c() >= chunkstatus.c();
|
|
}
|
|
diff --git a/src/main/java/net/minecraft/server/IAsyncTaskHandler.java b/src/main/java/net/minecraft/server/IAsyncTaskHandler.java
|
|
index 7e5ece9d5..cfe43e882 100644
|
|
--- a/src/main/java/net/minecraft/server/IAsyncTaskHandler.java
|
|
+++ b/src/main/java/net/minecraft/server/IAsyncTaskHandler.java
|
|
@@ -91,7 +91,7 @@ public abstract class IAsyncTaskHandler<R extends Runnable> implements Mailbox<R
|
|
|
|
}
|
|
|
|
- protected void executeAll() {
|
|
+ public void executeAll() { // Paper - protected -> public
|
|
while (this.executeNext()) {
|
|
;
|
|
}
|
|
diff --git a/src/main/java/net/minecraft/server/IChunkLoader.java b/src/main/java/net/minecraft/server/IChunkLoader.java
|
|
index 2f95174fc..134c76065 100644
|
|
--- a/src/main/java/net/minecraft/server/IChunkLoader.java
|
|
+++ b/src/main/java/net/minecraft/server/IChunkLoader.java
|
|
@@ -3,37 +3,49 @@ package net.minecraft.server;
|
|
import com.mojang.datafixers.DataFixer;
|
|
import java.io.File;
|
|
import java.io.IOException;
|
|
+// Paper start
|
|
+import java.util.concurrent.CompletableFuture;
|
|
+import java.util.concurrent.CompletionException;
|
|
+// Paper end
|
|
import java.util.function.Supplier;
|
|
import javax.annotation.Nullable;
|
|
|
|
-public class IChunkLoader implements AutoCloseable {
|
|
+public class IChunkLoader extends RegionFileCache implements AutoCloseable {
|
|
|
|
- private final IOWorker a; public IOWorker getIOWorker() { return a; } // Paper - OBFHELPER
|
|
+// private final IOWorker a; public IOWorker getIOWorker() { return a; } // Paper - OBFHELPER - nuke IOWorker
|
|
protected final DataFixer b;
|
|
@Nullable
|
|
- private PersistentStructureLegacy c;
|
|
+ private volatile PersistentStructureLegacy c; // Paper - async chunk loading
|
|
+
|
|
+ private final Object persistentDataLock = new Object(); // Paper
|
|
|
|
public IChunkLoader(File file, DataFixer datafixer) {
|
|
+ super(file);
|
|
this.b = datafixer;
|
|
- this.a = new IOWorker(new RegionFileCache(file), "chunk");
|
|
+// this.a = new IOWorker(new RegionFileCache(file), "chunk"); // Paper - nuke IOWorker
|
|
}
|
|
|
|
// CraftBukkit start
|
|
private boolean check(ChunkProviderServer cps, int x, int z) throws IOException {
|
|
ChunkCoordIntPair pos = new ChunkCoordIntPair(x, z);
|
|
if (cps != null) {
|
|
- com.google.common.base.Preconditions.checkState(org.bukkit.Bukkit.isPrimaryThread(), "primary thread");
|
|
- if (cps.isLoaded(x, z)) {
|
|
+ //com.google.common.base.Preconditions.checkState(org.bukkit.Bukkit.isPrimaryThread(), "primary thread"); // Paper - this function is now MT-Safe
|
|
+ if (cps.getChunkAtIfCachedImmediately(x, z) != null) { // Paper - isLoaded is a ticket level check, not a chunk loaded check!
|
|
return true;
|
|
}
|
|
}
|
|
|
|
- NBTTagCompound nbt = read(pos);
|
|
- if (nbt != null) {
|
|
- NBTTagCompound level = nbt.getCompound("Level");
|
|
- if (level.getBoolean("TerrainPopulated")) {
|
|
- return true;
|
|
- }
|
|
+
|
|
+ // Paper start - prioritize
|
|
+ NBTTagCompound nbt = cps == null ? read(pos) :
|
|
+ com.destroystokyo.paper.io.PaperFileIOThread.Holder.INSTANCE.loadChunkData((WorldServer)cps.getWorld(), x, z,
|
|
+ com.destroystokyo.paper.io.PrioritizedTaskQueue.HIGHER_PRIORITY, false, true).chunkData;
|
|
+ // Paper end
|
|
+ if (nbt != null) {
|
|
+ NBTTagCompound level = nbt.getCompound("Level");
|
|
+ if (level.getBoolean("TerrainPopulated")) {
|
|
+ return true;
|
|
+ }
|
|
|
|
ChunkStatus status = ChunkStatus.a(level.getString("Status"));
|
|
if (status != null && status.b(ChunkStatus.FEATURES)) {
|
|
@@ -64,11 +76,13 @@ public class IChunkLoader implements AutoCloseable {
|
|
if (i < 1493) {
|
|
nbttagcompound = GameProfileSerializer.a(this.b, DataFixTypes.CHUNK, nbttagcompound, i, 1493);
|
|
if (nbttagcompound.getCompound("Level").getBoolean("hasLegacyStructureData")) {
|
|
+ synchronized (this.persistentDataLock) { // Paper - Async chunk loading
|
|
if (this.c == null) {
|
|
this.c = PersistentStructureLegacy.a(dimensionmanager.getType(), (WorldPersistentData) supplier.get()); // CraftBukkit - getType
|
|
}
|
|
|
|
nbttagcompound = this.c.a(nbttagcompound);
|
|
+ } // Paper - Async chunk loading
|
|
}
|
|
}
|
|
|
|
@@ -84,24 +98,28 @@ public class IChunkLoader implements AutoCloseable {
|
|
return nbttagcompound.hasKeyOfType("DataVersion", 99) ? nbttagcompound.getInt("DataVersion") : -1;
|
|
}
|
|
|
|
- @Nullable
|
|
- public NBTTagCompound read(ChunkCoordIntPair chunkcoordintpair) throws IOException {
|
|
- return this.a.a(chunkcoordintpair);
|
|
- }
|
|
-
|
|
- public void a(ChunkCoordIntPair chunkcoordintpair, NBTTagCompound nbttagcompound) {
|
|
- this.a.a(chunkcoordintpair, nbttagcompound);
|
|
+// Paper start - nuke IOWorker
|
|
+// @Nullable
|
|
+// public NBTTagCompound read(ChunkCoordIntPair chunkcoordintpair) throws IOException {
|
|
+// return this.a.a(chunkcoordintpair);
|
|
+// }
|
|
+//
|
|
+ public void a(ChunkCoordIntPair chunkcoordintpair, NBTTagCompound nbttagcompound) throws IOException { write(chunkcoordintpair, nbttagcompound); } // Paper OBFHELPER
|
|
+ public void write(ChunkCoordIntPair chunkcoordintpair, NBTTagCompound nbttagcompound) throws IOException { // Paper - OBFHELPER - (Switched around for safety)
|
|
+ super.write(chunkcoordintpair, nbttagcompound);
|
|
if (this.c != null) {
|
|
- this.c.a(chunkcoordintpair.pair());
|
|
+ synchronized (this.persistentDataLock) { // Paper - Async chunk loading
|
|
+ this.c.a(chunkcoordintpair.pair()); } // Paper - Async chunk loading}
|
|
}
|
|
|
|
}
|
|
-
|
|
- public void i() {
|
|
- this.a.a().join();
|
|
- }
|
|
-
|
|
- public void close() throws IOException {
|
|
- this.a.close();
|
|
- }
|
|
+//
|
|
+// public void i() {
|
|
+// this.a.a().join();
|
|
+// }
|
|
+//
|
|
+// public void close() throws IOException {
|
|
+// this.a.close();
|
|
+// }
|
|
+// Paper end
|
|
}
|
|
diff --git a/src/main/java/net/minecraft/server/MCUtil.java b/src/main/java/net/minecraft/server/MCUtil.java
|
|
index 4851b9fc5..786e6c976 100644
|
|
--- a/src/main/java/net/minecraft/server/MCUtil.java
|
|
+++ b/src/main/java/net/minecraft/server/MCUtil.java
|
|
@@ -604,4 +604,9 @@ public final class MCUtil {
|
|
out.print(fileData);
|
|
}
|
|
}
|
|
+
|
|
+ public static int getTicketLevelFor(ChunkStatus status) {
|
|
+ // TODO make sure the constant `33` is correct on future updates. See getChunkAt(int, int, ChunkStatus, boolean)
|
|
+ return 33 + ChunkStatus.getTicketLevelOffset(status);
|
|
+ }
|
|
}
|
|
diff --git a/src/main/java/net/minecraft/server/MinecraftServer.java b/src/main/java/net/minecraft/server/MinecraftServer.java
|
|
index 6fa08c60b..baeaf772b 100644
|
|
--- a/src/main/java/net/minecraft/server/MinecraftServer.java
|
|
+++ b/src/main/java/net/minecraft/server/MinecraftServer.java
|
|
@@ -789,6 +789,7 @@ public abstract class MinecraftServer extends IAsyncTaskHandlerReentrant<TickTas
|
|
this.getUserCache().c(false); // Paper
|
|
}
|
|
// Spigot end
|
|
+ com.destroystokyo.paper.io.PaperFileIOThread.Holder.INSTANCE.close(true, true); // Paper
|
|
}
|
|
|
|
public String getServerIp() {
|
|
diff --git a/src/main/java/net/minecraft/server/NextTickListEntry.java b/src/main/java/net/minecraft/server/NextTickListEntry.java
|
|
index e9c405fb5..33cfeabde 100644
|
|
--- a/src/main/java/net/minecraft/server/NextTickListEntry.java
|
|
+++ b/src/main/java/net/minecraft/server/NextTickListEntry.java
|
|
@@ -4,7 +4,7 @@ import java.util.Comparator;
|
|
|
|
public class NextTickListEntry<T> {
|
|
|
|
- private static long d;
|
|
+ private static final java.util.concurrent.atomic.AtomicLong COUNTER = new java.util.concurrent.atomic.AtomicLong(); // Paper - async chunk loading
|
|
private final T e;
|
|
public final BlockPosition a;
|
|
public final long b;
|
|
@@ -16,7 +16,7 @@ public class NextTickListEntry<T> {
|
|
}
|
|
|
|
public NextTickListEntry(BlockPosition blockposition, T t0, long i, TickListPriority ticklistpriority) {
|
|
- this.f = (long) (NextTickListEntry.d++);
|
|
+ this.f = (long) (NextTickListEntry.COUNTER.getAndIncrement()); // Paper - async chunk loading
|
|
this.a = blockposition.immutableCopy();
|
|
this.e = t0;
|
|
this.b = i;
|
|
diff --git a/src/main/java/net/minecraft/server/NibbleArray.java b/src/main/java/net/minecraft/server/NibbleArray.java
|
|
index ed8c4a87b..996c83263 100644
|
|
--- a/src/main/java/net/minecraft/server/NibbleArray.java
|
|
+++ b/src/main/java/net/minecraft/server/NibbleArray.java
|
|
@@ -71,6 +71,7 @@ public class NibbleArray {
|
|
return this.a;
|
|
}
|
|
|
|
+ public NibbleArray copy() { return this.b(); } // Paper - OBFHELPER
|
|
public NibbleArray b() {
|
|
return this.a == null ? new NibbleArray() : new NibbleArray((byte[]) this.a.clone());
|
|
}
|
|
diff --git a/src/main/java/net/minecraft/server/PlayerChunk.java b/src/main/java/net/minecraft/server/PlayerChunk.java
|
|
index 50135446f..b38bc6775 100644
|
|
--- a/src/main/java/net/minecraft/server/PlayerChunk.java
|
|
+++ b/src/main/java/net/minecraft/server/PlayerChunk.java
|
|
@@ -127,6 +127,18 @@ public class PlayerChunk {
|
|
}
|
|
return null;
|
|
}
|
|
+
|
|
+ public ChunkStatus getChunkHolderStatus() {
|
|
+ for (ChunkStatus curr = ChunkStatus.FULL, next = curr.getPreviousStatus(); curr != next; curr = next, next = next.getPreviousStatus()) {
|
|
+ CompletableFuture<Either<IChunkAccess, PlayerChunk.Failure>> future = this.getStatusFutureUnchecked(curr);
|
|
+ Either<IChunkAccess, PlayerChunk.Failure> either = future.getNow(null);
|
|
+ if (either == null || !either.left().isPresent()) {
|
|
+ continue;
|
|
+ }
|
|
+ return curr;
|
|
+ }
|
|
+ return null;
|
|
+ }
|
|
// Paper end
|
|
|
|
public CompletableFuture<Either<IChunkAccess, PlayerChunk.Failure>> getStatusFutureUnchecked(ChunkStatus chunkstatus) {
|
|
@@ -354,7 +366,7 @@ public class PlayerChunk {
|
|
ChunkStatus chunkstatus = getChunkStatus(this.oldTicketLevel);
|
|
ChunkStatus chunkstatus1 = getChunkStatus(this.ticketLevel);
|
|
boolean flag = this.oldTicketLevel <= PlayerChunkMap.GOLDEN_TICKET;
|
|
- boolean flag1 = this.ticketLevel <= PlayerChunkMap.GOLDEN_TICKET;
|
|
+ boolean flag1 = this.ticketLevel <= PlayerChunkMap.GOLDEN_TICKET; // Paper - diff on change: (flag1 = new ticket level is in loadable range)
|
|
PlayerChunk.State playerchunk_state = getChunkState(this.oldTicketLevel);
|
|
PlayerChunk.State playerchunk_state1 = getChunkState(this.ticketLevel);
|
|
// CraftBukkit start
|
|
@@ -390,6 +402,12 @@ public class PlayerChunk {
|
|
}
|
|
});
|
|
|
|
+ // Paper start
|
|
+ if (!flag1) {
|
|
+ playerchunkmap.world.asyncChunkTaskManager.cancelChunkLoad(this.location.x, this.location.z);
|
|
+ }
|
|
+ // Paper end
|
|
+
|
|
for (int i = flag1 ? chunkstatus1.c() + 1 : 0; i <= chunkstatus.c(); ++i) {
|
|
completablefuture = (CompletableFuture) this.statusFutures.get(i);
|
|
if (completablefuture != null) {
|
|
diff --git a/src/main/java/net/minecraft/server/PlayerChunkMap.java b/src/main/java/net/minecraft/server/PlayerChunkMap.java
|
|
index b4c9d544f..7e5fa016c 100644
|
|
--- a/src/main/java/net/minecraft/server/PlayerChunkMap.java
|
|
+++ b/src/main/java/net/minecraft/server/PlayerChunkMap.java
|
|
@@ -63,7 +63,7 @@ public class PlayerChunkMap extends IChunkLoader implements PlayerChunk.d {
|
|
private final LightEngineThreaded lightEngine;
|
|
private final IAsyncTaskHandler<Runnable> executor;
|
|
public final ChunkGenerator<?> chunkGenerator;
|
|
- private final Supplier<WorldPersistentData> l;
|
|
+ private final Supplier<WorldPersistentData> l; public final Supplier<WorldPersistentData> getWorldPersistentDataSupplier() { return this.l; } // Paper - OBFHELPER
|
|
private final VillagePlace m;
|
|
public final LongSet unloadQueue;
|
|
private boolean updatingChunksModified;
|
|
@@ -73,7 +73,7 @@ public class PlayerChunkMap extends IChunkLoader implements PlayerChunk.d {
|
|
public final WorldLoadListener worldLoadListener;
|
|
public final PlayerChunkMap.a chunkDistanceManager; public final PlayerChunkMap.a getChunkMapDistanceManager() { return this.chunkDistanceManager; } // Paper - OBFHELPER
|
|
private final AtomicInteger u;
|
|
- private final DefinedStructureManager definedStructureManager;
|
|
+ public final DefinedStructureManager definedStructureManager; // Paper - private -> public
|
|
private final File w;
|
|
private final PlayerMap playerMap;
|
|
public final Int2ObjectMap<PlayerChunkMap.EntityTracker> trackedEntities;
|
|
@@ -163,7 +163,7 @@ public class PlayerChunkMap extends IChunkLoader implements PlayerChunk.d {
|
|
this.lightEngine = new LightEngineThreaded(ilightaccess, this, this.world.getWorldProvider().f(), threadedmailbox1, this.p.a(threadedmailbox1, false));
|
|
this.chunkDistanceManager = new PlayerChunkMap.a(executor, iasynctaskhandler);
|
|
this.l = supplier;
|
|
- this.m = new VillagePlace(new File(this.w, "poi"), datafixer);
|
|
+ this.m = new VillagePlace(new File(this.w, "poi"), datafixer, this.world); // Paper
|
|
this.setViewDistance(i);
|
|
}
|
|
|
|
@@ -210,7 +210,7 @@ public class PlayerChunkMap extends IChunkLoader implements PlayerChunk.d {
|
|
}
|
|
|
|
@Nullable
|
|
- protected PlayerChunk getVisibleChunk(long i) {
|
|
+ public PlayerChunk getVisibleChunk(long i) { // Paper - protected -> public
|
|
return (PlayerChunk) this.visibleChunks.get(i);
|
|
}
|
|
|
|
@@ -324,6 +324,7 @@ public class PlayerChunkMap extends IChunkLoader implements PlayerChunk.d {
|
|
public void close() throws IOException {
|
|
try {
|
|
this.p.close();
|
|
+ this.world.asyncChunkTaskManager.close(true); // Paper - Required since we're closing regionfiles in the next line
|
|
this.m.close();
|
|
} finally {
|
|
super.close();
|
|
@@ -415,7 +416,8 @@ public class PlayerChunkMap extends IChunkLoader implements PlayerChunk.d {
|
|
this.b(() -> {
|
|
return true;
|
|
});
|
|
- this.i();
|
|
+ this.world.asyncChunkTaskManager.flush(); // Paper - flush to preserve behavior compat with pre-async behaviour
|
|
+// this.i(); // Paper - nuke IOWorker
|
|
PlayerChunkMap.LOGGER.info("ThreadedAnvilChunkStorage ({}): All chunks are saved", this.w.getName());
|
|
} else {
|
|
this.visibleChunks.values().stream().filter(PlayerChunk::hasBeenLoaded).forEach((playerchunk) -> {
|
|
@@ -436,11 +438,15 @@ public class PlayerChunkMap extends IChunkLoader implements PlayerChunk.d {
|
|
protected void unloadChunks(BooleanSupplier booleansupplier) {
|
|
GameProfilerFiller gameprofilerfiller = this.world.getMethodProfiler();
|
|
|
|
+ try (Timing ignored = this.world.timings.poiUnload.startTiming()) { // Paper
|
|
gameprofilerfiller.enter("poi");
|
|
this.m.a(booleansupplier);
|
|
+ } // Paper
|
|
gameprofilerfiller.exitEnter("chunk_unload");
|
|
if (!this.world.isSavingDisabled()) {
|
|
+ try (Timing ignored = this.world.timings.chunkUnload.startTiming()) { // Paper
|
|
this.b(booleansupplier);
|
|
+ }// Paper
|
|
}
|
|
|
|
gameprofilerfiller.exit();
|
|
@@ -480,6 +486,60 @@ public class PlayerChunkMap extends IChunkLoader implements PlayerChunk.d {
|
|
|
|
}
|
|
|
|
+ // Paper start - async chunk save for unload
|
|
+ // Note: This is very unsafe to call if the chunk is still in use.
|
|
+ // This is also modeled after PlayerChunkMap#saveChunk(IChunkAccess, boolean), with the intentional difference being
|
|
+ // serializing the chunk is left to a worker thread.
|
|
+ private void asyncSave(IChunkAccess chunk) {
|
|
+ ChunkCoordIntPair chunkPos = chunk.getPos();
|
|
+ NBTTagCompound poiData;
|
|
+ try (Timing ignored = this.world.timings.chunkUnloadPOISerialization.startTiming()) {
|
|
+ poiData = this.getVillagePlace().getData(chunk.getPos());
|
|
+ }
|
|
+
|
|
+ com.destroystokyo.paper.io.PaperFileIOThread.Holder.INSTANCE.scheduleSave(this.world, chunkPos.x, chunkPos.z,
|
|
+ poiData, null, com.destroystokyo.paper.io.PrioritizedTaskQueue.LOW_PRIORITY);
|
|
+
|
|
+ if (!chunk.isNeedsSaving()) {
|
|
+ return;
|
|
+ }
|
|
+
|
|
+ ChunkStatus chunkstatus = chunk.getChunkStatus();
|
|
+
|
|
+ // Copied from PlayerChunkMap#saveChunk(IChunkAccess, boolean)
|
|
+ if (chunkstatus.getType() != ChunkStatus.Type.LEVELCHUNK) {
|
|
+ try (co.aikar.timings.Timing ignored1 = this.world.timings.chunkSaveOverwriteCheck.startTiming()) { // Paper
|
|
+ // Paper start - Optimize save by using status cache
|
|
+ try {
|
|
+ ChunkStatus statusOnDisk = this.getChunkStatusOnDisk(chunkPos);
|
|
+ if (statusOnDisk != null && statusOnDisk.getType() == ChunkStatus.Type.LEVELCHUNK) {
|
|
+ // Paper end
|
|
+ return;
|
|
+ }
|
|
+
|
|
+ if (chunkstatus == ChunkStatus.EMPTY && chunk.h().values().stream().noneMatch(StructureStart::e)) {
|
|
+ return;
|
|
+ }
|
|
+ } catch (IOException ex) {
|
|
+ ex.printStackTrace();
|
|
+ return;
|
|
+ }
|
|
+ }
|
|
+ }
|
|
+
|
|
+ ChunkRegionLoader.AsyncSaveData asyncSaveData;
|
|
+ try (Timing ignored = this.world.timings.chunkUnloadPrepareSave.startTiming()) {
|
|
+ asyncSaveData = ChunkRegionLoader.getAsyncSaveData(this.world, chunk);
|
|
+ }
|
|
+
|
|
+ this.world.asyncChunkTaskManager.scheduleChunkSave(chunkPos.x, chunkPos.z, com.destroystokyo.paper.io.PrioritizedTaskQueue.LOW_PRIORITY,
|
|
+ asyncSaveData, chunk);
|
|
+
|
|
+ chunk.setLastSaved(this.world.getTime());
|
|
+ chunk.setNeedsSaving(false);
|
|
+ }
|
|
+ // Paper end
|
|
+
|
|
private void a(long i, PlayerChunk playerchunk) {
|
|
CompletableFuture<IChunkAccess> completablefuture = playerchunk.getChunkSave();
|
|
Consumer<IChunkAccess> consumer = (ichunkaccess) -> { // CraftBukkit - decompile error
|
|
@@ -493,7 +553,7 @@ public class PlayerChunkMap extends IChunkLoader implements PlayerChunk.d {
|
|
((Chunk) ichunkaccess).setLoaded(false);
|
|
}
|
|
|
|
- this.saveChunk(ichunkaccess);
|
|
+ //this.saveChunk(ichunkaccess);// Paper - delay
|
|
if (this.loadedChunks.remove(i) && ichunkaccess instanceof Chunk) {
|
|
Chunk chunk = (Chunk) ichunkaccess;
|
|
|
|
@@ -501,6 +561,13 @@ public class PlayerChunkMap extends IChunkLoader implements PlayerChunk.d {
|
|
}
|
|
this.autoSaveQueue.remove(playerchunk); // Paper
|
|
|
|
+ try {
|
|
+ this.asyncSave(ichunkaccess); // Paper - async chunk saving
|
|
+ } catch (Throwable ex) {
|
|
+ LOGGER.fatal("Failed to prepare async save, attempting synchronous save", ex);
|
|
+ this.saveChunk(ichunkaccess);
|
|
+ }
|
|
+
|
|
this.lightEngine.a(ichunkaccess.getPos());
|
|
this.lightEngine.queueUpdate();
|
|
this.worldLoadListener.a(ichunkaccess.getPos(), (ChunkStatus) null);
|
|
@@ -570,27 +637,32 @@ public class PlayerChunkMap extends IChunkLoader implements PlayerChunk.d {
|
|
}
|
|
}
|
|
|
|
+ // Paper start - Async chunk io
|
|
+ public NBTTagCompound completeChunkData(NBTTagCompound compound, ChunkCoordIntPair chunkcoordintpair) throws IOException {
|
|
+ return compound == null ? null : this.getChunkData(this.world.getWorldProvider().getDimensionManager(), this.getWorldPersistentDataSupplier(), compound, chunkcoordintpair, this.world);
|
|
+ }
|
|
+ // Paper end
|
|
+
|
|
private CompletableFuture<Either<IChunkAccess, PlayerChunk.Failure>> f(ChunkCoordIntPair chunkcoordintpair) {
|
|
- return CompletableFuture.supplyAsync(() -> {
|
|
+ // Paper start - Async chunk io
|
|
+ final java.util.function.BiFunction<ChunkRegionLoader.InProgressChunkHolder, Throwable, Either<IChunkAccess, PlayerChunk.Failure>> syncLoadComplete = (chunkHolder, ioThrowable) -> {
|
|
try (Timing ignored = this.world.timings.syncChunkLoadTimer.startTimingIfSync()) { // Paper
|
|
this.world.getMethodProfiler().c("chunkLoad");
|
|
- NBTTagCompound nbttagcompound; // Paper
|
|
- try (Timing ignored2 = this.world.timings.chunkIOStage1.startTimingIfSync()) { // Paper start - timings
|
|
- nbttagcompound = this.readChunkData(chunkcoordintpair);
|
|
- } // Paper end
|
|
-
|
|
- if (nbttagcompound != null) {
|
|
- boolean flag = nbttagcompound.hasKeyOfType("Level", 10) && nbttagcompound.getCompound("Level").hasKeyOfType("Status", 8);
|
|
-
|
|
- if (flag) {
|
|
- ProtoChunk protochunk = ChunkRegionLoader.loadChunk(this.world, this.definedStructureManager, this.m, chunkcoordintpair, nbttagcompound);
|
|
+ if (ioThrowable != null) {
|
|
+ com.destroystokyo.paper.io.IOUtil.rethrow(ioThrowable);
|
|
+ }
|
|
|
|
- protochunk.setLastSaved(this.world.getTime());
|
|
- return Either.left(protochunk);
|
|
- }
|
|
+ this.getVillagePlace().loadInData(chunkcoordintpair, chunkHolder.poiData);
|
|
+ chunkHolder.tasks.forEach(Runnable::run);
|
|
+ // Paper - async load completes this
|
|
+ // Paper end
|
|
|
|
- PlayerChunkMap.LOGGER.error("Chunk file at {} is missing level data, skipping", chunkcoordintpair);
|
|
+ // Paper start - This is done async
|
|
+ if (chunkHolder.protoChunk != null) {
|
|
+ chunkHolder.protoChunk.setLastSaved(this.world.getTime());
|
|
+ return Either.left(chunkHolder.protoChunk);
|
|
}
|
|
+ // Paper end
|
|
} catch (ReportedException reportedexception) {
|
|
Throwable throwable = reportedexception.getCause();
|
|
|
|
@@ -604,7 +676,27 @@ public class PlayerChunkMap extends IChunkLoader implements PlayerChunk.d {
|
|
}
|
|
|
|
return Either.left(new ProtoChunk(chunkcoordintpair, ChunkConverter.a, this.world)); // Paper - Anti-Xray
|
|
- }, this.executor);
|
|
+ // Paper start - Async chunk io
|
|
+ };
|
|
+ CompletableFuture<Either<IChunkAccess, PlayerChunk.Failure>> ret = new CompletableFuture<>();
|
|
+
|
|
+ Consumer<ChunkRegionLoader.InProgressChunkHolder> chunkHolderConsumer = (ChunkRegionLoader.InProgressChunkHolder holder) -> {
|
|
+ PlayerChunkMap.this.executor.addTask(() -> {
|
|
+ ret.complete(syncLoadComplete.apply(holder, null));
|
|
+ });
|
|
+ };
|
|
+
|
|
+ CompletableFuture<NBTTagCompound> chunkSaveFuture = this.world.asyncChunkTaskManager.getChunkSaveFuture(chunkcoordintpair.x, chunkcoordintpair.z);
|
|
+ if (chunkSaveFuture != null) {
|
|
+ this.world.asyncChunkTaskManager.scheduleChunkLoad(chunkcoordintpair.x, chunkcoordintpair.z,
|
|
+ com.destroystokyo.paper.io.PrioritizedTaskQueue.HIGH_PRIORITY, chunkHolderConsumer, false, chunkSaveFuture);
|
|
+ this.world.asyncChunkTaskManager.raisePriority(chunkcoordintpair.x, chunkcoordintpair.z, com.destroystokyo.paper.io.PrioritizedTaskQueue.HIGH_PRIORITY);
|
|
+ } else {
|
|
+ this.world.asyncChunkTaskManager.scheduleChunkLoad(chunkcoordintpair.x, chunkcoordintpair.z,
|
|
+ com.destroystokyo.paper.io.PrioritizedTaskQueue.NORMAL_PRIORITY, chunkHolderConsumer, false);
|
|
+ }
|
|
+ return ret;
|
|
+ // Paper end
|
|
}
|
|
|
|
private CompletableFuture<Either<IChunkAccess, PlayerChunk.Failure>> b(PlayerChunk playerchunk, ChunkStatus chunkstatus) {
|
|
@@ -822,18 +914,43 @@ public class PlayerChunkMap extends IChunkLoader implements PlayerChunk.d {
|
|
return this.u.get();
|
|
}
|
|
|
|
+ // Paper start - async chunk io
|
|
+ private boolean writeDataAsync(ChunkCoordIntPair chunkPos, NBTTagCompound poiData, NBTTagCompound chunkData, boolean async) {
|
|
+ com.destroystokyo.paper.io.PaperFileIOThread.Holder.INSTANCE.scheduleSave(this.world, chunkPos.x, chunkPos.z,
|
|
+ poiData, chunkData, !async ? com.destroystokyo.paper.io.PrioritizedTaskQueue.HIGHEST_PRIORITY : com.destroystokyo.paper.io.PrioritizedTaskQueue.LOW_PRIORITY);
|
|
+
|
|
+ if (async) {
|
|
+ return true;
|
|
+ }
|
|
+
|
|
+ try (co.aikar.timings.Timing ignored = this.world.timings.chunkSaveIOWait.startTiming()) { // Paper
|
|
+ Boolean successPoi = com.destroystokyo.paper.io.PaperFileIOThread.Holder.INSTANCE.waitForIOToComplete(this.world, chunkPos.x, chunkPos.z, true, true);
|
|
+ Boolean successChunk = com.destroystokyo.paper.io.PaperFileIOThread.Holder.INSTANCE.waitForIOToComplete(this.world, chunkPos.x, chunkPos.z, true, false);
|
|
+
|
|
+ if (successPoi == Boolean.FALSE || successChunk == Boolean.FALSE) {
|
|
+ return false;
|
|
+ }
|
|
+
|
|
+ // null indicates no task existed, which means our write completed before we waited on it
|
|
+
|
|
+ return true;
|
|
+ } // Paper
|
|
+ }
|
|
+ // Paper end
|
|
+
|
|
public boolean saveChunk(IChunkAccess ichunkaccess) {
|
|
- this.m.a(ichunkaccess.getPos());
|
|
+ // Paper start - async param
|
|
+ return this.saveChunk(ichunkaccess, true);
|
|
+ }
|
|
+ public boolean saveChunk(IChunkAccess ichunkaccess, boolean async) {
|
|
+ try (co.aikar.timings.Timing ignored = this.world.timings.chunkSave.startTiming()) {
|
|
+ NBTTagCompound poiData = this.getVillagePlace().getData(ichunkaccess.getPos()); // Paper
|
|
+ //this.m.a(ichunkaccess.getPos()); // Delay
|
|
+ // Paper end
|
|
if (!ichunkaccess.isNeedsSaving()) {
|
|
return false;
|
|
} else {
|
|
- try {
|
|
- this.world.checkSession();
|
|
- } catch (ExceptionWorldConflict exceptionworldconflict) {
|
|
- PlayerChunkMap.LOGGER.error("Couldn't save chunk; already in use by another instance of Minecraft?", exceptionworldconflict);
|
|
- com.destroystokyo.paper.exception.ServerInternalException.reportInternalException(exceptionworldconflict); // Paper
|
|
- return false;
|
|
- }
|
|
+ // Paper - The save session check is performed on the IO thread
|
|
|
|
ichunkaccess.setLastSaved(this.world.getTime());
|
|
ichunkaccess.setNeedsSaving(false);
|
|
@@ -844,28 +961,35 @@ public class PlayerChunkMap extends IChunkLoader implements PlayerChunk.d {
|
|
NBTTagCompound nbttagcompound;
|
|
|
|
if (chunkstatus.getType() != ChunkStatus.Type.LEVELCHUNK) {
|
|
+ try (co.aikar.timings.Timing ignored1 = this.world.timings.chunkSaveOverwriteCheck.startTiming()) { // Paper
|
|
// Paper start - Optimize save by using status cache
|
|
ChunkStatus statusOnDisk = this.getChunkStatusOnDisk(chunkcoordintpair);
|
|
if (statusOnDisk != null && statusOnDisk.getType() == ChunkStatus.Type.LEVELCHUNK) {
|
|
// Paper end
|
|
+ this.writeDataAsync(ichunkaccess.getPos(), poiData, null, async); // Paper - Async chunk io
|
|
return false;
|
|
}
|
|
|
|
if (chunkstatus == ChunkStatus.EMPTY && ichunkaccess.h().values().stream().noneMatch(StructureStart::e)) {
|
|
+ this.writeDataAsync(ichunkaccess.getPos(), poiData, null, async); // Paper - Async chunk io
|
|
return false;
|
|
}
|
|
}
|
|
|
|
this.world.getMethodProfiler().c("chunkSave");
|
|
+ } // Paper
|
|
+ try (co.aikar.timings.Timing ignored1 = this.world.timings.chunkSaveDataSerialization.startTiming()) { // Paper
|
|
nbttagcompound = ChunkRegionLoader.saveChunk(this.world, ichunkaccess);
|
|
- this.a(chunkcoordintpair, nbttagcompound);
|
|
- return true;
|
|
+ } // Paper
|
|
+ return this.writeDataAsync(ichunkaccess.getPos(), poiData, nbttagcompound, async); // Paper - Async chunk io
|
|
+ //return true; // Paper
|
|
} catch (Exception exception) {
|
|
PlayerChunkMap.LOGGER.error("Failed to save chunk {},{}", chunkcoordintpair.x, chunkcoordintpair.z, exception);
|
|
com.destroystokyo.paper.exception.ServerInternalException.reportInternalException(exception); // Paper
|
|
return false;
|
|
}
|
|
}
|
|
+ } // Paper
|
|
}
|
|
|
|
protected void setViewDistance(int i) {
|
|
@@ -969,6 +1093,42 @@ public class PlayerChunkMap extends IChunkLoader implements PlayerChunk.d {
|
|
}
|
|
}
|
|
|
|
+ // Paper start - Asynchronous chunk io
|
|
+ @Nullable
|
|
+ @Override
|
|
+ public NBTTagCompound read(ChunkCoordIntPair chunkcoordintpair) throws IOException {
|
|
+ if (Thread.currentThread() != com.destroystokyo.paper.io.PaperFileIOThread.Holder.INSTANCE) {
|
|
+ NBTTagCompound ret = com.destroystokyo.paper.io.PaperFileIOThread.Holder.INSTANCE
|
|
+ .loadChunkDataAsyncFuture(this.world, chunkcoordintpair.x, chunkcoordintpair.z, com.destroystokyo.paper.io.IOUtil.getPriorityForCurrentThread(),
|
|
+ false, true, true).join().chunkData;
|
|
+
|
|
+ if (ret == com.destroystokyo.paper.io.PaperFileIOThread.FAILURE_VALUE) {
|
|
+ throw new IOException("See logs for further detail");
|
|
+ }
|
|
+ return ret;
|
|
+ }
|
|
+ return super.read(chunkcoordintpair);
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public void write(ChunkCoordIntPair chunkcoordintpair, NBTTagCompound nbttagcompound) throws IOException {
|
|
+ if (Thread.currentThread() != com.destroystokyo.paper.io.PaperFileIOThread.Holder.INSTANCE) {
|
|
+ com.destroystokyo.paper.io.PaperFileIOThread.Holder.INSTANCE.scheduleSave(
|
|
+ this.world, chunkcoordintpair.x, chunkcoordintpair.z, null, nbttagcompound,
|
|
+ com.destroystokyo.paper.io.IOUtil.getPriorityForCurrentThread());
|
|
+
|
|
+ Boolean ret = com.destroystokyo.paper.io.PaperFileIOThread.Holder.INSTANCE.waitForIOToComplete(this.world,
|
|
+ chunkcoordintpair.x, chunkcoordintpair.z, true, false);
|
|
+
|
|
+ if (ret == Boolean.FALSE) {
|
|
+ throw new IOException("See logs for further detail");
|
|
+ }
|
|
+ return;
|
|
+ }
|
|
+ super.write(chunkcoordintpair, nbttagcompound);
|
|
+ }
|
|
+ // Paper end
|
|
+
|
|
@Nullable
|
|
public NBTTagCompound readChunkData(ChunkCoordIntPair chunkcoordintpair) throws IOException { // Paper - private -> public
|
|
NBTTagCompound nbttagcompound = this.read(chunkcoordintpair);
|
|
@@ -991,33 +1151,55 @@ public class PlayerChunkMap extends IChunkLoader implements PlayerChunk.d {
|
|
|
|
// Paper start - chunk status cache "api"
|
|
public ChunkStatus getChunkStatusOnDiskIfCached(ChunkCoordIntPair chunkPos) {
|
|
- RegionFile regionFile = this.getIOWorker().getRegionFileCache().getRegionFileIfLoaded(chunkPos);
|
|
+ synchronized (this) { // Paper
|
|
+ RegionFile regionFile = this.getRegionFileIfLoaded(chunkPos);
|
|
|
|
return regionFile == null ? null : regionFile.getStatusIfCached(chunkPos.x, chunkPos.z);
|
|
+ } // Paper
|
|
}
|
|
|
|
public ChunkStatus getChunkStatusOnDisk(ChunkCoordIntPair chunkPos) throws IOException {
|
|
- RegionFile regionFile = this.getIOWorker().getRegionFileCache().getFile(chunkPos, false);
|
|
+ // Paper start - async chunk save for unload
|
|
+ IChunkAccess unloadingChunk = this.world.asyncChunkTaskManager.getChunkInSaveProgress(chunkPos.x, chunkPos.z);
|
|
+ if (unloadingChunk != null) {
|
|
+ return unloadingChunk.getChunkStatus();
|
|
+ }
|
|
+ // Paper end
|
|
+ // Paper start - async io
|
|
+ NBTTagCompound inProgressWrite = com.destroystokyo.paper.io.PaperFileIOThread.Holder.INSTANCE
|
|
+ .getPendingWrite(this.world, chunkPos.x, chunkPos.z, false);
|
|
|
|
- if (!regionFile.chunkExists(chunkPos)) {
|
|
- return null;
|
|
+ if (inProgressWrite != null) {
|
|
+ return ChunkRegionLoader.getStatus(inProgressWrite);
|
|
}
|
|
+ // Paper end
|
|
+ synchronized (this) { // Paper - async io
|
|
+ RegionFile regionFile = this.getFile(chunkPos, false);
|
|
+
|
|
+ if (!regionFile.chunkExists(chunkPos)) {
|
|
+ return null;
|
|
+ }
|
|
|
|
- ChunkStatus status = regionFile.getStatusIfCached(chunkPos.x, chunkPos.z);
|
|
+ ChunkStatus status = regionFile.getStatusIfCached(chunkPos.x, chunkPos.z);
|
|
|
|
- if (status != null) {
|
|
- return status;
|
|
+ if (status != null) {
|
|
+ return status;
|
|
+ }
|
|
+ // Paper start - async io
|
|
}
|
|
|
|
- this.readChunkData(chunkPos);
|
|
+ NBTTagCompound compound = this.readChunkData(chunkPos);
|
|
|
|
- return regionFile.getStatusIfCached(chunkPos.x, chunkPos.z);
|
|
+ return ChunkRegionLoader.getStatus(compound);
|
|
+ // Paper end
|
|
}
|
|
|
|
public void updateChunkStatusOnDisk(ChunkCoordIntPair chunkPos, @Nullable NBTTagCompound compound) throws IOException {
|
|
- RegionFile regionFile = this.getIOWorker().getRegionFileCache().getFile(chunkPos, false);
|
|
+ synchronized (this) {
|
|
+ RegionFile regionFile = this.getFile(chunkPos, false);
|
|
|
|
- regionFile.setStatus(chunkPos.x, chunkPos.z, ChunkRegionLoader.getStatus(compound));
|
|
+ regionFile.setStatus(chunkPos.x, chunkPos.z, ChunkRegionLoader.getStatus(compound));
|
|
+ }
|
|
}
|
|
|
|
public IChunkAccess getUnloadingChunk(int chunkX, int chunkZ) {
|
|
@@ -1026,6 +1208,39 @@ public class PlayerChunkMap extends IChunkLoader implements PlayerChunk.d {
|
|
}
|
|
// Paper end
|
|
|
|
+
|
|
+ // Paper start - async io
|
|
+ // this function will not load chunk data off disk to check for status
|
|
+ // ret null for unknown, empty for empty status on disk or absent from disk
|
|
+ public ChunkStatus getStatusOnDiskNoLoad(int x, int z) {
|
|
+ // Paper start - async chunk save for unload
|
|
+ IChunkAccess unloadingChunk = this.world.asyncChunkTaskManager.getChunkInSaveProgress(x, z);
|
|
+ if (unloadingChunk != null) {
|
|
+ return unloadingChunk.getChunkStatus();
|
|
+ }
|
|
+ // Paper end
|
|
+ // Paper start - async io
|
|
+ net.minecraft.server.NBTTagCompound inProgressWrite = com.destroystokyo.paper.io.PaperFileIOThread.Holder.INSTANCE
|
|
+ .getPendingWrite(this.world, x, z, false);
|
|
+
|
|
+ if (inProgressWrite != null) {
|
|
+ return net.minecraft.server.ChunkRegionLoader.getStatus(inProgressWrite);
|
|
+ }
|
|
+ // Paper end
|
|
+ // variant of PlayerChunkMap#getChunkStatusOnDisk that does not load data off disk, but loads the region file
|
|
+ ChunkCoordIntPair chunkPos = new ChunkCoordIntPair(x, z);
|
|
+ synchronized (world.getChunkProvider().playerChunkMap) {
|
|
+ net.minecraft.server.RegionFile file;
|
|
+ try {
|
|
+ file = world.getChunkProvider().playerChunkMap.getFile(chunkPos, false);
|
|
+ } catch (IOException ex) {
|
|
+ throw new RuntimeException(ex);
|
|
+ }
|
|
+
|
|
+ return !file.chunkExists(chunkPos) ? ChunkStatus.EMPTY : file.getStatusIfCached(x, z);
|
|
+ }
|
|
+ }
|
|
+
|
|
boolean isOutsideOfRange(ChunkCoordIntPair chunkcoordintpair) {
|
|
// Spigot start
|
|
return isOutsideOfRange(chunkcoordintpair, false);
|
|
@@ -1371,6 +1586,7 @@ public class PlayerChunkMap extends IChunkLoader implements PlayerChunk.d {
|
|
|
|
}
|
|
|
|
+ public VillagePlace getVillagePlace() { return this.h(); } // Paper - OBFHELPER
|
|
protected VillagePlace h() {
|
|
return this.m;
|
|
}
|
|
diff --git a/src/main/java/net/minecraft/server/RegionFile.java b/src/main/java/net/minecraft/server/RegionFile.java
|
|
index d37abf2cf..df728e2c0 100644
|
|
--- a/src/main/java/net/minecraft/server/RegionFile.java
|
|
+++ b/src/main/java/net/minecraft/server/RegionFile.java
|
|
@@ -36,6 +36,8 @@ public class RegionFile implements AutoCloseable {
|
|
private final RegionFileBitSet freeSectors;
|
|
public final File file;
|
|
|
|
+ public final java.util.concurrent.locks.ReentrantLock fileLock = new java.util.concurrent.locks.ReentrantLock(true); // Paper
|
|
+
|
|
// Paper start - Cache chunk status
|
|
private final ChunkStatus[] statuses = new ChunkStatus[32 * 32];
|
|
|
|
@@ -224,7 +226,7 @@ public class RegionFile implements AutoCloseable {
|
|
return (i + 4096 - 1) / 4096;
|
|
}
|
|
|
|
- public boolean b(ChunkCoordIntPair chunkcoordintpair) {
|
|
+ public synchronized boolean b(ChunkCoordIntPair chunkcoordintpair) { // Paper - synchronized
|
|
int i = this.getOffset(chunkcoordintpair);
|
|
|
|
if (i == 0) {
|
|
@@ -380,6 +382,11 @@ public class RegionFile implements AutoCloseable {
|
|
}
|
|
|
|
public void close() throws IOException {
|
|
+ // Paper start - Prevent regionfiles from being closed during use
|
|
+ this.fileLock.lock();
|
|
+ synchronized (this) {
|
|
+ try {
|
|
+ // Paper end
|
|
this.closed = true; // Paper
|
|
try {
|
|
this.c();
|
|
@@ -394,6 +401,10 @@ public class RegionFile implements AutoCloseable {
|
|
}
|
|
}
|
|
}
|
|
+ } finally { // Paper start - Prevent regionfiles from being closed during use
|
|
+ this.fileLock.unlock();
|
|
+ }
|
|
+ } // Paper end
|
|
|
|
}
|
|
|
|
diff --git a/src/main/java/net/minecraft/server/RegionFileCache.java b/src/main/java/net/minecraft/server/RegionFileCache.java
|
|
index e07ae9854..0f201000f 100644
|
|
--- a/src/main/java/net/minecraft/server/RegionFileCache.java
|
|
+++ b/src/main/java/net/minecraft/server/RegionFileCache.java
|
|
@@ -9,7 +9,7 @@ import java.io.File;
|
|
import java.io.IOException;
|
|
import javax.annotation.Nullable;
|
|
|
|
-public final class RegionFileCache implements AutoCloseable {
|
|
+public class RegionFileCache implements AutoCloseable { // Paper - no final
|
|
|
|
public final Long2ObjectLinkedOpenHashMap<RegionFile> cache = new Long2ObjectLinkedOpenHashMap();
|
|
private final File b;
|
|
@@ -20,16 +20,27 @@ public final class RegionFileCache implements AutoCloseable {
|
|
|
|
|
|
// Paper start
|
|
- public RegionFile getRegionFileIfLoaded(ChunkCoordIntPair chunkcoordintpair) {
|
|
+ public synchronized RegionFile getRegionFileIfLoaded(ChunkCoordIntPair chunkcoordintpair) { // Paper - synchronize for async io
|
|
return this.cache.getAndMoveToFirst(ChunkCoordIntPair.pair(chunkcoordintpair.getRegionX(), chunkcoordintpair.getRegionZ()));
|
|
}
|
|
|
|
// Paper end
|
|
- public RegionFile getFile(ChunkCoordIntPair chunkcoordintpair, boolean existingOnly) throws IOException { // CraftBukkit // Paper - private > public
|
|
+ public synchronized RegionFile getFile(ChunkCoordIntPair chunkcoordintpair, boolean existingOnly) throws IOException { // CraftBukkit // Paper - private > public, synchronize
|
|
+ // Paper start - add lock parameter
|
|
+ return this.getFile(chunkcoordintpair, existingOnly, false);
|
|
+ }
|
|
+ public synchronized RegionFile getFile(ChunkCoordIntPair chunkcoordintpair, boolean existingOnly, boolean lock) throws IOException {
|
|
+ // Paper end
|
|
long i = ChunkCoordIntPair.pair(chunkcoordintpair.getRegionX(), chunkcoordintpair.getRegionZ());
|
|
RegionFile regionfile = (RegionFile) this.cache.getAndMoveToFirst(i);
|
|
|
|
if (regionfile != null) {
|
|
+ // Paper start
|
|
+ if (lock) {
|
|
+ // must be in this synchronized block
|
|
+ regionfile.fileLock.lock();
|
|
+ }
|
|
+ // Paper end
|
|
return regionfile;
|
|
} else {
|
|
if (this.cache.size() >= com.destroystokyo.paper.PaperConfig.regionFileCacheSize) { // Paper - configurable
|
|
@@ -45,6 +56,12 @@ public final class RegionFileCache implements AutoCloseable {
|
|
RegionFile regionfile1 = new RegionFile(file, this.b);
|
|
|
|
this.cache.putAndMoveToFirst(i, regionfile1);
|
|
+ // Paper start
|
|
+ if (lock) {
|
|
+ // must be in this synchronized block
|
|
+ regionfile1.fileLock.lock();
|
|
+ }
|
|
+ // Paper end
|
|
return regionfile1;
|
|
}
|
|
}
|
|
@@ -119,7 +136,8 @@ public final class RegionFileCache implements AutoCloseable {
|
|
|
|
@Nullable
|
|
public NBTTagCompound read(ChunkCoordIntPair chunkcoordintpair) throws IOException {
|
|
- RegionFile regionfile = this.getFile(chunkcoordintpair, false); // CraftBukkit
|
|
+ RegionFile regionfile = this.getFile(chunkcoordintpair, false, true); // CraftBukkit // Paper
|
|
+ try { // Paper
|
|
DataInputStream datainputstream = regionfile.a(chunkcoordintpair);
|
|
// Paper start
|
|
if (regionfile.isOversized(chunkcoordintpair.x, chunkcoordintpair.z)) {
|
|
@@ -157,10 +175,14 @@ public final class RegionFileCache implements AutoCloseable {
|
|
}
|
|
|
|
return nbttagcompound;
|
|
+ } finally { // Paper start
|
|
+ regionfile.fileLock.unlock();
|
|
+ } // Paper end
|
|
}
|
|
|
|
protected void write(ChunkCoordIntPair chunkcoordintpair, NBTTagCompound nbttagcompound) throws IOException {
|
|
- RegionFile regionfile = this.getFile(chunkcoordintpair, false); // CraftBukkit
|
|
+ RegionFile regionfile = this.getFile(chunkcoordintpair, false, true); // CraftBukkit // Paper
|
|
+ try { // Paper
|
|
int attempts = 0; Exception laste = null; while (attempts++ < 5) { try { // Paper
|
|
DataOutputStream dataoutputstream = regionfile.c(chunkcoordintpair);
|
|
Throwable throwable = null;
|
|
@@ -199,9 +221,12 @@ public final class RegionFileCache implements AutoCloseable {
|
|
MinecraftServer.LOGGER.error("Failed to save chunk", laste);
|
|
}
|
|
// Paper end
|
|
+ } finally { // Paper start
|
|
+ regionfile.fileLock.unlock();
|
|
+ } // Paper end
|
|
}
|
|
|
|
- public void close() throws IOException {
|
|
+ public synchronized void close() throws IOException { // Paper -> synchronized
|
|
ObjectIterator objectiterator = this.cache.values().iterator();
|
|
|
|
while (objectiterator.hasNext()) {
|
|
@@ -213,7 +238,7 @@ public final class RegionFileCache implements AutoCloseable {
|
|
}
|
|
|
|
// CraftBukkit start
|
|
- public boolean chunkExists(ChunkCoordIntPair pos) throws IOException {
|
|
+ public synchronized boolean chunkExists(ChunkCoordIntPair pos) throws IOException { // Paper - synchronize
|
|
RegionFile regionfile = getFile(pos, true);
|
|
|
|
return regionfile != null ? regionfile.chunkExists(pos) : false;
|
|
diff --git a/src/main/java/net/minecraft/server/RegionFileSection.java b/src/main/java/net/minecraft/server/RegionFileSection.java
|
|
index db9f0196b..a6d8ef5eb 100644
|
|
--- a/src/main/java/net/minecraft/server/RegionFileSection.java
|
|
+++ b/src/main/java/net/minecraft/server/RegionFileSection.java
|
|
@@ -20,28 +20,29 @@ import javax.annotation.Nullable;
|
|
import org.apache.logging.log4j.LogManager;
|
|
import org.apache.logging.log4j.Logger;
|
|
|
|
-public class RegionFileSection<R extends MinecraftSerializable> implements AutoCloseable {
|
|
+public class RegionFileSection<R extends MinecraftSerializable> extends RegionFileCache implements AutoCloseable { // Paper - nuke IOWorker
|
|
|
|
private static final Logger LOGGER = LogManager.getLogger();
|
|
- private final IOWorker b;
|
|
+// private final IOWorker b;
|
|
private final Long2ObjectMap<Optional<R>> c = new Long2ObjectOpenHashMap();
|
|
- private final LongLinkedOpenHashSet d = new LongLinkedOpenHashSet();
|
|
+ protected final LongLinkedOpenHashSet d = new LongLinkedOpenHashSet(); // Paper - private -> protected
|
|
private final BiFunction<Runnable, Dynamic<?>, R> e;
|
|
private final Function<Runnable, R> f;
|
|
private final DataFixer g;
|
|
private final DataFixTypes h;
|
|
|
|
public RegionFileSection(File file, BiFunction<Runnable, Dynamic<?>, R> bifunction, Function<Runnable, R> function, DataFixer datafixer, DataFixTypes datafixtypes) {
|
|
+ super(file); // Paper - nuke IOWorker
|
|
this.e = bifunction;
|
|
this.f = function;
|
|
this.g = datafixer;
|
|
this.h = datafixtypes;
|
|
- this.b = new IOWorker(new RegionFileCache(file), file.getName());
|
|
+// this.b = new IOWorker(new RegionFileCache(file), file.getName()); // Paper - nuke IOWorker
|
|
}
|
|
|
|
protected void a(BooleanSupplier booleansupplier) {
|
|
- while (!this.d.isEmpty() && booleansupplier.getAsBoolean()) {
|
|
- ChunkCoordIntPair chunkcoordintpair = SectionPosition.a(this.d.firstLong()).u();
|
|
+ while (!this.d.isEmpty() && booleansupplier.getAsBoolean()) { // Paper - conflict here to avoid obfhelpers
|
|
+ ChunkCoordIntPair chunkcoordintpair = SectionPosition.a(this.d.firstLong()).u(); // Paper - conflict here to avoid obfhelpers
|
|
|
|
this.d(chunkcoordintpair);
|
|
}
|
|
@@ -95,13 +96,18 @@ public class RegionFileSection<R extends MinecraftSerializable> implements AutoC
|
|
}
|
|
|
|
private void b(ChunkCoordIntPair chunkcoordintpair) {
|
|
- this.a(chunkcoordintpair, DynamicOpsNBT.a, this.c(chunkcoordintpair));
|
|
+ // Paper start - load data in function
|
|
+ this.loadInData(chunkcoordintpair, this.c(chunkcoordintpair));
|
|
+ }
|
|
+ public void loadInData(ChunkCoordIntPair chunkPos, NBTTagCompound compound) {
|
|
+ this.a(chunkPos, DynamicOpsNBT.a, compound);
|
|
+ // Paper end
|
|
}
|
|
|
|
@Nullable
|
|
private NBTTagCompound c(ChunkCoordIntPair chunkcoordintpair) {
|
|
try {
|
|
- return this.b.a(chunkcoordintpair);
|
|
+ return this.read(chunkcoordintpair); // Paper - nuke IOWorker
|
|
} catch (IOException ioexception) {
|
|
RegionFileSection.LOGGER.error("Error reading chunk {} data from disk", chunkcoordintpair, ioexception);
|
|
return null;
|
|
@@ -143,17 +149,31 @@ public class RegionFileSection<R extends MinecraftSerializable> implements AutoC
|
|
}
|
|
|
|
private void d(ChunkCoordIntPair chunkcoordintpair) {
|
|
- Dynamic<NBTBase> dynamic = this.a(chunkcoordintpair, DynamicOpsNBT.a);
|
|
+ Dynamic<NBTBase> dynamic = this.a(chunkcoordintpair, DynamicOpsNBT.a); // Paper - conflict here to avoid adding obfhelpers :)
|
|
NBTBase nbtbase = (NBTBase) dynamic.getValue();
|
|
|
|
if (nbtbase instanceof NBTTagCompound) {
|
|
- this.b.a(chunkcoordintpair, (NBTTagCompound) nbtbase);
|
|
+ try { this.write(chunkcoordintpair, (NBTTagCompound) nbtbase); } catch (IOException ioexception) { RegionFileSection.LOGGER.error("Error writing data to disk", ioexception); } // Paper - nuke IOWorker // TODO make this write async
|
|
} else {
|
|
RegionFileSection.LOGGER.error("Expected compound tag, got {}", nbtbase);
|
|
}
|
|
|
|
}
|
|
|
|
+ // Paper start - internal get data function, copied from above
|
|
+ private NBTTagCompound getDataInternal(ChunkCoordIntPair chunkcoordintpair) {
|
|
+ Dynamic<NBTBase> dynamic = this.a(chunkcoordintpair, DynamicOpsNBT.a);
|
|
+ NBTBase nbtbase = (NBTBase) dynamic.getValue();
|
|
+
|
|
+ if (nbtbase instanceof NBTTagCompound) {
|
|
+ return (NBTTagCompound)nbtbase;
|
|
+ } else {
|
|
+ RegionFileSection.LOGGER.error("Expected compound tag, got {}", nbtbase);
|
|
+ }
|
|
+ return null;
|
|
+ }
|
|
+ // Paper end
|
|
+
|
|
private <T> Dynamic<T> a(ChunkCoordIntPair chunkcoordintpair, DynamicOps<T> dynamicops) {
|
|
Map<T, T> map = Maps.newHashMap();
|
|
|
|
@@ -190,9 +210,9 @@ public class RegionFileSection<R extends MinecraftSerializable> implements AutoC
|
|
public void a(ChunkCoordIntPair chunkcoordintpair) {
|
|
if (!this.d.isEmpty()) {
|
|
for (int i = 0; i < 16; ++i) {
|
|
- long j = SectionPosition.a(chunkcoordintpair, i).v();
|
|
+ long j = SectionPosition.a(chunkcoordintpair, i).v(); // Paper - conflict here to avoid obfhelpers
|
|
|
|
- if (this.d.contains(j)) {
|
|
+ if (this.d.contains(j)) { // Paper - conflict here to avoid obfhelpers
|
|
this.d(chunkcoordintpair);
|
|
return;
|
|
}
|
|
@@ -201,7 +221,26 @@ public class RegionFileSection<R extends MinecraftSerializable> implements AutoC
|
|
|
|
}
|
|
|
|
- public void close() throws IOException {
|
|
- this.b.close();
|
|
+// Paper start - nuke IOWorker
|
|
+// public void close() throws IOException {
|
|
+// this.b.close();
|
|
+// }
|
|
+// Paper end
|
|
+
|
|
+ // Paper start - get data function
|
|
+ public NBTTagCompound getData(ChunkCoordIntPair chunkcoordintpair) {
|
|
+ // Note: Copied from above
|
|
+ // This is checking if the data exists, then it builds it later in getDataInternal(ChunkCoordIntPair)
|
|
+ if (!this.d.isEmpty()) {
|
|
+ for (int i = 0; i < 16; ++i) {
|
|
+ long j = SectionPosition.a(chunkcoordintpair, i).v();
|
|
+
|
|
+ if (this.d.contains(j)) {
|
|
+ return this.getDataInternal(chunkcoordintpair);
|
|
+ }
|
|
+ }
|
|
+ }
|
|
+ return null;
|
|
}
|
|
+ // Paper end
|
|
}
|
|
diff --git a/src/main/java/net/minecraft/server/TicketType.java b/src/main/java/net/minecraft/server/TicketType.java
|
|
index 1d1b267f3..4b87ca2ec 100644
|
|
--- a/src/main/java/net/minecraft/server/TicketType.java
|
|
+++ b/src/main/java/net/minecraft/server/TicketType.java
|
|
@@ -22,6 +22,7 @@ public class TicketType<T> {
|
|
public static final TicketType<Unit> PLUGIN = a("plugin", (a, b) -> 0); // CraftBukkit
|
|
public static final TicketType<org.bukkit.plugin.Plugin> PLUGIN_TICKET = a("plugin_ticket", (plugin1, plugin2) -> plugin1.getClass().getName().compareTo(plugin2.getClass().getName())); // CraftBukkit
|
|
public static final TicketType<Integer> ANTIXRAY = a("antixray", Integer::compareTo); // Paper - Anti-Xray
|
|
+ public static final TicketType<Long> ASYNC_LOAD = a("async_load", Long::compareTo); // Paper
|
|
|
|
public static <T> TicketType<T> a(String s, Comparator<T> comparator) {
|
|
return new TicketType<>(s, comparator, 0L);
|
|
diff --git a/src/main/java/net/minecraft/server/VillagePlace.java b/src/main/java/net/minecraft/server/VillagePlace.java
|
|
index c999f8c9b..b59ef1a63 100644
|
|
--- a/src/main/java/net/minecraft/server/VillagePlace.java
|
|
+++ b/src/main/java/net/minecraft/server/VillagePlace.java
|
|
@@ -24,8 +24,16 @@ public class VillagePlace extends RegionFileSection<VillagePlaceSection> {
|
|
private final VillagePlace.a a = new VillagePlace.a();
|
|
private final LongSet b = new LongOpenHashSet();
|
|
|
|
+ private final WorldServer world; // Paper
|
|
+
|
|
public VillagePlace(File file, DataFixer datafixer) {
|
|
+ // Paper start
|
|
+ this(file, datafixer, null);
|
|
+ }
|
|
+ public VillagePlace(File file, DataFixer datafixer, WorldServer world) {
|
|
+ // Paper end
|
|
super(file, VillagePlaceSection::new, VillagePlaceSection::new, datafixer, DataFixTypes.POI_CHUNK);
|
|
+ this.world = world; // Paper
|
|
}
|
|
|
|
public void a(BlockPosition blockposition, VillagePlaceType villageplacetype) {
|
|
@@ -129,7 +137,23 @@ public class VillagePlace extends RegionFileSection<VillagePlaceSection> {
|
|
|
|
@Override
|
|
public void a(BooleanSupplier booleansupplier) {
|
|
- super.a(booleansupplier);
|
|
+ // Paper start - async chunk io
|
|
+ if (this.world == null) {
|
|
+ super.a(booleansupplier);
|
|
+ } else {
|
|
+ //super.a(booleansupplier); // re-implement below
|
|
+ while (!((RegionFileSection)this).d.isEmpty() && booleansupplier.getAsBoolean()) {
|
|
+ ChunkCoordIntPair chunkcoordintpair = SectionPosition.a(((RegionFileSection)this).d.firstLong()).u();
|
|
+
|
|
+ NBTTagCompound data;
|
|
+ try (co.aikar.timings.Timing ignored1 = this.world.timings.poiSaveDataSerialization.startTiming()) {
|
|
+ data = this.getData(chunkcoordintpair);
|
|
+ }
|
|
+ com.destroystokyo.paper.io.PaperFileIOThread.Holder.INSTANCE.scheduleSave(this.world,
|
|
+ chunkcoordintpair.x, chunkcoordintpair.z, data, null, com.destroystokyo.paper.io.PrioritizedTaskQueue.LOW_PRIORITY);
|
|
+ }
|
|
+ }
|
|
+ // Paper end
|
|
this.a.a();
|
|
}
|
|
|
|
@@ -229,6 +253,42 @@ public class VillagePlace extends RegionFileSection<VillagePlaceSection> {
|
|
}
|
|
}
|
|
|
|
+ // Paper start - Asynchronous chunk io
|
|
+ @javax.annotation.Nullable
|
|
+ @Override
|
|
+ public NBTTagCompound read(ChunkCoordIntPair chunkcoordintpair) throws java.io.IOException {
|
|
+ if (this.world != null && Thread.currentThread() != com.destroystokyo.paper.io.PaperFileIOThread.Holder.INSTANCE) {
|
|
+ NBTTagCompound ret = com.destroystokyo.paper.io.PaperFileIOThread.Holder.INSTANCE
|
|
+ .loadChunkDataAsyncFuture(this.world, chunkcoordintpair.x, chunkcoordintpair.z, com.destroystokyo.paper.io.IOUtil.getPriorityForCurrentThread(),
|
|
+ true, false, true).join().poiData;
|
|
+
|
|
+ if (ret == com.destroystokyo.paper.io.PaperFileIOThread.FAILURE_VALUE) {
|
|
+ throw new java.io.IOException("See logs for further detail");
|
|
+ }
|
|
+ return ret;
|
|
+ }
|
|
+ return super.read(chunkcoordintpair);
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public void write(ChunkCoordIntPair chunkcoordintpair, NBTTagCompound nbttagcompound) throws java.io.IOException {
|
|
+ if (this.world != null && Thread.currentThread() != com.destroystokyo.paper.io.PaperFileIOThread.Holder.INSTANCE) {
|
|
+ com.destroystokyo.paper.io.PaperFileIOThread.Holder.INSTANCE.scheduleSave(
|
|
+ this.world, chunkcoordintpair.x, chunkcoordintpair.z, nbttagcompound, null,
|
|
+ com.destroystokyo.paper.io.IOUtil.getPriorityForCurrentThread());
|
|
+
|
|
+ Boolean ret = com.destroystokyo.paper.io.PaperFileIOThread.Holder.INSTANCE.waitForIOToComplete(this.world,
|
|
+ chunkcoordintpair.x, chunkcoordintpair.z, true, true);
|
|
+
|
|
+ if (ret == Boolean.FALSE) {
|
|
+ throw new java.io.IOException("See logs for further detail");
|
|
+ }
|
|
+ return;
|
|
+ }
|
|
+ super.write(chunkcoordintpair, nbttagcompound);
|
|
+ }
|
|
+ // Paper end
|
|
+
|
|
public static enum Occupancy {
|
|
|
|
HAS_SPACE(VillagePlaceRecord::d), IS_OCCUPIED(VillagePlaceRecord::e), ANY((villageplacerecord) -> {
|
|
diff --git a/src/main/java/net/minecraft/server/WorldServer.java b/src/main/java/net/minecraft/server/WorldServer.java
|
|
index a79ccad0b..6634348e4 100644
|
|
--- a/src/main/java/net/minecraft/server/WorldServer.java
|
|
+++ b/src/main/java/net/minecraft/server/WorldServer.java
|
|
@@ -82,6 +82,79 @@ public class WorldServer extends World {
|
|
return new Throwable(entity + " Added to world at " + new java.util.Date());
|
|
}
|
|
|
|
+ // Paper start - Asynchronous IO
|
|
+ public final com.destroystokyo.paper.io.PaperFileIOThread.ChunkDataController poiDataController = new com.destroystokyo.paper.io.PaperFileIOThread.ChunkDataController() {
|
|
+ @Override
|
|
+ public void writeData(int x, int z, NBTTagCompound compound) throws java.io.IOException {
|
|
+ WorldServer.this.getChunkProvider().playerChunkMap.getVillagePlace().write(new ChunkCoordIntPair(x, z), compound);
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public NBTTagCompound readData(int x, int z) throws java.io.IOException {
|
|
+ return WorldServer.this.getChunkProvider().playerChunkMap.getVillagePlace().read(new ChunkCoordIntPair(x, z));
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public <T> T computeForRegionFile(int chunkX, int chunkZ, java.util.function.Function<RegionFile, T> function) {
|
|
+ synchronized (WorldServer.this.getChunkProvider().playerChunkMap.getVillagePlace()) {
|
|
+ RegionFile file;
|
|
+
|
|
+ try {
|
|
+ file = WorldServer.this.getChunkProvider().playerChunkMap.getVillagePlace().getFile(new ChunkCoordIntPair(chunkX, chunkZ), false);
|
|
+ } catch (java.io.IOException ex) {
|
|
+ throw new RuntimeException(ex);
|
|
+ }
|
|
+
|
|
+ return function.apply(file);
|
|
+ }
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public <T> T computeForRegionFileIfLoaded(int chunkX, int chunkZ, java.util.function.Function<RegionFile, T> function) {
|
|
+ synchronized (WorldServer.this.getChunkProvider().playerChunkMap.getVillagePlace()) {
|
|
+ RegionFile file = WorldServer.this.getChunkProvider().playerChunkMap.getVillagePlace().getRegionFileIfLoaded(new ChunkCoordIntPair(chunkX, chunkZ));
|
|
+ return function.apply(file);
|
|
+ }
|
|
+ }
|
|
+ };
|
|
+
|
|
+ public final com.destroystokyo.paper.io.PaperFileIOThread.ChunkDataController chunkDataController = new com.destroystokyo.paper.io.PaperFileIOThread.ChunkDataController() {
|
|
+ @Override
|
|
+ public void writeData(int x, int z, NBTTagCompound compound) throws java.io.IOException {
|
|
+ WorldServer.this.getChunkProvider().playerChunkMap.write(new ChunkCoordIntPair(x, z), compound);
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public NBTTagCompound readData(int x, int z) throws java.io.IOException {
|
|
+ return WorldServer.this.getChunkProvider().playerChunkMap.read(new ChunkCoordIntPair(x, z));
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public <T> T computeForRegionFile(int chunkX, int chunkZ, java.util.function.Function<RegionFile, T> function) {
|
|
+ synchronized (WorldServer.this.getChunkProvider().playerChunkMap) {
|
|
+ RegionFile file;
|
|
+
|
|
+ try {
|
|
+ file = WorldServer.this.getChunkProvider().playerChunkMap.getFile(new ChunkCoordIntPair(chunkX, chunkZ), false);
|
|
+ } catch (java.io.IOException ex) {
|
|
+ throw new RuntimeException(ex);
|
|
+ }
|
|
+
|
|
+ return function.apply(file);
|
|
+ }
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public <T> T computeForRegionFileIfLoaded(int chunkX, int chunkZ, java.util.function.Function<RegionFile, T> function) {
|
|
+ synchronized (WorldServer.this.getChunkProvider().playerChunkMap) {
|
|
+ RegionFile file = WorldServer.this.getChunkProvider().playerChunkMap.getRegionFileIfLoaded(new ChunkCoordIntPair(chunkX, chunkZ));
|
|
+ return function.apply(file);
|
|
+ }
|
|
+ }
|
|
+ };
|
|
+ public final com.destroystokyo.paper.io.chunk.ChunkTaskManager asyncChunkTaskManager;
|
|
+ // Paper end
|
|
+
|
|
// Add env and gen to constructor
|
|
public WorldServer(MinecraftServer minecraftserver, Executor executor, WorldNBTStorage worldnbtstorage, WorldData worlddata, DimensionManager dimensionmanager, GameProfilerFiller gameprofilerfiller, WorldLoadListener worldloadlistener, org.bukkit.World.Environment env, org.bukkit.generator.ChunkGenerator gen) {
|
|
super(worlddata, dimensionmanager, (world, worldprovider) -> {
|
|
@@ -125,6 +198,8 @@ public class WorldServer extends World {
|
|
|
|
this.mobSpawnerTrader = this.worldProvider.getDimensionManager().getType() == DimensionManager.OVERWORLD ? new MobSpawnerTrader(this) : null; // CraftBukkit - getType()
|
|
this.getServer().addWorld(this.getWorld()); // CraftBukkit
|
|
+
|
|
+ this.asyncChunkTaskManager = new com.destroystokyo.paper.io.chunk.ChunkTaskManager(this); // Paper
|
|
}
|
|
|
|
// CraftBukkit start
|
|
diff --git a/src/main/java/org/bukkit/craftbukkit/CraftWorld.java b/src/main/java/org/bukkit/craftbukkit/CraftWorld.java
|
|
index a71bb8650..1d275520f 100644
|
|
--- a/src/main/java/org/bukkit/craftbukkit/CraftWorld.java
|
|
+++ b/src/main/java/org/bukkit/craftbukkit/CraftWorld.java
|
|
@@ -554,22 +554,23 @@ public class CraftWorld implements World {
|
|
return true;
|
|
}
|
|
|
|
- net.minecraft.server.RegionFile file;
|
|
- try {
|
|
- file = world.getChunkProvider().playerChunkMap.getIOWorker().getRegionFileCache().getFile(chunkPos, false);
|
|
- } catch (IOException ex) {
|
|
- throw new RuntimeException(ex);
|
|
- }
|
|
+ ChunkStatus status = world.getChunkProvider().playerChunkMap.getStatusOnDiskNoLoad(x, z); // Paper - async io - move to own method
|
|
|
|
- ChunkStatus status = file.getStatusIfCached(x, z);
|
|
- if (!file.chunkExists(chunkPos) || (status != null && status != ChunkStatus.FULL)) {
|
|
+ // Paper start - async io
|
|
+ if (status == ChunkStatus.EMPTY) {
|
|
+ // does not exist on disk
|
|
return false;
|
|
}
|
|
|
|
+ if (status == null) { // at this stage we don't know what it is on disk
|
|
IChunkAccess chunk = world.getChunkProvider().getChunkAt(x, z, ChunkStatus.EMPTY, true);
|
|
if (!(chunk instanceof ProtoChunkExtension) && !(chunk instanceof net.minecraft.server.Chunk)) {
|
|
return false;
|
|
}
|
|
+ } else if (status != ChunkStatus.FULL) {
|
|
+ return false; // not full status on disk
|
|
+ }
|
|
+ // Paper end
|
|
|
|
// fall through to load
|
|
// we do this so we do not re-read the chunk data on disk
|
|
@@ -2439,6 +2440,24 @@ public class CraftWorld implements World {
|
|
|
|
return new CraftDragonBattle(((WorldProviderTheEnd) worldProvider).o()); // PAIL rename getDragonBattle
|
|
}
|
|
+ // Paper start
|
|
+ @Override
|
|
+ public CompletableFuture<Chunk> getChunkAtAsync(int x, int z, boolean gen) {
|
|
+ if (Bukkit.isPrimaryThread()) {
|
|
+ net.minecraft.server.Chunk immediate = this.world.getChunkProvider().getChunkAtIfLoadedImmediately(x, z);
|
|
+ if (immediate != null) {
|
|
+ return CompletableFuture.completedFuture(immediate.bukkitChunk);
|
|
+ }
|
|
+ }
|
|
+
|
|
+ CompletableFuture<Chunk> ret = new CompletableFuture<>();
|
|
+ this.world.getChunkProvider().getChunkAtAsynchronously(x, z, gen, (net.minecraft.server.Chunk chunk) -> {
|
|
+ ret.complete(chunk == null ? null : chunk.bukkitChunk);
|
|
+ });
|
|
+
|
|
+ return ret;
|
|
+ }
|
|
+ // Paper end
|
|
|
|
// Spigot start
|
|
@Override
|
|
diff --git a/src/main/java/org/spigotmc/WatchdogThread.java b/src/main/java/org/spigotmc/WatchdogThread.java
|
|
index 07936eeba..5bdcdcf9e 100644
|
|
--- a/src/main/java/org/spigotmc/WatchdogThread.java
|
|
+++ b/src/main/java/org/spigotmc/WatchdogThread.java
|
|
@@ -6,6 +6,7 @@ import java.lang.management.ThreadInfo;
|
|
import java.util.logging.Level;
|
|
import java.util.logging.Logger;
|
|
import com.destroystokyo.paper.PaperConfig;
|
|
+import com.destroystokyo.paper.io.chunk.ChunkTaskManager; // Paper
|
|
import net.minecraft.server.MinecraftServer;
|
|
import org.bukkit.Bukkit;
|
|
|
|
@@ -112,6 +113,7 @@ public class WatchdogThread extends Thread
|
|
// Paper end - Different message for short timeout
|
|
log.log( Level.SEVERE, "------------------------------" );
|
|
log.log( Level.SEVERE, "Server thread dump (Look for plugins here before reporting to Paper!):" ); // Paper
|
|
+ ChunkTaskManager.dumpAllChunkLoadInfo(); // Paper
|
|
dumpThread( ManagementFactory.getThreadMXBean().getThreadInfo( MinecraftServer.getServer().serverThread.getId(), Integer.MAX_VALUE ), log );
|
|
log.log( Level.SEVERE, "------------------------------" );
|
|
//
|
|
--
|
|
2.26.0
|
|
|