mirror of
https://github.com/PaperMC/Paper.git
synced 2024-11-05 10:20:53 +01:00
9038677c89
Upstream has released updates that appears to apply and compile correctly.
This update has not been tested by PaperMC and as with ANY update, please do your own testing
Bukkit Changes:
c71bb9ca Add PlayerRecipeDiscoverEvent and methods to (un/)discover recipes
CraftBukkit Changes:
7a2f4867
Implement PlayerRecipeDiscoverEvent and methods to (un/)discover recipes
1896 lines
83 KiB
Diff
1896 lines
83 KiB
Diff
From 446a8037bfb38116cc712bf89d52dde7cd5aff68 Mon Sep 17 00:00:00 2001
|
|
From: Aikar <aikar@aikar.co>
|
|
Date: Sat, 21 Jul 2018 16:55:04 -0400
|
|
Subject: [PATCH] Async Chunk Loading and Generation
|
|
|
|
This brings back parity to 1.12 and older versions in that any
|
|
chunk requested as part of the PlayerChunkMap can be loaded
|
|
asynchronously, since the chunk isn't needed "immediately".
|
|
|
|
The previous system used by CraftBukkit has been completely abandoned, as
|
|
mojang has put more concurrency checks into the process.
|
|
|
|
The new process is no longer lock free, but tries to maintain locks as
|
|
short as possible.
|
|
|
|
But with 1.13, we now have Chunk Conversions too. A main issue about
|
|
keeping just loading parity to 1.12 is that standard loads now
|
|
are treated as generation level events, to run the converter on
|
|
another thread.
|
|
|
|
However mojangs code was pretty bad here and doesn't actually provide
|
|
any concurrency...
|
|
|
|
Mojangs code is still not thread safe, and can only operate on
|
|
one world per thread safely, but this is still a major improvement
|
|
to get world generation off of the main thread for exploration.
|
|
|
|
This change brings Chunk Requests triggered by the Chunk Map to be
|
|
lazily loaded asynchronously.
|
|
|
|
Standard chunk loads can load in parallel across a shared executor.
|
|
|
|
However, chunk conversions and generations must only run one per world
|
|
at a time, so we have a single thread executor for those operations
|
|
per world, that all of those requests get scheduled to.
|
|
|
|
getChunkAt method is now thread safe, but has not been tested in
|
|
use by other threads for generations, but should be safe to do.
|
|
|
|
However, we are not encouraging plugins to go getting chunks async,
|
|
as while looking the chunk up may be safe, absolutely nothing about
|
|
reading or writing to the chunk will be safe, so plugins still
|
|
should not be touching chunks asynchronously!
|
|
|
|
diff --git a/src/main/java/com/destroystokyo/paper/PaperConfig.java b/src/main/java/com/destroystokyo/paper/PaperConfig.java
|
|
index da710cc6fe..6b65364e22 100644
|
|
--- a/src/main/java/com/destroystokyo/paper/PaperConfig.java
|
|
+++ b/src/main/java/com/destroystokyo/paper/PaperConfig.java
|
|
@@ -372,4 +372,15 @@ public class PaperConfig {
|
|
}
|
|
}
|
|
}
|
|
+
|
|
+ // Temporary opt out, will be removed later
|
|
+ public static boolean asyncChunks = false;
|
|
+ private static void disableAsyncChunks() {
|
|
+ asyncChunks = config.getBoolean("settings.async-chunks", true);
|
|
+ if (!asyncChunks) {
|
|
+ log("Async Chunks: Disabled - Chunks will be managed synchronosuly, and will cause tremendous lag.");
|
|
+ } else {
|
|
+ log("Async Chunks: Enabled - Chunks will be loaded and generated much faster, without lag.");
|
|
+ }
|
|
+ }
|
|
}
|
|
diff --git a/src/main/java/com/destroystokyo/paper/util/PriorityQueuedExecutor.java b/src/main/java/com/destroystokyo/paper/util/PriorityQueuedExecutor.java
|
|
new file mode 100644
|
|
index 0000000000..0a9fd5d662
|
|
--- /dev/null
|
|
+++ b/src/main/java/com/destroystokyo/paper/util/PriorityQueuedExecutor.java
|
|
@@ -0,0 +1,277 @@
|
|
+package com.destroystokyo.paper.util;
|
|
+
|
|
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
+import net.minecraft.server.NamedIncrementingThreadFactory;
|
|
+
|
|
+import javax.annotation.Nonnull;
|
|
+import java.util.ArrayList;
|
|
+import java.util.List;
|
|
+import java.util.concurrent.AbstractExecutorService;
|
|
+import java.util.concurrent.CompletableFuture;
|
|
+import java.util.concurrent.ConcurrentLinkedQueue;
|
|
+import java.util.concurrent.RejectedExecutionException;
|
|
+import java.util.concurrent.ThreadFactory;
|
|
+import java.util.concurrent.TimeUnit;
|
|
+import java.util.concurrent.atomic.AtomicBoolean;
|
|
+import java.util.concurrent.atomic.AtomicInteger;
|
|
+import java.util.function.Supplier;
|
|
+
|
|
+/**
|
|
+ * Implements an Executor Service that allows specifying Task Priority
|
|
+ * and bumping of task priority.
|
|
+ *
|
|
+ * @author aikar
|
|
+ */
|
|
+@SuppressWarnings({"WeakerAccess", "UnusedReturnValue", "unused"})
|
|
+public class PriorityQueuedExecutor extends AbstractExecutorService {
|
|
+ private final ConcurrentLinkedQueue<Runnable> high = new ConcurrentLinkedQueue<>();
|
|
+ private final ConcurrentLinkedQueue<Runnable> normal = new ConcurrentLinkedQueue<>();
|
|
+ private final RejectionHandler handler;
|
|
+ private volatile boolean shuttingDown = false;
|
|
+ private volatile boolean shuttingDownNow = false;
|
|
+ private final List<Thread> threads = new ArrayList<>();
|
|
+
|
|
+ public PriorityQueuedExecutor(String name) {
|
|
+ this(name, Runtime.getRuntime().availableProcessors(), null);
|
|
+ }
|
|
+
|
|
+ public PriorityQueuedExecutor(String name, int threads) {
|
|
+ this(name, threads, null);
|
|
+ }
|
|
+
|
|
+ public PriorityQueuedExecutor(String name, int threads, RejectionHandler handler) {
|
|
+ ThreadFactory factory = new ThreadFactoryBuilder()
|
|
+ .setThreadFactory(new NamedIncrementingThreadFactory(name))
|
|
+ .setDaemon(true)
|
|
+ .build();
|
|
+ for (int i = 0; i < threads; i++) {
|
|
+ final Thread thread = factory.newThread(this::processQueues);
|
|
+ thread.start();
|
|
+ this.threads.add(thread);
|
|
+ }
|
|
+ if (handler == null) {
|
|
+ handler = ABORT_POLICY;
|
|
+ }
|
|
+ this.handler = handler;
|
|
+ }
|
|
+
|
|
+ public void shutdown() {
|
|
+ shuttingDown = true;
|
|
+ synchronized (this) {
|
|
+ this.notifyAll();
|
|
+ }
|
|
+ }
|
|
+
|
|
+ @Nonnull
|
|
+ @Override
|
|
+ public List<Runnable> shutdownNow() {
|
|
+ shuttingDown = true;
|
|
+ shuttingDownNow = true;
|
|
+ List<Runnable> tasks = new ArrayList<>(high.size() + normal.size());
|
|
+ Runnable run;
|
|
+ while ((run = getTask()) != null) {
|
|
+ tasks.add(run);
|
|
+ }
|
|
+
|
|
+ return tasks;
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public boolean isShutdown() {
|
|
+ return shuttingDown;
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public boolean isTerminated() {
|
|
+ if (!shuttingDown) {
|
|
+ return false;
|
|
+ }
|
|
+ return high.isEmpty() && normal.isEmpty();
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public boolean awaitTermination(long timeout, @Nonnull TimeUnit unit) {
|
|
+ synchronized (this) {
|
|
+ this.notifyAll();
|
|
+ }
|
|
+ final long wait = unit.toNanos(timeout);
|
|
+ final long max = System.nanoTime() + wait;
|
|
+ for (;!threads.isEmpty() && System.nanoTime() < max;) {
|
|
+ threads.removeIf(thread -> !thread.isAlive());
|
|
+ }
|
|
+ return isTerminated();
|
|
+ }
|
|
+
|
|
+
|
|
+ public PendingTask<Void> createPendingTask(Runnable task) {
|
|
+ return createPendingTask(task, Priority.NORMAL);
|
|
+ }
|
|
+ public PendingTask<Void> createPendingTask(Runnable task, Priority priority) {
|
|
+ return createPendingTask(() -> {
|
|
+ task.run();
|
|
+ return null;
|
|
+ }, priority);
|
|
+ }
|
|
+
|
|
+ public <T> PendingTask<T> createPendingTask(Supplier<T> task) {
|
|
+ return createPendingTask(task, Priority.NORMAL);
|
|
+ }
|
|
+
|
|
+ public <T> PendingTask<T> createPendingTask(Supplier<T> task, Priority priority) {
|
|
+ return new PendingTask<>(task, priority);
|
|
+ }
|
|
+
|
|
+ public PendingTask<Void> submitTask(Runnable run) {
|
|
+ return submitTask(createPendingTask(run));
|
|
+ }
|
|
+
|
|
+ public PendingTask<Void> submitTask(Runnable run, Priority priority) {
|
|
+ return submitTask(createPendingTask(run, priority));
|
|
+ }
|
|
+
|
|
+ public <T> PendingTask<T> submitTask(Supplier<T> run) {
|
|
+ return submitTask(createPendingTask(run));
|
|
+ }
|
|
+
|
|
+ public <T> PendingTask<T> submitTask(Supplier<T> run, Priority priority) {
|
|
+ return submitTask(createPendingTask(run, priority));
|
|
+ }
|
|
+
|
|
+ public <T> PendingTask<T> submitTask(PendingTask<T> task) {
|
|
+ if (shuttingDown) {
|
|
+ handler.onRejection(task, this);
|
|
+ return task;
|
|
+ }
|
|
+ task.submit(this);
|
|
+ return task;
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public void execute(@Nonnull Runnable command) {
|
|
+ submitTask(command);
|
|
+ }
|
|
+
|
|
+ private Runnable getTask() {
|
|
+ Runnable run = high.poll();
|
|
+ if (run != null) {
|
|
+ return run;
|
|
+ }
|
|
+ return normal.poll();
|
|
+ }
|
|
+
|
|
+ private void processQueues() {
|
|
+ Runnable run = null;
|
|
+ while (true) {
|
|
+ if (run != null) {
|
|
+ run.run();
|
|
+ }
|
|
+ if (shuttingDownNow) {
|
|
+ return;
|
|
+ }
|
|
+ if ((run = getTask()) != null) {
|
|
+ continue;
|
|
+ }
|
|
+ synchronized (PriorityQueuedExecutor.this) {
|
|
+ if ((run = getTask()) != null) {
|
|
+ continue;
|
|
+ }
|
|
+
|
|
+ if (shuttingDown || shuttingDownNow) {
|
|
+ return;
|
|
+ }
|
|
+ try {
|
|
+ PriorityQueuedExecutor.this.wait();
|
|
+ } catch (InterruptedException ignored) {
|
|
+ }
|
|
+ }
|
|
+ }
|
|
+ }
|
|
+
|
|
+ public enum Priority {
|
|
+ NORMAL, HIGH
|
|
+ }
|
|
+
|
|
+ public class PendingTask <T> implements Runnable {
|
|
+
|
|
+ private final AtomicBoolean hasRan = new AtomicBoolean();
|
|
+ private final AtomicInteger submitted = new AtomicInteger(-1);
|
|
+ private final AtomicInteger priority;
|
|
+ private final Supplier<T> run;
|
|
+ private final CompletableFuture<T> future = new CompletableFuture<>();
|
|
+ private volatile PriorityQueuedExecutor executor;
|
|
+
|
|
+ public PendingTask(Supplier<T> run) {
|
|
+ this(run, Priority.NORMAL);
|
|
+ }
|
|
+
|
|
+ public PendingTask(Supplier<T> run, Priority priority) {
|
|
+ this.priority = new AtomicInteger(priority.ordinal());
|
|
+ this.run = run;
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public void run() {
|
|
+ if (!hasRan.compareAndSet(false, true)) {
|
|
+ return;
|
|
+ }
|
|
+
|
|
+ try {
|
|
+ future.complete(run.get());
|
|
+ } catch (Throwable e) {
|
|
+ future.completeExceptionally(e);
|
|
+ }
|
|
+ }
|
|
+
|
|
+ public void bumpPriority() {
|
|
+ if (!priority.compareAndSet(Priority.NORMAL.ordinal(), Priority.HIGH.ordinal())) {
|
|
+ return;
|
|
+ }
|
|
+
|
|
+ if (this.executor == null) {
|
|
+ return;
|
|
+ }
|
|
+ // If we have already been submitted, resubmit with new priority
|
|
+ submit(this.executor);
|
|
+ }
|
|
+
|
|
+ public CompletableFuture<T> onDone() {
|
|
+ return future;
|
|
+ }
|
|
+
|
|
+ public void submit(PriorityQueuedExecutor executor) {
|
|
+ for (;;) {
|
|
+ final int submitted = this.submitted.get();
|
|
+ final int priority = this.priority.get();
|
|
+ if (submitted == priority) {
|
|
+ return;
|
|
+ }
|
|
+ if (this.submitted.compareAndSet(submitted, priority)) {
|
|
+ if (priority == Priority.HIGH.ordinal()) {
|
|
+ high.add(this);
|
|
+ } else {
|
|
+ normal.add(this);
|
|
+ }
|
|
+
|
|
+ break;
|
|
+ }
|
|
+ }
|
|
+
|
|
+ //noinspection SynchronizationOnLocalVariableOrMethodParameter
|
|
+ synchronized (executor) {
|
|
+ // Wake up a thread to take this work
|
|
+ executor.notify();
|
|
+ }
|
|
+ }
|
|
+ }
|
|
+ public interface RejectionHandler {
|
|
+ void onRejection(Runnable run, PriorityQueuedExecutor executor);
|
|
+ }
|
|
+
|
|
+ public static final RejectionHandler ABORT_POLICY = (run, executor) -> {
|
|
+ throw new RejectedExecutionException("Executor has been shutdown");
|
|
+ };
|
|
+ public static final RejectionHandler CALLER_RUNS_POLICY = (run, executor) -> {
|
|
+ run.run();
|
|
+ };
|
|
+
|
|
+}
|
|
diff --git a/src/main/java/net/minecraft/server/ChunkProviderServer.java b/src/main/java/net/minecraft/server/ChunkProviderServer.java
|
|
index 5b57ea93c8..5d5834ba7f 100644
|
|
--- a/src/main/java/net/minecraft/server/ChunkProviderServer.java
|
|
+++ b/src/main/java/net/minecraft/server/ChunkProviderServer.java
|
|
@@ -38,9 +38,9 @@ public class ChunkProviderServer implements IChunkProvider {
|
|
public final Long2ObjectMap<Chunk> chunks = Long2ObjectMaps.synchronize(new ChunkMap(8192));
|
|
private Chunk lastChunk;
|
|
private final ChunkTaskScheduler chunkScheduler;
|
|
- private final SchedulerBatch<ChunkCoordIntPair, ChunkStatus, ProtoChunk> batchScheduler;
|
|
+ final SchedulerBatch<ChunkCoordIntPair, ChunkStatus, ProtoChunk> batchScheduler; // Paper
|
|
public final WorldServer world;
|
|
- private final IAsyncTaskHandler asyncTaskHandler;
|
|
+ final IAsyncTaskHandler asyncTaskHandler; // Paper
|
|
|
|
public ChunkProviderServer(WorldServer worldserver, IChunkLoader ichunkloader, ChunkGenerator<?> chunkgenerator, IAsyncTaskHandler iasynctaskhandler) {
|
|
this.world = worldserver;
|
|
@@ -77,10 +77,61 @@ public class ChunkProviderServer implements IChunkProvider {
|
|
this.unloadQueue.remove(ChunkCoordIntPair.a(i, j));
|
|
}
|
|
|
|
+ // Paper start - defaults if Async Chunks is not enabled
|
|
+ boolean chunkGoingToExists(int x, int z) {
|
|
+ final long k = ChunkCoordIntPair.asLong(x, z);
|
|
+ return chunkScheduler.progressCache.containsKey(k);
|
|
+ }
|
|
+ public void bumpPriority(ChunkCoordIntPair coords) {
|
|
+ // do nothing, override in async
|
|
+ }
|
|
+
|
|
+ public List<ChunkCoordIntPair> getSpiralOutChunks(BlockPosition blockposition, int radius) {
|
|
+ List<ChunkCoordIntPair> list = com.google.common.collect.Lists.newArrayList();
|
|
+
|
|
+ for (int r = 1; r <= radius; r++) {
|
|
+ int x = -r;
|
|
+ int z = r;
|
|
+ list.add(new ChunkCoordIntPair(blockposition.getX(), blockposition.getZ()));
|
|
+ // Iterates the edge of half of the box; then negates for other half.
|
|
+ while (x <= r && z > -r) {
|
|
+ list.add(new ChunkCoordIntPair(blockposition.getX() + x, blockposition.getZ() + z));
|
|
+ list.add(new ChunkCoordIntPair(blockposition.getX() - x, blockposition.getZ() - z));
|
|
+
|
|
+ if (x < r) {
|
|
+ x++;
|
|
+ } else {
|
|
+ z--;
|
|
+ }
|
|
+ }
|
|
+ }
|
|
+ return list;
|
|
+ }
|
|
+
|
|
+ public Chunk getChunkAt(int x, int z, boolean load, boolean gen, Consumer<Chunk> consumer) {
|
|
+ return getChunkAt(x, z, load, gen, false, consumer);
|
|
+ }
|
|
+ public Chunk getChunkAt(int x, int z, boolean load, boolean gen, boolean priority, Consumer<Chunk> consumer) {
|
|
+ Chunk chunk = getChunkAt(x, z, load, gen);
|
|
+ if (consumer != null) {
|
|
+ consumer.accept(chunk);
|
|
+ }
|
|
+ return chunk;
|
|
+ }
|
|
+ // Paper end
|
|
+
|
|
@Nullable
|
|
public Chunk getChunkAt(int i, int j, boolean flag, boolean flag1) {
|
|
IChunkLoader ichunkloader = this.chunkLoader;
|
|
Chunk chunk;
|
|
+ // Paper start - do already loaded checks before synchronize
|
|
+ long k = ChunkCoordIntPair.a(i, j);
|
|
+ chunk = (Chunk) this.chunks.get(k);
|
|
+ if (chunk != null) {
|
|
+ //this.lastChunk = chunk; // Paper remove vanilla lastChunk
|
|
+ return chunk;
|
|
+ }
|
|
+ // Paper end
|
|
|
|
synchronized (this.chunkLoader) {
|
|
// Paper start - remove vanilla lastChunk, we do it more accurately
|
|
@@ -88,13 +139,15 @@ public class ChunkProviderServer implements IChunkProvider {
|
|
return this.lastChunk;
|
|
}*/ // Paper end
|
|
|
|
- long k = ChunkCoordIntPair.a(i, j);
|
|
+ // Paper start - move up
|
|
+ //long k = ChunkCoordIntPair.a(i, j);
|
|
|
|
- chunk = (Chunk) this.chunks.get(k);
|
|
+ /*chunk = (Chunk) this.chunks.get(k);
|
|
if (chunk != null) {
|
|
//this.lastChunk = chunk; // Paper remove vanilla lastChunk
|
|
return chunk;
|
|
- }
|
|
+ }*/
|
|
+ // Paper end
|
|
|
|
if (flag) {
|
|
try (co.aikar.timings.Timing timing = world.timings.syncChunkLoadTimer.startTiming()) { // Paper
|
|
@@ -150,7 +203,8 @@ public class ChunkProviderServer implements IChunkProvider {
|
|
return (IChunkAccess) (chunk != null ? chunk : (IChunkAccess) this.chunkScheduler.b(new ChunkCoordIntPair(i, j), flag));
|
|
}
|
|
|
|
- public CompletableFuture<ProtoChunk> a(Iterable<ChunkCoordIntPair> iterable, Consumer<Chunk> consumer) {
|
|
+ public CompletableFuture<Void> loadAllChunks(Iterable<ChunkCoordIntPair> iterable, Consumer<Chunk> consumer) { return a(iterable, consumer).thenCompose(protoChunk -> null); } // Paper - overriden in async chunk provider
|
|
+ private CompletableFuture<ProtoChunk> a(Iterable<ChunkCoordIntPair> iterable, Consumer<Chunk> consumer) { // Paper - mark private, use above method
|
|
this.batchScheduler.b();
|
|
Iterator iterator = iterable.iterator();
|
|
|
|
@@ -168,6 +222,7 @@ public class ChunkProviderServer implements IChunkProvider {
|
|
return this.batchScheduler.c();
|
|
}
|
|
|
|
+ ReportedException generateChunkError(int i, int j, Throwable throwable) { return a(i, j, throwable); } // Paper - OBFHELPER
|
|
private ReportedException a(int i, int j, Throwable throwable) {
|
|
CrashReport crashreport = CrashReport.a(throwable, "Exception generating new chunk");
|
|
CrashReportSystemDetails crashreportsystemdetails = crashreport.a("Chunk to be generated");
|
|
@@ -287,11 +342,13 @@ public class ChunkProviderServer implements IChunkProvider {
|
|
}
|
|
|
|
public void close() {
|
|
- try {
|
|
+ // Paper start - we do not need to wait for chunk generations to finish on close
|
|
+ /*try {
|
|
this.batchScheduler.a();
|
|
} catch (InterruptedException interruptedexception) {
|
|
ChunkProviderServer.a.error("Couldn\'t stop taskManager", interruptedexception);
|
|
- }
|
|
+ }*/
|
|
+ // Paper end
|
|
|
|
}
|
|
|
|
diff --git a/src/main/java/net/minecraft/server/ChunkRegionLoader.java b/src/main/java/net/minecraft/server/ChunkRegionLoader.java
|
|
index c233b7e903..edd0742527 100644
|
|
--- a/src/main/java/net/minecraft/server/ChunkRegionLoader.java
|
|
+++ b/src/main/java/net/minecraft/server/ChunkRegionLoader.java
|
|
@@ -120,7 +120,7 @@ public class ChunkRegionLoader implements IChunkLoader, IAsyncChunkSaver {
|
|
// CraftBukkit start
|
|
private boolean check(ChunkProviderServer cps, int x, int z) throws IOException {
|
|
if (cps != null) {
|
|
- com.google.common.base.Preconditions.checkState(org.bukkit.Bukkit.isPrimaryThread(), "primary thread");
|
|
+ //com.google.common.base.Preconditions.checkState(org.bukkit.Bukkit.isPrimaryThread(), "primary thread"); // Paper - this is safe
|
|
if (cps.isLoaded(x, z)) {
|
|
return true;
|
|
}
|
|
@@ -380,11 +380,12 @@ public class ChunkRegionLoader implements IChunkLoader, IAsyncChunkSaver {
|
|
}
|
|
};
|
|
} else {
|
|
+ /* // Paper start - we will never invoke this in an unsafe way
|
|
NBTTagCompound nbttagcompound2 = this.a(world, chunkcoordintpair.x, chunkcoordintpair.z);
|
|
|
|
if (nbttagcompound2 != null && this.a(nbttagcompound2) == ChunkStatus.Type.LEVELCHUNK) {
|
|
return;
|
|
- }
|
|
+ }*/ // Paper end
|
|
|
|
completion = new Supplier<NBTTagCompound>() {
|
|
public NBTTagCompound get() {
|
|
diff --git a/src/main/java/net/minecraft/server/ChunkTaskScheduler.java b/src/main/java/net/minecraft/server/ChunkTaskScheduler.java
|
|
index 34019bd1b3..4ca977645f 100644
|
|
--- a/src/main/java/net/minecraft/server/ChunkTaskScheduler.java
|
|
+++ b/src/main/java/net/minecraft/server/ChunkTaskScheduler.java
|
|
@@ -20,7 +20,7 @@ public class ChunkTaskScheduler extends Scheduler<ChunkCoordIntPair, ChunkStatus
|
|
private final ChunkGenerator<?> d;
|
|
private final IChunkLoader e;
|
|
private final IAsyncTaskHandler f;
|
|
- private final Long2ObjectMap<Scheduler.a> progressCache = new ExpiringMap<Scheduler.a>(8192, 5000) { // CraftBukkit - decompile error
|
|
+ final Long2ObjectMap<Scheduler.a> progressCache = new ExpiringMap<Scheduler.a>(8192, 5000) { // CraftBukkit - decompile error // Paper - synchronize
|
|
protected boolean a(Scheduler.a scheduler_a) {
|
|
ProtoChunk protochunk = (ProtoChunk) scheduler_a.a();
|
|
|
|
@@ -50,7 +50,7 @@ public class ChunkTaskScheduler extends Scheduler<ChunkCoordIntPair, ChunkStatus
|
|
protected Scheduler.a a(ChunkCoordIntPair chunkcoordintpair, boolean flag) {
|
|
IChunkLoader ichunkloader = this.e;
|
|
|
|
- synchronized (this.e) {
|
|
+ synchronized (progressCache) { // Paper - synchronize on progress cache instead
|
|
return flag ? (Scheduler.a) this.progressCache.computeIfAbsent(chunkcoordintpair.a(), (i) -> {
|
|
ProtoChunk protochunk;
|
|
|
|
diff --git a/src/main/java/net/minecraft/server/DataPaletteBlock.java b/src/main/java/net/minecraft/server/DataPaletteBlock.java
|
|
index 71a3636be6..b0170db9ca 100644
|
|
--- a/src/main/java/net/minecraft/server/DataPaletteBlock.java
|
|
+++ b/src/main/java/net/minecraft/server/DataPaletteBlock.java
|
|
@@ -3,7 +3,7 @@ package net.minecraft.server;
|
|
import com.destroystokyo.paper.antixray.ChunkPacketInfo; // Paper - Anti-Xray
|
|
import java.util.Arrays;
|
|
import java.util.Objects;
|
|
-import java.util.concurrent.locks.ReentrantLock;
|
|
+import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
import java.util.function.Function;
|
|
import java.util.stream.Collectors;
|
|
|
|
@@ -20,25 +20,16 @@ public class DataPaletteBlock<T> implements DataPaletteExpandable<T> {
|
|
protected DataBits a; protected DataBits getDataBits() { return this.a; } // Paper - OBFHELPER
|
|
private DataPalette<T> h; private DataPalette<T> getDataPalette() { return this.h; } // Paper - OBFHELPER
|
|
private int i; private int getBitsPerObject() { return this.i; } // Paper - OBFHELPER
|
|
- private final ReentrantLock j = new ReentrantLock();
|
|
+ // Paper start - use read write locks
|
|
+ private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
|
|
|
|
private void b() {
|
|
- if (this.j.isLocked() && !this.j.isHeldByCurrentThread()) {
|
|
- String s = (String)Thread.getAllStackTraces().keySet().stream().filter(Objects::nonNull).map((thread) -> {
|
|
- return thread.getName() + ": \n\tat " + (String)Arrays.stream(thread.getStackTrace()).map(Object::toString).collect(Collectors.joining("\n\tat "));
|
|
- }).collect(Collectors.joining("\n"));
|
|
- CrashReport crashreport = new CrashReport("Writing into PalettedContainer from multiple threads", new IllegalStateException());
|
|
- CrashReportSystemDetails crashreportsystemdetails = crashreport.a("Thread dumps");
|
|
- crashreportsystemdetails.a("Thread dumps", s);
|
|
- throw new ReportedException(crashreport);
|
|
- } else {
|
|
- this.j.lock();
|
|
- }
|
|
+ lock.writeLock().lock();
|
|
}
|
|
-
|
|
private void c() {
|
|
- this.j.unlock();
|
|
+ lock.writeLock().unlock();
|
|
}
|
|
+ // Paper end
|
|
|
|
public DataPaletteBlock(DataPalette<T> datapalette, RegistryBlockID<T> registryblockid, Function<NBTTagCompound, T> function, Function<T, NBTTagCompound> function1, T object) {
|
|
// Paper start - Anti-Xray - Support default constructor
|
|
@@ -147,8 +138,13 @@ public class DataPaletteBlock<T> implements DataPaletteExpandable<T> {
|
|
}
|
|
|
|
protected T a(int ix) {
|
|
- T object = this.h.a(this.a.a(ix)); // Paper - decompile fix
|
|
- return (T)(object == null ? this.g : object);
|
|
+ try { // Paper
|
|
+ lock.readLock().lock();
|
|
+ T object = this.h.a(this.a.a(ix)); // Paper - decompile fix
|
|
+ return (T)(object == null ? this.g : object);
|
|
+ } finally {
|
|
+ lock.readLock().unlock();
|
|
+ } // Paper
|
|
}
|
|
|
|
// Paper start - Anti-Xray - Support default methods
|
|
diff --git a/src/main/java/net/minecraft/server/DefinedStructureManager.java b/src/main/java/net/minecraft/server/DefinedStructureManager.java
|
|
index 271dc41d45..bd15534c23 100644
|
|
--- a/src/main/java/net/minecraft/server/DefinedStructureManager.java
|
|
+++ b/src/main/java/net/minecraft/server/DefinedStructureManager.java
|
|
@@ -19,7 +19,7 @@ import org.apache.logging.log4j.Logger;
|
|
|
|
public class DefinedStructureManager implements IResourcePackListener {
|
|
private static final Logger a = LogManager.getLogger();
|
|
- private final Map<MinecraftKey, DefinedStructure> b = Maps.newHashMap();
|
|
+ private final Map<MinecraftKey, DefinedStructure> b = Maps.newConcurrentMap(); // Paper
|
|
private final DataFixer c;
|
|
private final MinecraftServer d;
|
|
private final java.nio.file.Path e;
|
|
diff --git a/src/main/java/net/minecraft/server/Entity.java b/src/main/java/net/minecraft/server/Entity.java
|
|
index 0237049a40..cd601f29a3 100644
|
|
--- a/src/main/java/net/minecraft/server/Entity.java
|
|
+++ b/src/main/java/net/minecraft/server/Entity.java
|
|
@@ -209,7 +209,7 @@ public abstract class Entity implements INamableTileEntity, ICommandListener, Ke
|
|
this.random = SHARED_RANDOM; // Paper
|
|
this.fireTicks = -this.getMaxFireTicks();
|
|
this.justCreated = true;
|
|
- this.uniqueID = MathHelper.a(this.random);
|
|
+ this.uniqueID = MathHelper.a(java.util.concurrent.ThreadLocalRandom.current()); // Paper
|
|
this.au = this.uniqueID.toString();
|
|
this.aJ = Sets.newHashSet();
|
|
this.aL = new double[] { 0.0D, 0.0D, 0.0D};
|
|
diff --git a/src/main/java/net/minecraft/server/IChunkLoader.java b/src/main/java/net/minecraft/server/IChunkLoader.java
|
|
index 4698ee99f8..dfb45cc4ea 100644
|
|
--- a/src/main/java/net/minecraft/server/IChunkLoader.java
|
|
+++ b/src/main/java/net/minecraft/server/IChunkLoader.java
|
|
@@ -6,6 +6,8 @@ import javax.annotation.Nullable;
|
|
|
|
public interface IChunkLoader {
|
|
|
|
+ void loadEntities(NBTTagCompound nbttagcompound, Chunk chunk); // Paper - Async Chunks
|
|
+ Object[] loadChunk(GeneratorAccess generatoraccess, int i, int j, Consumer<Chunk> consumer) throws IOException; // Paper - Async Chunks
|
|
@Nullable
|
|
Chunk a(GeneratorAccess generatoraccess, int i, int j, Consumer<Chunk> consumer) throws IOException;
|
|
|
|
diff --git a/src/main/java/net/minecraft/server/MathHelper.java b/src/main/java/net/minecraft/server/MathHelper.java
|
|
index 49fba0979e..9ad646f8d4 100644
|
|
--- a/src/main/java/net/minecraft/server/MathHelper.java
|
|
+++ b/src/main/java/net/minecraft/server/MathHelper.java
|
|
@@ -142,6 +142,7 @@ public class MathHelper {
|
|
return Math.floorMod(i, j);
|
|
}
|
|
|
|
+ public static float normalizeYaw(float fx) { return g(fx); } // Paper - OBFHELPER
|
|
public static float g(float fx) {
|
|
fx = fx % 360.0F;
|
|
if (fx >= 180.0F) {
|
|
diff --git a/src/main/java/net/minecraft/server/MinecraftServer.java b/src/main/java/net/minecraft/server/MinecraftServer.java
|
|
index 98d182fdb8..487d98eb1b 100644
|
|
--- a/src/main/java/net/minecraft/server/MinecraftServer.java
|
|
+++ b/src/main/java/net/minecraft/server/MinecraftServer.java
|
|
@@ -503,6 +503,7 @@ public abstract class MinecraftServer implements IAsyncTaskHandler, IMojangStati
|
|
|
|
// CraftBukkit start - fire WorldLoadEvent and handle whether or not to keep the spawn in memory
|
|
Stopwatch stopwatch = Stopwatch.createStarted();
|
|
+ boolean waitForChunks = Boolean.getBoolean("paper.waitforchunks"); // Paper
|
|
for (WorldServer worldserver : this.getWorlds()) {
|
|
MinecraftServer.LOGGER.info("Preparing start region for level " + worldserver.dimension + " (Seed: " + worldserver.getSeed() + ")");
|
|
if (!worldserver.getWorld().getKeepSpawnInMemory()) {
|
|
@@ -510,29 +511,25 @@ public abstract class MinecraftServer implements IAsyncTaskHandler, IMojangStati
|
|
}
|
|
|
|
BlockPosition blockposition = worldserver.getSpawn();
|
|
- ArrayList arraylist = Lists.newArrayList();
|
|
+ List<ChunkCoordIntPair> arraylist = worldserver.getChunkProviderServer().getSpiralOutChunks(blockposition, worldserver.paperConfig.keepLoadedRange >> 4); // Paper
|
|
Set set = Sets.newConcurrentHashSet();
|
|
|
|
- // Paper start
|
|
- short radius = worldserver.paperConfig.keepLoadedRange;
|
|
- for (int i = -radius; i <= radius && this.isRunning(); i += 16) {
|
|
- for (int j = -radius; j <= radius && this.isRunning(); j += 16) {
|
|
- // Paper end
|
|
- arraylist.add(new ChunkCoordIntPair(blockposition.getX() + i >> 4, blockposition.getZ() + j >> 4));
|
|
- }
|
|
- } // Paper
|
|
+ // Paper - remove arraylist creation, call spiral above
|
|
if (this.isRunning()) { // Paper
|
|
int expected = arraylist.size(); // Paper
|
|
|
|
|
|
- CompletableFuture completablefuture = worldserver.getChunkProviderServer().a((Iterable) arraylist, (chunk) -> {
|
|
+ CompletableFuture completablefuture = worldserver.getChunkProviderServer().loadAllChunks(arraylist, (chunk) -> { // Paper
|
|
set.add(chunk.getPos());
|
|
- if (set.size() < expected && set.size() % 25 == 0) this.a(new ChatMessage("menu.preparingSpawn", new Object[0]), set.size() * 100 / expected); // Paper
|
|
+ if (waitForChunks && (set.size() == expected || (set.size() < expected && set.size() % (set.size() / 10) == 0))) {
|
|
+ this.a(new ChatMessage("menu.preparingSpawn", new Object[0]), set.size() * 100 / expected); // Paper
|
|
+ }
|
|
});
|
|
|
|
- while (!completablefuture.isDone()) {
|
|
+ while (waitForChunks && !completablefuture.isDone() && isRunning()) { // Paper
|
|
try {
|
|
- completablefuture.get(1L, TimeUnit.SECONDS);
|
|
+ PaperAsyncChunkProvider.processChunkLoads(worldserver); // Paper
|
|
+ completablefuture.get(50L, TimeUnit.MILLISECONDS); // Paper
|
|
} catch (InterruptedException interruptedexception) {
|
|
throw new RuntimeException(interruptedexception);
|
|
} catch (ExecutionException executionexception) {
|
|
@@ -542,11 +539,11 @@ public abstract class MinecraftServer implements IAsyncTaskHandler, IMojangStati
|
|
|
|
throw new RuntimeException(executionexception.getCause());
|
|
} catch (TimeoutException timeoutexception) {
|
|
- this.a(new ChatMessage("menu.preparingSpawn", new Object[0]), set.size() * 100 / expected); // Paper
|
|
+ //this.a(new ChatMessage("menu.preparingSpawn", new Object[0]), set.size() * 100 / expected); // Paper
|
|
}
|
|
}
|
|
|
|
- this.a(new ChatMessage("menu.preparingSpawn", new Object[0]), set.size() * 100 / expected); // Paper
|
|
+ if (waitForChunks) this.a(new ChatMessage("menu.preparingSpawn", new Object[0]), set.size() * 100 / expected); // Paper
|
|
}
|
|
}
|
|
|
|
@@ -650,6 +647,7 @@ public abstract class MinecraftServer implements IAsyncTaskHandler, IMojangStati
|
|
if (hasStopped) return;
|
|
hasStopped = true;
|
|
}
|
|
+ PaperAsyncChunkProvider.stop(this); // Paper
|
|
// CraftBukkit end
|
|
MinecraftServer.LOGGER.info("Stopping server");
|
|
MinecraftTimings.stopServer(); // Paper
|
|
@@ -1017,6 +1015,7 @@ public abstract class MinecraftServer implements IAsyncTaskHandler, IMojangStati
|
|
while ((futuretask = (FutureTask) this.f.poll()) != null) {
|
|
SystemUtils.a(futuretask, MinecraftServer.LOGGER);
|
|
}
|
|
+ PaperAsyncChunkProvider.processChunkLoads(this); // Paper
|
|
MinecraftTimings.minecraftSchedulerTimer.stopTiming(); // Paper
|
|
|
|
this.methodProfiler.c("commandFunctions");
|
|
@@ -1053,6 +1052,7 @@ public abstract class MinecraftServer implements IAsyncTaskHandler, IMojangStati
|
|
// CraftBukkit - dropTickTime
|
|
for (Iterator iterator = this.getWorlds().iterator(); iterator.hasNext();) {
|
|
WorldServer worldserver = (WorldServer) iterator.next();
|
|
+ PaperAsyncChunkProvider.processChunkLoads(worldserver); // Paper
|
|
TileEntityHopper.skipHopperEvents = worldserver.paperConfig.disableHopperMoveEvents || org.bukkit.event.inventory.InventoryMoveItemEvent.getHandlerList().getRegisteredListeners().length == 0; // Paper
|
|
i = SystemUtils.c();
|
|
if (true || worldserver.worldProvider.getDimensionManager() == DimensionManager.OVERWORLD || this.getAllowNether()) { // CraftBukkit
|
|
@@ -1109,6 +1109,7 @@ public abstract class MinecraftServer implements IAsyncTaskHandler, IMojangStati
|
|
this.methodProfiler.e();
|
|
this.methodProfiler.e();
|
|
worldserver.explosionDensityCache.clear(); // Paper - Optimize explosions
|
|
+ PaperAsyncChunkProvider.processChunkLoads(worldserver); // Paper
|
|
}
|
|
}
|
|
|
|
diff --git a/src/main/java/net/minecraft/server/PaperAsyncChunkProvider.java b/src/main/java/net/minecraft/server/PaperAsyncChunkProvider.java
|
|
new file mode 100644
|
|
index 0000000000..a3fc989832
|
|
--- /dev/null
|
|
+++ b/src/main/java/net/minecraft/server/PaperAsyncChunkProvider.java
|
|
@@ -0,0 +1,478 @@
|
|
+/*
|
|
+ * This file is licensed under the MIT License (MIT).
|
|
+ *
|
|
+ * Copyright (c) 2018 Daniel Ennis <http://aikar.co>
|
|
+ *
|
|
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
|
|
+ * of this software and associated documentation files (the "Software"), to deal
|
|
+ * in the Software without restriction, including without limitation the rights
|
|
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
|
|
+ * copies of the Software, and to permit persons to whom the Software is
|
|
+ * furnished to do so, subject to the following conditions:
|
|
+ *
|
|
+ * The above copyright notice and this permission notice shall be included in
|
|
+ * all copies or substantial portions of the Software.
|
|
+ *
|
|
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
|
|
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
|
|
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
|
|
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
|
|
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
|
|
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
|
|
+ * THE SOFTWARE.
|
|
+ */
|
|
+package net.minecraft.server;
|
|
+
|
|
+import com.destroystokyo.paper.PaperConfig;
|
|
+import com.destroystokyo.paper.util.PriorityQueuedExecutor;
|
|
+import com.destroystokyo.paper.util.PriorityQueuedExecutor.Priority;
|
|
+import it.unimi.dsi.fastutil.longs.Long2ObjectMap;
|
|
+import it.unimi.dsi.fastutil.longs.Long2ObjectMaps;
|
|
+import it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap;
|
|
+import org.bukkit.Bukkit;
|
|
+import org.bukkit.craftbukkit.generator.CustomChunkGenerator;
|
|
+import org.bukkit.craftbukkit.generator.InternalChunkGenerator;
|
|
+
|
|
+import javax.annotation.Nullable;
|
|
+import java.io.IOException;
|
|
+import java.util.ArrayList;
|
|
+import java.util.Iterator;
|
|
+import java.util.List;
|
|
+import java.util.concurrent.CompletableFuture;
|
|
+import java.util.concurrent.ConcurrentLinkedQueue;
|
|
+import java.util.function.Consumer;
|
|
+
|
|
+@SuppressWarnings("unused")
|
|
+public class PaperAsyncChunkProvider extends ChunkProviderServer {
|
|
+
|
|
+ private static final PriorityQueuedExecutor.RejectionHandler IGNORE_HANDLER = (run, executor) -> {};
|
|
+ private static final int CHUNK_THREADS = (int) Math.min(Integer.getInteger("paper.maxchunkthreads", 8), Runtime.getRuntime().availableProcessors() * 1.5);
|
|
+ private static final PriorityQueuedExecutor EXECUTOR = new PriorityQueuedExecutor("PaperChunkLoader", PaperConfig.asyncChunks ? CHUNK_THREADS : 0, IGNORE_HANDLER);
|
|
+
|
|
+ private final PriorityQueuedExecutor generationExecutor;
|
|
+ //private static final PriorityQueuedExecutor generationExecutor = new PriorityQueuedExecutor("PaperChunkGen", 1);
|
|
+ private final Long2ObjectMap<PendingChunk> pendingChunks = Long2ObjectMaps.synchronize(new Long2ObjectOpenHashMap<>());
|
|
+ private final ConcurrentLinkedQueue<Runnable> mainThreadQueue = new ConcurrentLinkedQueue<>();
|
|
+ private final IAsyncTaskHandler asyncHandler;
|
|
+
|
|
+ private final WorldServer world;
|
|
+ private final IChunkLoader chunkLoader;
|
|
+ private final MinecraftServer server;
|
|
+ private final boolean shouldGenSync;
|
|
+
|
|
+ public PaperAsyncChunkProvider(WorldServer world, IChunkLoader chunkLoader, InternalChunkGenerator generator, MinecraftServer server) {
|
|
+ super(world, chunkLoader, generator, server);
|
|
+
|
|
+ this.server = world.getMinecraftServer();
|
|
+ this.world = world;
|
|
+ this.asyncHandler = server;
|
|
+ this.chunkLoader = chunkLoader;
|
|
+ String worldName = this.world.getWorld().getName();
|
|
+ this.shouldGenSync = generator instanceof CustomChunkGenerator && !(((CustomChunkGenerator) generator).asyncSupported);
|
|
+ this.generationExecutor = new PriorityQueuedExecutor("PaperChunkGen-" + worldName, shouldGenSync ? 0 : 1, IGNORE_HANDLER);
|
|
+ }
|
|
+
|
|
+ static void processChunkLoads(MinecraftServer server) {
|
|
+ for (WorldServer world : server.getWorlds()) {
|
|
+ processChunkLoads(world);
|
|
+ }
|
|
+ }
|
|
+
|
|
+ static void processChunkLoads(World world) {
|
|
+ IChunkProvider chunkProvider = world.getChunkProvider();
|
|
+ if (chunkProvider instanceof PaperAsyncChunkProvider) {
|
|
+ ((PaperAsyncChunkProvider) chunkProvider).processChunkLoads();
|
|
+ }
|
|
+ }
|
|
+
|
|
+ static void stop(MinecraftServer server) {
|
|
+ EXECUTOR.shutdownNow();
|
|
+ for (WorldServer world : server.getWorlds()) {
|
|
+ IChunkProvider chunkProvider = world.getChunkProvider();
|
|
+ if (chunkProvider instanceof PaperAsyncChunkProvider) {
|
|
+ ((PaperAsyncChunkProvider) chunkProvider).generationExecutor.shutdownNow();
|
|
+ }
|
|
+ }
|
|
+ }
|
|
+
|
|
+ private boolean processChunkLoads() {
|
|
+ Runnable run;
|
|
+ boolean hadLoad = false;
|
|
+ while ((run = mainThreadQueue.poll()) != null) {
|
|
+ run.run();
|
|
+ hadLoad = true;
|
|
+ }
|
|
+ return hadLoad;
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public void bumpPriority(ChunkCoordIntPair coords) {
|
|
+ PendingChunk pending = pendingChunks.get(coords.asLong());
|
|
+ if (pending != null) {
|
|
+ pending.bumpPriority();
|
|
+ }
|
|
+ }
|
|
+
|
|
+ @Nullable
|
|
+ @Override
|
|
+ public Chunk getChunkAt(int x, int z, boolean load, boolean gen) {
|
|
+ return getChunkAt(x, z, load, gen, null);
|
|
+ }
|
|
+
|
|
+ @Nullable
|
|
+ @Override
|
|
+ public Chunk getChunkAt(int x, int z, boolean load, boolean gen, boolean priority, Consumer<Chunk> consumer) {
|
|
+ long key = ChunkCoordIntPair.asLong(x, z);
|
|
+ Chunk chunk = this.chunks.get(key);
|
|
+ if (chunk != null || !load) { // return null if we aren't loading
|
|
+ if (consumer != null) {
|
|
+ consumer.accept(chunk);
|
|
+ }
|
|
+ return chunk;
|
|
+ }
|
|
+ return loadOrGenerateChunk(x, z, gen, priority, consumer); // Async overrides this method
|
|
+ }
|
|
+
|
|
+ private Chunk loadOrGenerateChunk(int x, int z, boolean gen, boolean priority, Consumer<Chunk> consumer) {
|
|
+ final long key = ChunkCoordIntPair.asLong(x, z);
|
|
+
|
|
+ // Obtain a PendingChunk
|
|
+ final PendingChunk pending;
|
|
+ final boolean isBlockingMain = consumer == null && server.isMainThread();
|
|
+ synchronized (pendingChunks) {
|
|
+ PendingChunk pendingChunk = pendingChunks.get(key);
|
|
+ // DO NOT CALL ANY METHODS ON PENDING CHUNK IN THIS BLOCK - WILL DEADLOCK
|
|
+ if (pendingChunk == null) {
|
|
+ pending = new PendingChunk(x, z, key, gen, priority || isBlockingMain);
|
|
+ pendingChunks.put(key, pending);
|
|
+ } else if (pendingChunk.hasFinished && gen && !pendingChunk.canGenerate && pendingChunk.chunk == null) {
|
|
+ // need to overwrite the old
|
|
+ pending = new PendingChunk(x, z, key, true, priority || isBlockingMain);
|
|
+ pendingChunks.put(key, pending);
|
|
+ } else {
|
|
+ pending = pendingChunk;
|
|
+ if (priority || isBlockingMain) {
|
|
+ pending.bumpPriority();
|
|
+ }
|
|
+ }
|
|
+ }
|
|
+ // Listen for when result is ready
|
|
+ final CompletableFuture<Chunk> future = new CompletableFuture<>();
|
|
+ pending.addListener(future, gen);
|
|
+
|
|
+ if (isBlockingMain && pending.hasFinished) {
|
|
+ processChunkLoads();
|
|
+ return pending.postChunk();
|
|
+ }
|
|
+
|
|
+ if (isBlockingMain) {
|
|
+ try (co.aikar.timings.Timing timing = world.timings.syncChunkLoadTimer.startTiming()) {
|
|
+ while (!future.isDone()) {
|
|
+ // We aren't done, obtain lock on queue
|
|
+ synchronized (mainThreadQueue) {
|
|
+ // We may of received our request now, check it
|
|
+ if (processChunkLoads()) {
|
|
+ // If we processed SOMETHING, don't wait
|
|
+ continue;
|
|
+ }
|
|
+ try {
|
|
+ // We got nothing from the queue, wait until something has been added
|
|
+ mainThreadQueue.wait(1);
|
|
+ } catch (InterruptedException ignored) {
|
|
+ }
|
|
+ }
|
|
+ // Queue has been notified or timed out, process it
|
|
+ processChunkLoads();
|
|
+ }
|
|
+ // We should be done AND posted into chunk map now, return it
|
|
+ return future.join();
|
|
+ }
|
|
+ } else if (consumer == null) {
|
|
+ // This is on another thread
|
|
+ return future.join();
|
|
+ } else {
|
|
+ future.thenAccept((c) -> this.asyncHandler.postToMainThread(() -> consumer.accept(c)));
|
|
+ }
|
|
+ return null;
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public CompletableFuture<Void> loadAllChunks(Iterable<ChunkCoordIntPair> iterable, Consumer<Chunk> consumer) {
|
|
+ Iterator<ChunkCoordIntPair> iterator = iterable.iterator();
|
|
+
|
|
+ List<CompletableFuture<Chunk>> all = new ArrayList<>();
|
|
+ while (iterator.hasNext()) {
|
|
+ ChunkCoordIntPair chunkcoordintpair = iterator.next();
|
|
+ CompletableFuture<Chunk> future = new CompletableFuture<>();
|
|
+ all.add(future);
|
|
+ this.getChunkAt(chunkcoordintpair.x, chunkcoordintpair.z, true, true, chunk -> {
|
|
+ future.complete(chunk);
|
|
+ if (consumer != null) {
|
|
+ consumer.accept(chunk);
|
|
+ }
|
|
+ });
|
|
+ }
|
|
+ return CompletableFuture.allOf(all.toArray(new CompletableFuture[0]));
|
|
+ }
|
|
+
|
|
+ boolean chunkGoingToExists(int x, int z) {
|
|
+ synchronized (pendingChunks) {
|
|
+ long key = ChunkCoordIntPair.asLong(x, z);
|
|
+ PendingChunk pendingChunk = pendingChunks.get(key);
|
|
+ return pendingChunk != null && pendingChunk.canGenerate;
|
|
+ }
|
|
+ }
|
|
+
|
|
+ private enum PendingStatus {
|
|
+ /**
|
|
+ * Request has just started
|
|
+ */
|
|
+ STARTED,
|
|
+ /**
|
|
+ * Chunk is attempting to be loaded from disk
|
|
+ */
|
|
+ LOADING,
|
|
+ /**
|
|
+ * Chunk must generate on main and is pending main
|
|
+ */
|
|
+ GENERATION_PENDING,
|
|
+ /**
|
|
+ * Chunk is generating
|
|
+ */
|
|
+ GENERATING,
|
|
+ /**
|
|
+ * Chunk is ready and is pending post to main
|
|
+ */
|
|
+ PENDING_MAIN,
|
|
+ /**
|
|
+ * Could not load chunk, and did not need to generat
|
|
+ */
|
|
+ FAIL,
|
|
+ /**
|
|
+ * Fully done with this request (may or may not of loaded)
|
|
+ */
|
|
+ DONE
|
|
+ }
|
|
+
|
|
+ private class PendingChunk implements Runnable {
|
|
+ private final int x;
|
|
+ private final int z;
|
|
+ private final long key;
|
|
+ private final PriorityQueuedExecutor.PendingTask<Void> task;
|
|
+ private final PriorityQueuedExecutor.PendingTask<Chunk> chunkGenTask;
|
|
+ private final CompletableFuture<Chunk> loadOnly = new CompletableFuture<>();
|
|
+ private final CompletableFuture<Chunk> generate = new CompletableFuture<>();
|
|
+
|
|
+ private volatile PendingStatus status = PendingStatus.STARTED;
|
|
+ private volatile boolean generating;
|
|
+ private volatile boolean canGenerate;
|
|
+ private volatile boolean isHighPriority;
|
|
+ private volatile boolean hasPosted;
|
|
+ private volatile boolean hasFinished;
|
|
+ private volatile Chunk chunk;
|
|
+ private volatile NBTTagCompound pendingLevel;
|
|
+
|
|
+ PendingChunk(int x, int z, long key, boolean canGenerate, boolean priority) {
|
|
+ this.x = x;
|
|
+ this.z = z;
|
|
+ this.key = key;
|
|
+ this.canGenerate = canGenerate;
|
|
+ Priority taskPriority = priority ? Priority.HIGH : Priority.NORMAL;
|
|
+ this.chunkGenTask = generationExecutor.createPendingTask(this::generateChunk, taskPriority);
|
|
+ this.task = EXECUTOR.submitTask(this, taskPriority);
|
|
+ }
|
|
+
|
|
+ private synchronized void setStatus(PendingStatus status) {
|
|
+ this.status = status;
|
|
+ }
|
|
+
|
|
+ private Chunk loadChunk(int x, int z) throws IOException {
|
|
+ setStatus(PendingStatus.LOADING);
|
|
+ Object[] data = chunkLoader.loadChunk(world, x, z, null);
|
|
+ if (data != null) {
|
|
+ // Level must be loaded on main
|
|
+ this.pendingLevel = ((NBTTagCompound) data[1]).getCompound("Level");
|
|
+ return (Chunk) data[0];
|
|
+ } else {
|
|
+ return null;
|
|
+ }
|
|
+ }
|
|
+
|
|
+ private Chunk generateChunk() {
|
|
+ CompletableFuture<Chunk> pending = new CompletableFuture<>();
|
|
+ batchScheduler.startBatch();
|
|
+ batchScheduler.add(new ChunkCoordIntPair(x, z));
|
|
+ try {
|
|
+ ProtoChunk protoChunk = batchScheduler.executeBatch().join();
|
|
+ boolean saved = false;
|
|
+ if (!Bukkit.isPrimaryThread()) {
|
|
+ // If we are async, dispatch later
|
|
+ try {
|
|
+ chunkLoader.saveChunk(world, protoChunk, true);
|
|
+ saved = true;
|
|
+ } catch (IOException | ExceptionWorldConflict e) {
|
|
+ e.printStackTrace();
|
|
+ }
|
|
+ }
|
|
+ Chunk chunk = new Chunk(world, protoChunk, x, z);
|
|
+ if (saved) {
|
|
+ chunk.setLastSaved(world.getTime());
|
|
+ }
|
|
+ generateFinished(chunk);
|
|
+
|
|
+ return chunk;
|
|
+ } catch (Exception e) {
|
|
+ MinecraftServer.LOGGER.error("Couldn't generate chunk (" +world.getWorld().getName() + ":" + x + "," + z + ")", e);
|
|
+ generateFinished(null);
|
|
+ return null;
|
|
+ }
|
|
+ }
|
|
+
|
|
+ boolean loadFinished(Chunk chunk) {
|
|
+ if (chunk != null) {
|
|
+ postChunkToMain(chunk);
|
|
+ return false;
|
|
+ }
|
|
+ loadOnly.complete(null);
|
|
+
|
|
+ synchronized (this) {
|
|
+ if (!canGenerate) {
|
|
+ setStatus(PendingStatus.FAIL);
|
|
+ this.chunk = null;
|
|
+ this.hasFinished = true;
|
|
+ pendingChunks.remove(key);
|
|
+ return false;
|
|
+ } else {
|
|
+ setStatus(PendingStatus.GENERATING);
|
|
+ generating = true;
|
|
+ return true;
|
|
+ }
|
|
+ }
|
|
+ }
|
|
+
|
|
+ void generateFinished(Chunk chunk) {
|
|
+ synchronized (this) {
|
|
+ this.chunk = chunk;
|
|
+ this.hasFinished = true;
|
|
+ }
|
|
+ if (chunk != null) {
|
|
+ postChunkToMain(chunk);
|
|
+ } else {
|
|
+ synchronized (this) {
|
|
+ pendingChunks.remove(key);
|
|
+ completeFutures(null);
|
|
+ }
|
|
+ }
|
|
+ }
|
|
+
|
|
+ synchronized private void completeFutures(Chunk chunk) {
|
|
+ loadOnly.complete(chunk);
|
|
+ generate.complete(chunk);
|
|
+ }
|
|
+
|
|
+ private void postChunkToMain(Chunk chunk) {
|
|
+ synchronized (this) {
|
|
+ setStatus(PendingStatus.PENDING_MAIN);
|
|
+ this.chunk = chunk;
|
|
+ this.hasFinished = true;
|
|
+ }
|
|
+ if (Bukkit.isPrimaryThread()) {
|
|
+ postChunk();
|
|
+ } else {
|
|
+ synchronized (mainThreadQueue) {
|
|
+ mainThreadQueue.add(this::postChunk);
|
|
+ mainThreadQueue.notify();
|
|
+ }
|
|
+ }
|
|
+ }
|
|
+
|
|
+ Chunk postChunk() {
|
|
+ if (!server.isMainThread()) {
|
|
+ throw new IllegalStateException("Must post from main");
|
|
+ }
|
|
+ synchronized (this) {
|
|
+ if (hasPosted) {
|
|
+ return chunk;
|
|
+ }
|
|
+ hasPosted = true;
|
|
+ }
|
|
+ try {
|
|
+ if (chunk == null) {
|
|
+ chunk = chunks.get(key);
|
|
+ completeFutures(chunk);
|
|
+ return chunk;
|
|
+ }
|
|
+ if (pendingLevel != null) {
|
|
+ chunkLoader.loadEntities(pendingLevel, chunk);
|
|
+ pendingLevel = null;
|
|
+ }
|
|
+ synchronized (chunks) {
|
|
+ final Chunk other = chunks.get(key);
|
|
+ if (other != null) {
|
|
+ this.chunk = other;
|
|
+ completeFutures(other);
|
|
+ return other;
|
|
+ }
|
|
+ if (chunk != null) {
|
|
+ chunks.put(key, chunk);
|
|
+ }
|
|
+ }
|
|
+
|
|
+ chunk.addEntities();
|
|
+
|
|
+ completeFutures(chunk);
|
|
+ return chunk;
|
|
+ } finally {
|
|
+ pendingChunks.remove(key);
|
|
+ setStatus(PendingStatus.DONE);
|
|
+ }
|
|
+ }
|
|
+
|
|
+ synchronized void addListener(CompletableFuture<Chunk> future, boolean gen) {
|
|
+ if (hasFinished) {
|
|
+ future.complete(chunk);
|
|
+ } else if (gen) {
|
|
+ canGenerate = true;
|
|
+ generate.thenAccept(future::complete);
|
|
+ } else {
|
|
+ if (generating) {
|
|
+ future.complete(null);
|
|
+ } else {
|
|
+ loadOnly.thenAccept(future::complete);
|
|
+ }
|
|
+ }
|
|
+ }
|
|
+
|
|
+ @Override
|
|
+ public void run() {
|
|
+ try {
|
|
+ if (!loadFinished(loadChunk(x, z))) {
|
|
+ return;
|
|
+ }
|
|
+ } catch (Exception ex) {
|
|
+ MinecraftServer.LOGGER.error("Couldn't load chunk (" +world.getWorld().getName() + ":" + x + "," + z + ")", ex);
|
|
+ if (!(ex instanceof IOException)) {
|
|
+ return;
|
|
+ }
|
|
+ }
|
|
+
|
|
+ if (shouldGenSync) {
|
|
+ synchronized (this) {
|
|
+ setStatus(PendingStatus.GENERATION_PENDING);
|
|
+ mainThreadQueue.add(() -> generateFinished(this.generateChunk()));
|
|
+ }
|
|
+ synchronized (mainThreadQueue) {
|
|
+ mainThreadQueue.notify();
|
|
+ }
|
|
+ } else {
|
|
+ generationExecutor.submitTask(chunkGenTask);
|
|
+ }
|
|
+ }
|
|
+
|
|
+ void bumpPriority() {
|
|
+ task.bumpPriority();
|
|
+ chunkGenTask.bumpPriority();
|
|
+ }
|
|
+ }
|
|
+
|
|
+}
|
|
diff --git a/src/main/java/net/minecraft/server/PlayerChunk.java b/src/main/java/net/minecraft/server/PlayerChunk.java
|
|
index 2c7c8adf7c..04ad94e171 100644
|
|
--- a/src/main/java/net/minecraft/server/PlayerChunk.java
|
|
+++ b/src/main/java/net/minecraft/server/PlayerChunk.java
|
|
@@ -30,13 +30,42 @@ public class PlayerChunk {
|
|
// All may seem good at first, but there's deeper issues if you play for a bit
|
|
boolean chunkExists; // Paper
|
|
private boolean loadInProgress = false;
|
|
- private Runnable loadedRunnable = new Runnable() {
|
|
- public void run() {
|
|
- loadInProgress = false;
|
|
- PlayerChunk.this.chunk = PlayerChunk.this.playerChunkMap.getWorld().getChunkProviderServer().getChunkAt(location.x, location.z, true, true);
|
|
- markChunkUsed(); // Paper - delay chunk unloads
|
|
- }
|
|
+ // Paper start
|
|
+ private java.util.function.Consumer<Chunk> chunkLoadedConsumer = chunk -> {
|
|
+ loadInProgress = false;
|
|
+ PlayerChunk pChunk = PlayerChunk.this;
|
|
+ pChunk.chunk = chunk;
|
|
+ markChunkUsed(); // Paper - delay chunk unloads
|
|
};
|
|
+ private boolean markedHigh = false;
|
|
+ void checkHighPriority(EntityPlayer player) {
|
|
+ if (done || markedHigh || chunk != null) {
|
|
+ return;
|
|
+ }
|
|
+ final double dist = getDistance(player.locX, player.locZ);
|
|
+ if (dist > 8) {
|
|
+ return;
|
|
+ }
|
|
+ if (dist >= 3 && getDistance(player, 5) > 3.5) {
|
|
+ return;
|
|
+ }
|
|
+
|
|
+ markedHigh = true;
|
|
+ playerChunkMap.getWorld().getChunkProviderServer().bumpPriority(location);
|
|
+ }
|
|
+ private double getDistance(EntityPlayer player, int inFront) {
|
|
+ final float yaw = MathHelper.normalizeYaw(player.yaw);
|
|
+ final double x = player.locX + (inFront * Math.cos(Math.toRadians(yaw)));
|
|
+ final double z = player.locZ + (inFront * Math.sin(Math.toRadians(yaw)));
|
|
+ return getDistance(x, z);
|
|
+ }
|
|
+
|
|
+ private double getDistance(double blockX, double blockZ) {
|
|
+ final double x = location.x - ((int)Math.floor(blockX) >> 4);
|
|
+ final double z = location.z - ((int)Math.floor(blockZ) >> 4);
|
|
+ return Math.sqrt((x * x) + (z * z));
|
|
+ }
|
|
+ // Paper end
|
|
// Paper start - delay chunk unloads
|
|
private void markChunkUsed() {
|
|
if (chunk == null) {
|
|
@@ -58,8 +87,8 @@ public class PlayerChunk {
|
|
ChunkProviderServer chunkproviderserver = playerchunkmap.getWorld().getChunkProviderServer();
|
|
|
|
chunkproviderserver.a(i, j);
|
|
- this.chunk = chunkproviderserver.getChunkAt(i, j, true, false);
|
|
- this.chunkExists = this.chunk != null || ChunkIOExecutor.hasQueuedChunkLoad(playerChunkMap.getWorld(), i, j); // Paper
|
|
+ this.chunk = chunkproviderserver.getChunkAt(i, j, false, false); // Paper
|
|
+ this.chunkExists = this.chunk != null || chunkproviderserver.chunkGoingToExists(i, j); // Paper
|
|
markChunkUsed(); // Paper - delay chunk unloads
|
|
}
|
|
|
|
@@ -80,7 +109,7 @@ public class PlayerChunk {
|
|
this.c.add(entityplayer);
|
|
if (this.done) {
|
|
this.sendChunk(entityplayer);
|
|
- }
|
|
+ } else checkHighPriority(entityplayer); // Paper
|
|
|
|
}
|
|
}
|
|
@@ -105,8 +134,13 @@ public class PlayerChunk {
|
|
if (this.chunk != null) {
|
|
return true;
|
|
} else {
|
|
- this.chunk = this.playerChunkMap.getWorld().getChunkProviderServer().getChunkAt(this.location.x, this.location.z, true, flag);
|
|
- markChunkUsed(); // Paper - delay chunk unloads
|
|
+ // Paper start - async chunks
|
|
+ if (!loadInProgress) {
|
|
+ loadInProgress = true;
|
|
+ this.chunk = this.playerChunkMap.getWorld().getChunkProviderServer().getChunkAt(this.location.x, this.location.z, true, flag, markedHigh, chunkLoadedConsumer); // Paper)
|
|
+ markChunkUsed(); // Paper - delay chunk unloads
|
|
+ }
|
|
+ // Paper end
|
|
return this.chunk != null;
|
|
}
|
|
}
|
|
diff --git a/src/main/java/net/minecraft/server/PlayerChunkMap.java b/src/main/java/net/minecraft/server/PlayerChunkMap.java
|
|
index d1a443ca8d..6c54294d3f 100644
|
|
--- a/src/main/java/net/minecraft/server/PlayerChunkMap.java
|
|
+++ b/src/main/java/net/minecraft/server/PlayerChunkMap.java
|
|
@@ -349,7 +349,13 @@ public class PlayerChunkMap {
|
|
if (playerchunk != null) {
|
|
playerchunk.b(entityplayer);
|
|
}
|
|
+ } else { // Paper start
|
|
+ PlayerChunk playerchunk = this.getChunk(l1 - j1, i2 - k1);
|
|
+ if (playerchunk != null) {
|
|
+ playerchunk.checkHighPriority(entityplayer); // Paper
|
|
+ }
|
|
}
|
|
+ // Paper end
|
|
}
|
|
}
|
|
|
|
@@ -360,7 +366,11 @@ public class PlayerChunkMap {
|
|
// CraftBukkit start - send nearest chunks first
|
|
Collections.sort(chunksToLoad, new ChunkCoordComparator(entityplayer));
|
|
for (ChunkCoordIntPair pair : chunksToLoad) {
|
|
- this.c(pair.x, pair.z).a(entityplayer);
|
|
+ // Paper start
|
|
+ PlayerChunk c = this.c(pair.x, pair.z);
|
|
+ c.checkHighPriority(entityplayer);
|
|
+ c.a(entityplayer);
|
|
+ // Paper end
|
|
}
|
|
// CraftBukkit end
|
|
}
|
|
diff --git a/src/main/java/net/minecraft/server/RegionLimitedWorldAccess.java b/src/main/java/net/minecraft/server/RegionLimitedWorldAccess.java
|
|
index 3c35c0f481..187ca2813a 100644
|
|
--- a/src/main/java/net/minecraft/server/RegionLimitedWorldAccess.java
|
|
+++ b/src/main/java/net/minecraft/server/RegionLimitedWorldAccess.java
|
|
@@ -35,7 +35,7 @@ public class RegionLimitedWorldAccess implements GeneratorAccess {
|
|
this.d = l;
|
|
this.e = i;
|
|
this.f = j;
|
|
- this.g = world;
|
|
+ this.g = world.regionLimited(this); // Paper
|
|
this.h = world.getSeed();
|
|
this.m = world.getChunkProvider().getChunkGenerator().getSettings();
|
|
this.i = world.getSeaLevel();
|
|
diff --git a/src/main/java/net/minecraft/server/SchedulerBatch.java b/src/main/java/net/minecraft/server/SchedulerBatch.java
|
|
index d868149d1a..0d94b262ac 100644
|
|
--- a/src/main/java/net/minecraft/server/SchedulerBatch.java
|
|
+++ b/src/main/java/net/minecraft/server/SchedulerBatch.java
|
|
@@ -9,6 +9,7 @@ public class SchedulerBatch<K, T extends SchedulerTask<K, T>, R> {
|
|
private final Scheduler<K, T, R> b;
|
|
private boolean c;
|
|
private int d = 1000;
|
|
+ private final java.util.concurrent.locks.ReentrantLock lock = new java.util.concurrent.locks.ReentrantLock(true); // Paper
|
|
|
|
public SchedulerBatch(Scheduler<K, T, R> scheduler) {
|
|
this.b = scheduler;
|
|
@@ -18,7 +19,9 @@ public class SchedulerBatch<K, T extends SchedulerTask<K, T>, R> {
|
|
this.b.b();
|
|
}
|
|
|
|
+ public void startBatch() { b(); } // Paper - OBFHELPER
|
|
public void b() {
|
|
+ lock.lock(); // Paper
|
|
if (this.c) {
|
|
throw new RuntimeException("Batch already started.");
|
|
} else {
|
|
@@ -27,6 +30,7 @@ public class SchedulerBatch<K, T extends SchedulerTask<K, T>, R> {
|
|
}
|
|
}
|
|
|
|
+ public CompletableFuture<R> add(K k0) { return a(k0); } // Paper - OBFHELPER
|
|
public CompletableFuture<R> a(K object) {
|
|
if (!this.c) {
|
|
throw new RuntimeException("Batch not properly started. Please use startBatch to create a new batch.");
|
|
@@ -42,7 +46,13 @@ public class SchedulerBatch<K, T extends SchedulerTask<K, T>, R> {
|
|
}
|
|
}
|
|
|
|
+ public CompletableFuture<R> executeBatch() { return c(); } // Paper - OBFHELPER
|
|
public CompletableFuture<R> c() {
|
|
+ // Paper start
|
|
+ if (!lock.isHeldByCurrentThread()) {
|
|
+ throw new IllegalStateException("Current thread does not hold the write lock");
|
|
+ }
|
|
+ try {// Paper end
|
|
if (!this.c) {
|
|
throw new RuntimeException("Batch not properly started. Please use startBatch to create a new batch.");
|
|
} else {
|
|
@@ -53,5 +63,6 @@ public class SchedulerBatch<K, T extends SchedulerTask<K, T>, R> {
|
|
this.c = false;
|
|
return this.b.c();
|
|
}
|
|
+ } finally { lock.unlock(); } // Paper
|
|
}
|
|
}
|
|
diff --git a/src/main/java/net/minecraft/server/StructurePiece.java b/src/main/java/net/minecraft/server/StructurePiece.java
|
|
index a5cf017da1..def8730b86 100644
|
|
--- a/src/main/java/net/minecraft/server/StructurePiece.java
|
|
+++ b/src/main/java/net/minecraft/server/StructurePiece.java
|
|
@@ -14,7 +14,7 @@ public abstract class StructurePiece {
|
|
private EnumBlockMirror b;
|
|
private EnumBlockRotation c;
|
|
protected int o;
|
|
- private static final Set<Block> d = ImmutableSet.builder().add(Blocks.NETHER_BRICK_FENCE).add(Blocks.TORCH).add(Blocks.WALL_TORCH).add(Blocks.OAK_FENCE).add(Blocks.SPRUCE_FENCE).add(Blocks.DARK_OAK_FENCE).add(Blocks.ACACIA_FENCE).add(Blocks.BIRCH_FENCE).add(Blocks.JUNGLE_FENCE).add(Blocks.LADDER).add(Blocks.IRON_BARS).build();
|
|
+ private static final Set<Block> d = ImmutableSet.<Block>builder().add(Blocks.NETHER_BRICK_FENCE).add(Blocks.TORCH).add(Blocks.WALL_TORCH).add(Blocks.OAK_FENCE).add(Blocks.SPRUCE_FENCE).add(Blocks.DARK_OAK_FENCE).add(Blocks.ACACIA_FENCE).add(Blocks.BIRCH_FENCE).add(Blocks.JUNGLE_FENCE).add(Blocks.LADDER).add(Blocks.IRON_BARS).build(); // Paper - decompile error
|
|
|
|
public StructurePiece() {
|
|
}
|
|
@@ -63,11 +63,11 @@ public abstract class StructurePiece {
|
|
}
|
|
|
|
public static StructurePiece a(List<StructurePiece> list, StructureBoundingBox structureboundingbox) {
|
|
- for(StructurePiece structurepiece : list) {
|
|
+ synchronized (list) { for(StructurePiece structurepiece : list) { // Paper - synchronize main structure list
|
|
if (structurepiece.d() != null && structurepiece.d().a(structureboundingbox)) {
|
|
return structurepiece;
|
|
}
|
|
- }
|
|
+ }} // Paper
|
|
|
|
return null;
|
|
}
|
|
diff --git a/src/main/java/net/minecraft/server/StructureStart.java b/src/main/java/net/minecraft/server/StructureStart.java
|
|
index f87182b5c4..574930f5fe 100644
|
|
--- a/src/main/java/net/minecraft/server/StructureStart.java
|
|
+++ b/src/main/java/net/minecraft/server/StructureStart.java
|
|
@@ -6,7 +6,7 @@ import java.util.List;
|
|
import java.util.Random;
|
|
|
|
public abstract class StructureStart {
|
|
- protected final List<StructurePiece> a = Lists.newArrayList();
|
|
+ protected final List<StructurePiece> a = java.util.Collections.synchronizedList(Lists.newArrayList()); // Paper
|
|
protected StructureBoundingBox b;
|
|
protected int c;
|
|
protected int d;
|
|
@@ -49,9 +49,9 @@ public abstract class StructureStart {
|
|
protected void a(IBlockAccess var1) {
|
|
this.b = StructureBoundingBox.a();
|
|
|
|
- for(StructurePiece structurepiece : this.a) {
|
|
+ synchronized (this.a) {for(StructurePiece structurepiece : this.a) { // Paper - synchronize
|
|
this.b.b(structurepiece.d());
|
|
- }
|
|
+ }} // Paper
|
|
|
|
}
|
|
|
|
@@ -114,9 +114,9 @@ public abstract class StructureStart {
|
|
int l = k - this.b.e;
|
|
this.b.a(0, l, 0);
|
|
|
|
- for(StructurePiece structurepiece : this.a) {
|
|
+ synchronized (this.a) {for(StructurePiece structurepiece : this.a) { // Paper - synchronize
|
|
structurepiece.a(0, l, 0);
|
|
- }
|
|
+ }} // Paper
|
|
|
|
}
|
|
|
|
@@ -132,9 +132,9 @@ public abstract class StructureStart {
|
|
int i1 = l - this.b.b;
|
|
this.b.a(0, i1, 0);
|
|
|
|
- for(StructurePiece structurepiece : this.a) {
|
|
+ synchronized (this.a) {for(StructurePiece structurepiece : this.a) { // Paper - synchronize
|
|
structurepiece.a(0, i1, 0);
|
|
- }
|
|
+ }} // Paper
|
|
|
|
}
|
|
|
|
diff --git a/src/main/java/net/minecraft/server/World.java b/src/main/java/net/minecraft/server/World.java
|
|
index e52e4bb458..13f69f1b82 100644
|
|
--- a/src/main/java/net/minecraft/server/World.java
|
|
+++ b/src/main/java/net/minecraft/server/World.java
|
|
@@ -46,7 +46,7 @@ import org.bukkit.event.entity.CreatureSpawnEvent.SpawnReason;
|
|
import org.bukkit.generator.ChunkGenerator;
|
|
// CraftBukkit end
|
|
|
|
-public abstract class World implements IEntityAccess, GeneratorAccess, IIBlockAccess, AutoCloseable {
|
|
+public abstract class World implements IEntityAccess, GeneratorAccess, IIBlockAccess, AutoCloseable, Cloneable { // Paper
|
|
|
|
protected static final Logger e = LogManager.getLogger();
|
|
private static final EnumDirection[] a = EnumDirection.values();
|
|
@@ -109,6 +109,24 @@ public abstract class World implements IEntityAccess, GeneratorAccess, IIBlockAc
|
|
protected PersistentVillage villages;
|
|
public final MethodProfiler methodProfiler;
|
|
public final boolean isClientSide;
|
|
+ // Paper start - yes this is hacky as shit
|
|
+ RegionLimitedWorldAccess regionLimited;
|
|
+ World originalWorld;
|
|
+ public World regionLimited(RegionLimitedWorldAccess limitedWorldAccess) {
|
|
+ try {
|
|
+ World clone = (World) super.clone();
|
|
+ clone.regionLimited = limitedWorldAccess;
|
|
+ clone.originalWorld = this;
|
|
+ return clone;
|
|
+ } catch (CloneNotSupportedException e1) {
|
|
+ }
|
|
+ return null;
|
|
+ }
|
|
+ ChunkCoordIntPair[] strongholdCoords;
|
|
+ final java.util.concurrent.atomic.AtomicBoolean
|
|
+ strongholdInit = new java.util.concurrent.atomic.AtomicBoolean
|
|
+ (false);
|
|
+ // Paper end
|
|
public boolean allowMonsters;
|
|
public boolean allowAnimals;
|
|
private boolean J;
|
|
@@ -741,17 +759,42 @@ public abstract class World implements IEntityAccess, GeneratorAccess, IIBlockAc
|
|
|
|
}
|
|
|
|
- public IBlockData getType(BlockPosition blockposition) {
|
|
- // CraftBukkit start - tree generation
|
|
+ // Paper - async variant
|
|
+ public java.util.concurrent.CompletableFuture<IBlockData> getTypeAsync(BlockPosition blockposition) {
|
|
+ int x = blockposition.getX();
|
|
+ int z = blockposition.getZ();
|
|
if (captureTreeGeneration) {
|
|
Iterator<CraftBlockState> it = capturedBlockStates.iterator();
|
|
while (it.hasNext()) {
|
|
CraftBlockState previous = it.next();
|
|
- if (previous.getX() == blockposition.getX() && previous.getY() == blockposition.getY() && previous.getZ() == blockposition.getZ()) {
|
|
- return previous.getHandle();
|
|
+ if (previous.getX() == x && previous.getY() == blockposition.getY() && previous.getZ() == z) {
|
|
+ return java.util.concurrent.CompletableFuture.completedFuture(previous.getHandle());
|
|
}
|
|
}
|
|
}
|
|
+ if (blockposition.isInvalidYLocation()) {
|
|
+ return java.util.concurrent.CompletableFuture.completedFuture(Blocks.VOID_AIR.getBlockData());
|
|
+ } else {
|
|
+ java.util.concurrent.CompletableFuture<IBlockData> future = new java.util.concurrent.CompletableFuture<>();
|
|
+ ((ChunkProviderServer) chunkProvider).getChunkAt(x << 4, z << 4, true, true, (chunk) -> {
|
|
+ future.complete(chunk.getType(blockposition));
|
|
+ });
|
|
+ return future;
|
|
+ }
|
|
+ }
|
|
+ // Paper end
|
|
+
|
|
+ public IBlockData getType(BlockPosition blockposition) {
|
|
+ // CraftBukkit start - tree generation
|
|
+ if (captureTreeGeneration) { // If any of this logic updates, update async variant above
|
|
+ Iterator<CraftBlockState> it = capturedBlockStates.iterator();
|
|
+ while (it.hasNext()) { // If any of this logic updates, update async variant above
|
|
+ CraftBlockState previous = it.next();
|
|
+ if (previous.getX() == blockposition.getX() && previous.getY() == blockposition.getY() && previous.getZ() == blockposition.getZ()) {
|
|
+ return previous.getHandle(); // If any of this logic updates, update async variant above
|
|
+ }
|
|
+ } // If any of this logic updates, update async variant above
|
|
+ }
|
|
// CraftBukkit end
|
|
if (blockposition.isInvalidYLocation()) { // Paper
|
|
return Blocks.VOID_AIR.getBlockData();
|
|
@@ -1022,6 +1065,11 @@ public abstract class World implements IEntityAccess, GeneratorAccess, IIBlockAc
|
|
}
|
|
|
|
public boolean addEntity(Entity entity, SpawnReason spawnReason) { // Changed signature, added SpawnReason
|
|
+ // Paper start
|
|
+ if (regionLimited != null) {
|
|
+ return regionLimited.addEntity(entity, spawnReason);
|
|
+ }
|
|
+ // Paper end
|
|
org.spigotmc.AsyncCatcher.catchOp( "entity add"); // Spigot
|
|
if (entity == null) return false;
|
|
if (entity.valid) { MinecraftServer.LOGGER.error("Attempted Double World add on " + entity, new Throwable()); return true; } // Paper
|
|
diff --git a/src/main/java/net/minecraft/server/WorldGenStronghold.java b/src/main/java/net/minecraft/server/WorldGenStronghold.java
|
|
index fa99fe0146..4f49786aa3 100644
|
|
--- a/src/main/java/net/minecraft/server/WorldGenStronghold.java
|
|
+++ b/src/main/java/net/minecraft/server/WorldGenStronghold.java
|
|
@@ -9,24 +9,29 @@ import java.util.Random;
|
|
import javax.annotation.Nullable;
|
|
|
|
public class WorldGenStronghold extends StructureGenerator<WorldGenFeatureStrongholdConfiguration> {
|
|
- private boolean b;
|
|
- private ChunkCoordIntPair[] c;
|
|
- private long d;
|
|
+ // Paper start - no shared state
|
|
+ //private boolean b;
|
|
+ //private ChunkCoordIntPair[] c;
|
|
+ //private long d;
|
|
+ // Paper end
|
|
|
|
public WorldGenStronghold() {
|
|
}
|
|
|
|
protected boolean a(ChunkGenerator<?> chunkgenerator, Random var2, int i, int j) {
|
|
- if (this.d != chunkgenerator.getSeed()) {
|
|
+ // Paper start
|
|
+ /*if (this.d != chunkgenerator.getSeed()) {
|
|
this.c();
|
|
- }
|
|
+ }*/
|
|
|
|
- if (!this.b) {
|
|
+ synchronized (chunkgenerator.getWorld().strongholdInit) {
|
|
+ if (chunkgenerator.getWorld().strongholdInit.compareAndSet(false, true)) { // Paper
|
|
this.a(chunkgenerator);
|
|
- this.b = true;
|
|
- }
|
|
+ //this.b = true;
|
|
+ }} // Paper
|
|
+ // Paper end
|
|
|
|
- for(ChunkCoordIntPair chunkcoordintpair : this.c) {
|
|
+ for(ChunkCoordIntPair chunkcoordintpair : chunkgenerator.getWorld().strongholdCoords) { // Paper
|
|
if (i == chunkcoordintpair.x && j == chunkcoordintpair.z) {
|
|
return true;
|
|
}
|
|
@@ -36,8 +41,8 @@ public class WorldGenStronghold extends StructureGenerator<WorldGenFeatureStrong
|
|
}
|
|
|
|
private void c() {
|
|
- this.b = false;
|
|
- this.c = null;
|
|
+ //this.b = false; // Paper
|
|
+ //this.c = null; // Paper
|
|
}
|
|
|
|
protected boolean a(GeneratorAccess generatoraccess) {
|
|
@@ -69,20 +74,23 @@ public class WorldGenStronghold extends StructureGenerator<WorldGenFeatureStrong
|
|
if (!chunkgenerator.getWorldChunkManager().a(this)) {
|
|
return null;
|
|
} else {
|
|
- if (this.d != world.getSeed()) {
|
|
- this.c();
|
|
- }
|
|
+ // Paper start
|
|
+ /*if (this.d != chunkgenerator.getSeed()) {
|
|
+ this.c();
|
|
+ }*/
|
|
|
|
- if (!this.b) {
|
|
- this.a(chunkgenerator);
|
|
- this.b = true;
|
|
- }
|
|
+ synchronized (chunkgenerator.getWorld().strongholdInit) {
|
|
+ if (chunkgenerator.getWorld().strongholdInit.compareAndSet(false, true)) { // Paper
|
|
+ this.a(chunkgenerator);
|
|
+ //this.b = true;
|
|
+ }} // Paper
|
|
+ // Paper end
|
|
|
|
BlockPosition blockposition1 = null;
|
|
BlockPosition.MutableBlockPosition blockposition$mutableblockposition = new BlockPosition.MutableBlockPosition(0, 0, 0);
|
|
double d0 = Double.MAX_VALUE;
|
|
|
|
- for(ChunkCoordIntPair chunkcoordintpair : this.c) {
|
|
+ for(ChunkCoordIntPair chunkcoordintpair : chunkgenerator.getWorld().strongholdCoords) { // Paper
|
|
blockposition$mutableblockposition.c((chunkcoordintpair.x << 4) + 8, 32, (chunkcoordintpair.z << 4) + 8);
|
|
double d1 = blockposition$mutableblockposition.n(blockposition);
|
|
if (blockposition1 == null) {
|
|
@@ -99,7 +107,7 @@ public class WorldGenStronghold extends StructureGenerator<WorldGenFeatureStrong
|
|
}
|
|
|
|
private void a(ChunkGenerator<?> chunkgenerator) {
|
|
- this.d = chunkgenerator.getSeed();
|
|
+ //this.d = chunkgenerator.getSeed(); // Paper
|
|
ArrayList arraylist = Lists.newArrayList();
|
|
|
|
for(BiomeBase biomebase : IRegistry.BIOME) {
|
|
@@ -111,7 +119,7 @@ public class WorldGenStronghold extends StructureGenerator<WorldGenFeatureStrong
|
|
int i2 = chunkgenerator.getSettings().e();
|
|
int j2 = chunkgenerator.getSettings().f();
|
|
int i = chunkgenerator.getSettings().g();
|
|
- this.c = new ChunkCoordIntPair[j2];
|
|
+ ChunkCoordIntPair[] strongholdCoords = chunkgenerator.getWorld().strongholdCoords = new ChunkCoordIntPair[j2]; // Paper
|
|
int j = 0;
|
|
Long2ObjectMap long2objectmap = chunkgenerator.getStructureStartCache(this);
|
|
synchronized(long2objectmap) {
|
|
@@ -119,8 +127,8 @@ public class WorldGenStronghold extends StructureGenerator<WorldGenFeatureStrong
|
|
|
|
while(objectiterator.hasNext()) {
|
|
StructureStart structurestart = (StructureStart)objectiterator.next();
|
|
- if (j < this.c.length) {
|
|
- this.c[j++] = new ChunkCoordIntPair(structurestart.e(), structurestart.f());
|
|
+ if (j < strongholdCoords.length) { // Paper
|
|
+ strongholdCoords[j++] = new ChunkCoordIntPair(structurestart.e(), structurestart.f()); // Paper
|
|
}
|
|
}
|
|
}
|
|
@@ -129,11 +137,11 @@ public class WorldGenStronghold extends StructureGenerator<WorldGenFeatureStrong
|
|
random.setSeed(chunkgenerator.getSeed());
|
|
double d1 = random.nextDouble() * Math.PI * 2.0D;
|
|
int k = long2objectmap.size();
|
|
- if (k < this.c.length) {
|
|
+ if (k < strongholdCoords.length) { // Paper
|
|
int l = 0;
|
|
int i1 = 0;
|
|
|
|
- for(int j1 = 0; j1 < this.c.length; ++j1) {
|
|
+ for(int j1 = 0; j1 < strongholdCoords.length; ++j1) { // Paper
|
|
double d0 = (double)(4 * i2 + i2 * i1 * 6) + (random.nextDouble() - 0.5D) * (double)i2 * 2.5D;
|
|
int k1 = (int)Math.round(Math.cos(d1) * d0);
|
|
int l1 = (int)Math.round(Math.sin(d1) * d0);
|
|
@@ -144,7 +152,7 @@ public class WorldGenStronghold extends StructureGenerator<WorldGenFeatureStrong
|
|
}
|
|
|
|
if (j1 >= k) {
|
|
- this.c[j1] = new ChunkCoordIntPair(k1, l1);
|
|
+ strongholdCoords[j1] = new ChunkCoordIntPair(k1, l1); // Paper
|
|
}
|
|
|
|
d1 += (Math.PI * 2D) / (double)i;
|
|
@@ -153,7 +161,7 @@ public class WorldGenStronghold extends StructureGenerator<WorldGenFeatureStrong
|
|
++i1;
|
|
l = 0;
|
|
i = i + 2 * i / (i1 + 1);
|
|
- i = Math.min(i, this.c.length - j1);
|
|
+ i = Math.min(i, strongholdCoords.length - j1); // Paper
|
|
d1 += random.nextDouble() * Math.PI * 2.0D;
|
|
}
|
|
}
|
|
diff --git a/src/main/java/net/minecraft/server/WorldServer.java b/src/main/java/net/minecraft/server/WorldServer.java
|
|
index 14905fceb0..c0aeccffbf 100644
|
|
--- a/src/main/java/net/minecraft/server/WorldServer.java
|
|
+++ b/src/main/java/net/minecraft/server/WorldServer.java
|
|
@@ -732,7 +732,7 @@ public class WorldServer extends World implements IAsyncTaskHandler {
|
|
gen = new org.bukkit.craftbukkit.generator.NormalChunkGenerator(this, this.getSeed());
|
|
}
|
|
|
|
- return new ChunkProviderServer(this, ichunkloader, gen, this.server);
|
|
+ return com.destroystokyo.paper.PaperConfig.asyncChunks ? new PaperAsyncChunkProvider(this, ichunkloader, gen, this.server) : new ChunkProviderServer(this, ichunkloader, gen, this.server); // Paper - async chunks
|
|
// CraftBukkit end
|
|
}
|
|
|
|
diff --git a/src/main/java/org/bukkit/craftbukkit/CraftServer.java b/src/main/java/org/bukkit/craftbukkit/CraftServer.java
|
|
index cb2255a5d0..19f2f2bbd8 100644
|
|
--- a/src/main/java/org/bukkit/craftbukkit/CraftServer.java
|
|
+++ b/src/main/java/org/bukkit/craftbukkit/CraftServer.java
|
|
@@ -1014,8 +1014,12 @@ public final class CraftServer implements Server {
|
|
if (internal.getWorld().getKeepSpawnInMemory()) {
|
|
short short1 = internal.paperConfig.keepLoadedRange; // Paper
|
|
long i = System.currentTimeMillis();
|
|
- for (int j = -short1; j <= short1; j += 16) {
|
|
- for (int k = -short1; k <= short1; k += 16) {
|
|
+ // Paper start
|
|
+ for (ChunkCoordIntPair coords : internal.getChunkProviderServer().getSpiralOutChunks(internal.getSpawn(), short1 >> 4)) {{
|
|
+ int j = coords.x;
|
|
+ int k = coords.z;
|
|
+ // Paper end
|
|
+
|
|
long l = System.currentTimeMillis();
|
|
|
|
if (l < i) {
|
|
@@ -1031,7 +1035,7 @@ public final class CraftServer implements Server {
|
|
}
|
|
|
|
BlockPosition chunkcoordinates = internal.getSpawn();
|
|
- internal.getChunkProviderServer().getChunkAt(chunkcoordinates.getX() + j >> 4, chunkcoordinates.getZ() + k >> 4, true, true);
|
|
+ internal.getChunkProviderServer().getChunkAt(chunkcoordinates.getX() + j >> 4, chunkcoordinates.getZ() + k >> 4, true, true, c -> {}); // Paper
|
|
}
|
|
}
|
|
}
|
|
diff --git a/src/main/java/org/bukkit/craftbukkit/CraftWorld.java b/src/main/java/org/bukkit/craftbukkit/CraftWorld.java
|
|
index d0110070a9..02b6bf2990 100644
|
|
--- a/src/main/java/org/bukkit/craftbukkit/CraftWorld.java
|
|
+++ b/src/main/java/org/bukkit/craftbukkit/CraftWorld.java
|
|
@@ -157,6 +157,16 @@ public class CraftWorld implements World {
|
|
}
|
|
}
|
|
|
|
+ // Paper start - Async chunk load API
|
|
+ @Override
|
|
+ public java.util.concurrent.CompletableFuture<Chunk> getChunkAtAsync(final int x, final int z, final boolean gen) {
|
|
+ final ChunkProviderServer cps = this.world.getChunkProviderServer();
|
|
+ java.util.concurrent.CompletableFuture<Chunk> future = new java.util.concurrent.CompletableFuture<>();
|
|
+ cps.getChunkAt(x, z, true, gen, chunk -> future.complete(chunk != null ? chunk.bukkitChunk : null));
|
|
+ return future;
|
|
+ }
|
|
+ // Paper end
|
|
+
|
|
public Chunk getChunkAt(int x, int z) {
|
|
return this.world.getChunkProviderServer().getChunkAt(x, z, true, true).bukkitChunk;
|
|
}
|
|
@@ -1333,10 +1343,13 @@ public class CraftWorld implements World {
|
|
int chunkCoordZ = chunkcoordinates.getZ() >> 4;
|
|
// Cycle through the 25x25 Chunks around it to load/unload the chunks.
|
|
int radius = world.paperConfig.keepLoadedRange / 16; // Paper
|
|
- for (int x = -radius; x <= radius; x++) { // Paper
|
|
- for (int z = -radius; z <= radius; z++) { // Paper
|
|
+ // Paper start
|
|
+ for (ChunkCoordIntPair coords : world.getChunkProviderServer().getSpiralOutChunks(world.getSpawn(), radius)) {{
|
|
+ int x = coords.x;
|
|
+ int z = coords.z;
|
|
+ // Paper end
|
|
if (keepLoaded) {
|
|
- loadChunk(chunkCoordX + x, chunkCoordZ + z);
|
|
+ getChunkAtAsync(chunkCoordX + x, chunkCoordZ + z, chunk -> {}); // Paper - Async Chunks
|
|
} else {
|
|
if (isChunkLoaded(chunkCoordX + x, chunkCoordZ + z)) {
|
|
unloadChunk(chunkCoordX + x, chunkCoordZ + z);
|
|
diff --git a/src/main/java/org/bukkit/craftbukkit/event/CraftEventFactory.java b/src/main/java/org/bukkit/craftbukkit/event/CraftEventFactory.java
|
|
index 9e903159d9..4ead18b66c 100644
|
|
--- a/src/main/java/org/bukkit/craftbukkit/event/CraftEventFactory.java
|
|
+++ b/src/main/java/org/bukkit/craftbukkit/event/CraftEventFactory.java
|
|
@@ -78,6 +78,7 @@ public class CraftEventFactory {
|
|
public static final DamageSource POISON = CraftDamageSource.copyOf(DamageSource.MAGIC);
|
|
public static org.bukkit.block.Block blockDamage; // For use in EntityDamageByBlockEvent
|
|
public static Entity entityDamage; // For use in EntityDamageByEntityEvent
|
|
+ public static boolean isWorldGen(GeneratorAccess world) { return world instanceof net.minecraft.server.RegionLimitedWorldAccess; } // Paper
|
|
|
|
// helper methods
|
|
private static boolean canBuild(CraftWorld world, Player player, int x, int z) {
|
|
@@ -301,6 +302,7 @@ public class CraftEventFactory {
|
|
CraftServer craftServer = (CraftServer) entity.getServer();
|
|
|
|
CreatureSpawnEvent event = new CreatureSpawnEvent(entity, spawnReason);
|
|
+ if (isWorldGen(entityliving.world)) return event; // Paper - do not call during world gen
|
|
craftServer.getPluginManager().callEvent(event);
|
|
return event;
|
|
}
|
|
@@ -948,6 +950,7 @@ public class CraftEventFactory {
|
|
}
|
|
|
|
BlockIgniteEvent event = new BlockIgniteEvent(bukkitWorld.getBlockAt(block.getX(), block.getY(), block.getZ()), cause, igniter);
|
|
+ if (isWorldGen(world)) return event; // Paper - do not call during world gen
|
|
world.getServer().getPluginManager().callEvent(event);
|
|
return event;
|
|
}
|
|
@@ -972,6 +975,7 @@ public class CraftEventFactory {
|
|
}
|
|
|
|
BlockIgniteEvent event = new BlockIgniteEvent(bukkitWorld.getBlockAt(pos.getX(), pos.getY(), pos.getZ()), cause, bukkitIgniter);
|
|
+ if (isWorldGen(world)) return event; // Paper - do not call during world gen
|
|
world.getServer().getPluginManager().callEvent(event);
|
|
return event;
|
|
}
|
|
@@ -1179,7 +1183,8 @@ public class CraftEventFactory {
|
|
public static BlockPhysicsEvent callBlockPhysicsEvent(GeneratorAccess world, BlockPosition blockposition) {
|
|
org.bukkit.block.Block block = CraftBlock.at(world, blockposition);
|
|
BlockPhysicsEvent event = new BlockPhysicsEvent(block, block.getBlockData());
|
|
- world.getMinecraftWorld().getMinecraftServer().server.getPluginManager().callEvent(event);
|
|
+ if (isWorldGen(world)) return event; // Paper - do not call during world gen
|
|
+ Bukkit.getPluginManager().callEvent(event); // Paper
|
|
return event;
|
|
}
|
|
|
|
@@ -1215,6 +1220,7 @@ public class CraftEventFactory {
|
|
}
|
|
|
|
EntityPotionEffectEvent event = new EntityPotionEffectEvent((LivingEntity) entity.getBukkitEntity(), bukkitOldEffect, bukkitNewEffect, cause, action, willOverride);
|
|
+ if (isWorldGen(entity.world)) return event; // Paper - do not call during world gen
|
|
Bukkit.getPluginManager().callEvent(event);
|
|
|
|
return event;
|
|
@@ -1233,6 +1239,7 @@ public class CraftEventFactory {
|
|
blockState.setData(block);
|
|
|
|
BlockFormEvent event = (entity == null) ? new BlockFormEvent(blockState.getBlock(), blockState) : new EntityBlockFormEvent(entity.getBukkitEntity(), blockState.getBlock(), blockState);
|
|
+ if (isWorldGen(world)) return true; // Paper - do not call during world gen
|
|
world.getServer().getPluginManager().callEvent(event);
|
|
|
|
if (!event.isCancelled()) {
|
|
diff --git a/src/main/java/org/bukkit/craftbukkit/generator/CustomChunkGenerator.java b/src/main/java/org/bukkit/craftbukkit/generator/CustomChunkGenerator.java
|
|
index 9c2adb2351..62c197b80d 100644
|
|
--- a/src/main/java/org/bukkit/craftbukkit/generator/CustomChunkGenerator.java
|
|
+++ b/src/main/java/org/bukkit/craftbukkit/generator/CustomChunkGenerator.java
|
|
@@ -21,6 +21,7 @@ public class CustomChunkGenerator extends InternalChunkGenerator<GeneratorSettin
|
|
private final WorldServer world;
|
|
private final long seed;
|
|
private final Random random;
|
|
+ public final boolean asyncSupported; // Paper
|
|
private final WorldChunkManager chunkManager;
|
|
private final WorldGenStronghold strongholdGen = new WorldGenStronghold();
|
|
private final GeneratorSettingsDefault settings = new GeneratorSettingsDefault();
|
|
@@ -43,6 +44,15 @@ public class CustomChunkGenerator extends InternalChunkGenerator<GeneratorSettin
|
|
this.world = (WorldServer) world;
|
|
this.generator = generator;
|
|
this.seed = seed;
|
|
+ // Paper start
|
|
+ boolean asyncSupported = false;
|
|
+ try {
|
|
+ java.lang.reflect.Field asyncSafe = generator.getClass().getDeclaredField("PAPER_ASYNC_SAFE");
|
|
+ asyncSafe.setAccessible(true);
|
|
+ asyncSupported = asyncSafe.getBoolean(generator);
|
|
+ } catch (NoSuchFieldException | IllegalAccessException ignored) {}
|
|
+ this.asyncSupported = asyncSupported;
|
|
+ // Paper end
|
|
|
|
this.random = new Random(seed);
|
|
this.chunkManager = world.worldProvider.getChunkGenerator().getWorldChunkManager();
|
|
--
|
|
2.19.0
|
|
|