mirror of
https://github.com/Jozufozu/Flywheel.git
synced 2025-01-04 03:16:24 +01:00
Pretty lit
- Add LitVisual interface. - A NON MOVING visual that provides the sections it wishes to receive updates for upon creation. - Cannibalize LightUpdaterImpl into LitVisualStorage. - Remove everything else related to LightUpdater. - EntityVisual no longer receives light updates. - MinecartVisual updates light every tick. - Add nullability annotations to AbstractVisual#relight. - Check instanceof before removing from the various lists in Storage. - Also, don't immediately run SyncedPlan on the main thread. It makes profiling too difficult as the call stack for its runnable ends up starting from multiple points.
This commit is contained in:
parent
a0eab9a250
commit
59cb5f7dbf
17 changed files with 280 additions and 386 deletions
|
@ -0,0 +1,30 @@
|
|||
package com.jozufozu.flywheel.api.visual;
|
||||
|
||||
import java.util.function.LongConsumer;
|
||||
|
||||
import net.minecraft.core.SectionPos;
|
||||
|
||||
/**
|
||||
* A non-moving visual that listens to light updates.
|
||||
* <br>
|
||||
* If your visual moves around in the world at all, you should use {@link TickableVisual} or {@link DynamicVisual},
|
||||
* and poll for light yourself rather than listening for updates.
|
||||
*/
|
||||
public interface LitVisual extends Visual {
|
||||
/**
|
||||
* Called when a section this visual is contained in receives a light update.
|
||||
* <br>
|
||||
* Even if multiple sections are updated at the same time, this method will only be called once.
|
||||
*/
|
||||
void updateLight();
|
||||
|
||||
/**
|
||||
* Collect the sections that this visual is contained in.
|
||||
* <br>
|
||||
* This method is only called upon visual creation.
|
||||
*
|
||||
* @param consumer The consumer to provide the sections to.
|
||||
* @see SectionPos#asLong
|
||||
*/
|
||||
void collectLightSections(LongConsumer consumer);
|
||||
}
|
|
@ -1,9 +0,0 @@
|
|||
package com.jozufozu.flywheel.api.visualization;
|
||||
|
||||
import com.jozufozu.flywheel.lib.light.LightListener;
|
||||
|
||||
public interface LightUpdater {
|
||||
void addListener(LightListener listener);
|
||||
|
||||
void removeListener(LightListener listener);
|
||||
}
|
|
@ -11,5 +11,5 @@ import net.minecraft.core.Vec3i;
|
|||
* @param renderOrigin The origin of the renderer as a world position.
|
||||
* All models render as if this position is (0, 0, 0).
|
||||
*/
|
||||
public record VisualizationContext(InstancerProvider instancerProvider, LightUpdater lightUpdater, Vec3i renderOrigin) {
|
||||
public record VisualizationContext(InstancerProvider instancerProvider, Vec3i renderOrigin) {
|
||||
}
|
||||
|
|
|
@ -35,7 +35,6 @@ import com.jozufozu.flywheel.impl.visualization.manager.VisualManagerImpl;
|
|||
import com.jozufozu.flywheel.impl.visualization.ratelimit.BandedPrimeLimiter;
|
||||
import com.jozufozu.flywheel.impl.visualization.ratelimit.DistanceUpdateLimiterImpl;
|
||||
import com.jozufozu.flywheel.impl.visualization.ratelimit.NonLimiter;
|
||||
import com.jozufozu.flywheel.lib.light.LightUpdaterImpl;
|
||||
import com.jozufozu.flywheel.lib.task.Flag;
|
||||
import com.jozufozu.flywheel.lib.task.IfElsePlan;
|
||||
import com.jozufozu.flywheel.lib.task.MapContextPlan;
|
||||
|
@ -46,6 +45,7 @@ import com.jozufozu.flywheel.lib.task.SimplePlan;
|
|||
import com.jozufozu.flywheel.lib.util.LevelAttached;
|
||||
|
||||
import it.unimi.dsi.fastutil.longs.Long2ObjectMap;
|
||||
import it.unimi.dsi.fastutil.longs.LongSet;
|
||||
import net.minecraft.client.Minecraft;
|
||||
import net.minecraft.core.Vec3i;
|
||||
import net.minecraft.server.level.BlockDestructionProgress;
|
||||
|
@ -62,7 +62,6 @@ public class VisualizationManagerImpl implements VisualizationManager, Supplier<
|
|||
|
||||
private final Engine engine;
|
||||
private final TaskExecutor taskExecutor;
|
||||
private final LightUpdaterImpl lightUpdater;
|
||||
|
||||
private final VisualManagerImpl<BlockEntity, BlockEntityStorage> blockEntities;
|
||||
private final VisualManagerImpl<Entity, EntityStorage> entities;
|
||||
|
@ -85,37 +84,29 @@ public class VisualizationManagerImpl implements VisualizationManager, Supplier<
|
|||
engine = BackendManager.getBackend()
|
||||
.createEngine(level);
|
||||
taskExecutor = FlwTaskExecutor.get();
|
||||
lightUpdater = new LightUpdaterImpl();
|
||||
|
||||
blockEntities = new VisualManagerImpl<>(new BlockEntityStorage(this));
|
||||
entities = new VisualManagerImpl<>(new EntityStorage(this));
|
||||
effects = new VisualManagerImpl<>(new EffectStorage(this));
|
||||
|
||||
var blockEntitiesStorage = blockEntities.getStorage();
|
||||
var entitiesStorage = entities.getStorage();
|
||||
var effectsStorage = effects.getStorage();
|
||||
tickPlan = MapContextPlan.map(this::createVisualTickContext)
|
||||
.to(NestedPlan.of(SimplePlan.<VisualTickContext>of(context -> blockEntities.processQueue(0))
|
||||
.then(blockEntities.getStorage()
|
||||
.getTickPlan()), SimplePlan.<VisualTickContext>of(context -> entities.processQueue(0))
|
||||
.then(entities.getStorage()
|
||||
.getTickPlan()), SimplePlan.<VisualTickContext>of(context -> effects.processQueue(0))
|
||||
.then(effects.getStorage()
|
||||
.getTickPlan())))
|
||||
.then(blockEntitiesStorage.getTickPlan()), SimplePlan.<VisualTickContext>of(context -> entities.processQueue(0))
|
||||
.then(entitiesStorage.getTickPlan()), SimplePlan.<VisualTickContext>of(context -> effects.processQueue(0))
|
||||
.then(effectsStorage.getTickPlan())))
|
||||
.then(RaisePlan.raise(tickFlag))
|
||||
.simplify();
|
||||
|
||||
var lightUpdatePlan = lightUpdater.plan();
|
||||
var recreate = SimplePlan.<RenderContext>of(context -> blockEntitiesStorage.recreateAll(context.partialTick()), context -> entitiesStorage.recreateAll(context.partialTick()), context -> effectsStorage.recreateAll(context.partialTick()));
|
||||
|
||||
var recreate = SimplePlan.<RenderContext>of(context -> blockEntities.getStorage()
|
||||
.recreateAll(context.partialTick()), context -> entities.getStorage()
|
||||
.recreateAll(context.partialTick()), context -> effects.getStorage()
|
||||
.recreateAll(context.partialTick()))
|
||||
.then(lightUpdatePlan);
|
||||
|
||||
var update = SimplePlan.<RenderContext>of(context -> blockEntities.processQueue(context.partialTick()), context -> entities.processQueue(context.partialTick()), context -> effects.processQueue(context.partialTick()))
|
||||
.then(lightUpdatePlan.and(MapContextPlan.map(this::createVisualContext)
|
||||
.to(NestedPlan.of(blockEntities.getStorage()
|
||||
.getFramePlan(), entities.getStorage()
|
||||
.getFramePlan(), effects.getStorage()
|
||||
.getFramePlan()))));
|
||||
var update = MapContextPlan.map(this::createVisualFrameContext)
|
||||
.to(NestedPlan.of(SimplePlan.<VisualFrameContext>of(context -> blockEntities.processQueue(0))
|
||||
.then(blockEntitiesStorage.getFramePlan()), SimplePlan.<VisualFrameContext>of(context -> entities.processQueue(0))
|
||||
.then(entitiesStorage.getFramePlan()), SimplePlan.<VisualFrameContext>of(context -> effects.processQueue(0))
|
||||
.then(effectsStorage.getFramePlan())));
|
||||
|
||||
framePlan = IfElsePlan.on((RenderContext ctx) -> engine.updateRenderOrigin(ctx.camera()))
|
||||
.ifTrue(recreate)
|
||||
|
@ -132,7 +123,7 @@ public class VisualizationManagerImpl implements VisualizationManager, Supplier<
|
|||
}
|
||||
}
|
||||
|
||||
private VisualFrameContext createVisualContext(RenderContext ctx) {
|
||||
private VisualFrameContext createVisualFrameContext(RenderContext ctx) {
|
||||
Vec3i renderOrigin = engine.renderOrigin();
|
||||
var cameraPos = ctx.camera()
|
||||
.getPosition();
|
||||
|
@ -209,7 +200,7 @@ public class VisualizationManagerImpl implements VisualizationManager, Supplier<
|
|||
|
||||
@Override
|
||||
public VisualizationContext get() {
|
||||
return new VisualizationContext(engine, lightUpdater, engine.renderOrigin());
|
||||
return new VisualizationContext(engine, engine.renderOrigin());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -232,10 +223,6 @@ public class VisualizationManagerImpl implements VisualizationManager, Supplier<
|
|||
return effects;
|
||||
}
|
||||
|
||||
public LightUpdaterImpl getLightUpdater() {
|
||||
return lightUpdater;
|
||||
}
|
||||
|
||||
/**
|
||||
* Tick the visuals after the game has ticked:
|
||||
* <p>
|
||||
|
@ -343,4 +330,13 @@ public class VisualizationManagerImpl implements VisualizationManager, Supplier<
|
|||
effects.invalidate();
|
||||
engine.delete();
|
||||
}
|
||||
|
||||
public void enqueueLightUpdateSections(LongSet sections) {
|
||||
blockEntities.getStorage()
|
||||
.enqueueLightUpdateSections(sections);
|
||||
entities.getStorage()
|
||||
.enqueueLightUpdateSections(sections);
|
||||
effects.getStorage()
|
||||
.enqueueLightUpdateSections(sections);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,163 @@
|
|||
package com.jozufozu.flywheel.impl.visualization.storage;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.WeakHashMap;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import com.jozufozu.flywheel.api.task.Plan;
|
||||
import com.jozufozu.flywheel.api.task.TaskExecutor;
|
||||
import com.jozufozu.flywheel.api.visual.LitVisual;
|
||||
import com.jozufozu.flywheel.api.visual.VisualFrameContext;
|
||||
import com.jozufozu.flywheel.lib.task.PlanUtil;
|
||||
import com.jozufozu.flywheel.lib.task.SimplyComposedPlan;
|
||||
import com.jozufozu.flywheel.lib.task.Synchronizer;
|
||||
|
||||
import it.unimi.dsi.fastutil.longs.Long2ObjectMap;
|
||||
import it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap;
|
||||
import it.unimi.dsi.fastutil.longs.LongArraySet;
|
||||
import it.unimi.dsi.fastutil.longs.LongOpenHashSet;
|
||||
import it.unimi.dsi.fastutil.longs.LongSet;
|
||||
import it.unimi.dsi.fastutil.objects.ObjectArrayList;
|
||||
|
||||
/**
|
||||
* Keeps track of what chunks/sections each listener is in, so we can update exactly what needs to be updated.
|
||||
*/
|
||||
public class LitVisualStorage {
|
||||
private static final long NEVER_UPDATED = Long.MIN_VALUE;
|
||||
private static final long INITIAL_UPDATE_ID = NEVER_UPDATED + 1;
|
||||
|
||||
private final Map<LitVisual, LongSet> visuals2Sections = new WeakHashMap<>();
|
||||
private final Long2ObjectMap<List<Updater>> sections2Visuals = new Long2ObjectOpenHashMap<>();
|
||||
|
||||
private final LongSet sectionsQueue = new LongOpenHashSet();
|
||||
|
||||
private long updateId = INITIAL_UPDATE_ID;
|
||||
|
||||
public Plan<VisualFrameContext> plan() {
|
||||
return (SimplyComposedPlan<VisualFrameContext>) (TaskExecutor taskExecutor, VisualFrameContext context, Runnable onCompletion) -> {
|
||||
if (sectionsQueue.isEmpty()) {
|
||||
onCompletion.run();
|
||||
return;
|
||||
}
|
||||
|
||||
var sync = new Synchronizer(sectionsQueue.size(), () -> {
|
||||
sectionsQueue.clear();
|
||||
onCompletion.run();
|
||||
});
|
||||
|
||||
long updateId = getNextUpdateId();
|
||||
|
||||
for (long section : sectionsQueue) {
|
||||
var visuals = sections2Visuals.get(section);
|
||||
if (visuals != null && !visuals.isEmpty()) {
|
||||
taskExecutor.execute(() -> PlanUtil.distribute(taskExecutor, updateId, sync, visuals, Updater::updateLight));
|
||||
} else {
|
||||
sync.decrementAndEventuallyRun();
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private long getNextUpdateId() {
|
||||
long out = this.updateId;
|
||||
|
||||
this.updateId++;
|
||||
if (this.updateId == NEVER_UPDATED) {
|
||||
// Somehow we were running long enough to wrap around. Go back to the initial value.
|
||||
this.updateId = INITIAL_UPDATE_ID;
|
||||
}
|
||||
|
||||
return out;
|
||||
}
|
||||
|
||||
public boolean isEmpty() {
|
||||
return visuals2Sections.isEmpty();
|
||||
}
|
||||
|
||||
public void add(LitVisual visual) {
|
||||
LongSet sections = new LongArraySet();
|
||||
|
||||
visual.collectLightSections(sections::add);
|
||||
|
||||
Updater updater;
|
||||
if (sections.isEmpty()) {
|
||||
return;
|
||||
} else if (sections.size() == 1) {
|
||||
updater = new Updater.Simple(visual);
|
||||
} else {
|
||||
updater = new Updater.Synced(visual, new AtomicLong(NEVER_UPDATED));
|
||||
}
|
||||
|
||||
for (long section : sections) {
|
||||
sections2Visuals.computeIfAbsent(section, $ -> new ObjectArrayList<>())
|
||||
.add(updater);
|
||||
}
|
||||
|
||||
visuals2Sections.put(visual, sections);
|
||||
}
|
||||
|
||||
public void enqueueLightUpdateSections(LongSet sections) {
|
||||
sectionsQueue.addAll(sections);
|
||||
}
|
||||
|
||||
public void remove(LitVisual visual) {
|
||||
var sections = visuals2Sections.remove(visual);
|
||||
|
||||
if (sections == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
for (long section : sections) {
|
||||
List<Updater> listeners = sections2Visuals.get(section);
|
||||
if (listeners != null) {
|
||||
listeners.remove(indexOfUpdater(listeners, visual));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void clear() {
|
||||
visuals2Sections.clear();
|
||||
sections2Visuals.clear();
|
||||
sectionsQueue.clear();
|
||||
}
|
||||
|
||||
private static int indexOfUpdater(List<Updater> listeners, LitVisual visual) {
|
||||
for (int i = 0; i < listeners.size(); i++) {
|
||||
if (listeners.get(i)
|
||||
.visual() == visual) {
|
||||
return i;
|
||||
}
|
||||
}
|
||||
return -1;
|
||||
}
|
||||
|
||||
// Breaking this into 2 separate cases allows us to avoid the sync overhead in the common case.
|
||||
// TODO: is it faster to only use the synced variant to avoid virtual dispatches?
|
||||
sealed interface Updater {
|
||||
void updateLight(long updateId);
|
||||
|
||||
LitVisual visual();
|
||||
|
||||
// The visual is only in one section. In this case, we can just update the visual directly.
|
||||
record Simple(LitVisual visual) implements Updater {
|
||||
@Override
|
||||
public void updateLight(long updateId) {
|
||||
visual.updateLight();
|
||||
}
|
||||
}
|
||||
|
||||
// The visual is in multiple sections. Here we need to make sure that the visual only gets updated once,
|
||||
// even when multiple sections it was contained in are updated at the same time.
|
||||
record Synced(LitVisual visual, AtomicLong updateId) implements Updater {
|
||||
@Override
|
||||
public void updateLight(long updateId) {
|
||||
// Different update ID means we won, so we can update the visual.
|
||||
// Same update ID means another thread beat us to the update.
|
||||
if (this.updateId.getAndSet(updateId) != updateId) {
|
||||
visual.updateLight();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -10,6 +10,7 @@ import org.jetbrains.annotations.Nullable;
|
|||
|
||||
import com.jozufozu.flywheel.api.task.Plan;
|
||||
import com.jozufozu.flywheel.api.visual.DynamicVisual;
|
||||
import com.jozufozu.flywheel.api.visual.LitVisual;
|
||||
import com.jozufozu.flywheel.api.visual.PlannedVisual;
|
||||
import com.jozufozu.flywheel.api.visual.TickableVisual;
|
||||
import com.jozufozu.flywheel.api.visual.Visual;
|
||||
|
@ -18,6 +19,7 @@ import com.jozufozu.flywheel.api.visual.VisualTickContext;
|
|||
import com.jozufozu.flywheel.api.visualization.VisualizationContext;
|
||||
import com.jozufozu.flywheel.lib.task.ForEachPlan;
|
||||
|
||||
import it.unimi.dsi.fastutil.longs.LongSet;
|
||||
import it.unimi.dsi.fastutil.objects.Reference2ObjectOpenHashMap;
|
||||
|
||||
public abstract class Storage<T> {
|
||||
|
@ -25,6 +27,7 @@ public abstract class Storage<T> {
|
|||
protected final List<TickableVisual> tickableVisuals = new ArrayList<>();
|
||||
protected final List<DynamicVisual> dynamicVisuals = new ArrayList<>();
|
||||
protected final List<PlannedVisual> plannedVisuals = new ArrayList<>();
|
||||
protected final LitVisualStorage litVisuals = new LitVisualStorage();
|
||||
protected final VisualUpdatePlan<VisualFrameContext> framePlan = new VisualUpdatePlan<>(() -> plannedVisuals.stream()
|
||||
.map(PlannedVisual::planFrame)
|
||||
.toList());
|
||||
|
@ -57,10 +60,20 @@ public abstract class Storage<T> {
|
|||
return;
|
||||
}
|
||||
|
||||
tickableVisuals.remove(visual);
|
||||
dynamicVisuals.remove(visual);
|
||||
if (plannedVisuals.remove(visual)) {
|
||||
framePlan.triggerReInitialize();
|
||||
if (visual instanceof TickableVisual tickable) {
|
||||
tickableVisuals.remove(tickable);
|
||||
}
|
||||
if (visual instanceof DynamicVisual dynamic) {
|
||||
dynamicVisuals.remove(dynamic);
|
||||
}
|
||||
if (visual instanceof PlannedVisual planned) {
|
||||
if (plannedVisuals.remove(planned)) {
|
||||
framePlan.triggerReInitialize();
|
||||
tickPlan.triggerReInitialize();
|
||||
}
|
||||
}
|
||||
if (visual instanceof LitVisual lit) {
|
||||
litVisuals.remove(lit);
|
||||
}
|
||||
visual.delete();
|
||||
}
|
||||
|
@ -87,6 +100,7 @@ public abstract class Storage<T> {
|
|||
tickableVisuals.clear();
|
||||
dynamicVisuals.clear();
|
||||
plannedVisuals.clear();
|
||||
litVisuals.clear();
|
||||
visuals.replaceAll((obj, visual) -> {
|
||||
visual.delete();
|
||||
|
||||
|
@ -104,6 +118,7 @@ public abstract class Storage<T> {
|
|||
tickableVisuals.clear();
|
||||
dynamicVisuals.clear();
|
||||
plannedVisuals.clear();
|
||||
litVisuals.clear();
|
||||
framePlan.triggerReInitialize();
|
||||
tickPlan.triggerReInitialize();
|
||||
visuals.values()
|
||||
|
@ -124,13 +139,18 @@ public abstract class Storage<T> {
|
|||
protected abstract Visual createRaw(T obj);
|
||||
|
||||
public Plan<VisualFrameContext> getFramePlan() {
|
||||
return framePlan.and(ForEachPlan.of(() -> dynamicVisuals, DynamicVisual::beginFrame));
|
||||
return framePlan.and(ForEachPlan.of(() -> dynamicVisuals, DynamicVisual::beginFrame))
|
||||
.and(litVisuals.plan());
|
||||
}
|
||||
|
||||
public Plan<VisualTickContext> getTickPlan() {
|
||||
return tickPlan.and(ForEachPlan.of(() -> tickableVisuals, TickableVisual::tick));
|
||||
}
|
||||
|
||||
public void enqueueLightUpdateSections(LongSet sections) {
|
||||
litVisuals.enqueueLightUpdateSections(sections);
|
||||
}
|
||||
|
||||
private void setup(Visual visual, float partialTick) {
|
||||
visual.init(partialTick);
|
||||
|
||||
|
@ -147,6 +167,10 @@ public abstract class Storage<T> {
|
|||
framePlan.add(planned.planFrame());
|
||||
tickPlan.add(planned.planTick());
|
||||
}
|
||||
|
||||
if (visual instanceof LitVisual lit) {
|
||||
litVisuals.add(lit);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -1,28 +0,0 @@
|
|||
package com.jozufozu.flywheel.lib.light;
|
||||
|
||||
import com.jozufozu.flywheel.lib.box.Box;
|
||||
|
||||
import net.minecraft.core.SectionPos;
|
||||
import net.minecraft.world.level.LightLayer;
|
||||
|
||||
/**
|
||||
* Implementors of this interface may choose to subscribe to light updates by calling
|
||||
* {@link LightUpdaterImpl#addListener(LightListener)}.<p>
|
||||
*
|
||||
* It is the responsibility of the implementor to keep a reference to the level an object is contained in.
|
||||
*/
|
||||
public interface LightListener {
|
||||
Box getVolume();
|
||||
|
||||
/**
|
||||
* Check the status of the light listener.
|
||||
* @return {@code true} if the listener is invalid/removed/deleted,
|
||||
* and should no longer receive updates.
|
||||
*/
|
||||
boolean isInvalid();
|
||||
|
||||
/**
|
||||
* Called when light updates in a section the implementor cares about.
|
||||
*/
|
||||
void onLightUpdate(LightLayer type, SectionPos pos);
|
||||
}
|
|
@ -1,126 +0,0 @@
|
|||
package com.jozufozu.flywheel.lib.light;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Queue;
|
||||
import java.util.WeakHashMap;
|
||||
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import com.jozufozu.flywheel.api.event.RenderContext;
|
||||
import com.jozufozu.flywheel.api.task.Plan;
|
||||
import com.jozufozu.flywheel.api.task.TaskExecutor;
|
||||
import com.jozufozu.flywheel.api.visualization.LightUpdater;
|
||||
import com.jozufozu.flywheel.lib.box.Box;
|
||||
import com.jozufozu.flywheel.lib.task.PlanUtil;
|
||||
import com.jozufozu.flywheel.lib.task.SimplyComposedPlan;
|
||||
import com.jozufozu.flywheel.lib.task.Synchronizer;
|
||||
|
||||
import it.unimi.dsi.fastutil.longs.Long2ObjectMap;
|
||||
import it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap;
|
||||
import it.unimi.dsi.fastutil.longs.LongArraySet;
|
||||
import it.unimi.dsi.fastutil.longs.LongOpenHashSet;
|
||||
import it.unimi.dsi.fastutil.longs.LongSet;
|
||||
import net.minecraft.core.SectionPos;
|
||||
import net.minecraft.world.level.LightLayer;
|
||||
|
||||
/**
|
||||
* Keeps track of what chunks/sections each listener is in, so we can update exactly what needs to be updated.
|
||||
*/
|
||||
public class LightUpdaterImpl implements LightUpdater {
|
||||
private final Map<LightListener, LongSet> listenersAndTheirSections = new WeakHashMap<>();
|
||||
private final Long2ObjectMap<List<LightListener>> listenersBySection = new Long2ObjectOpenHashMap<>();
|
||||
|
||||
private final Queue<LightListener> additionQueue = new ConcurrentLinkedQueue<>();
|
||||
private final LongSet sectionsQueue = new LongOpenHashSet();
|
||||
|
||||
/**
|
||||
* Add a listener.
|
||||
*
|
||||
* @param listener The object that wants to receive light update notifications.
|
||||
*/
|
||||
public void addListener(LightListener listener) {
|
||||
additionQueue.add(listener);
|
||||
}
|
||||
|
||||
public void removeListener(LightListener listener) {
|
||||
listenersAndTheirSections.remove(listener);
|
||||
}
|
||||
|
||||
public Plan<RenderContext> plan() {
|
||||
return (SimplyComposedPlan<RenderContext>) (TaskExecutor taskExecutor, RenderContext context, Runnable onCompletion) -> {
|
||||
processQueue();
|
||||
|
||||
if (sectionsQueue.isEmpty()) {
|
||||
onCompletion.run();
|
||||
return;
|
||||
}
|
||||
|
||||
var sync = new Synchronizer(sectionsQueue.size(), () -> {
|
||||
sectionsQueue.clear();
|
||||
onCompletion.run();
|
||||
});
|
||||
|
||||
sectionsQueue.forEach((long section) -> {
|
||||
List<LightListener> listeners = listenersBySection.get(section);
|
||||
if (listeners != null && !listeners.isEmpty()) {
|
||||
taskExecutor.execute(() -> {
|
||||
PlanUtil.distribute(taskExecutor, SectionPos.of(section), sync, listeners, (listener, pos) -> {
|
||||
listener.onLightUpdate(LightLayer.BLOCK, pos);
|
||||
});
|
||||
});
|
||||
} else {
|
||||
sync.decrementAndEventuallyRun();
|
||||
}
|
||||
});
|
||||
};
|
||||
}
|
||||
|
||||
public Stream<Box> getAllBoxes() {
|
||||
return listenersAndTheirSections.keySet()
|
||||
.stream()
|
||||
.map(LightListener::getVolume);
|
||||
}
|
||||
|
||||
public boolean isEmpty() {
|
||||
return listenersAndTheirSections.isEmpty();
|
||||
}
|
||||
|
||||
private synchronized void processQueue() {
|
||||
LightListener listener;
|
||||
while ((listener = additionQueue.poll()) != null) {
|
||||
doAdd(listener);
|
||||
}
|
||||
}
|
||||
|
||||
private void doAdd(LightListener listener) {
|
||||
Box box = listener.getVolume();
|
||||
|
||||
LongSet sections = new LongArraySet();
|
||||
|
||||
int minX = SectionPos.blockToSectionCoord(box.getMinX());
|
||||
int minY = SectionPos.blockToSectionCoord(box.getMinY());
|
||||
int minZ = SectionPos.blockToSectionCoord(box.getMinZ());
|
||||
int maxX = SectionPos.blockToSectionCoord(box.getMaxX());
|
||||
int maxY = SectionPos.blockToSectionCoord(box.getMaxY());
|
||||
int maxZ = SectionPos.blockToSectionCoord(box.getMaxZ());
|
||||
|
||||
for (int x = minX; x <= maxX; x++) {
|
||||
for (int y = minY; y <= maxY; y++) {
|
||||
for (int z = minZ; z <= maxZ; z++) {
|
||||
var longPos = SectionPos.asLong(x, y, z);
|
||||
sections.add(longPos);
|
||||
listenersBySection.computeIfAbsent(longPos, $ -> new ArrayList<>())
|
||||
.add(listener);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
listenersAndTheirSections.put(listener, sections);
|
||||
}
|
||||
|
||||
public void notifySectionUpdates(LongSet sections) {
|
||||
sectionsQueue.addAll(sections);
|
||||
}
|
||||
}
|
|
@ -11,7 +11,7 @@ import net.minecraft.core.SectionPos;
|
|||
import net.minecraft.world.level.BlockAndTintGetter;
|
||||
import net.minecraft.world.level.LightLayer;
|
||||
|
||||
public class LightVolume implements Box, LightListener {
|
||||
public class LightVolume implements Box {
|
||||
|
||||
protected final BlockAndTintGetter level;
|
||||
protected final MutableBox box = new MutableBox();
|
||||
|
@ -24,7 +24,6 @@ public class LightVolume implements Box, LightListener {
|
|||
this.lightData = MemoryBlock.malloc(this.box.volume() * 2);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Box getVolume() {
|
||||
return box;
|
||||
}
|
||||
|
@ -59,7 +58,6 @@ public class LightVolume implements Box, LightListener {
|
|||
return box.getMaxZ();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isInvalid() {
|
||||
return lightData == null;
|
||||
}
|
||||
|
@ -207,7 +205,6 @@ public class LightVolume implements Box, LightListener {
|
|||
return (x + box.sizeX() * (y + z * box.sizeY())) * 2;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onLightUpdate(LightLayer type, SectionPos pos) {
|
||||
if (lightData == null) return;
|
||||
|
||||
|
|
|
@ -1,10 +0,0 @@
|
|||
package com.jozufozu.flywheel.lib.light;
|
||||
|
||||
// TODO: remove
|
||||
public interface TickingLightListener extends LightListener {
|
||||
/**
|
||||
* Called every tick for active listeners.
|
||||
* @return {@code true} if the listener changed.
|
||||
*/
|
||||
boolean tickLightListener();
|
||||
}
|
|
@ -1,89 +0,0 @@
|
|||
package com.jozufozu.flywheel.lib.light;
|
||||
|
||||
import java.util.AbstractCollection;
|
||||
import java.util.Iterator;
|
||||
import java.util.Set;
|
||||
import java.util.WeakHashMap;
|
||||
import java.util.function.LongConsumer;
|
||||
|
||||
import com.jozufozu.flywheel.lib.util.FlwUtil;
|
||||
|
||||
import it.unimi.dsi.fastutil.longs.Long2ObjectMap;
|
||||
import it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap;
|
||||
import it.unimi.dsi.fastutil.longs.LongRBTreeSet;
|
||||
import it.unimi.dsi.fastutil.longs.LongSet;
|
||||
|
||||
class WeakContainmentMultiMap<T> extends AbstractCollection<T> {
|
||||
private final Long2ObjectMap<Set<T>> forward;
|
||||
private final WeakHashMap<T, LongSet> reverse;
|
||||
|
||||
public WeakContainmentMultiMap() {
|
||||
forward = new Long2ObjectOpenHashMap<>();
|
||||
reverse = new WeakHashMap<>();
|
||||
}
|
||||
|
||||
/**
|
||||
* This is a confusing function, but it maintains the internal state of the section maps.
|
||||
*
|
||||
* <p>
|
||||
* First, uses the reverse lookup map to remove listener from all sets in the lookup map.<br>
|
||||
* Then, clears the listeners containment set.
|
||||
* </p>
|
||||
*
|
||||
* @param listener The listener to clean up.
|
||||
* @return An empty set that should be populated with the sections the listener is contained in.
|
||||
*/
|
||||
public LongSet getAndResetContainment(T listener) {
|
||||
LongSet containmentSet = reverse.computeIfAbsent(listener, $ -> new LongRBTreeSet());
|
||||
|
||||
containmentSet.forEach((LongConsumer) l -> {
|
||||
Set<T> listeners = forward.get(l);
|
||||
|
||||
if (listeners != null) {
|
||||
listeners.remove(listener);
|
||||
}
|
||||
});
|
||||
|
||||
containmentSet.clear();
|
||||
|
||||
return containmentSet;
|
||||
}
|
||||
|
||||
public Set<T> get(long l) {
|
||||
return forward.get(l);
|
||||
}
|
||||
|
||||
public void put(long sectionPos, T listener) {
|
||||
forward.computeIfAbsent(sectionPos, $ -> FlwUtil.createWeakHashSet()).add(listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean remove(Object o) {
|
||||
LongSet containmentSet = reverse.remove(o);
|
||||
|
||||
if (containmentSet != null) {
|
||||
containmentSet.forEach((LongConsumer) l -> {
|
||||
Set<T> listeners = forward.get(l);
|
||||
|
||||
if (listeners != null) {
|
||||
listeners.remove(o);
|
||||
}
|
||||
});
|
||||
|
||||
containmentSet.clear();
|
||||
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<T> iterator() {
|
||||
return reverse.keySet().iterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size() {
|
||||
return reverse.size();
|
||||
}
|
||||
}
|
|
@ -15,11 +15,6 @@ public record SyncedPlan<C>(RunnableWithContext<C> task) implements SimplyCompos
|
|||
|
||||
@Override
|
||||
public void execute(TaskExecutor taskExecutor, C context, Runnable onCompletion) {
|
||||
if (taskExecutor.isMainThread()) {
|
||||
task.run(context);
|
||||
onCompletion.run();
|
||||
return;
|
||||
}
|
||||
taskExecutor.scheduleForMainThread(() -> {
|
||||
task.run(context);
|
||||
onCompletion.run();
|
||||
|
|
|
@ -1,19 +1,21 @@
|
|||
package com.jozufozu.flywheel.lib.visual;
|
||||
|
||||
import java.util.function.LongConsumer;
|
||||
|
||||
import org.joml.FrustumIntersection;
|
||||
|
||||
import com.jozufozu.flywheel.api.visual.BlockEntityVisual;
|
||||
import com.jozufozu.flywheel.api.visual.DynamicVisual;
|
||||
import com.jozufozu.flywheel.api.visual.LitVisual;
|
||||
import com.jozufozu.flywheel.api.visual.PlannedVisual;
|
||||
import com.jozufozu.flywheel.api.visual.TickableVisual;
|
||||
import com.jozufozu.flywheel.api.visual.VisualFrameContext;
|
||||
import com.jozufozu.flywheel.api.visualization.VisualManager;
|
||||
import com.jozufozu.flywheel.api.visualization.VisualizationContext;
|
||||
import com.jozufozu.flywheel.lib.box.Box;
|
||||
import com.jozufozu.flywheel.lib.box.MutableBox;
|
||||
import com.jozufozu.flywheel.lib.math.MoreMath;
|
||||
|
||||
import net.minecraft.core.BlockPos;
|
||||
import net.minecraft.core.SectionPos;
|
||||
import net.minecraft.world.level.block.entity.BlockEntity;
|
||||
import net.minecraft.world.level.block.state.BlockState;
|
||||
|
||||
|
@ -33,7 +35,7 @@ import net.minecraft.world.level.block.state.BlockState;
|
|||
*
|
||||
* @param <T> The type of {@link BlockEntity}.
|
||||
*/
|
||||
public abstract class AbstractBlockEntityVisual<T extends BlockEntity> extends AbstractVisual implements BlockEntityVisual<T> {
|
||||
public abstract class AbstractBlockEntityVisual<T extends BlockEntity> extends AbstractVisual implements BlockEntityVisual<T>, LitVisual {
|
||||
protected final T blockEntity;
|
||||
protected final BlockPos pos;
|
||||
protected final BlockPos visualPos;
|
||||
|
@ -48,13 +50,18 @@ public abstract class AbstractBlockEntityVisual<T extends BlockEntity> extends A
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean shouldReset() {
|
||||
return blockEntity.getBlockState() != blockState;
|
||||
public void init(float partialTick) {
|
||||
updateLight();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Box getVolume() {
|
||||
return MutableBox.from(pos);
|
||||
public void collectLightSections(LongConsumer consumer) {
|
||||
consumer.accept(SectionPos.asLong(pos));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean shouldReset() {
|
||||
return blockEntity.getBlockState() != blockState;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -9,13 +9,9 @@ import com.jozufozu.flywheel.api.visual.PlannedVisual;
|
|||
import com.jozufozu.flywheel.api.visual.TickableVisual;
|
||||
import com.jozufozu.flywheel.api.visualization.VisualizationContext;
|
||||
import com.jozufozu.flywheel.api.visualization.VisualizationManager;
|
||||
import com.jozufozu.flywheel.lib.box.Box;
|
||||
import com.jozufozu.flywheel.lib.box.MutableBox;
|
||||
import com.jozufozu.flywheel.lib.light.TickingLightListener;
|
||||
|
||||
import net.minecraft.util.Mth;
|
||||
import net.minecraft.world.entity.Entity;
|
||||
import net.minecraft.world.phys.AABB;
|
||||
import net.minecraft.world.phys.Vec3;
|
||||
|
||||
/**
|
||||
|
@ -34,18 +30,20 @@ import net.minecraft.world.phys.Vec3;
|
|||
*
|
||||
* @param <T> The type of {@link Entity}.
|
||||
*/
|
||||
public abstract class AbstractEntityVisual<T extends Entity> extends AbstractVisual implements EntityVisual<T>, TickingLightListener {
|
||||
public abstract class AbstractEntityVisual<T extends Entity> extends AbstractVisual implements EntityVisual<T> {
|
||||
protected final T entity;
|
||||
protected final MutableBox bounds;
|
||||
protected final EntityVisibilityTester visibilityTester;
|
||||
|
||||
public AbstractEntityVisual(VisualizationContext ctx, T entity) {
|
||||
super(ctx, entity.level());
|
||||
this.entity = entity;
|
||||
bounds = MutableBox.from(entity.getBoundingBox());
|
||||
visibilityTester = new EntityVisibilityTester(entity, ctx.renderOrigin(), 1.5f);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init(float partialTick) {
|
||||
}
|
||||
|
||||
/**
|
||||
* Calculate the distance squared between this visual and the given <em>world</em> position.
|
||||
*
|
||||
|
@ -58,26 +56,6 @@ public abstract class AbstractEntityVisual<T extends Entity> extends AbstractVis
|
|||
return entity.distanceToSqr(x, y, z);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Box getVolume() {
|
||||
return bounds;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean tickLightListener() {
|
||||
AABB boundsNow = entity.getBoundingBox();
|
||||
|
||||
if (bounds.sameAs(boundsNow)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
bounds.assign(boundsNow);
|
||||
|
||||
updateLight();
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* In order to accommodate for floating point precision errors at high coordinates,
|
||||
* {@link VisualizationManager}s are allowed to arbitrarily adjust the origin, and
|
||||
|
|
|
@ -3,19 +3,19 @@ package com.jozufozu.flywheel.lib.visual;
|
|||
import java.util.Objects;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import org.jetbrains.annotations.Nullable;
|
||||
|
||||
import com.jozufozu.flywheel.api.instance.InstancerProvider;
|
||||
import com.jozufozu.flywheel.api.visual.Visual;
|
||||
import com.jozufozu.flywheel.api.visualization.VisualizationContext;
|
||||
import com.jozufozu.flywheel.lib.instance.FlatLit;
|
||||
import com.jozufozu.flywheel.lib.light.LightListener;
|
||||
|
||||
import net.minecraft.core.BlockPos;
|
||||
import net.minecraft.core.SectionPos;
|
||||
import net.minecraft.core.Vec3i;
|
||||
import net.minecraft.world.level.Level;
|
||||
import net.minecraft.world.level.LightLayer;
|
||||
|
||||
public abstract class AbstractVisual implements Visual, LightListener {
|
||||
public abstract class AbstractVisual implements Visual {
|
||||
/**
|
||||
* The visualization context used to construct this visual.
|
||||
* <br>
|
||||
|
@ -35,13 +35,6 @@ public abstract class AbstractVisual implements Visual, LightListener {
|
|||
this.level = level;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init(float partialTick) {
|
||||
visualizationContext.lightUpdater()
|
||||
.addListener(this);
|
||||
updateLight();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void update(float partialTick) {
|
||||
}
|
||||
|
@ -51,14 +44,6 @@ public abstract class AbstractVisual implements Visual, LightListener {
|
|||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Called after initialization and when a light update occurs in the world.
|
||||
* <br>
|
||||
* If your instances need it, update light here.
|
||||
*/
|
||||
public void updateLight() {
|
||||
}
|
||||
|
||||
protected abstract void _delete();
|
||||
|
||||
@Override
|
||||
|
@ -68,26 +53,14 @@ public abstract class AbstractVisual implements Visual, LightListener {
|
|||
}
|
||||
|
||||
_delete();
|
||||
visualizationContext.lightUpdater()
|
||||
.removeListener(this);
|
||||
deleted = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onLightUpdate(LightLayer type, SectionPos pos) {
|
||||
updateLight();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isInvalid() {
|
||||
return deleted;
|
||||
}
|
||||
|
||||
protected void relight(BlockPos pos, FlatLit... instances) {
|
||||
protected void relight(BlockPos pos, @Nullable FlatLit... instances) {
|
||||
relight(level.getBrightness(LightLayer.BLOCK, pos), level.getBrightness(LightLayer.SKY, pos), instances);
|
||||
}
|
||||
|
||||
protected void relight(int block, int sky, FlatLit... instances) {
|
||||
protected void relight(int block, int sky, @Nullable FlatLit... instances) {
|
||||
for (FlatLit instance : instances) {
|
||||
if (instance == null) {
|
||||
continue;
|
||||
|
@ -99,22 +72,22 @@ public abstract class AbstractVisual implements Visual, LightListener {
|
|||
}
|
||||
}
|
||||
|
||||
protected void relight(BlockPos pos, Stream<? extends FlatLit> instances) {
|
||||
protected void relight(BlockPos pos, Stream<? extends @Nullable FlatLit> instances) {
|
||||
relight(level.getBrightness(LightLayer.BLOCK, pos), level.getBrightness(LightLayer.SKY, pos), instances);
|
||||
}
|
||||
|
||||
protected void relight(int block, int sky, Stream<? extends FlatLit> instances) {
|
||||
protected void relight(int block, int sky, Stream<? extends @Nullable FlatLit> instances) {
|
||||
instances.filter(Objects::nonNull)
|
||||
.forEach(instance -> instance.setLight(block, sky)
|
||||
.handle()
|
||||
.setChanged());
|
||||
}
|
||||
|
||||
protected void relight(BlockPos pos, Iterable<? extends FlatLit> instances) {
|
||||
protected void relight(BlockPos pos, Iterable<? extends @Nullable FlatLit> instances) {
|
||||
relight(level.getBrightness(LightLayer.BLOCK, pos), level.getBrightness(LightLayer.SKY, pos), instances);
|
||||
}
|
||||
|
||||
protected void relight(int block, int sky, Iterable<? extends FlatLit> instances) {
|
||||
protected void relight(int block, int sky, Iterable<? extends @Nullable FlatLit> instances) {
|
||||
for (FlatLit instance : instances) {
|
||||
if (instance == null) {
|
||||
continue;
|
||||
|
|
|
@ -34,8 +34,7 @@ public abstract class LayerLightSectionStorageMixin {
|
|||
var manager = VisualizationManagerImpl.get((LevelAccessor) this.chunkSource.getLevel());
|
||||
|
||||
if (manager != null) {
|
||||
manager.getLightUpdater()
|
||||
.notifySectionUpdates(this.sectionsAffectedByLightUpdates);
|
||||
manager.enqueueLightUpdateSections(this.sectionsAffectedByLightUpdates);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -82,10 +82,9 @@ public class MinecartVisual<T extends AbstractMinecart> extends AbstractEntityVi
|
|||
blockState = displayBlockState;
|
||||
contents.delete();
|
||||
contents = createContentsInstance();
|
||||
if (contents != null) {
|
||||
relight(entity.blockPosition(), contents);
|
||||
}
|
||||
}
|
||||
|
||||
updateLight();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -173,13 +172,8 @@ public class MinecartVisual<T extends AbstractMinecart> extends AbstractEntityVi
|
|||
.setChanged();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateLight() {
|
||||
if (contents == null) {
|
||||
relight(entity.blockPosition(), body);
|
||||
} else {
|
||||
relight(entity.blockPosition(), body, contents);
|
||||
}
|
||||
relight(entity.blockPosition(), body, contents);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
Loading…
Reference in a new issue