mirror of
https://github.com/PaperMC/Paper.git
synced 2024-11-06 10:49:40 +01:00
7fe98bd520
* more patches * even moar patches
1467 lines
76 KiB
Diff
1467 lines
76 KiB
Diff
From 0000000000000000000000000000000000000000 Mon Sep 17 00:00:00 2001
|
|
From: Aikar <aikar@aikar.co>
|
|
Date: Thu, 4 Jun 2020 22:43:29 -0400
|
|
Subject: [PATCH] Optimize Light Engine
|
|
|
|
Massive update to light to improve performance and chunk loading/generation.
|
|
|
|
1) Massive bit packing/unpacking optimizations and inlining.
|
|
A lot of performance has to do with constant packing and unpacking of bits.
|
|
We now inline a most bit operations, and re-use base x/y/z bits in many places.
|
|
This helps with cpu level processing to just do all the math at once instead
|
|
of having to jump in and out of function calls.
|
|
|
|
This much logic also is likely over the JVM Inline limit for JIT too.
|
|
2) Applied a few of JellySquid's Phosphor mod optimizations such as
|
|
- ensuring we don't notify neighbor chunks when neighbor chunk doesn't need to be notified
|
|
- reduce hasLight checks in initializing light, and prob some more, they are tagged JellySquid where phosphor influence was used.
|
|
3) Optimize hot path accesses to getting updating chunk to have less branching
|
|
4) Optimize getBlock accesses to have less branching, and less unpacking
|
|
5) Have a separate urgent bucket for chunk light tasks. These tasks will always cut in line over non blocking light tasks.
|
|
6) Retain chunk priority while light tasks are enqueued. So if a task comes in at high priority but the queue is full
|
|
of tasks already at a lower priority, before the task was simply added to the end. Now it can cut in line to the front.
|
|
this applies for both urgent and non urgent tasks.
|
|
7) Buffer non urgent tasks even if queueUpdate is called multiple times to improve efficiency.
|
|
8) Fix NPE risk that crashes server in getting nibble data
|
|
|
|
1.17: Depends on chunk urgency patch as well
|
|
|
|
diff --git a/src/main/java/net/minecraft/server/level/ChunkHolder.java b/src/main/java/net/minecraft/server/level/ChunkHolder.java
|
|
index 8260636da673ef095728c208db2d6237bab2db19..9e3629884709126574a52ad44fe7523f01dbcce9 100644
|
|
--- a/src/main/java/net/minecraft/server/level/ChunkHolder.java
|
|
+++ b/src/main/java/net/minecraft/server/level/ChunkHolder.java
|
|
@@ -753,6 +753,7 @@ public class ChunkHolder {
|
|
ioPriority = com.destroystokyo.paper.io.PrioritizedTaskQueue.HIGH_PRIORITY;
|
|
}
|
|
chunkMap.level.asyncChunkTaskManager.raisePriority(pos.x, pos.z, ioPriority);
|
|
+ chunkMap.level.getChunkSource().getLightEngine().queue.changePriority(pos.toLong(), getCurrentPriority(), priority);
|
|
}
|
|
if (getCurrentPriority() != priority) {
|
|
this.onLevelChange.onLevelChange(this.pos, this::getCurrentPriority, priority, this::setPriority); // use preferred priority
|
|
diff --git a/src/main/java/net/minecraft/server/level/ChunkMap.java b/src/main/java/net/minecraft/server/level/ChunkMap.java
|
|
index acc566d14926dcf9e88f3e0837884e4c823d777c..f4dd30c8b3326db72d3b3068ee2291de6f15de7c 100644
|
|
--- a/src/main/java/net/minecraft/server/level/ChunkMap.java
|
|
+++ b/src/main/java/net/minecraft/server/level/ChunkMap.java
|
|
@@ -98,6 +98,7 @@ import net.minecraft.world.level.levelgen.structure.StructureStart;
|
|
import net.minecraft.world.level.levelgen.structure.templatesystem.StructureManager;
|
|
import net.minecraft.world.level.storage.DimensionDataStorage;
|
|
import net.minecraft.world.level.storage.LevelStorageSource;
|
|
+import net.minecraft.world.level.storage.PrimaryLevelData;
|
|
import it.unimi.dsi.fastutil.objects.ObjectRBTreeSet; // Paper
|
|
import org.apache.commons.lang3.mutable.MutableBoolean;
|
|
import org.apache.logging.log4j.LogManager;
|
|
@@ -328,6 +329,7 @@ public class ChunkMap extends ChunkStorage implements ChunkHolder.PlayerProvider
|
|
}
|
|
// Paper end
|
|
|
|
+ private final java.util.concurrent.ExecutorService lightThread;
|
|
public ChunkMap(ServerLevel worldserver, LevelStorageSource.LevelStorageAccess convertable_conversionsession, DataFixer dataFixer, StructureManager definedstructuremanager, Executor workerExecutor, BlockableEventLoop<Runnable> mainThreadExecutor, LightChunkGetter chunkProvider, ChunkGenerator chunkGenerator, ChunkProgressListener worldGenerationProgressListener, Supplier<DimensionDataStorage> supplier, int i, boolean flag) {
|
|
super(new File(convertable_conversionsession.getDimensionPath(worldserver.dimension()), "region"), dataFixer, flag);
|
|
//this.visibleChunks = this.updatingChunks.clone(); // Paper - no more cloning
|
|
@@ -359,7 +361,15 @@ public class ChunkMap extends ChunkStorage implements ChunkHolder.PlayerProvider
|
|
ProcessorHandle<Runnable> mailbox = ProcessorHandle.of("main", mainThreadExecutor::tell);
|
|
|
|
this.progressListener = worldGenerationProgressListener;
|
|
- ProcessorMailbox<Runnable> lightthreaded; ProcessorMailbox<Runnable> threadedmailbox1 = lightthreaded = ProcessorMailbox.create(workerExecutor, "light"); // Paper
|
|
+ // Paper start - use light thread
|
|
+ ProcessorMailbox<Runnable> lightthreaded; ProcessorMailbox<Runnable> threadedmailbox1 = lightthreaded = ProcessorMailbox.create(lightThread = java.util.concurrent.Executors.newSingleThreadExecutor(r -> {
|
|
+ Thread thread = new Thread(r);
|
|
+ thread.setName(((PrimaryLevelData)level.getLevelData()).getLevelName() + " - Light");
|
|
+ thread.setDaemon(true);
|
|
+ thread.setPriority(Thread.NORM_PRIORITY+1);
|
|
+ return thread;
|
|
+ }), "light");
|
|
+ // Paper end
|
|
|
|
this.queueSorter = new ChunkTaskPriorityQueueSorter(ImmutableList.of(threadedmailbox, mailbox, threadedmailbox1), workerExecutor, Integer.MAX_VALUE);
|
|
this.worldgenMailbox = this.queueSorter.getProcessor(threadedmailbox, false);
|
|
@@ -705,6 +715,7 @@ public class ChunkMap extends ChunkStorage implements ChunkHolder.PlayerProvider
|
|
// Paper end
|
|
}
|
|
|
|
+ protected final IntSupplier getPrioritySupplier(long i) { return getChunkQueueLevel(i); } // Paper - OBFHELPER
|
|
protected IntSupplier getChunkQueueLevel(long pos) {
|
|
return () -> {
|
|
ChunkHolder playerchunk = this.getVisibleChunkIfPresent(pos);
|
|
@@ -832,6 +843,7 @@ public class ChunkMap extends ChunkStorage implements ChunkHolder.PlayerProvider
|
|
@Override
|
|
public void close() throws IOException {
|
|
try {
|
|
+ this.lightThread.shutdown(); // Paper
|
|
this.queueSorter.close();
|
|
this.level.asyncChunkTaskManager.close(true); // Paper - Required since we're closing regionfiles in the next line
|
|
this.poiManager.close();
|
|
diff --git a/src/main/java/net/minecraft/server/level/SectionTracker.java b/src/main/java/net/minecraft/server/level/SectionTracker.java
|
|
index 125ae965bb539ae24c60cb992eb7cfc35fd65b25..9fa6c290373b0e0cc0e7ed84c0c2363c8ad14dd3 100644
|
|
--- a/src/main/java/net/minecraft/server/level/SectionTracker.java
|
|
+++ b/src/main/java/net/minecraft/server/level/SectionTracker.java
|
|
@@ -1,6 +1,5 @@
|
|
package net.minecraft.server.level;
|
|
|
|
-import net.minecraft.core.SectionPos;
|
|
import net.minecraft.world.level.lighting.DynamicGraphMinFixedPoint;
|
|
|
|
public abstract class SectionTracker extends DynamicGraphMinFixedPoint {
|
|
@@ -16,14 +15,20 @@ public abstract class SectionTracker extends DynamicGraphMinFixedPoint {
|
|
|
|
@Override
|
|
protected void checkNeighborsAfterUpdate(long id, int level, boolean decrease) {
|
|
+ // Paper start
|
|
+ int x = (int) (id >> 42);
|
|
+ int y = (int) (id << 44 >> 44);
|
|
+ int z = (int) (id << 22 >> 42);
|
|
+ // Paper end
|
|
for (int k = -1; k <= 1; ++k) {
|
|
for (int l = -1; l <= 1; ++l) {
|
|
for (int i1 = -1; i1 <= 1; ++i1) {
|
|
- long j1 = SectionPos.offset(id, k, l, i1);
|
|
+ if (k == 0 && l == 0 && i1 == 0) continue; // Paper
|
|
+ long j1 = (((long) (x + k) & 4194303L) << 42) | (((long) (y + l) & 1048575L)) | (((long) (z + i1) & 4194303L) << 20); // Paper
|
|
|
|
- if (j1 != id) {
|
|
+ //if (j1 != i) { // Paper - checked above
|
|
this.checkNeighbor(id, j1, level, decrease);
|
|
- }
|
|
+ //} // Paper
|
|
}
|
|
}
|
|
}
|
|
@@ -34,10 +39,15 @@ public abstract class SectionTracker extends DynamicGraphMinFixedPoint {
|
|
protected int getComputedLevel(long id, long excludedId, int maxLevel) {
|
|
int l = maxLevel;
|
|
|
|
+ // Paper start
|
|
+ int x = (int) (id >> 42);
|
|
+ int y = (int) (id << 44 >> 44);
|
|
+ int z = (int) (id << 22 >> 42);
|
|
+ // Paper end
|
|
for (int i1 = -1; i1 <= 1; ++i1) {
|
|
for (int j1 = -1; j1 <= 1; ++j1) {
|
|
for (int k1 = -1; k1 <= 1; ++k1) {
|
|
- long l1 = SectionPos.offset(id, i1, j1, k1);
|
|
+ long l1 = (((long) (x + i1) & 4194303L) << 42) | (((long) (y + j1) & 1048575L)) | (((long) (z + k1) & 4194303L) << 20); // Paper
|
|
|
|
if (l1 == id) {
|
|
l1 = Long.MAX_VALUE;
|
|
diff --git a/src/main/java/net/minecraft/server/level/ServerChunkCache.java b/src/main/java/net/minecraft/server/level/ServerChunkCache.java
|
|
index f36badcafbad7fb4537ffdf54d9e266ae3d72459..7a615a18f1f297adfe7e046407a019d8933e9ed9 100644
|
|
--- a/src/main/java/net/minecraft/server/level/ServerChunkCache.java
|
|
+++ b/src/main/java/net/minecraft/server/level/ServerChunkCache.java
|
|
@@ -1072,7 +1072,7 @@ public class ServerChunkCache extends ChunkSource {
|
|
if (ServerChunkCache.this.runDistanceManagerUpdates()) {
|
|
return true;
|
|
} else {
|
|
- ServerChunkCache.this.lightEngine.tryScheduleUpdate();
|
|
+ ServerChunkCache.this.lightEngine.tryScheduleUpdate(); // Paper - not needed
|
|
return super.pollTask() || execChunkTask; // Paper
|
|
}
|
|
} finally {
|
|
diff --git a/src/main/java/net/minecraft/server/level/ThreadedLevelLightEngine.java b/src/main/java/net/minecraft/server/level/ThreadedLevelLightEngine.java
|
|
index cc4190b3a8904d1eaae0f542a3b3090583f5ff82..14835bfab300d305faee2db705d7386dc16427f5 100644
|
|
--- a/src/main/java/net/minecraft/server/level/ThreadedLevelLightEngine.java
|
|
+++ b/src/main/java/net/minecraft/server/level/ThreadedLevelLightEngine.java
|
|
@@ -1,6 +1,7 @@
|
|
package net.minecraft.server.level;
|
|
|
|
import com.mojang.datafixers.util.Pair;
|
|
+import it.unimi.dsi.fastutil.longs.Long2ObjectLinkedOpenHashMap; // Paper
|
|
import it.unimi.dsi.fastutil.objects.ObjectArrayList;
|
|
import it.unimi.dsi.fastutil.objects.ObjectList;
|
|
import it.unimi.dsi.fastutil.objects.ObjectListIterator;
|
|
@@ -16,6 +17,7 @@ import net.minecraft.util.thread.ProcessorMailbox;
|
|
import net.minecraft.world.level.ChunkPos;
|
|
import net.minecraft.world.level.LightLayer;
|
|
import net.minecraft.world.level.chunk.ChunkAccess;
|
|
+import net.minecraft.world.level.chunk.ChunkStatus;
|
|
import net.minecraft.world.level.chunk.DataLayer;
|
|
import net.minecraft.world.level.chunk.LevelChunkSection;
|
|
import net.minecraft.world.level.chunk.LightChunkGetter;
|
|
@@ -27,15 +29,149 @@ public class ThreadedLevelLightEngine extends LevelLightEngine implements AutoCl
|
|
|
|
private static final Logger LOGGER = LogManager.getLogger();
|
|
private final ProcessorMailbox<Runnable> taskMailbox;
|
|
- private final ObjectList<Pair<ThreadedLevelLightEngine.TaskType, Runnable>> lightTasks = new ObjectArrayList();
|
|
- private final ChunkMap chunkMap;
|
|
+ // Paper start
|
|
+ private static final int MAX_PRIORITIES = ChunkMap.MAX_CHUNK_DISTANCE + 2;
|
|
+
|
|
+ private boolean isChunkLightStatus(long pair) {
|
|
+ ChunkHolder playerChunk = playerChunkMap.getVisibleChunkIfPresent(pair);
|
|
+ if (playerChunk == null) {
|
|
+ return false;
|
|
+ }
|
|
+ ChunkStatus status = ChunkHolder.getStatus(playerChunk.getTicketLevel());
|
|
+ return status != null && status.isAtLeastStatus(ChunkStatus.LIGHT);
|
|
+ }
|
|
+
|
|
+ static class ChunkLightQueue {
|
|
+ public boolean shouldFastUpdate;
|
|
+ java.util.ArrayDeque<Runnable> pre = new java.util.ArrayDeque<Runnable>();
|
|
+ java.util.ArrayDeque<Runnable> post = new java.util.ArrayDeque<Runnable>();
|
|
+
|
|
+ ChunkLightQueue(long chunk) {}
|
|
+ }
|
|
+
|
|
+ static class PendingLightTask {
|
|
+ long chunkId;
|
|
+ IntSupplier priority;
|
|
+ Runnable pre;
|
|
+ Runnable post;
|
|
+ boolean fastUpdate;
|
|
+
|
|
+ public PendingLightTask(long chunkId, IntSupplier priority, Runnable pre, Runnable post, boolean fastUpdate) {
|
|
+ this.chunkId = chunkId;
|
|
+ this.priority = priority;
|
|
+ this.pre = pre;
|
|
+ this.post = post;
|
|
+ this.fastUpdate = fastUpdate;
|
|
+ }
|
|
+ }
|
|
+
|
|
+
|
|
+ // Retain the chunks priority level for queued light tasks
|
|
+ class LightQueue {
|
|
+ private int size = 0;
|
|
+ private final Long2ObjectLinkedOpenHashMap<ChunkLightQueue>[] buckets = new Long2ObjectLinkedOpenHashMap[MAX_PRIORITIES];
|
|
+ private final java.util.concurrent.ConcurrentLinkedQueue<PendingLightTask> pendingTasks = new java.util.concurrent.ConcurrentLinkedQueue<>();
|
|
+ private final java.util.concurrent.ConcurrentLinkedQueue<Runnable> priorityChanges = new java.util.concurrent.ConcurrentLinkedQueue<>();
|
|
+
|
|
+ private LightQueue() {
|
|
+ for (int i = 0; i < buckets.length; i++) {
|
|
+ buckets[i] = new Long2ObjectLinkedOpenHashMap<>();
|
|
+ }
|
|
+ }
|
|
+
|
|
+ public void changePriority(long pair, int currentPriority, int priority) {
|
|
+ this.priorityChanges.add(() -> {
|
|
+ ChunkLightQueue remove = this.buckets[currentPriority].remove(pair);
|
|
+ if (remove != null) {
|
|
+ ChunkLightQueue existing = this.buckets[Math.max(1, priority)].put(pair, remove);
|
|
+ if (existing != null) {
|
|
+ remove.pre.addAll(existing.pre);
|
|
+ remove.post.addAll(existing.post);
|
|
+ }
|
|
+ }
|
|
+ });
|
|
+ }
|
|
+
|
|
+ public final void addChunk(long chunkId, IntSupplier priority, Runnable pre, Runnable post) {
|
|
+ pendingTasks.add(new PendingLightTask(chunkId, priority, pre, post, true));
|
|
+ tryScheduleUpdate();
|
|
+ }
|
|
+
|
|
+ public final void add(long chunkId, IntSupplier priority, ThreadedLevelLightEngine.TaskType type, Runnable run) {
|
|
+ pendingTasks.add(new PendingLightTask(chunkId, priority, type == TaskType.PRE_UPDATE ? run : null, type == TaskType.POST_UPDATE ? run : null, false));
|
|
+ }
|
|
+ public final void add(PendingLightTask update) {
|
|
+ int priority = update.priority.getAsInt();
|
|
+ ChunkLightQueue lightQueue = this.buckets[priority].computeIfAbsent(update.chunkId, ChunkLightQueue::new);
|
|
+
|
|
+ if (update.pre != null) {
|
|
+ this.size++;
|
|
+ lightQueue.pre.add(update.pre);
|
|
+ }
|
|
+ if (update.post != null) {
|
|
+ this.size++;
|
|
+ lightQueue.post.add(update.post);
|
|
+ }
|
|
+ if (update.fastUpdate) {
|
|
+ lightQueue.shouldFastUpdate = true;
|
|
+ }
|
|
+ }
|
|
+
|
|
+ public final boolean isEmpty() {
|
|
+ return this.size == 0 && this.pendingTasks.isEmpty();
|
|
+ }
|
|
+
|
|
+ public final int size() {
|
|
+ return this.size;
|
|
+ }
|
|
+
|
|
+ public boolean poll(java.util.List<Runnable> pre, java.util.List<Runnable> post) {
|
|
+ PendingLightTask pending;
|
|
+ while ((pending = pendingTasks.poll()) != null) {
|
|
+ add(pending);
|
|
+ }
|
|
+ Runnable run;
|
|
+ while ((run = priorityChanges.poll()) != null) {
|
|
+ run.run();
|
|
+ }
|
|
+ boolean hasWork = false;
|
|
+ Long2ObjectLinkedOpenHashMap<ChunkLightQueue>[] buckets = this.buckets;
|
|
+ int priority = 0;
|
|
+ while (priority < MAX_PRIORITIES && !isEmpty()) {
|
|
+ Long2ObjectLinkedOpenHashMap<ChunkLightQueue> bucket = buckets[priority];
|
|
+ if (bucket.isEmpty()) {
|
|
+ priority++;
|
|
+ if (hasWork) {
|
|
+ return true;
|
|
+ } else {
|
|
+ continue;
|
|
+ }
|
|
+ }
|
|
+ ChunkLightQueue queue = bucket.removeFirst();
|
|
+ this.size -= queue.pre.size() + queue.post.size();
|
|
+ pre.addAll(queue.pre);
|
|
+ post.addAll(queue.post);
|
|
+ queue.pre.clear();
|
|
+ queue.post.clear();
|
|
+ hasWork = true;
|
|
+ if (queue.shouldFastUpdate) {
|
|
+ return true;
|
|
+ }
|
|
+ }
|
|
+ return hasWork;
|
|
+ }
|
|
+ }
|
|
+
|
|
+ final LightQueue queue = new LightQueue();
|
|
+ // Paper end
|
|
+ private final ChunkMap chunkMap; private final ChunkMap playerChunkMap; // Paper
|
|
private final ProcessorHandle<ChunkTaskPriorityQueueSorter.Message<Runnable>> sorterMailbox;
|
|
private volatile int taskPerBatch = 5;
|
|
private final AtomicBoolean scheduled = new AtomicBoolean();
|
|
|
|
public ThreadedLevelLightEngine(LightChunkGetter chunkProvider, ChunkMap chunkStorage, boolean hasBlockLight, ProcessorMailbox<Runnable> processor, ProcessorHandle<ChunkTaskPriorityQueueSorter.Message<Runnable>> executor) {
|
|
super(chunkProvider, true, hasBlockLight);
|
|
- this.chunkMap = chunkStorage;
|
|
+ this.chunkMap = chunkStorage; this.playerChunkMap = chunkMap; // Paper
|
|
this.sorterMailbox = executor;
|
|
this.taskMailbox = processor;
|
|
}
|
|
@@ -122,13 +258,9 @@ public class ThreadedLevelLightEngine extends LevelLightEngine implements AutoCl
|
|
}
|
|
|
|
private void addTask(int x, int z, IntSupplier completedLevelSupplier, ThreadedLevelLightEngine.TaskType stage, Runnable task) {
|
|
- this.sorterMailbox.tell(ChunkTaskPriorityQueueSorter.message(() -> {
|
|
- this.lightTasks.add(Pair.of(stage, task));
|
|
- if (this.lightTasks.size() >= this.taskPerBatch) {
|
|
- this.runUpdate();
|
|
- }
|
|
-
|
|
- }, ChunkPos.asLong(x, z), completedLevelSupplier));
|
|
+ // Paper start - replace method
|
|
+ this.queue.add(ChunkPos.asLong(x, z), completedLevelSupplier, stage, task);
|
|
+ // Paper end
|
|
}
|
|
|
|
@Override
|
|
@@ -145,8 +277,19 @@ public class ThreadedLevelLightEngine extends LevelLightEngine implements AutoCl
|
|
public CompletableFuture<ChunkAccess> lightChunk(ChunkAccess chunk, boolean excludeBlocks) {
|
|
ChunkPos chunkcoordintpair = chunk.getPos();
|
|
|
|
- chunk.setLightCorrect(false);
|
|
- this.addTask(chunkcoordintpair.x, chunkcoordintpair.z, ThreadedLevelLightEngine.TaskType.PRE_UPDATE, Util.name(() -> {
|
|
+ // Paper start
|
|
+ //ichunkaccess.b(false); // Don't need to disable this
|
|
+ long pair = chunkcoordintpair.toLong();
|
|
+ CompletableFuture<ChunkAccess> future = new CompletableFuture<>();
|
|
+ IntSupplier prioritySupplier = playerChunkMap.getPrioritySupplier(pair);
|
|
+ boolean[] skippedPre = {false};
|
|
+ this.queue.addChunk(pair, prioritySupplier, Util.name(() -> {
|
|
+ if (!isChunkLightStatus(pair)) {
|
|
+ future.complete(chunk);
|
|
+ skippedPre[0] = true;
|
|
+ return;
|
|
+ }
|
|
+ // Paper end
|
|
LevelChunkSection[] achunksection = chunk.getSections();
|
|
|
|
for (int i = 0; i < 16; ++i) {
|
|
@@ -164,55 +307,48 @@ public class ThreadedLevelLightEngine extends LevelLightEngine implements AutoCl
|
|
});
|
|
}
|
|
|
|
- this.chunkMap.releaseLightTicket(chunkcoordintpair);
|
|
+ // this.d.c(chunkcoordintpair); // Paper - move into post task below
|
|
}, () -> {
|
|
return "lightChunk " + chunkcoordintpair + " " + excludeBlocks;
|
|
- }));
|
|
- return CompletableFuture.supplyAsync(() -> {
|
|
+ // Paper start - merge the 2 together
|
|
+ }), () -> {
|
|
+ this.chunkMap.releaseLightTicket(chunkcoordintpair); // Paper - release light tickets as post task to ensure they stay loaded until fully done
|
|
+ if (skippedPre[0]) return; // Paper - future's already complete
|
|
chunk.setLightCorrect(true);
|
|
super.retainData(chunkcoordintpair, false);
|
|
- return chunk;
|
|
- }, (runnable) -> {
|
|
- this.addTask(chunkcoordintpair.x, chunkcoordintpair.z, ThreadedLevelLightEngine.TaskType.POST_UPDATE, runnable);
|
|
+ // Paper start
|
|
+ future.complete(chunk);
|
|
});
|
|
+ return future;
|
|
+ // Paper end
|
|
}
|
|
|
|
public void tryScheduleUpdate() {
|
|
- if ((!this.lightTasks.isEmpty() || super.hasLightWork()) && this.scheduled.compareAndSet(false, true)) {
|
|
+ if ((!this.queue.isEmpty() || super.hasLightWork()) && this.scheduled.compareAndSet(false, true)) { // Paper
|
|
this.taskMailbox.tell((() -> { // Paper - decompile error
|
|
this.runUpdate();
|
|
this.scheduled.set(false);
|
|
+ tryScheduleUpdate(); // Paper - if we still have work to do, do it!
|
|
}));
|
|
}
|
|
|
|
}
|
|
|
|
+ // Paper start - replace impl
|
|
+ private final java.util.List<Runnable> pre = new java.util.ArrayList<>();
|
|
+ private final java.util.List<Runnable> post = new java.util.ArrayList<>();
|
|
private void runUpdate() {
|
|
- int i = Math.min(this.lightTasks.size(), this.taskPerBatch);
|
|
- ObjectListIterator<Pair<ThreadedLevelLightEngine.TaskType, Runnable>> objectlistiterator = this.lightTasks.iterator();
|
|
-
|
|
- Pair pair;
|
|
- int j;
|
|
-
|
|
- for (j = 0; objectlistiterator.hasNext() && j < i; ++j) {
|
|
- pair = (Pair) objectlistiterator.next();
|
|
- if (pair.getFirst() == ThreadedLevelLightEngine.TaskType.PRE_UPDATE) {
|
|
- ((Runnable) pair.getSecond()).run();
|
|
- }
|
|
+ if (queue.poll(pre, post)) {
|
|
+ pre.forEach(Runnable::run);
|
|
+ pre.clear();
|
|
+ super.runUpdates(Integer.MAX_VALUE, true, true);
|
|
+ post.forEach(Runnable::run);
|
|
+ post.clear();
|
|
+ } else {
|
|
+ // might have level updates to go still
|
|
+ super.runUpdates(Integer.MAX_VALUE, true, true);
|
|
}
|
|
-
|
|
- objectlistiterator.back(j);
|
|
- super.runUpdates(Integer.MAX_VALUE, true, true);
|
|
-
|
|
- for (j = 0; objectlistiterator.hasNext() && j < i; ++j) {
|
|
- pair = (Pair) objectlistiterator.next();
|
|
- if (pair.getFirst() == ThreadedLevelLightEngine.TaskType.POST_UPDATE) {
|
|
- ((Runnable) pair.getSecond()).run();
|
|
- }
|
|
-
|
|
- objectlistiterator.remove();
|
|
- }
|
|
-
|
|
+ // Paper end
|
|
}
|
|
|
|
public void setTaskPerBatch(int taskBatchSize) {
|
|
diff --git a/src/main/java/net/minecraft/util/thread/ProcessorMailbox.java b/src/main/java/net/minecraft/util/thread/ProcessorMailbox.java
|
|
index c763aa0c0cf49dd844af94a820103258b49021ae..195535835bdc63f7cfdebeaa957dde590262ea42 100644
|
|
--- a/src/main/java/net/minecraft/util/thread/ProcessorMailbox.java
|
|
+++ b/src/main/java/net/minecraft/util/thread/ProcessorMailbox.java
|
|
@@ -110,7 +110,8 @@ public class ProcessorMailbox<T> implements ProcessorHandle<T>, AutoCloseable, R
|
|
|
|
}
|
|
|
|
- @Override
|
|
+
|
|
+ public final void queue(T t0) { tell(t0); } @Override // Paper - OBFHELPER
|
|
public void tell(T message) {
|
|
this.queue.push(message);
|
|
this.registerForExecution();
|
|
diff --git a/src/main/java/net/minecraft/world/level/chunk/DataLayer.java b/src/main/java/net/minecraft/world/level/chunk/DataLayer.java
|
|
index 83e9d8bff9a31fe13a0e22445cd6eecb7abe8561..1e8ce9894fd0a121da83020c6064b7833af1c5f2 100644
|
|
--- a/src/main/java/net/minecraft/world/level/chunk/DataLayer.java
|
|
+++ b/src/main/java/net/minecraft/world/level/chunk/DataLayer.java
|
|
@@ -11,6 +11,13 @@ import net.minecraft.server.MCUtil;
|
|
public class DataLayer {
|
|
|
|
// Paper start
|
|
+ public static final DataLayer EMPTY_NIBBLE_ARRAY = new DataLayer() {
|
|
+ @Override
|
|
+ public byte[] getData() {
|
|
+ throw new IllegalStateException();
|
|
+ }
|
|
+ };
|
|
+ public long lightCacheKey = Long.MIN_VALUE;
|
|
public static byte[] EMPTY_NIBBLE = new byte[2048];
|
|
private static final int nibbleBucketSizeMultiplier = Integer.getInteger("Paper.nibbleBucketSize", 3072);
|
|
private static final int maxPoolSize = Integer.getInteger("Paper.maxNibblePoolSize", (int) Math.min(6, Math.max(1, Runtime.getRuntime().maxMemory() / 1024 / 1024 / 1024)) * (nibbleBucketSizeMultiplier * 8));
|
|
diff --git a/src/main/java/net/minecraft/world/level/lighting/BlockLightEngine.java b/src/main/java/net/minecraft/world/level/lighting/BlockLightEngine.java
|
|
index 709fc42057f8a0282c3c942067e63abb874d9042..eaaaecb67966e5e366cf59f92674c82d1d87552e 100644
|
|
--- a/src/main/java/net/minecraft/world/level/lighting/BlockLightEngine.java
|
|
+++ b/src/main/java/net/minecraft/world/level/lighting/BlockLightEngine.java
|
|
@@ -23,9 +23,11 @@ public final class BlockLightEngine extends LayerLightEngine<BlockLightSectionSt
|
|
}
|
|
|
|
private int getLightEmission(long blockPos) {
|
|
- int j = BlockPos.getX(blockPos);
|
|
- int k = BlockPos.getY(blockPos);
|
|
- int l = BlockPos.getZ(blockPos);
|
|
+ // Paper start - inline math
|
|
+ int j = (int) (blockPos >> 38);
|
|
+ int k = (int) ((blockPos << 52) >> 52);
|
|
+ int l = (int) ((blockPos << 26) >> 38);
|
|
+ // Paper end
|
|
BlockGetter iblockaccess = this.chunkSource.getChunkForLighting(j >> 4, l >> 4);
|
|
|
|
return iblockaccess != null ? iblockaccess.getLightEmission(this.pos.set(j, k, l)) : 0;
|
|
@@ -40,25 +42,33 @@ public final class BlockLightEngine extends LayerLightEngine<BlockLightSectionSt
|
|
} else if (level >= 15) {
|
|
return level;
|
|
} else {
|
|
- int l = Integer.signum(BlockPos.getX(targetId) - BlockPos.getX(sourceId));
|
|
- int i1 = Integer.signum(BlockPos.getY(targetId) - BlockPos.getY(sourceId));
|
|
- int j1 = Integer.signum(BlockPos.getZ(targetId) - BlockPos.getZ(sourceId));
|
|
+ // Paper start - reuse math - credit to JellySquid for idea
|
|
+ int jx = (int) (targetId >> 38);
|
|
+ int jy = (int) ((targetId << 52) >> 52);
|
|
+ int jz = (int) ((targetId << 26) >> 38);
|
|
+ int ix = (int) (sourceId >> 38);
|
|
+ int iy = (int) ((sourceId << 52) >> 52);
|
|
+ int iz = (int) ((sourceId << 26) >> 38);
|
|
+ int l = Integer.signum(jx - ix);
|
|
+ int i1 = Integer.signum(jy - iy);
|
|
+ int j1 = Integer.signum(jz - iz);
|
|
+ // Paper end
|
|
Direction enumdirection = Direction.fromNormal(l, i1, j1);
|
|
|
|
if (enumdirection == null) {
|
|
return 15;
|
|
} else {
|
|
//MutableInt mutableint = new MutableInt(); // Paper - share mutableint, single threaded
|
|
- BlockState iblockdata = this.getStateAndOpacity(targetId, mutableint);
|
|
-
|
|
- if (mutableint.getValue() >= 15) {
|
|
+ BlockState iblockdata = this.getBlockOptimized(jx, jy, jz, mutableint); // Paper
|
|
+ int blockedLight = mutableint.getValue(); // Paper
|
|
+ if (blockedLight >= 15) { // Paper
|
|
return 15;
|
|
} else {
|
|
- BlockState iblockdata1 = this.getStateAndOpacity(sourceId, (MutableInt) null);
|
|
+ BlockState iblockdata1 = this.getBlockOptimized(ix, iy, iz); // Paper
|
|
VoxelShape voxelshape = this.getShape(iblockdata1, sourceId, enumdirection);
|
|
VoxelShape voxelshape1 = this.getShape(iblockdata, targetId, enumdirection.getOpposite());
|
|
|
|
- return Shapes.faceShapeOccludes(voxelshape, voxelshape1) ? 15 : level + Math.max(1, mutableint.getValue());
|
|
+ return Shapes.faceShapeOccludes(voxelshape, voxelshape1) ? 15 : level + Math.max(1, blockedLight); // Paper
|
|
}
|
|
}
|
|
}
|
|
@@ -66,14 +76,19 @@ public final class BlockLightEngine extends LayerLightEngine<BlockLightSectionSt
|
|
|
|
@Override
|
|
protected void checkNeighborsAfterUpdate(long id, int level, boolean decrease) {
|
|
- long k = SectionPos.blockToSection(id);
|
|
+ // Paper start - reuse unpacking, credit to JellySquid (Didn't do full optimization though)
|
|
+ int x = (int) (id >> 38);
|
|
+ int y = (int) ((id << 52) >> 52);
|
|
+ int z = (int) ((id << 26) >> 38);
|
|
+ long k = SectionPos.blockPosAsSectionLong(x, y, z);
|
|
+ // Paper end
|
|
Direction[] aenumdirection = BlockLightEngine.DIRECTIONS;
|
|
int l = aenumdirection.length;
|
|
|
|
for (int i1 = 0; i1 < l; ++i1) {
|
|
Direction enumdirection = aenumdirection[i1];
|
|
- long j1 = BlockPos.offset(id, enumdirection);
|
|
- long k1 = SectionPos.blockToSection(j1);
|
|
+ long j1 = BlockPos.getAdjacent(x, y, z, enumdirection); // Paper
|
|
+ long k1 = SectionPos.blockToSection(j1); // Paper
|
|
|
|
if (k == k1 || ((BlockLightSectionStorage) this.storage).storingLightForSection(k1)) {
|
|
this.checkNeighbor(id, j1, level, decrease);
|
|
@@ -98,27 +113,37 @@ public final class BlockLightEngine extends LayerLightEngine<BlockLightSectionSt
|
|
}
|
|
}
|
|
|
|
- long j1 = SectionPos.blockToSection(id);
|
|
- DataLayer nibblearray = ((BlockLightSectionStorage) this.storage).getDataLayer(j1, true);
|
|
+ // Paper start
|
|
+ int baseX = (int) (id >> 38);
|
|
+ int baseY = (int) ((id << 52) >> 52);
|
|
+ int baseZ = (int) ((id << 26) >> 38);
|
|
+ long j1 = SectionPos.blockPosAsSectionLong(baseX, baseY, baseZ);
|
|
+ DataLayer nibblearray = this.storage.updating.getUpdatingOptimized(j1);
|
|
+ // Paper end
|
|
Direction[] aenumdirection = BlockLightEngine.DIRECTIONS;
|
|
int k1 = aenumdirection.length;
|
|
|
|
for (int l1 = 0; l1 < k1; ++l1) {
|
|
Direction enumdirection = aenumdirection[l1];
|
|
- long i2 = BlockPos.offset(id, enumdirection);
|
|
+ // Paper start
|
|
+ int newX = baseX + enumdirection.getStepX();
|
|
+ int newY = baseY + enumdirection.getStepY();
|
|
+ int newZ = baseZ + enumdirection.getStepZ();
|
|
+ long i2 = BlockPos.asLong(newX, newY, newZ);
|
|
|
|
if (i2 != excludedId) {
|
|
- long j2 = SectionPos.blockToSection(i2);
|
|
+ long j2 = SectionPos.blockPosAsSectionLong(newX, newY, newZ);
|
|
+ // Paper end
|
|
DataLayer nibblearray1;
|
|
|
|
if (j1 == j2) {
|
|
nibblearray1 = nibblearray;
|
|
} else {
|
|
- nibblearray1 = ((BlockLightSectionStorage) this.storage).getDataLayer(j2, true);
|
|
+ nibblearray1 = ((BlockLightSectionStorage) this.storage).updating.getUpdatingOptimized(j2); // Paper
|
|
}
|
|
|
|
if (nibblearray1 != null) {
|
|
- int k2 = this.computeLevelFromNeighbor(i2, id, this.getLevel(nibblearray1, i2));
|
|
+ int k2 = this.computeLevelFromNeighbor(i2, id, this.getNibbleLightInverse(nibblearray1, newX, newY, newZ)); // Paper
|
|
|
|
if (l > k2) {
|
|
l = k2;
|
|
diff --git a/src/main/java/net/minecraft/world/level/lighting/BlockLightSectionStorage.java b/src/main/java/net/minecraft/world/level/lighting/BlockLightSectionStorage.java
|
|
index a1ad4d73ddaf6afe97a1f1ff7e0622b52fac8761..f771ef8d841567b421b6c0529af3f0713c79eb7c 100644
|
|
--- a/src/main/java/net/minecraft/world/level/lighting/BlockLightSectionStorage.java
|
|
+++ b/src/main/java/net/minecraft/world/level/lighting/BlockLightSectionStorage.java
|
|
@@ -1,8 +1,6 @@
|
|
package net.minecraft.world.level.lighting;
|
|
|
|
import it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap;
|
|
-import net.minecraft.core.BlockPos;
|
|
-import net.minecraft.core.SectionPos;
|
|
import net.minecraft.world.level.LightLayer;
|
|
import net.minecraft.world.level.chunk.DataLayer;
|
|
import net.minecraft.world.level.chunk.LightChunkGetter;
|
|
@@ -15,10 +13,14 @@ public class BlockLightSectionStorage extends LayerLightSectionStorage<BlockLigh
|
|
|
|
@Override
|
|
protected int getLightValue(long blockPos) {
|
|
- long j = SectionPos.blockToSection(blockPos);
|
|
- DataLayer nibblearray = this.getDataLayer(j, false);
|
|
-
|
|
- return nibblearray == null ? 0 : nibblearray.get(SectionPos.sectionRelative(BlockPos.getX(blockPos)), SectionPos.sectionRelative(BlockPos.getY(blockPos)), SectionPos.sectionRelative(BlockPos.getZ(blockPos)));
|
|
+ // Paper start
|
|
+ int baseX = (int) (blockPos >> 38);
|
|
+ int baseY = (int) ((blockPos << 52) >> 52);
|
|
+ int baseZ = (int) ((blockPos << 26) >> 38);
|
|
+ long j = (((long) (baseX >> 4) & 4194303L) << 42) | (((long) (baseY >> 4) & 1048575L)) | (((long) (baseZ >> 4) & 4194303L) << 20);
|
|
+ DataLayer nibblearray = this.e_visible.lookup.apply(j);
|
|
+ return nibblearray == null ? 0 : nibblearray.get(baseX & 15, baseY & 15, baseZ & 15);
|
|
+ // Paper end
|
|
}
|
|
|
|
public static final class BlockDataLayerStorageMap extends DataLayerStorageMap<BlockLightSectionStorage.BlockDataLayerStorageMap> {
|
|
diff --git a/src/main/java/net/minecraft/world/level/lighting/DataLayerStorageMap.java b/src/main/java/net/minecraft/world/level/lighting/DataLayerStorageMap.java
|
|
index 54cca3b376e5ce02936edc8b9c17e67e17f07147..ed2ed6194670016086be580dc4514d5d3d1b235b 100644
|
|
--- a/src/main/java/net/minecraft/world/level/lighting/DataLayerStorageMap.java
|
|
+++ b/src/main/java/net/minecraft/world/level/lighting/DataLayerStorageMap.java
|
|
@@ -7,13 +7,18 @@ import net.minecraft.world.level.chunk.DataLayer;
|
|
|
|
public abstract class DataLayerStorageMap<M extends DataLayerStorageMap<M>> {
|
|
|
|
- private final long[] lastSectionKeys = new long[2];
|
|
- private final DataLayer[] lastSections = new DataLayer[2];
|
|
+ // private final long[] b = new long[2]; // Paper - unused
|
|
+ private final DataLayer[] lastSections = new DataLayer[]{DataLayer.EMPTY_NIBBLE_ARRAY, DataLayer.EMPTY_NIBBLE_ARRAY}; private final DataLayer[] cache = lastSections; // Paper - OBFHELPER
|
|
private boolean cacheEnabled;
|
|
protected final com.destroystokyo.paper.util.map.QueuedChangesMapLong2Object<DataLayer> data; // Paper - avoid copying light data
|
|
protected final boolean isVisible; // Paper - avoid copying light data
|
|
- java.util.function.Function<Long, DataLayer> lookup; // Paper - faster branchless lookup
|
|
|
|
+ // Paper start - faster lookups with less branching, use interface to avoid boxing instead of Function
|
|
+ public final NibbleArrayAccess lookup;
|
|
+ public interface NibbleArrayAccess {
|
|
+ DataLayer apply(long id);
|
|
+ }
|
|
+ // Paper end
|
|
// Paper start - avoid copying light data
|
|
protected DataLayerStorageMap(com.destroystokyo.paper.util.map.QueuedChangesMapLong2Object<DataLayer> data, boolean isVisible) {
|
|
if (isVisible) {
|
|
@@ -21,12 +26,14 @@ public abstract class DataLayerStorageMap<M extends DataLayerStorageMap<M>> {
|
|
}
|
|
this.data = data;
|
|
this.isVisible = isVisible;
|
|
+ // Paper end - avoid copying light data
|
|
+ // Paper start - faster lookups with less branching
|
|
if (isVisible) {
|
|
lookup = data::getVisibleAsync;
|
|
} else {
|
|
- lookup = data::getUpdating;
|
|
+ lookup = data.getUpdatingMap()::get; // jump straight the sub map
|
|
}
|
|
- // Paper end - avoid copying light data
|
|
+ // Paper end
|
|
this.clearCache();
|
|
this.cacheEnabled = true;
|
|
}
|
|
@@ -36,7 +43,9 @@ public abstract class DataLayerStorageMap<M extends DataLayerStorageMap<M>> {
|
|
public void copyDataLayer(long pos) {
|
|
if (this.isVisible) { throw new IllegalStateException("writing to visible data"); } // Paper - avoid copying light data
|
|
DataLayer updating = this.data.getUpdating(pos); // Paper - pool nibbles
|
|
- this.data.queueUpdate(pos, new DataLayer().markPoolSafe(updating.getCloneIfSet())); // Paper - avoid copying light data - pool safe clone
|
|
+ DataLayer nibblearray = new DataLayer().markPoolSafe(updating.getCloneIfSet()); // Paper
|
|
+ nibblearray.lightCacheKey = pos; // Paper
|
|
+ this.data.queueUpdate(pos, nibblearray); // Paper - avoid copying light data - pool safe clone
|
|
if (updating.cleaner != null) MCUtil.scheduleTask(2, updating.cleaner, "Light Engine Release"); // Paper - delay clean incase anything holding ref was still using it
|
|
this.clearCache();
|
|
}
|
|
@@ -45,34 +54,34 @@ public abstract class DataLayerStorageMap<M extends DataLayerStorageMap<M>> {
|
|
return lookup.apply(chunkPos) != null; // Paper - avoid copying light data
|
|
}
|
|
|
|
- @Nullable
|
|
- public final DataLayer getLayer(long chunkPos) { // Paper - final
|
|
- if (this.cacheEnabled) {
|
|
- for (int j = 0; j < 2; ++j) {
|
|
- if (chunkPos == this.lastSectionKeys[j]) {
|
|
- return this.lastSections[j];
|
|
- }
|
|
- }
|
|
- }
|
|
-
|
|
- DataLayer nibblearray = lookup.apply(chunkPos); // Paper - avoid copying light data
|
|
+ // Paper start - less branching as we know we are using cache and updating
|
|
+ public final DataLayer getUpdatingOptimized(final long i) { // Paper - final
|
|
+ final DataLayer[] cache = this.cache;
|
|
+ if (cache[0].lightCacheKey == i) return cache[0];
|
|
+ if (cache[1].lightCacheKey == i) return cache[1];
|
|
|
|
+ final DataLayer nibblearray = this.lookup.apply(i); // Paper - avoid copying light data
|
|
if (nibblearray == null) {
|
|
return null;
|
|
} else {
|
|
- if (this.cacheEnabled) {
|
|
- for (int k = 1; k > 0; --k) {
|
|
- this.lastSectionKeys[k] = this.lastSectionKeys[k - 1];
|
|
- this.lastSections[k] = this.lastSections[k - 1];
|
|
- }
|
|
-
|
|
- this.lastSectionKeys[0] = chunkPos;
|
|
- this.lastSections[0] = nibblearray;
|
|
- }
|
|
-
|
|
+ cache[1] = cache[0];
|
|
+ cache[0] = nibblearray;
|
|
return nibblearray;
|
|
}
|
|
}
|
|
+ // Paper end
|
|
+
|
|
+ @Nullable
|
|
+ public final DataLayer getLayer(final long chunkPos) { // Paper - final
|
|
+ // Paper start - optimize visible case or missed updating cases
|
|
+ if (this.cacheEnabled) {
|
|
+ // short circuit to optimized
|
|
+ return getUpdatingOptimized(chunkPos);
|
|
+ }
|
|
+
|
|
+ return this.lookup.apply(chunkPos);
|
|
+ // Paper end
|
|
+ }
|
|
|
|
@Nullable
|
|
public DataLayer removeLayer(long chunkPos) {
|
|
@@ -82,13 +91,14 @@ public abstract class DataLayerStorageMap<M extends DataLayerStorageMap<M>> {
|
|
|
|
public void setLayer(long pos, DataLayer data) {
|
|
if (this.isVisible) { throw new IllegalStateException("writing to visible data"); } // Paper - avoid copying light data
|
|
+ data.lightCacheKey = pos; // Paper
|
|
this.data.queueUpdate(pos, data); // Paper - avoid copying light data
|
|
}
|
|
|
|
public void clearCache() {
|
|
for (int i = 0; i < 2; ++i) {
|
|
- this.lastSectionKeys[i] = Long.MAX_VALUE;
|
|
- this.lastSections[i] = null;
|
|
+ // this.b[i] = Long.MAX_VALUE; // Paper - Unused
|
|
+ this.lastSections[i] = DataLayer.EMPTY_NIBBLE_ARRAY; // Paper
|
|
}
|
|
}
|
|
|
|
diff --git a/src/main/java/net/minecraft/world/level/lighting/LayerLightEngine.java b/src/main/java/net/minecraft/world/level/lighting/LayerLightEngine.java
|
|
index 53f38fa95f4ffad12c73d94ab1d7ecf7ee78af09..088ea8a14f1bb264b59fcec626b1a28d7f6d7c47 100644
|
|
--- a/src/main/java/net/minecraft/world/level/lighting/LayerLightEngine.java
|
|
+++ b/src/main/java/net/minecraft/world/level/lighting/LayerLightEngine.java
|
|
@@ -10,6 +10,7 @@ import net.minecraft.world.level.ChunkPos;
|
|
import net.minecraft.world.level.LightLayer;
|
|
import net.minecraft.world.level.block.Blocks;
|
|
import net.minecraft.world.level.block.state.BlockState;
|
|
+import net.minecraft.world.level.chunk.ChunkAccess;
|
|
import net.minecraft.world.level.chunk.DataLayer;
|
|
import net.minecraft.world.level.chunk.LightChunkGetter;
|
|
import net.minecraft.world.phys.shapes.Shapes;
|
|
@@ -23,10 +24,37 @@ public abstract class LayerLightEngine<M extends DataLayerStorageMap<M>, S exten
|
|
protected final LightLayer layer;
|
|
protected final S storage;
|
|
private boolean runningLightUpdates;
|
|
- protected final BlockPos.MutableBlockPos pos = new BlockPos.MutableBlockPos();
|
|
+ protected final BlockPos.MutableBlockPos pos = new BlockPos.MutableBlockPos(); protected final BlockPos.MutableBlockPos pos = pos; // Paper
|
|
private final long[] lastChunkPos = new long[2];
|
|
- private final BlockGetter[] lastChunk = new BlockGetter[2];
|
|
+ private final ChunkAccess[] h = new ChunkAccess[2]; // Paper
|
|
|
|
+ // Paper start - see fully commented out method below (look for Bedrock)
|
|
+ // optimized method with less branching for when scenarios arent needed.
|
|
+ // avoid using mutable version if can
|
|
+ protected final BlockState getBlockOptimized(int x, int y, int z, MutableInt mutableint) {
|
|
+ ChunkAccess iblockaccess = this.a(x >> 4, z >> 4);
|
|
+
|
|
+ if (iblockaccess == null) {
|
|
+ mutableint.setValue(16);
|
|
+ return Blocks.BEDROCK.defaultBlockState();
|
|
+ } else {
|
|
+ this.pos.setValues(x, y, z);
|
|
+ BlockState iblockdata = iblockaccess.getType(x, y, z);
|
|
+ mutableint.setValue(iblockdata.getLightBlock(this.chunkSource.getLevel(), this.pos));
|
|
+ return iblockdata.canOcclude() && iblockdata.useShapeForLightOcclusion() ? iblockdata : Blocks.AIR.defaultBlockState();
|
|
+ }
|
|
+ }
|
|
+ protected final BlockState getBlockOptimized(int x, int y, int z) {
|
|
+ ChunkAccess iblockaccess = this.a(x >> 4, z >> 4);
|
|
+
|
|
+ if (iblockaccess == null) {
|
|
+ return Blocks.BEDROCK.defaultBlockState();
|
|
+ } else {
|
|
+ BlockState iblockdata = iblockaccess.getType(x, y, z);
|
|
+ return iblockdata.canOcclude() && iblockdata.useShapeForLightOcclusion() ? iblockdata : Blocks.AIR.defaultBlockState();
|
|
+ }
|
|
+ }
|
|
+ // Paper end
|
|
public LayerLightEngine(LightChunkGetter chunkProvider, LightLayer type, S lightStorage) {
|
|
super(16, 256, 8192);
|
|
this.chunkSource = chunkProvider;
|
|
@@ -45,63 +73,65 @@ public abstract class LayerLightEngine<M extends DataLayerStorageMap<M>, S exten
|
|
}
|
|
|
|
@Nullable
|
|
- private BlockGetter getChunk(int chunkX, int chunkZ) {
|
|
- long k = ChunkPos.asLong(chunkX, chunkZ);
|
|
+ private ChunkAccess a(int i, int j) { // Paper
|
|
+ long k = ChunkPos.asLong(i, j);
|
|
|
|
for (int l = 0; l < 2; ++l) {
|
|
if (k == this.lastChunkPos[l]) {
|
|
- return this.lastChunk[l];
|
|
+ return this.h[l];
|
|
}
|
|
}
|
|
|
|
- BlockGetter iblockaccess = this.chunkSource.getChunkForLighting(chunkX, chunkZ);
|
|
+ ChunkAccess iblockaccess = (ChunkAccess) this.chunkSource.getChunkForLighting(i, j); // Paper
|
|
|
|
for (int i1 = 1; i1 > 0; --i1) {
|
|
this.lastChunkPos[i1] = this.lastChunkPos[i1 - 1];
|
|
- this.lastChunk[i1] = this.lastChunk[i1 - 1];
|
|
+ this.h[i1] = this.h[i1 - 1];
|
|
}
|
|
|
|
this.lastChunkPos[0] = k;
|
|
- this.lastChunk[0] = iblockaccess;
|
|
+ this.h[0] = iblockaccess;
|
|
return iblockaccess;
|
|
}
|
|
|
|
private void clearCache() {
|
|
Arrays.fill(this.lastChunkPos, ChunkPos.INVALID_CHUNK_POS);
|
|
- Arrays.fill(this.lastChunk, (Object) null);
|
|
+ Arrays.fill(this.h, (Object) null);
|
|
}
|
|
|
|
- protected BlockState getStateAndOpacity(long pos, @Nullable MutableInt mutableint) {
|
|
- if (pos == Long.MAX_VALUE) {
|
|
- if (mutableint != null) {
|
|
- mutableint.setValue(0);
|
|
- }
|
|
-
|
|
- return Blocks.AIR.defaultBlockState();
|
|
- } else {
|
|
- int j = SectionPos.blockToSectionCoord(BlockPos.getX(pos));
|
|
- int k = SectionPos.blockToSectionCoord(BlockPos.getZ(pos));
|
|
- BlockGetter iblockaccess = this.getChunk(j, k);
|
|
-
|
|
- if (iblockaccess == null) {
|
|
- if (mutableint != null) {
|
|
- mutableint.setValue(16);
|
|
- }
|
|
-
|
|
- return Blocks.BEDROCK.defaultBlockState();
|
|
- } else {
|
|
- this.pos.set(pos);
|
|
- BlockState iblockdata = iblockaccess.getBlockState(this.pos);
|
|
- boolean flag = iblockdata.canOcclude() && iblockdata.useShapeForLightOcclusion();
|
|
-
|
|
- if (mutableint != null) {
|
|
- mutableint.setValue(iblockdata.getLightBlock(this.chunkSource.getLevel(), (BlockPos) this.pos));
|
|
- }
|
|
-
|
|
- return flag ? iblockdata : Blocks.AIR.defaultBlockState();
|
|
- }
|
|
- }
|
|
- }
|
|
+ // Paper start - comment out, see getBlockOptimized
|
|
+// protected IBlockData a(long i, @Nullable MutableInt mutableint) {
|
|
+// if (i == Long.MAX_VALUE) {
|
|
+// if (mutableint != null) {
|
|
+// mutableint.setValue(0);
|
|
+// }
|
|
+//
|
|
+// return Blocks.AIR.getBlockData();
|
|
+// } else {
|
|
+// int j = SectionPosition.a(BlockPosition.b(i));
|
|
+// int k = SectionPosition.a(BlockPosition.d(i));
|
|
+// IBlockAccess iblockaccess = this.a(j, k);
|
|
+//
|
|
+// if (iblockaccess == null) {
|
|
+// if (mutableint != null) {
|
|
+// mutableint.setValue(16);
|
|
+// }
|
|
+//
|
|
+// return Blocks.BEDROCK.getBlockData();
|
|
+// } else {
|
|
+// this.d.g(i);
|
|
+// IBlockData iblockdata = iblockaccess.getType(this.d);
|
|
+// boolean flag = iblockdata.l() && iblockdata.e();
|
|
+//
|
|
+// if (mutableint != null) {
|
|
+// mutableint.setValue(iblockdata.b(this.a.getWorld(), (BlockPosition) this.d));
|
|
+// }
|
|
+//
|
|
+// return flag ? iblockdata : Blocks.AIR.getBlockData();
|
|
+// }
|
|
+// }
|
|
+// }
|
|
+ // Paper end
|
|
|
|
protected VoxelShape getShape(BlockState world, long pos, Direction facing) {
|
|
return world.canOcclude() ? world.getFaceOcclusionShape(this.chunkSource.getLevel(), this.pos.set(pos), facing) : Shapes.empty();
|
|
@@ -136,8 +166,9 @@ public abstract class LayerLightEngine<M extends DataLayerStorageMap<M>, S exten
|
|
return id == Long.MAX_VALUE ? 0 : 15 - this.storage.getStoredLevel(id);
|
|
}
|
|
|
|
+ protected int getNibbleLightInverse(DataLayer nibblearray, int x, int y, int z) { return 15 - nibblearray.get(x & 15, y & 15, z & 15); } // Paper - x/y/z version of below
|
|
protected int getLevel(DataLayer section, long blockPos) {
|
|
- return 15 - section.get(SectionPos.sectionRelative(BlockPos.getX(blockPos)), SectionPos.sectionRelative(BlockPos.getY(blockPos)), SectionPos.sectionRelative(BlockPos.getZ(blockPos)));
|
|
+ return 15 - section.get((int) (blockPos >> 38) & 15, (int) ((blockPos << 52) >> 52) & 15, (int) ((blockPos << 26) >> 38) & 15); // Paper
|
|
}
|
|
|
|
@Override
|
|
diff --git a/src/main/java/net/minecraft/world/level/lighting/LayerLightSectionStorage.java b/src/main/java/net/minecraft/world/level/lighting/LayerLightSectionStorage.java
|
|
index 5757bcfded35f112d52a7c81586850ba50e0d8dd..17a6610b352af5d3e2cbcdf9b4d9b0d4d356b5cf 100644
|
|
--- a/src/main/java/net/minecraft/world/level/lighting/LayerLightSectionStorage.java
|
|
+++ b/src/main/java/net/minecraft/world/level/lighting/LayerLightSectionStorage.java
|
|
@@ -27,9 +27,9 @@ public abstract class LayerLightSectionStorage<M extends DataLayerStorageMap<M>>
|
|
protected final LongSet toMarkNoData = new LongOpenHashSet();
|
|
protected final LongSet toMarkData = new LongOpenHashSet();
|
|
protected volatile M e_visible; protected final Object visibleUpdateLock = new Object(); // Paper - diff on change, should be "visible" - force compile fail on usage change
|
|
- protected final M updatingSectionData; // Paper - diff on change, should be "updating"
|
|
+ protected final M updatingSectionData; protected final M updating; // Paper - diff on change, should be "updating"
|
|
protected final LongSet changedSections = new LongOpenHashSet();
|
|
- protected final LongSet sectionsAffectedByLightUpdates = new LongOpenHashSet();
|
|
+ protected final LongSet sectionsAffectedByLightUpdates = new LongOpenHashSet(); LongSet dirty = sectionsAffectedByLightUpdates; // Paper - OBFHELPER
|
|
protected final Long2ObjectMap<DataLayer> queuedSections = Long2ObjectMaps.synchronize(new Long2ObjectOpenHashMap());
|
|
private final LongSet untrustedSections = new LongOpenHashSet();
|
|
private final LongSet columnsToRetainQueuedDataFor = new LongOpenHashSet();
|
|
@@ -37,33 +37,33 @@ public abstract class LayerLightSectionStorage<M extends DataLayerStorageMap<M>>
|
|
protected volatile boolean hasToRemove;
|
|
|
|
protected LayerLightSectionStorage(LightLayer lightType, LightChunkGetter chunkProvider, M lightData) {
|
|
- super(3, 16, 256);
|
|
+ super(3, 256, 256); // Paper - bump expected size of level sets to improve collisions and reduce rehashing (seen a lot of it)
|
|
this.layer = lightType;
|
|
this.chunkSource = chunkProvider;
|
|
- this.updatingSectionData = lightData;
|
|
+ this.updatingSectionData = lightData; updating = lightData; // Paper
|
|
this.e_visible = lightData.copy(); // Paper - avoid copying light data
|
|
this.e_visible.disableCache(); // Paper - avoid copying light data
|
|
}
|
|
|
|
- protected boolean storingLightForSection(long sectionPos) {
|
|
- return this.getDataLayer(sectionPos, true) != null;
|
|
+ protected final boolean storingLightForSection(long sectionPos) { // Paper - final to help inlining
|
|
+ return this.updating.getUpdatingOptimized(sectionPos) != null; // Paper - inline to avoid branching
|
|
}
|
|
|
|
@Nullable
|
|
protected DataLayer getDataLayer(long sectionPos, boolean cached) {
|
|
// Paper start - avoid copying light data
|
|
if (cached) {
|
|
- return this.getDataLayer(this.updatingSectionData, sectionPos);
|
|
+ return this.updating.getUpdatingOptimized(sectionPos);
|
|
} else {
|
|
synchronized (this.visibleUpdateLock) {
|
|
- return this.getDataLayer(this.e_visible, sectionPos);
|
|
+ return this.e_visible.lookup.apply(sectionPos);
|
|
}
|
|
}
|
|
// Paper end - avoid copying light data
|
|
}
|
|
|
|
@Nullable
|
|
- protected DataLayer getDataLayer(M storage, long sectionPos) {
|
|
+ protected final DataLayer getDataLayer(M storage, long sectionPos) { // Paper
|
|
return storage.getLayer(sectionPos);
|
|
}
|
|
|
|
@@ -77,27 +77,57 @@ public abstract class LayerLightSectionStorage<M extends DataLayerStorageMap<M>>
|
|
protected abstract int getLightValue(long blockPos);
|
|
|
|
protected int getStoredLevel(long blockPos) {
|
|
- long j = SectionPos.blockToSection(blockPos);
|
|
- DataLayer nibblearray = this.getDataLayer(j, true);
|
|
+ // Paper start - reuse and inline math, use Optimized Updating path
|
|
+ final int x = (int) (blockPos >> 38);
|
|
+ final int y = (int) ((blockPos << 52) >> 52);
|
|
+ final int z = (int) ((blockPos << 26) >> 38);
|
|
+ long j = SectionPos.blockPosAsSectionLong(x, y, z);
|
|
+ DataLayer nibblearray = this.updating.getUpdatingOptimized(j);
|
|
+ // BUG: Sometimes returns null and crashes, try to recover, but to prevent crash just return no light.
|
|
+ if (nibblearray == null) {
|
|
+ nibblearray = this.e_visible.lookup.apply(j);
|
|
+ }
|
|
+ if (nibblearray == null) {
|
|
+ System.err.println("Null nibble, preventing crash " + BlockPos.of(blockPos));
|
|
+ return 0;
|
|
+ }
|
|
|
|
- return nibblearray.get(SectionPos.sectionRelative(BlockPos.getX(blockPos)), SectionPos.sectionRelative(BlockPos.getY(blockPos)), SectionPos.sectionRelative(BlockPos.getZ(blockPos)));
|
|
+ return nibblearray.get(x & 15, y & 15, z & 15); // Paper - inline operations
|
|
+ // Paper end
|
|
}
|
|
|
|
protected void setStoredLevel(long blockPos, int value) {
|
|
- long k = SectionPos.blockToSection(blockPos);
|
|
+ // Paper start - cache part of the math done in loop below
|
|
+ int x = (int) (blockPos >> 38);
|
|
+ int y = (int) ((blockPos << 52) >> 52);
|
|
+ int z = (int) ((blockPos << 26) >> 38);
|
|
+ long k = SectionPos.blockPosAsSectionLong(x, y, z);
|
|
+ // Paper end
|
|
|
|
if (this.changedSections.add(k)) {
|
|
this.updatingSectionData.copyDataLayer(k);
|
|
}
|
|
|
|
DataLayer nibblearray = this.getDataLayer(k, true);
|
|
-
|
|
- nibblearray.set(SectionPos.sectionRelative(BlockPos.getX(blockPos)), SectionPos.sectionRelative(BlockPos.getY(blockPos)), SectionPos.sectionRelative(BlockPos.getZ(blockPos)), value);
|
|
-
|
|
- for (int l = -1; l <= 1; ++l) {
|
|
- for (int i1 = -1; i1 <= 1; ++i1) {
|
|
- for (int j1 = -1; j1 <= 1; ++j1) {
|
|
- this.sectionsAffectedByLightUpdates.add(SectionPos.blockToSection(BlockPos.offset(blockPos, i1, j1, l)));
|
|
+ nibblearray.set(x & 15, y & 15, z & 15, value); // Paper - use already calculated x/y/z
|
|
+
|
|
+ // Paper start - credit to JellySquid for a major optimization here:
|
|
+ /*
|
|
+ * An extremely important optimization is made here in regards to adding items to the pending notification set. The
|
|
+ * original implementation attempts to add the coordinate of every chunk which contains a neighboring block position
|
|
+ * even though a huge number of loop iterations will simply map to block positions within the same updating chunk.
|
|
+ *
|
|
+ * Our implementation here avoids this by pre-calculating the min/max chunk coordinates so we can iterate over only
|
|
+ * the relevant chunk positions once. This reduces what would always be 27 iterations to just 1-8 iterations.
|
|
+ *
|
|
+ * @reason Use faster implementation
|
|
+ * @author JellySquid
|
|
+ */
|
|
+ for (int z2 = (z - 1) >> 4; z2 <= (z + 1) >> 4; ++z2) {
|
|
+ for (int x2 = (x - 1) >> 4; x2 <= (x + 1) >> 4; ++x2) {
|
|
+ for (int y2 = (y - 1) >> 4; y2 <= (y + 1) >> 4; ++y2) {
|
|
+ this.dirty.add(SectionPos.asLong(x2, y2, z2));
|
|
+ // Paper end
|
|
}
|
|
}
|
|
}
|
|
@@ -129,17 +159,23 @@ public abstract class LayerLightSectionStorage<M extends DataLayerStorageMap<M>>
|
|
}
|
|
|
|
if (k >= 2 && level != 2) {
|
|
- if (this.toRemove.contains(id)) {
|
|
- this.toRemove.remove(id);
|
|
- } else {
|
|
+ if (!this.toRemove.remove(id)) { // Paper - remove useless contains - credit to JellySquid
|
|
+ //this.p.remove(i); // Paper
|
|
+ //} else { // Paper
|
|
this.updatingSectionData.setLayer(id, this.createDataLayer(id));
|
|
this.changedSections.add(id);
|
|
this.onNodeAdded(id);
|
|
|
|
- for (int l = -1; l <= 1; ++l) {
|
|
- for (int i1 = -1; i1 <= 1; ++i1) {
|
|
- for (int j1 = -1; j1 <= 1; ++j1) {
|
|
- this.sectionsAffectedByLightUpdates.add(SectionPos.blockToSection(BlockPos.offset(id, i1, j1, l)));
|
|
+ // Paper start - reuse x/y/z and only notify valid chunks - Credit to JellySquid (See above method for notes)
|
|
+ int x = (int) (id >> 38);
|
|
+ int y = (int) ((id << 52) >> 52);
|
|
+ int z = (int) ((id << 26) >> 38);
|
|
+
|
|
+ for (int z2 = (z - 1) >> 4; z2 <= (z + 1) >> 4; ++z2) {
|
|
+ for (int x2 = (x - 1) >> 4; x2 <= (x + 1) >> 4; ++x2) {
|
|
+ for (int y2 = (y - 1) >> 4; y2 <= (y + 1) >> 4; ++y2) {
|
|
+ this.dirty.add(SectionPos.asLong(x2, y2, z2));
|
|
+ // Paper end
|
|
}
|
|
}
|
|
}
|
|
@@ -165,9 +201,9 @@ public abstract class LayerLightSectionStorage<M extends DataLayerStorageMap<M>>
|
|
return SectionPos.blockToSection(j) == sectionPos;
|
|
});
|
|
} else {
|
|
- int j = SectionPos.sectionToBlockCoord(SectionPos.x(sectionPos));
|
|
- int k = SectionPos.sectionToBlockCoord(SectionPos.y(sectionPos));
|
|
- int l = SectionPos.sectionToBlockCoord(SectionPos.z(sectionPos));
|
|
+ int j = (int) (sectionPos >> 42) << 4; // Paper - inline
|
|
+ int k = (int) (sectionPos << 44 >> 44) << 4; // Paper - inline
|
|
+ int l = (int) (sectionPos << 22 >> 42) << 4; // Paper - inline
|
|
|
|
for (int i1 = 0; i1 < 16; ++i1) {
|
|
for (int j1 = 0; j1 < 16; ++j1) {
|
|
@@ -194,7 +230,7 @@ public abstract class LayerLightSectionStorage<M extends DataLayerStorageMap<M>>
|
|
DataLayer nibblearray;
|
|
|
|
while (longiterator.hasNext()) {
|
|
- i = (Long) longiterator.next();
|
|
+ i = longiterator.nextLong(); // Paper
|
|
this.clearQueuedSectionBlocks(lightProvider, i);
|
|
DataLayer nibblearray1 = (DataLayer) this.queuedSections.remove(i);
|
|
|
|
@@ -212,7 +248,7 @@ public abstract class LayerLightSectionStorage<M extends DataLayerStorageMap<M>>
|
|
longiterator = this.toRemove.iterator();
|
|
|
|
while (longiterator.hasNext()) {
|
|
- i = (Long) longiterator.next();
|
|
+ i = longiterator.nextLong(); // Paper
|
|
this.onNodeRemoved(i);
|
|
}
|
|
|
|
@@ -223,12 +259,13 @@ public abstract class LayerLightSectionStorage<M extends DataLayerStorageMap<M>>
|
|
Entry entry;
|
|
long j;
|
|
|
|
+ DataLayer test = null; // Paper
|
|
while (objectiterator.hasNext()) {
|
|
entry = (Entry) objectiterator.next();
|
|
j = entry.getLongKey();
|
|
- if (this.storingLightForSection(j)) {
|
|
+ if ((test = this.updating.getUpdatingOptimized(j)) != null) { // Paper - dont look up nibble twice
|
|
nibblearray = (DataLayer) entry.getValue();
|
|
- if (this.updatingSectionData.getLayer(j) != nibblearray) {
|
|
+ if (test != nibblearray) { // Paper
|
|
this.clearQueuedSectionBlocks(lightProvider, j);
|
|
this.updatingSectionData.setLayer(j, nibblearray);
|
|
this.changedSections.add(j);
|
|
@@ -241,14 +278,14 @@ public abstract class LayerLightSectionStorage<M extends DataLayerStorageMap<M>>
|
|
longiterator = this.queuedSections.keySet().iterator();
|
|
|
|
while (longiterator.hasNext()) {
|
|
- i = (Long) longiterator.next();
|
|
+ i = longiterator.nextLong(); // Paper
|
|
this.checkEdgesForSection(lightProvider, i);
|
|
}
|
|
} else {
|
|
longiterator = this.untrustedSections.iterator();
|
|
|
|
while (longiterator.hasNext()) {
|
|
- i = (Long) longiterator.next();
|
|
+ i = longiterator.nextLong(); // Paper
|
|
this.checkEdgesForSection(lightProvider, i);
|
|
}
|
|
}
|
|
@@ -269,15 +306,20 @@ public abstract class LayerLightSectionStorage<M extends DataLayerStorageMap<M>>
|
|
|
|
private void checkEdgesForSection(LayerLightEngine<M, ?> lightProvider, long sectionPos) {
|
|
if (this.storingLightForSection(sectionPos)) {
|
|
- int j = SectionPos.sectionToBlockCoord(SectionPos.x(sectionPos));
|
|
- int k = SectionPos.sectionToBlockCoord(SectionPos.y(sectionPos));
|
|
- int l = SectionPos.sectionToBlockCoord(SectionPos.z(sectionPos));
|
|
+ // Paper start
|
|
+ int secX = (int) (sectionPos >> 42);
|
|
+ int secY = (int) (sectionPos << 44 >> 44);
|
|
+ int secZ = (int) (sectionPos << 22 >> 42);
|
|
+ int j = secX << 4; // baseX
|
|
+ int k = secY << 4; // baseY
|
|
+ int l = secZ << 4; // baseZ
|
|
+ // Paper end
|
|
Direction[] aenumdirection = LayerLightSectionStorage.DIRECTIONS;
|
|
int i1 = aenumdirection.length;
|
|
|
|
for (int j1 = 0; j1 < i1; ++j1) {
|
|
Direction enumdirection = aenumdirection[j1];
|
|
- long k1 = SectionPos.offset(sectionPos, enumdirection);
|
|
+ long k1 = SectionPos.getAdjacentFromSectionPos(secX, secY, secZ, enumdirection); // Paper - avoid extra unpacking
|
|
|
|
if (!this.queuedSections.containsKey(k1) && this.storingLightForSection(k1)) {
|
|
for (int l1 = 0; l1 < 16; ++l1) {
|
|
diff --git a/src/main/java/net/minecraft/world/level/lighting/SkyLightEngine.java b/src/main/java/net/minecraft/world/level/lighting/SkyLightEngine.java
|
|
index ff1fbc46776b26ca56c3293e40ed55028230ec46..da4003aebc8d5ffce695071af9a27139568d773f 100644
|
|
--- a/src/main/java/net/minecraft/world/level/lighting/SkyLightEngine.java
|
|
+++ b/src/main/java/net/minecraft/world/level/lighting/SkyLightEngine.java
|
|
@@ -4,6 +4,7 @@ import net.minecraft.core.BlockPos;
|
|
import net.minecraft.core.Direction;
|
|
import net.minecraft.core.SectionPos;
|
|
import net.minecraft.world.level.LightLayer;
|
|
+import net.minecraft.world.level.block.Blocks;
|
|
import net.minecraft.world.level.block.state.BlockState;
|
|
import net.minecraft.world.level.chunk.DataLayer;
|
|
import net.minecraft.world.level.chunk.LightChunkGetter;
|
|
@@ -38,21 +39,25 @@ public final class SkyLightEngine extends LayerLightEngine<SkyLightSectionStorag
|
|
return level;
|
|
} else {
|
|
//MutableInt mutableint = new MutableInt(); // Paper - share mutableint, single threaded
|
|
- BlockState iblockdata = this.getStateAndOpacity(targetId, mutableint);
|
|
-
|
|
- if (mutableint.getValue() >= 15) {
|
|
+ // Paper start - use x/y/z and optimized block lookup
|
|
+ int jx = (int) (targetId >> 38);
|
|
+ int jy = (int) ((targetId << 52) >> 52);
|
|
+ int jz = (int) ((targetId << 26) >> 38);
|
|
+ BlockState iblockdata = this.getBlockOptimized(jx, jy, jz, mutableint);
|
|
+ int blockedLight = mutableint.getValue();
|
|
+ if (blockedLight >= 15) {
|
|
+ // Paper end
|
|
return 15;
|
|
} else {
|
|
- int l = BlockPos.getX(sourceId);
|
|
- int i1 = BlockPos.getY(sourceId);
|
|
- int j1 = BlockPos.getZ(sourceId);
|
|
- int k1 = BlockPos.getX(targetId);
|
|
- int l1 = BlockPos.getY(targetId);
|
|
- int i2 = BlockPos.getZ(targetId);
|
|
- boolean flag = l == k1 && j1 == i2;
|
|
- int j2 = Integer.signum(k1 - l);
|
|
- int k2 = Integer.signum(l1 - i1);
|
|
- int l2 = Integer.signum(i2 - j1);
|
|
+ // Paper start - inline math
|
|
+ int ix = (int) (sourceId >> 38);
|
|
+ int iy = (int) ((sourceId << 52) >> 52);
|
|
+ int iz = (int) ((sourceId << 26) >> 38);
|
|
+ boolean flag = ix == jx && iz == jz;
|
|
+ int j2 = Integer.signum(jx - ix);
|
|
+ int k2 = Integer.signum(jy - iy);
|
|
+ int l2 = Integer.signum(jz - iz);
|
|
+ // Paper end
|
|
Direction enumdirection;
|
|
|
|
if (sourceId == Long.MAX_VALUE) {
|
|
@@ -61,7 +66,7 @@ public final class SkyLightEngine extends LayerLightEngine<SkyLightSectionStorag
|
|
enumdirection = Direction.fromNormal(j2, k2, l2);
|
|
}
|
|
|
|
- BlockState iblockdata1 = this.getStateAndOpacity(sourceId, (MutableInt) null);
|
|
+ BlockState iblockdata1 = sourceId == Long.MAX_VALUE ? Blocks.AIR.defaultBlockState() : this.getBlockOptimized(ix, iy, iz); // Paper
|
|
VoxelShape voxelshape;
|
|
|
|
if (enumdirection != null) {
|
|
@@ -91,9 +96,9 @@ public final class SkyLightEngine extends LayerLightEngine<SkyLightSectionStorag
|
|
}
|
|
}
|
|
|
|
- boolean flag1 = sourceId == Long.MAX_VALUE || flag && i1 > l1;
|
|
+ boolean flag1 = sourceId == Long.MAX_VALUE || flag && iy > jy; // Paper rename vars to iy > jy
|
|
|
|
- return flag1 && level == 0 && mutableint.getValue() == 0 ? 0 : level + Math.max(1, mutableint.getValue());
|
|
+ return flag1 && level == 0 && blockedLight == 0 ? 0 : level + Math.max(1, blockedLight); // Paper
|
|
}
|
|
}
|
|
}
|
|
@@ -101,10 +106,14 @@ public final class SkyLightEngine extends LayerLightEngine<SkyLightSectionStorag
|
|
|
|
@Override
|
|
protected void checkNeighborsAfterUpdate(long id, int level, boolean decrease) {
|
|
- long k = SectionPos.blockToSection(id);
|
|
- int l = BlockPos.getY(id);
|
|
- int i1 = SectionPos.sectionRelative(l);
|
|
- int j1 = SectionPos.blockToSectionCoord(l);
|
|
+ // Paper start
|
|
+ int baseX = (int) (id >> 38);
|
|
+ int baseY = (int) ((id << 52) >> 52);
|
|
+ int baseZ = (int) ((id << 26) >> 38);
|
|
+ long k = SectionPos.blockPosAsSectionLong(baseX, baseY, baseZ);
|
|
+ int i1 = baseY & 15;
|
|
+ int j1 = baseY >> 4;
|
|
+ // Paper end
|
|
int k1;
|
|
|
|
if (i1 != 0) {
|
|
@@ -119,15 +128,16 @@ public final class SkyLightEngine extends LayerLightEngine<SkyLightSectionStorag
|
|
k1 = l1;
|
|
}
|
|
|
|
- long i2 = BlockPos.offset(id, 0, -1 - k1 * 16, 0);
|
|
- long j2 = SectionPos.blockToSection(i2);
|
|
+ int newBaseY = baseY + (-1 - k1 * 16); // Paper
|
|
+ long i2 = BlockPos.asLong(baseX, newBaseY, baseZ); // Paper
|
|
+ long j2 = SectionPos.blockPosAsSectionLong(baseX, newBaseY, baseZ); // Paper
|
|
|
|
if (k == j2 || ((SkyLightSectionStorage) this.storage).storingLightForSection(j2)) {
|
|
this.checkNeighbor(id, i2, level, decrease);
|
|
}
|
|
|
|
- long k2 = BlockPos.offset(id, Direction.UP);
|
|
- long l2 = SectionPos.blockToSection(k2);
|
|
+ long k2 = BlockPos.asLong(baseX, baseY + 1, baseZ); // Paper
|
|
+ long l2 = SectionPos.blockPosAsSectionLong(baseX, baseY + 1, baseZ); // Paper
|
|
|
|
if (k == l2 || ((SkyLightSectionStorage) this.storage).storingLightForSection(l2)) {
|
|
this.checkNeighbor(id, k2, level, decrease);
|
|
@@ -142,8 +152,8 @@ public final class SkyLightEngine extends LayerLightEngine<SkyLightSectionStorag
|
|
int k3 = 0;
|
|
|
|
while (true) {
|
|
- long l3 = BlockPos.offset(id, enumdirection.getStepX(), -k3, enumdirection.getStepZ());
|
|
- long i4 = SectionPos.blockToSection(l3);
|
|
+ long l3 = BlockPos.asLong(baseX + enumdirection.getStepX(), baseY - k3, baseZ + enumdirection.getStepZ()); // Paper
|
|
+ long i4 = SectionPos.blockPosAsSectionLong(baseX + enumdirection.getStepX(), baseY - k3, baseZ + enumdirection.getStepZ()); // Paper
|
|
|
|
if (k == i4) {
|
|
this.checkNeighbor(id, l3, level, decrease);
|
|
@@ -181,26 +191,36 @@ public final class SkyLightEngine extends LayerLightEngine<SkyLightSectionStorag
|
|
}
|
|
}
|
|
|
|
- long j1 = SectionPos.blockToSection(id);
|
|
- DataLayer nibblearray = ((SkyLightSectionStorage) this.storage).getDataLayer(j1, true);
|
|
+ // Paper start
|
|
+ int baseX = (int) (id >> 38);
|
|
+ int baseY = (int) ((id << 52) >> 52);
|
|
+ int baseZ = (int) ((id << 26) >> 38);
|
|
+ long j1 = SectionPos.blockPosAsSectionLong(baseX, baseY, baseZ);
|
|
+ DataLayer nibblearray = this.storage.updating.getUpdatingOptimized(j1);
|
|
+ // Paper end
|
|
Direction[] aenumdirection = SkyLightEngine.DIRECTIONS;
|
|
int k1 = aenumdirection.length;
|
|
|
|
for (int l1 = 0; l1 < k1; ++l1) {
|
|
Direction enumdirection = aenumdirection[l1];
|
|
- long i2 = BlockPos.offset(id, enumdirection);
|
|
- long j2 = SectionPos.blockToSection(i2);
|
|
+ // Paper start
|
|
+ int newX = baseX + enumdirection.getStepX();
|
|
+ int newY = baseY + enumdirection.getStepY();
|
|
+ int newZ = baseZ + enumdirection.getStepZ();
|
|
+ long i2 = BlockPos.asLong(newX, newY, newZ);
|
|
+ long j2 = SectionPos.blockPosAsSectionLong(newX, newY, newZ);
|
|
+ // Paper end
|
|
DataLayer nibblearray1;
|
|
|
|
if (j1 == j2) {
|
|
nibblearray1 = nibblearray;
|
|
} else {
|
|
- nibblearray1 = ((SkyLightSectionStorage) this.storage).getDataLayer(j2, true);
|
|
+ nibblearray1 = ((SkyLightSectionStorage) this.storage).updating.getUpdatingOptimized(j2); // Paper
|
|
}
|
|
|
|
if (nibblearray1 != null) {
|
|
if (i2 != excludedId) {
|
|
- int k2 = this.computeLevelFromNeighbor(i2, id, this.getLevel(nibblearray1, i2));
|
|
+ int k2 = this.computeLevelFromNeighbor(i2, id, this.getNibbleLightInverse(nibblearray1, newX, newY, newZ)); // Paper
|
|
|
|
if (l > k2) {
|
|
l = k2;
|
|
@@ -215,7 +235,7 @@ public final class SkyLightEngine extends LayerLightEngine<SkyLightSectionStorag
|
|
j2 = SectionPos.offset(j2, Direction.UP);
|
|
}
|
|
|
|
- DataLayer nibblearray2 = ((SkyLightSectionStorage) this.storage).getDataLayer(j2, true);
|
|
+ DataLayer nibblearray2 = this.storage.updating.getUpdatingOptimized(j2); // Paper
|
|
|
|
if (i2 != excludedId) {
|
|
int l2;
|
|
diff --git a/src/main/java/net/minecraft/world/level/lighting/SkyLightSectionStorage.java b/src/main/java/net/minecraft/world/level/lighting/SkyLightSectionStorage.java
|
|
index 88f168f9d4c29cfc93500227bf8a60de4b6e4d8a..0dcf3ee97fd596f57ca74d7dc75f8ded2faad515 100644
|
|
--- a/src/main/java/net/minecraft/world/level/lighting/SkyLightSectionStorage.java
|
|
+++ b/src/main/java/net/minecraft/world/level/lighting/SkyLightSectionStorage.java
|
|
@@ -28,7 +28,12 @@ public class SkyLightSectionStorage extends LayerLightSectionStorage<SkyLightSec
|
|
|
|
@Override
|
|
protected int getLightValue(long blockPos) {
|
|
- long j = SectionPos.blockToSection(blockPos);
|
|
+ // Paper start
|
|
+ int baseX = (int) (blockPos >> 38);
|
|
+ int baseY = (int) ((blockPos << 52) >> 52);
|
|
+ int baseZ = (int) ((blockPos << 26) >> 38);
|
|
+ long j = SectionPos.blockPosAsSectionLong(baseX, baseY, baseZ);
|
|
+ // Paper end
|
|
int k = SectionPos.y(j);
|
|
synchronized (this.visibleUpdateLock) { // Paper - avoid copying light data
|
|
SkyLightSectionStorage.SkyDataLayerStorageMap lightenginestoragesky_a = (SkyLightSectionStorage.SkyDataLayerStorageMap) this.e_visible; // Paper - avoid copying light data - must be after lock acquire
|
|
@@ -49,7 +54,7 @@ public class SkyLightSectionStorage extends LayerLightSectionStorage<SkyLightSec
|
|
}
|
|
}
|
|
|
|
- return nibblearray.get(SectionPos.sectionRelative(BlockPos.getX(blockPos)), SectionPos.sectionRelative(BlockPos.getY(blockPos)), SectionPos.sectionRelative(BlockPos.getZ(blockPos)));
|
|
+ return nibblearray.get(baseX & 15, (int) ((blockPos << 52) >> 52) & 15, (int) baseZ & 15); // Paper - y changed above
|
|
} else {
|
|
return 15;
|
|
}
|
|
@@ -168,7 +173,7 @@ public class SkyLightSectionStorage extends LayerLightSectionStorage<SkyLightSec
|
|
if (k != ((SkyLightSectionStorage.SkyDataLayerStorageMap) this.updatingSectionData).currentLowestY && SectionPos.y(j) < k) {
|
|
DataLayer nibblearray1;
|
|
|
|
- while ((nibblearray1 = this.getDataLayer(j, true)) == null) {
|
|
+ while ((nibblearray1 = this.updating.getUpdatingOptimized(j)) == null) { // Paper
|
|
j = SectionPos.offset(j, Direction.UP);
|
|
}
|
|
|
|
@@ -192,7 +197,10 @@ public class SkyLightSectionStorage extends LayerLightSectionStorage<SkyLightSec
|
|
longiterator = this.sectionsToAddSourcesTo.iterator();
|
|
|
|
while (longiterator.hasNext()) {
|
|
- i = (Long) longiterator.next();
|
|
+ i = longiterator.nextLong(); // Paper
|
|
+ int baseX = (int) (i >> 42) << 4; // Paper
|
|
+ int baseY = (int) (i << 44 >> 44) << 4; // Paper
|
|
+ int baseZ = (int) (i << 22 >> 42) << 4; // Paper
|
|
j = this.getLevel(i);
|
|
if (j != 2 && !this.sectionsToRemoveSourcesFrom.contains(i) && this.sectionsWithSources.add(i)) {
|
|
int l;
|
|
@@ -203,10 +211,10 @@ public class SkyLightSectionStorage extends LayerLightSectionStorage<SkyLightSec
|
|
((SkyLightSectionStorage.SkyDataLayerStorageMap) this.updatingSectionData).copyDataLayer(i);
|
|
}
|
|
|
|
- Arrays.fill(this.getDataLayer(i, true).asBytesPoolSafe(), (byte) -1); // Paper
|
|
- k = SectionPos.sectionToBlockCoord(SectionPos.x(i));
|
|
- l = SectionPos.sectionToBlockCoord(SectionPos.y(i));
|
|
- int i1 = SectionPos.sectionToBlockCoord(SectionPos.z(i));
|
|
+ Arrays.fill(this.updating.getUpdatingOptimized(i).asBytesPoolSafe(), (byte) -1); // Paper - use optimized
|
|
+ k = baseX; // Paper
|
|
+ l = baseY; // Paper
|
|
+ int i1 = baseZ; // Paper
|
|
Direction[] aenumdirection = SkyLightSectionStorage.HORIZONTALS;
|
|
int j1 = aenumdirection.length;
|
|
|
|
@@ -215,7 +223,7 @@ public class SkyLightSectionStorage extends LayerLightSectionStorage<SkyLightSec
|
|
for (int l1 = 0; l1 < j1; ++l1) {
|
|
Direction enumdirection = aenumdirection[l1];
|
|
|
|
- k1 = SectionPos.offset(i, enumdirection);
|
|
+ k1 = SectionPos.getAdjacentFromBlockPos(baseX, baseY, baseZ, enumdirection); // Paper
|
|
if ((this.sectionsToRemoveSourcesFrom.contains(k1) || !this.sectionsWithSources.contains(k1) && !this.sectionsToAddSourcesTo.contains(k1)) && this.storingLightForSection(k1)) {
|
|
for (int i2 = 0; i2 < 16; ++i2) {
|
|
for (int j2 = 0; j2 < 16; ++j2) {
|
|
@@ -248,16 +256,16 @@ public class SkyLightSectionStorage extends LayerLightSectionStorage<SkyLightSec
|
|
|
|
for (int i3 = 0; i3 < 16; ++i3) {
|
|
for (j1 = 0; j1 < 16; ++j1) {
|
|
- long j3 = BlockPos.asLong(SectionPos.sectionToBlockCoord(SectionPos.x(i)) + i3, SectionPos.sectionToBlockCoord(SectionPos.y(i)), SectionPos.sectionToBlockCoord(SectionPos.z(i)) + j1);
|
|
+ long j3 = BlockPos.asLong(baseX + i3, baseY, baseZ + j1); // Paper
|
|
|
|
- k1 = BlockPos.asLong(SectionPos.sectionToBlockCoord(SectionPos.x(i)) + i3, SectionPos.sectionToBlockCoord(SectionPos.y(i)) - 1, SectionPos.sectionToBlockCoord(SectionPos.z(i)) + j1);
|
|
+ k1 = BlockPos.asLong(baseX + i3, baseY - 1, baseZ + j1); // Paper
|
|
lightProvider.checkEdge(j3, k1, lightProvider.computeLevelFromNeighbor(j3, k1, 0), true);
|
|
}
|
|
}
|
|
} else {
|
|
for (k = 0; k < 16; ++k) {
|
|
for (l = 0; l < 16; ++l) {
|
|
- long k3 = BlockPos.asLong(SectionPos.sectionToBlockCoord(SectionPos.x(i)) + k, SectionPos.sectionToBlockCoord(SectionPos.y(i)) + 16 - 1, SectionPos.sectionToBlockCoord(SectionPos.z(i)) + l);
|
|
+ long k3 = BlockPos.asLong(baseX + k, baseY + 16 - 1, baseZ + l); // Paper
|
|
|
|
lightProvider.checkEdge(Long.MAX_VALUE, k3, 0, true);
|
|
}
|
|
@@ -272,11 +280,14 @@ public class SkyLightSectionStorage extends LayerLightSectionStorage<SkyLightSec
|
|
longiterator = this.sectionsToRemoveSourcesFrom.iterator();
|
|
|
|
while (longiterator.hasNext()) {
|
|
- i = (Long) longiterator.next();
|
|
+ i = longiterator.nextLong(); // Paper
|
|
+ int baseX = (int) (i >> 42) << 4; // Paper
|
|
+ int baseY = (int) (i << 44 >> 44) << 4; // Paper
|
|
+ int baseZ = (int) (i << 22 >> 42) << 4; // Paper
|
|
if (this.sectionsWithSources.remove(i) && this.storingLightForSection(i)) {
|
|
for (j = 0; j < 16; ++j) {
|
|
for (k = 0; k < 16; ++k) {
|
|
- long l3 = BlockPos.asLong(SectionPos.sectionToBlockCoord(SectionPos.x(i)) + j, SectionPos.sectionToBlockCoord(SectionPos.y(i)) + 16 - 1, SectionPos.sectionToBlockCoord(SectionPos.z(i)) + k);
|
|
+ long l3 = BlockPos.asLong(baseX + j, baseY + 16 - 1, baseZ + k); // Paper
|
|
|
|
lightProvider.checkEdge(Long.MAX_VALUE, l3, 15, false);
|
|
}
|