From dd8542b5c3b84b6ce175108435b2c1aee3950cc5 Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Fri, 10 Oct 2025 12:57:41 -0400 Subject: [PATCH 01/36] Allow unused FieldsProducers and DocValuesProducers to be unloaded --- .../lucene/index/SegmentCoreReaders.java | 4 +- .../apache/lucene/index/SegmentDocValues.java | 3 +- .../org/apache/lucene/index/Unloader.java | 1141 +++++++++++++++++ .../index/UnloadingDocValuesProducer.java | 137 ++ .../lucene/index/UnloadingFieldsProducer.java | 130 ++ .../lucene/store/ByteBuffersDirectory.java | 35 +- .../org/apache/lucene/store/FSDirectory.java | 36 +- .../MappedByteBufferIndexInputProvider.java | 1 + .../store/UnloaderCoordinationPoint.java | 51 + 9 files changed, 1534 insertions(+), 4 deletions(-) create mode 100644 lucene/core/src/java/org/apache/lucene/index/Unloader.java create mode 100644 lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java create mode 100644 lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java create mode 100644 lucene/core/src/java/org/apache/lucene/store/UnloaderCoordinationPoint.java diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java b/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java index a63f0a8a5efe..20fdc82701a0 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java +++ b/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java @@ -116,7 +116,9 @@ protected TermVectorsReader initialValue() { if (coreFieldInfos.hasPostings()) { final PostingsFormat format = codec.postingsFormat(); // Ask codec for its Fields - fields = format.fieldsProducer(segmentReadState); + fields = + Unloader.fieldsProducer( + () -> format.fieldsProducer(segmentReadState), dir, segmentReadState); assert fields != null; } else { fields = null; diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java b/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java index 1d90783da393..c3dd49b0ce3d 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java +++ b/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java @@ -49,7 +49,8 @@ private RefCount newDocValuesProducer( SegmentReadState srs = new SegmentReadState(dvDir, si.info, infos, IOContext.READ, segmentSuffix); DocValuesFormat dvFormat = si.info.getCodec().docValuesFormat(); - return new RefCount(dvFormat.fieldsProducer(srs)) { + return new RefCount( + Unloader.docValuesProducer(() -> dvFormat.fieldsProducer(srs), si.info.dir, srs)) { @SuppressWarnings("synthetic-access") @Override protected void release() throws IOException { diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java new file mode 100644 index 000000000000..8dc95154170a --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -0,0 +1,1141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.lucene.index; + +import java.io.ByteArrayOutputStream; +import java.io.Closeable; +import java.io.IOException; +import java.io.PrintStream; +import java.lang.ref.ReferenceQueue; +import java.lang.ref.WeakReference; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Random; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.atomic.LongAdder; +import java.util.function.Consumer; +import java.util.function.IntUnaryOperator; +import java.util.function.LongSupplier; +import java.util.function.Supplier; +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.codecs.FieldsProducer; +import org.apache.lucene.store.AlreadyClosedException; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.UnloaderCoordinationPoint; +import org.apache.lucene.util.IOFunction; +import org.apache.lucene.util.IOSupplier; +import org.apache.lucene.util.InfoStream; +import org.apache.lucene.util.RamUsageEstimator; +import org.apache.lucene.util.ThreadInterruptedException; + +/** Handles thread-safe dynamic unloading and on-demand reloading of backing resource. */ +public class Unloader implements Closeable { + + private InfoStream out = InfoStream.getDefault(); + + private static final DelegateFuture CLOSED = new DelegateFuture<>(true, null, 0); + + static { + CLOSED.complete(null); + } + + private final IOFunction, T> reopen; + + private volatile long lastAccessNanos = System.nanoTime(); + + private final AtomicReference> backing; + private final String description; + private final UnloadHelper reporter; + private final ScheduledExecutorService exec; + + private static final class DelegateFuture extends CompletableFuture { + private final boolean unloading; + private final WeakReference> prev; + private final AtomicInteger refCount; + + @SuppressWarnings("unused") + private DelegateFuture hardRef; // kept to prevent collection + + private DelegateFuture(boolean unloading, DelegateFuture prev, int initialRefCount) { + this.unloading = unloading; + this.prev = new WeakReference<>(prev); + if (unloading) { + this.refCount = null; + hardRef = prev; + whenComplete( + (r, e) -> { + if (e == null) { + // if we've completed normally (no exception), then release the hard reference + // (we keep the reference if we complete exceptionally, because we may want to + // retry unloading. This should be rare anyway. + hardRef = null; + } + }); + } else { + refCount = new AtomicInteger(initialRefCount); + } + } + + /** + * true if a reservation was acquired for this instance. Reservation release must be handled + * elsewhere. + */ + private boolean acquire() { + return refCount.updateAndGet(ACQUIRE) > 0; + } + + private boolean unload(boolean force) { + assert !unloading; + if (force) { + refCount.set(UNLOADED_REFCOUNT); + return true; + } else { + return refCount.updateAndGet(UNLOAD) == UNLOADED_REFCOUNT; + } + } + } + + @SuppressWarnings("unchecked") + private final DelegateFuture closedSentinel = (DelegateFuture) CLOSED; + + private static final AtomicBoolean EXTERNAL_REFQUEUE_HANDLING = new AtomicBoolean(); + + private static final LongAdder OUTSTANDING_SIZE = new LongAdder(); + + private static final LongSupplier OUTSTANDING_SIZE_SUPPLIER = OUTSTANDING_SIZE::sum; + + private static final Consumer REF_REMOVER = (r) -> remove((Ref) r); + + /** + * Creates a new unloader to handle unloading and on-demand reloading a backing resource + * + * @param reopen function with this as parameter; returns a newly loaded instance of + * the backing resource, and (if applicable) schedules a task to check eligibility to unload + * at a point in the future (determined by `keepAliveNanos`) + * @param keepAliveNanos the time threshold (in nanos) since last access, at which the backing + * resource will be eligible for unloading + * @param receiveFirstInstance informs the calling (shim resource) of the first backing resource, + * returning a string description. This may be used to initialize state on the shim resource + * according to information about the backing resource. The caller should take care to not + * hold any references to the initial object that would prevent it from being GC'ed. + * @throws IOException e.g., on error reading index + */ + public Unloader( + UnloadHelper unloadHelper, + IOFunction, T> reopen, + long keepAliveNanos, + IOFunction receiveFirstInstance) + throws IOException { + if (!EXTERNAL_REFQUEUE_HANDLING.get()) { + unloadHelper.maybeHandleRefQueues( + removeOutstanding, REF_REMOVER, EXTERNAL_REFQUEUE_HANDLING, OUTSTANDING_SIZE_SUPPLIER); + } + this.reporter = unloadHelper; + this.exec = unloadHelper.onCreation(this); + T in = reopen.apply(this); + try { + description = receiveFirstInstance.apply(in); + DelegateFuture holder = new DelegateFuture<>(false, null, 0); + holder.complete(in); + backing = new AtomicReference<>(holder); + this.reopen = reopen; + this.keepAliveNanos = keepAliveNanos; + } catch (Throwable t) { + try (in) { + unloadHelper.onClose(); + throw t; + } + } + } + + /** Sets the infostream for {@link Unloader}. */ + public void setInfoStream(InfoStream out) { + this.out = out; + List deferred; + if (out != InfoStream.NO_OUTPUT + && out.isEnabled("UN") + && (deferred = DEFERRED_INIT_MESSAGES.getAndSet(null)) != null) { + for (String m : deferred) { + out.message("UN", m); + } + } + } + + private final long keepAliveNanos; + + /** This resource has already been unloaded */ + public static final long ALREADY_UNLOADED = -2; + + /** This resource was unloaded as a result of this invocation of {@link #maybeUnload()}. */ + public static final long UNLOADED = -1; + + /** This resource is still referenced, so was not unloaded. */ + public static final long STILL_REFERENCED = 0; + + private static void drainRemoveOutstanding() { + for (ReferenceQueue q : removeOutstanding) { + Ref collected; + while ((collected = (Ref) q.poll()) != null) { + remove(collected); + } + } + } + + private final Random unloadRandom = new Random(); // single-threaded access + + private static boolean injectDelay(Random r, int oneIn, int millis) { + if (r.nextInt(oneIn) == 0) { + try { + Thread.sleep(millis); + } catch ( + @SuppressWarnings("unused") + InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + return true; + } + + private static DelegateFuture unloadRef( + AtomicReference> ref, boolean[] unloading) { + DelegateFuture extant; + while (!(extant = ref.get()).unloading) { + if (!extant.unload(false)) { + // still referenced + return null; + } + DelegateFuture candidate = new DelegateFuture<>(true, extant, 0); + if (ref.compareAndSet(extant, candidate)) { + return candidate; + } + } + if (extant == CLOSED) { + throw new AlreadyClosedException(""); + } + // already unloading + unloading[0] = true; + return null; + } + + private static DelegateFuture loadRef( + AtomicReference> ref, boolean[] weCompute) { + DelegateFuture extant; + while ((extant = ref.get()).unloading || !extant.acquire()) { + if (extant.unloading) { + if (extant == CLOSED) { + throw new AlreadyClosedException(""); + } + DelegateFuture candidate = new DelegateFuture<>(false, extant, 1); + if (ref.compareAndSet(extant, candidate)) { + weCompute[0] = true; + return candidate; + } + } + } + assert !extant.unloading; + return extant; + } + + private static DelegateFuture retry( + AtomicReference> ref, DelegateFuture replace, boolean[] weCompute) { + DelegateFuture prev = replace.prev.get(); + if (prev == null) { + replace.unload(true); + ref.compareAndSet(replace, new DelegateFuture<>(true, replace, 0)); + return loadRef(ref, weCompute); + } + DelegateFuture candidate = new DelegateFuture<>(false, prev, 1); + DelegateFuture extant = ref.compareAndExchange(replace, candidate); + if (extant == replace) { + weCompute[0] = true; + return candidate; + } else if (!extant.unloading && extant.acquire()) { + return extant; + } else { + replace.unload(true); + ref.compareAndSet(replace, new DelegateFuture<>(true, replace, 0)); + return loadRef(ref, weCompute); + } + } + + /** + * Conditionally unloads (closes) the delegate {@link FieldsProducer}. Returns {@link #UNLOADED} + * if resources were unloaded, otherwise returns the number of nanos remaining until the resources + * might be eligible for unloading. + * + *

The special value {@link #STILL_REFERENCED} indicates that based on last known access time, + * this resource should be eligible for unloading -- but for some reason (e.g., refCount?) + * did not permit unloading. If this happens a lot, it probably indicates an error in logic + * somewhere. + */ + public long maybeUnload() throws IOException { + if (!EXTERNAL_REFQUEUE_HANDLING.get()) drainRemoveOutstanding(); + long nanosSinceLastAccess = System.nanoTime() - lastAccessNanos; + if (nanosSinceLastAccess < keepAliveNanos) { + // don't unload + return keepAliveNanos - nanosSinceLastAccess; + } + final boolean[] unloaded = new boolean[1]; + DelegateFuture holder = unloadRef(backing, unloaded); + if (holder == null) { + return unloaded[0] ? ALREADY_UNLOADED : STILL_REFERENCED; + } + // try to unload + try { + T weUnloaded = doUnload(holder, unloaded); + holder.complete(weUnloaded); + if (weUnloaded != null) { + return UNLOADED; + } else { + return unloaded[0] ? ALREADY_UNLOADED : STILL_REFERENCED; + } + } catch (Throwable t) { + holder.completeExceptionally(t); + throw t; + } + } + + private T doUnload(DelegateFuture holder, boolean[] unloaded) throws IOException { + assert injectDelay(unloadRandom, 5, 20); + DelegateFuture active; + T toClose; + try { + active = holder.prev.get(); + assert active != null; + toClose = active.getNow(null); + } catch ( + @SuppressWarnings("unused") + Exception ex) { + // exception during loading means there's nothing to unload + unloaded[0] = true; + return null; + } + if (toClose == null) { + // this can happen if (and should be _only_ if) we're trying to close + // a value that's still loading for some reason. This might cause a + // problem for foreground threads (resource access or close), but from + // background `maybeUnload()` perspective we don't _want_ to wait for + // it to finish loading. But it's our responsibility to ensure that the + // value _does_ get closed: either by putting it back onto `backing`, + // or as a last resort, waiting for it to load and then closing it. + + // first try to put our value back + if (backing.compareAndSet(holder, active)) { + // if CAS succeeds, it's guaranteed that state has not changed since + // we fetched this value for closing, so we have safely put it back + // as "still referenced" + return null; + } else { + // state has changed; it's our responsibility to wait for and close + // the resource we already pulled. Here we wait an _absurdly_ long + // time, because it's a leak at this point if we don't close the + // resource we've pulled. + try { + toClose = active.get(10, TimeUnit.MINUTES); + } catch (InterruptedException ex) { + // we're probably shutting down + throw new ThreadInterruptedException(ex); + } catch ( + @SuppressWarnings("unused") + TimeoutException ex) { + if (out.isEnabled("UN")) + out.message("UN", "ERROR: stuck waiting to close loading resource!"); + // TODO: we could put this into a queue somewhere that retries closing? + // that said, if properly implemented, this should literally _never_ happen. + return null; + } catch ( + @SuppressWarnings("unused") + ExecutionException ex) { + // exception during loading means there's nothing to unload + unloaded[0] = true; + return null; + } + } + } + toClose.close(); + unloaded[0] = true; + return toClose; + } + + private static final long CLOSE_WAIT_FOR_LOAD_SECONDS = 10; + private static final long CLOSE_WAIT_FOR_BACKGROUND_UNLOAD_SECONDS = 1; + + @Override + @SuppressWarnings("try") + public void close() throws IOException { + this.reporter.onClose(); + DelegateFuture holder = backing.getAndSet(closedSentinel); + if (holder == CLOSED) { + throw new AlreadyClosedException(""); + } + if (holder.unloading) { + closeUnloading(holder); + } else { + // it's an active or loading instance + DelegateFuture maybeUnloadingHolder = holder.prev.get(); + assert maybeUnloadingHolder == null || maybeUnloadingHolder.unloading; + try (T toClose = + interruptProtectedGet(holder, CLOSE_WAIT_FOR_LOAD_SECONDS, TimeUnit.SECONDS)) { + closeUnloading(maybeUnloadingHolder); + // if we're closing ourselves, it should also count as an unload. + reporter.onUnload(System.nanoTime() - lastAccessNanos); + return; + } catch ( + @SuppressWarnings("unused") + ExecutionException e) { + // an exception while loading means there's nothing to close, and that's ok + } catch (TimeoutException e) { + // probably deadlock; we have no way to get the value to close + throw new IOException("timed out waiting to close loading value ", e); + } + closeUnloading(maybeUnloadingHolder); + } + } + + @SuppressWarnings("try") + private void closeUnloading(DelegateFuture unloading) throws IOException { + if (unloading == null) { + // must be done + return; + } + DelegateFuture toClose = unloading.prev.get(); + try { + // first, wait for unloading to complete + interruptProtectedGet(unloading, CLOSE_WAIT_FOR_BACKGROUND_UNLOAD_SECONDS, TimeUnit.SECONDS); + return; + } catch (ExecutionException e) { + Throwable cause = e.getCause(); + if (cause instanceof IOException) { + throw (IOException) cause; + } + // fallthrough to try to close it ourselves + } catch ( + @SuppressWarnings("unused") + TimeoutException e) { + if (out.isEnabled("UN")) + out.message( + "UN", "WARN: timeout out waiting for background unload to complete " + description); + // fallthrough to try to close it ourselves + } + if (toClose != null) { + // (if `toClose == null`, we've done all we can) + // otherwise, `toClose` represents the resource that `unloading` was trying + // to close. As a last-ditch effort, we'll try to close it ourselves here. + // The risk here is double-close, but that's preferable to a resource leak. + try { + interruptProtectedGet(toClose, CLOSE_WAIT_FOR_LOAD_SECONDS, TimeUnit.SECONDS).close(); + reporter.onUnload(System.nanoTime() - lastAccessNanos); + } catch ( + @SuppressWarnings("unused") + ExecutionException e) { + // exception during compute means there's nothing to close + } catch (TimeoutException e) { + throw new IOException("timeout out waiting to close loading value " + description, e); + } + } + } + + /** + * Circumvents {@link InterruptedException} and forces the calling thread to block for the full + * allotted time. This should only be invoked from synchronous close, where the risk of a resource + * leak outweighs the risk from delayed thread exit. + * + *

If this method swallows an {@link InterruptedException}, it will re-set the thread's + * interrupted status before returning. + * + *

TODO: evaluate whether this behavior is actually desired in non-test context. + */ + private static T interruptProtectedGet( + DelegateFuture future, long longWaitSeconds, TimeUnit timeUnit) + throws ExecutionException, TimeoutException { + boolean interrupted = false; + long now = System.nanoTime(); + final long until = now + timeUnit.toNanos(longWaitSeconds); + long waitNanos; + try { + while ((waitNanos = until - now) >= 0) { + try { + return future.get(waitNanos, TimeUnit.NANOSECONDS); + } catch ( + @SuppressWarnings("unused") + InterruptedException ex) { + interrupted = true; + } + now = System.nanoTime(); + } + throw new TimeoutException(); + } finally { + if (interrupted) { + Thread.currentThread().interrupt(); + } + } + } + + private static final class CloseableVal implements Supplier, Closeable { + + private final T val; + private final AtomicInteger refCount; + + private CloseableVal(T val, AtomicInteger refCount) { + this.val = val; + this.refCount = refCount; + } + + @Override + public T get() { + return val; + } + + @Override + public void close() throws IOException { + refCount.updateAndGet(RELEASE); + } + } + + private static final long TOTAL_BLOCK_NANOS = TimeUnit.SECONDS.toNanos(10); + + private CloseableVal backing() throws IOException { + boolean[] weCompute = new boolean[1]; + DelegateFuture holder = loadRef(backing, weCompute); + long now = System.nanoTime(); + long until = now + TOTAL_BLOCK_NANOS; + while (!weCompute[0]) { + try { + return new CloseableVal<>(holder.get(until - now, TimeUnit.NANOSECONDS), holder.refCount); + } catch (ExecutionException e) { + Throwable t = e.getCause(); + if (t instanceof IOException) { + throw (IOException) t; + } + holder = retry(backing, holder, weCompute); + } catch (InterruptedException e) { + throw new ThreadInterruptedException(e); + } catch ( + @SuppressWarnings("unused") + TimeoutException e) { + throw new IOException("timed out waiting to load backing resource"); + } + now = System.nanoTime(); + } + // we compute the result + boolean successfullyComputed = false; + T candidate = null; + try { + candidate = reopen.apply(this); + holder.complete(candidate); + successfullyComputed = true; + return new CloseableVal<>(candidate, holder.refCount); + } catch (Throwable t) { + holder.completeExceptionally(t); + throw t; + } finally { + if (candidate != null && !successfullyComputed) { + candidate.close(); + } + } + } + + // Arbitrary negative values that we won't hit accidentally + private static final int UNLOADED_REFCOUNT = ~(Integer.MAX_VALUE >> 1); + private static final int FORCE_UNLOADED_REFCOUNT = UNLOADED_REFCOUNT + 10; + + private static final IntUnaryOperator ACQUIRE = + (extant) -> { + switch (extant) { + case FORCE_UNLOADED_REFCOUNT: + return FORCE_UNLOADED_REFCOUNT; + case UNLOADED_REFCOUNT: + return UNLOADED_REFCOUNT; + default: + assert extant >= 0; + return extant + 1; + } + }; + + private static final IntUnaryOperator RELEASE = + (extant) -> { + switch (extant) { + case FORCE_UNLOADED_REFCOUNT: + return FORCE_UNLOADED_REFCOUNT; + case UNLOADED_REFCOUNT: + case 0: + throw new IllegalStateException(); + default: + assert extant > 0; + return extant - 1; + } + }; + + private static final IntUnaryOperator UNLOAD = + (extant) -> { + switch (extant) { + case FORCE_UNLOADED_REFCOUNT: + return FORCE_UNLOADED_REFCOUNT; + case UNLOADED_REFCOUNT: + throw new IllegalStateException("already unloaded"); + case 0: + return UNLOADED_REFCOUNT; + default: + assert extant > 0; + return extant; + } + }; + + private static final int PARALLEL_HEAD_FACTOR = 32; + private static final int PARALLEL_HEAD_MASK = PARALLEL_HEAD_FACTOR - 1; + + @SuppressWarnings({"unchecked", "rawtypes"}) + private static final ReferenceQueue[] removeOutstanding = + new ReferenceQueue[PARALLEL_HEAD_FACTOR]; + + static { + for (int i = PARALLEL_HEAD_FACTOR - 1; i >= 0; i--) { + removeOutstanding[i] = new ReferenceQueue<>(); + } + } + + /** + * Number of ram bytes per instance of {@link Ref}. This can be used in conjunction with {@link + * #OUTSTANDING_SIZE_SUPPLIER} (accessed via the final arg to {@link + * UnloadHelper#maybeHandleRefQueues(ReferenceQueue[], Consumer, AtomicBoolean, LongSupplier)}) to + * determine the point-in-time heap usage associated with refQueue reference tracking. + */ + public static final long RAMBYTES_PER_REF = + RamUsageEstimator.shallowSizeOfInstance(Ref.class) + + RamUsageEstimator.shallowSizeOfInstance(AtomicReference.class); + + private static final class Ref extends WeakReference { + private final AtomicInteger refCount; + private final AtomicReference next = new AtomicReference<>(); + private volatile Ref prev; + + public Ref( + Object referent, ReferenceQueue q, AtomicInteger refCount, Ref prev) { + super(referent, q); + this.refCount = refCount; + this.prev = prev; + } + } + + private static final Ref[] HEAD = new Ref[PARALLEL_HEAD_FACTOR]; + + static { + for (int i = PARALLEL_HEAD_FACTOR - 1; i >= 0; i--) { + HEAD[i] = new Ref(null, null, null, null); + } + } + + private static final Ref RESERVED = new Ref(null, null, null, null); + private static final Ref REMOVED = new Ref(null, null, null, null); + + private static void add(final Object o, AtomicInteger refCount) { + int parallelIdx = Thread.currentThread().hashCode() & PARALLEL_HEAD_MASK; + OUTSTANDING_SIZE.increment(); + if (!EXTERNAL_REFQUEUE_HANDLING.get()) drainRemoveOutstanding(); + Ref head = HEAD[parallelIdx]; + final Ref ref = new Ref(o, removeOutstanding[parallelIdx], refCount, head); + Ref next = reserve(head, RESERVED); + if (next != null) { + next.prev = ref; + ref.next.set(next); + } + if (!head.next.compareAndSet(RESERVED, ref)) { + throw new IllegalStateException(); + } + @SuppressWarnings("unused") + Object dummy = o; + } + + private static Ref reserve(Ref ref, Ref reservation) { + Ref next = ref.next.get(); + for (; ; ) { + while (next == RESERVED) { + if (reservation == REMOVED) { + Thread.yield(); + } + next = ref.next.get(); + } + Ref extant = ref.next.compareAndExchange(next, reservation); + if (extant == next) { + return next; + } else { + next = extant; + } + } + } + + private static void remove(final Ref ref) { + Ref next = reserve(ref, REMOVED); + OUTSTANDING_SIZE.decrement(); + ref.refCount.updateAndGet(RELEASE); + // now we have a lock on the link to next + Ref prev; + for (; ; ) { + prev = ref.prev; + if (prev.next.compareAndSet(ref, RESERVED)) { + break; + } else { + Thread.yield(); + } + } + // now we have a lock on the link from prev + if (next != null) { + next.prev = prev; + } + if (!prev.next.compareAndSet(RESERVED, next)) { + throw new IllegalStateException(); + } + } + + static TermsEnum wrap(TermsEnum te, Consumer registerRef) { + TermsEnum ret = + new FilterLeafReader.FilterTermsEnum(te) { + @Override + public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException { + PostingsEnum ret = super.postings(reuse, flags); + registerRef.accept(ret); + return ret; + } + + @Override + public ImpactsEnum impacts(int flags) throws IOException { + ImpactsEnum ret = super.impacts(flags); + registerRef.accept(ret); + return ret; + } + }; + registerRef.accept(ret); + return ret; + } + + interface RefTrackShim { + V shim(V in, Consumer registerRef); + } + + V execute(FPIOFunction function, K arg) throws IOException { + return execute(function, arg, null); + } + + V execute(FPIOFunction function, K arg, RefTrackShim shim) throws IOException { + try (CloseableVal active = backing()) { + V ret = function.apply(active.get(), arg); + if (ret == null) { + return null; + } else { + AtomicInteger refCount = active.refCount; + refCount.getAndUpdate(ACQUIRE); + ret = + shim == null + ? ret + : shim.shim( + ret, + (v) -> { + refCount.getAndUpdate(ACQUIRE); + add(v, refCount); + }); + add(ret, refCount); + return ret; + } + } finally { + lastAccessNanos = System.nanoTime(); + } + } + + interface FPIOFunction { + V apply(T fp, K arg) throws IOException; + } + + /** + * This should be set to true for Lucene tests (where the only lifecycle hook we have + * is per-{@link Directory}), and set to false from contexts that call {@link + * UnloaderCoordinationPoint#setUnloadHelperSupplier(Supplier)} with an external executor whose + * lifecycle is managed by other means (e.g., from Solr). + */ + public static final boolean EXECUTOR_PER_DIRECTORY; + + /** + * Time threshold at which a resource becomes eligible for unloading. Set this very low (0 or 1) + * for stress testing. + */ + private static final long KEEP_ALIVE_NANOS; + + private static final String DEFAULT_KEEP_ALIVE_SPEC = "60m"; + private static final long DEFAULT_KEEP_ALIVE_NANOS = TimeUnit.MINUTES.toNanos(60); + + /** + * Additional time allowance for the first use of resource after load/reload. Set this very low + * (e.g., 0) for stress testing. + */ + private static final long INITIAL_NANOS; + + private static final String DEFAULT_INITIAL_SPEC = "1m"; + private static final long DFEAULT_INITIAL_NANOS = TimeUnit.MINUTES.toNanos(1); + + /** + * Visible for testing. This may be used to override/disable resource unloading for specific tests + * that are known to not play well with unloading. + * + *

NOTE: this is known to be the case only for tests that evaluate opening index over a + * directory without communication with the {@link IndexWriter}. In such cases, with an + * active {@link IndexWriter}, there is no way to "refcount" index files to prevent them from + * being deleted. Lucene is ok with this by design, since, once opened, segment readers will hold + * filehandles to files even if they are deleted; but it doesn't play well with {@link Unloader}, + * which has to re-load from disk. This would be problematic for some supported uses of Lucene, + * but not for Lucene as used by Solr, e.g., where {@link DirectoryReader} instances are + * always acquired from (and incRef) {@link IndexWriter}. + */ + static boolean DISABLE = false; + + private static final AtomicReference> DEFERRED_INIT_MESSAGES = + new AtomicReference<>(new ArrayList<>()); + + static { + List deferred = DEFERRED_INIT_MESSAGES.get(); + EXECUTOR_PER_DIRECTORY = + "true".equals(System.getProperty("lucene.unload.executorPerDirectory")); // default to false + deferred.add("INFO: set static property EXECUTOR_PER_DIRECTORY=" + EXECUTOR_PER_DIRECTORY); + KEEP_ALIVE_NANOS = + getNanos("lucene.unload.ttl", DEFAULT_KEEP_ALIVE_SPEC, DEFAULT_KEEP_ALIVE_NANOS, deferred); + deferred.add( + "INFO: set static property DEFAULT_KEEP_ALIVE_MILLIS=" + + TimeUnit.NANOSECONDS.toMillis(KEEP_ALIVE_NANOS)); + INITIAL_NANOS = + getNanos("lucene.unload.initial", DEFAULT_INITIAL_SPEC, DFEAULT_INITIAL_NANOS, deferred); + deferred.add( + "INFO: set static property DEFAULT_INITIAL_MILLIS=" + + TimeUnit.NANOSECONDS.toMillis(INITIAL_NANOS)); + } + + private static long getNanos( + String syspropName, String defaultSpec, long defaultNanos, List deferred) { + try { + String unloadSpec = System.getProperty(syspropName, defaultSpec); + int endIdx = unloadSpec.length() - 1; + if (unloadSpec.isEmpty()) { + deferred.add("WARN: empty " + syspropName + " spec"); + return defaultNanos; + } else { + TimeUnit t; + char c = unloadSpec.charAt(endIdx); + switch (c) { + case 's': + t = TimeUnit.SECONDS; + break; + case 'm': + t = TimeUnit.MINUTES; + break; + case 'h': + t = TimeUnit.HOURS; + break; + case 'd': + t = TimeUnit.DAYS; + break; + default: + if (c >= '0' && c <= '9') { + endIdx++; + t = TimeUnit.MILLISECONDS; + } else { + deferred.add("WARN: bad " + syspropName + " spec: " + unloadSpec); + return defaultNanos; + } + } + try { + int v = Integer.parseInt(unloadSpec, 0, endIdx, 10); + return t.toNanos(v); + } catch (NumberFormatException ex) { + deferred.add("WARN: bad " + syspropName + " spec: " + unloadSpec + " " + ex); + throw ex; + } + } + } catch ( + @SuppressWarnings("unused") + Exception ex) { + return defaultNanos; + } + } + + // visible for testing + static int nonEmptyRefQueueHeadCount() { + return Math.toIntExact(Arrays.stream(HEAD).filter((r) -> r.next.get() != null).count()); + } + + // visible for testing + static void addDummyReference(int byteSize) { + add(new byte[byteSize], new AtomicInteger(1)); + } + + /** + * Passed to {@link Unloader#Unloader(UnloadHelper, IOFunction, long, IOFunction)} ctor. This + * provides a means for reporting unload/reload lifecycle events to higher-level components. This + * can be useful if used in a framework that wants to track metrics around load/unload, or manage + * the underlying components that handle load/unload according to framework lifecycles. + * + *

e.g., framework may supply its own {@link ScheduledExecutorService} for running unload + * checks, and may (via {@link #maybeHandleRefQueues(ReferenceQueue[], Consumer, AtomicBoolean, + * LongSupplier)} manage the handling of reference tracking as well. + */ + public interface UnloadHelper { + /** + * Called once, from the {@link Unloader#Unloader(UnloadHelper, IOFunction, long, IOFunction)} + * ctor, to provide the {@link ScheduledExecutorService} for scheduling periodic unload tasks. + * The {@link Unloader} under construction is passed as an arg. Implementations should call + * {@link Unloader#setInfoStream(InfoStream)} on the specified instance, and may also be used to + * update metrics about number of created instances, etc. + */ + ScheduledExecutorService onCreation(Unloader u); + + /** + * Called for each load/reload of backing resource + * + * @param nanosSincePriorAccess how long it's been since this resource was last accessed before + * reload + * @param loadTime how long did it take to load this resource (nanos) + */ + default void onLoad(long nanosSincePriorAccess, long loadTime) {} + + /** + * Called for each unload of backing resource + * + * @param nanosSinceLastAccess how long it's bene since this resource was last accessed before + * unload + */ + default void onUnload(long nanosSinceLastAccess) {} + + /** + * called when the associated top-level resource is closed. Any backing resources held open at + * time of close will also be unloaded (closed); this will be separately reported via {@link + * #onUnload(long)}. + */ + default void onClose() {} + + /** + * A callback that allows a framework to handle refQueue management (and provides a window into + * the size of the refQueue(s) for metrics purposes. + * + * @param queues refQueue instances + * @param handler to be called for each {@link java.lang.ref.Reference} removed from a refQueue + * @param handleRefQueue implementations should update this to true if they plan to + * handle the refQueues, and should set it back to false if/when they stop + * handling any of the provided refQueues. + * @param outstandingSize for metrics; the number of references tracked but not yet collected + * off a refQueue. + */ + default void maybeHandleRefQueues( + ReferenceQueue[] queues, + Consumer handler, + AtomicBoolean handleRefQueue, + LongSupplier outstandingSize) {} + ; + } + + /** + * {@link UnloadHelper} base impl that handles setting {@link ScheduledExecutorService} and {@link + * InfoStream} on {@link Unloader} callers of {@link #onCreation(Unloader)}. + */ + public abstract static class AbstractUnloadHelper implements UnloadHelper { + private volatile ScheduledExecutorService exec; + private volatile InfoStream infoStream; + + /** + * Provides executor (for scheduling periodic unload tasks) and infoStream for setting on + * calling {@link Unloader}s. Both args must be non-null. + */ + public AbstractUnloadHelper(ScheduledExecutorService exec, InfoStream infoStream) { + this.exec = exec; + this.infoStream = infoStream; + } + + @Override + public ScheduledExecutorService onCreation(Unloader u) { + ScheduledExecutorService ret = exec; + InfoStream infoStream = this.infoStream; + u.setInfoStream(infoStream); + exec = null; + this.infoStream = null; + return ret; + } + } + + /** + * For testing; provides a hook so that tests of refQueue functionality may directly manage + * refQueue handling. + */ + static void configure(UnloadHelper unloadHelper) { + unloadHelper.maybeHandleRefQueues( + removeOutstanding, REF_REMOVER, EXTERNAL_REFQUEUE_HANDLING, OUTSTANDING_SIZE_SUPPLIER); + } + + /** + * Returns a {@link FieldsProducer} over the specified {@link SegmentReadState}, conditionally + * wrapped to allow dynamic unloading and on-demand reloading of the backing resource. + * + *

The backing resource is initially loaded, and will be reloaded if applicable, via the + * provided `open` {@link IOSupplier}. The {@link Directory} is passed only to be used as an + * {@link UnloaderCoordinationPoint}. + * + *

NOTE: the segment files specified by {@link SegmentReadState}, which must be present upon + * initialization, must still be accessible on disk if/when the backing resource is reloaded + * (after having been unloaded). In practice, this means that {@link + * IndexWriter#incRefDeleter(SegmentInfos)} must have been called for the {@link SegmentInfos} + * associated with the specified {@link SegmentReadState}. This happens organically in many + * contexts, but not all -- particularly in tests. + */ + public static FieldsProducer fieldsProducer( + IOSupplier open, Directory dir, SegmentReadState srs) throws IOException { + UnloadHelper unloadHelper; + if (srs.context.mergeInfo != null + || srs.context.flushInfo != null + || DISABLE + || (unloadHelper = UnloaderCoordinationPoint.getUnloadHelper(dir)) == null) { + return open.get(); + } + String type = FieldsProducer.class.getSimpleName(); + return new UnloadingFieldsProducer( + unloadHelper, + (u) -> { + long start = System.nanoTime(); + FieldsProducer fp = open.get(); + try { + u.exec.schedule( + maybeUnloadTask(u, type, u.reporter), + KEEP_ALIVE_NANOS + INITIAL_NANOS, + TimeUnit.NANOSECONDS); + } catch ( + @SuppressWarnings("unused") + RejectedExecutionException ex) { + // shutting down; log and swallow + if (u.out.isEnabled("UN")) + u.out.message("UN", "WARN: new FieldsProducer while shutting down"); + } catch (Throwable t) { + try (fp) { + throw t; + } + } + u.reporter.onLoad(start - u.lastAccessNanos, System.nanoTime() - start); + return fp; + }, + KEEP_ALIVE_NANOS); + } + + /** + * Returns a {@link DocValuesProducer} over the specified {@link SegmentReadState}, conditionally + * wrapped to allow dynamic unloading and on-demand reloading of the backing resource. + * + *

The backing resource is initially loaded, and will be reloaded if applicable, via the + * provided `open` {@link IOSupplier}. The {@link Directory} is passed only to be used as an + * {@link UnloaderCoordinationPoint}. + * + *

NOTE: the segment files specified by {@link SegmentReadState}, which must be present upon + * initialization, must still be accessible on disk if/when the backing resource is reloaded + * (after having been unloaded). In practice, this means that {@link + * IndexWriter#incRefDeleter(SegmentInfos)} must have been called for the {@link SegmentInfos} + * associated with the specified {@link SegmentReadState}. This happens organically in many + * contexts, but not all -- particularly in tests. + */ + public static DocValuesProducer docValuesProducer( + IOSupplier open, Directory dir, SegmentReadState srs) throws IOException { + UnloadHelper unloadHelper; + if (srs.context.mergeInfo != null + || srs.context.flushInfo != null + || DISABLE + || (unloadHelper = UnloaderCoordinationPoint.getUnloadHelper(dir)) == null) { + return open.get(); + } + String type = DocValuesProducer.class.getSimpleName(); + return new UnloadingDocValuesProducer( + unloadHelper, + (u) -> { + long start = System.nanoTime(); + DocValuesProducer dvp = open.get(); + try { + u.exec.schedule( + maybeUnloadTask(u, type, u.reporter), + KEEP_ALIVE_NANOS + INITIAL_NANOS, + TimeUnit.NANOSECONDS); + } catch ( + @SuppressWarnings("unused") + RejectedExecutionException ex) { + // shutting down; log and swallow + if (u.out.isEnabled("UN")) + u.out.message("UN", "WARN: new DocValuesProducer while shutting down"); + } catch (Throwable t) { + try (dvp) { + throw t; + } + } + u.reporter.onLoad(start - u.lastAccessNanos, System.nanoTime() - start); + return dvp; + }, + KEEP_ALIVE_NANOS); + } + + private static void printStackTrace(Throwable t, InfoStream out) { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + t.printStackTrace(new PrintStream(baos, true, StandardCharsets.UTF_8)); + if (out.isEnabled("UN")) out.message("UN", baos.toString(StandardCharsets.UTF_8)); + } + + private static Callable maybeUnloadTask(Unloader u, String type, UnloadHelper reporter) { + return () -> { + long remaining; + try { + if (u.backing.get() == CLOSED || u.exec.isShutdown()) { + return null; + } + remaining = u.maybeUnload(); + } catch (Throwable t) { + if (!(t instanceof AlreadyClosedException)) { + if (u.out.isEnabled("UN")) + u.out.message( + "UN", + "WARN: exception in maybeUnload(); recheck in " + + TimeUnit.NANOSECONDS.toMillis(KEEP_ALIVE_NANOS) + + "ms " + + t); + printStackTrace(t, u.out); + u.exec.schedule( + maybeUnloadTask(u, type, reporter), KEEP_ALIVE_NANOS, TimeUnit.NANOSECONDS); + } + throw t; + } + if (remaining > 0) { + u.exec.schedule(maybeUnloadTask(u, type, reporter), remaining, TimeUnit.NANOSECONDS); + } else if (remaining == STILL_REFERENCED) { + u.exec.schedule(maybeUnloadTask(u, type, reporter), KEEP_ALIVE_NANOS, TimeUnit.NANOSECONDS); + } else if (remaining == UNLOADED) { + reporter.onUnload(System.nanoTime() - u.lastAccessNanos); + } else if (remaining == ALREADY_UNLOADED) { + // already unloaded + } else { + if (u.out.isEnabled("UN")) + u.out.message("UN", "ERROR: unexpected return value: " + remaining); + } + return null; + }; + } +} diff --git a/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java new file mode 100644 index 000000000000..37a55b40cbea --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.lucene.index; + +import static org.apache.lucene.index.Unloader.FPIOFunction; + +import java.io.IOException; +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.util.IOFunction; +import org.apache.lucene.util.automaton.CompiledAutomaton; + +/** + * A {@link DocValuesProducer} that conditionally unloads (and subsequently reloads on-demand) + * backing resources (via {@link Unloader}). + */ +public class UnloadingDocValuesProducer extends DocValuesProducer { + + private final Unloader u; + + /** + * Creates a new instance + * + * @param reopen opens/reopens the backing resource + * @param keepAliveNanos time threshold (since last access) at which the backing resource is + * eligible to be unloaded + * @throws IOException e.g., on error opening backing resource + */ + public UnloadingDocValuesProducer( + Unloader.UnloadHelper reporter, + IOFunction, DocValuesProducer> reopen, + long keepAliveNanos) + throws IOException { + u = new Unloader<>(reporter, reopen, keepAliveNanos, Object::toString); + } + + @Override + public void close() throws IOException { + u.close(); + } + + private final FPIOFunction checkIntegrity = + (dvp, ignored) -> { + dvp.checkIntegrity(); + return null; + }; + + @Override + public void checkIntegrity() throws IOException { + u.execute(checkIntegrity, null); + } + + private final FPIOFunction getNumeric = + DocValuesProducer::getNumeric; + + @Override + public NumericDocValues getNumeric(FieldInfo field) throws IOException { + return u.execute(getNumeric, field); + } + + private final FPIOFunction getBinary = + DocValuesProducer::getBinary; + + @Override + public BinaryDocValues getBinary(FieldInfo field) throws IOException { + return u.execute(getBinary, field); + } + + private final FPIOFunction getSorted = + DocValuesProducer::getSorted; + + @Override + public SortedDocValues getSorted(FieldInfo field) throws IOException { + return u.execute( + getSorted, + field, + (rawSorted, registerRef) -> { + // wrap so that we can track refs for returned `TermsEnum` instances + return new FilterSortedDocValues(rawSorted) { + @Override + public TermsEnum intersect(CompiledAutomaton automaton) throws IOException { + return Unloader.wrap(super.intersect(automaton), registerRef); + } + + @Override + public TermsEnum termsEnum() throws IOException { + return Unloader.wrap(super.termsEnum(), registerRef); + } + }; + }); + } + + private final FPIOFunction + getSortedNumeric = DocValuesProducer::getSortedNumeric; + + @Override + public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException { + return u.execute(getSortedNumeric, field); + } + + private final FPIOFunction getSortedSet = + DocValuesProducer::getSortedSet; + + @Override + public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException { + return u.execute( + getSortedSet, + field, + (rawSorted, registerRef) -> { + // wrap so that we can track refs for returned `TermsEnum` instances + return new FilterSortedSetDocValues(rawSorted) { + @Override + public TermsEnum intersect(CompiledAutomaton automaton) throws IOException { + return Unloader.wrap(super.intersect(automaton), registerRef); + } + + @Override + public TermsEnum termsEnum() throws IOException { + return Unloader.wrap(super.termsEnum(), registerRef); + } + }; + }); + } +} diff --git a/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java b/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java new file mode 100644 index 000000000000..6c6a40aaa20f --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.lucene.index; + +import static org.apache.lucene.index.Unloader.FPIOFunction; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Iterator; +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.codecs.FieldsProducer; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IOFunction; +import org.apache.lucene.util.automaton.CompiledAutomaton; + +/** + * A {@link DocValuesProducer} that conditionally unloads (and subsequently reloads on-demand) + * backing resources (via {@link Unloader}). + */ +public class UnloadingFieldsProducer extends FieldsProducer { + + private final int size; + + private final Unloader u; + + /** + * Creates a new instance + * + * @param reopen opens/reopens the backing resource + * @param keepAliveNanos time threshold (since last access) at which the backing resource is + * eligible to be unloaded + * @throws IOException e.g., on error opening backing resource + */ + public UnloadingFieldsProducer( + Unloader.UnloadHelper unloadHelper, + IOFunction, FieldsProducer> reopen, + long keepAliveNanos) + throws IOException { + final int[] size = new int[1]; + u = + new Unloader<>( + unloadHelper, + reopen, + keepAliveNanos, + (fp) -> { + size[0] = fp.size(); + return fp.toString(); + }); + this.size = size[0]; + } + + @Override + public void close() throws IOException { + u.close(); + } + + private final FPIOFunction checkIntegrity = + (fp, ignored) -> { + fp.checkIntegrity(); + return null; + }; + + @Override + public void checkIntegrity() throws IOException { + u.execute(checkIntegrity, null); + } + + private final FPIOFunction> iterator = + (fp, ignored) -> fp.iterator(); + + @Override + public Iterator iterator() { + try { + return u.execute(iterator, null); + } catch (IOException e) { + // this should never happen + throw new UncheckedIOException(e); + } + } + + private final FPIOFunction terms = Fields::terms; + + @Override + public Terms terms(String field) throws IOException { + return u.execute( + terms, + field, + (rawTerms, registerRef) -> { + // NOTE: we have to wrap here because a reference to the raw value may be + // retained internal to the backing `FieldsProducer`. This can generate a + // profusion of redundant references that never get collected. This is a + // memory leak, and also prevents resources from being unloaded, even when + // they should be eligible for unloading. + // + // This particular rationale for wrapping only applies to `Terms` -- other + // resources are already created as one-offs. + return new FilterLeafReader.FilterTerms(rawTerms) { + @Override + public TermsEnum iterator() throws IOException { + return Unloader.wrap(super.iterator(), registerRef); + } + + @Override + public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) + throws IOException { + return Unloader.wrap(super.intersect(compiled, startTerm), registerRef); + } + }; + }); + } + + @Override + public int size() { + return size; + } +} diff --git a/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDirectory.java b/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDirectory.java index dbefd6a03cc9..7f7ee28f401b 100644 --- a/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDirectory.java +++ b/lucene/core/src/java/org/apache/lucene/store/ByteBuffersDirectory.java @@ -30,13 +30,18 @@ import java.util.Objects; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicLong; import java.util.function.BiFunction; import java.util.function.Function; import java.util.function.Supplier; import java.util.zip.CRC32; import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.Unloader; import org.apache.lucene.util.BitUtil; +import org.apache.lucene.util.InfoStream; +import org.apache.lucene.util.NamedThreadFactory; /** * A {@link ByteBuffer}-based {@link Directory} implementation that can be used to store index files @@ -49,7 +54,7 @@ * * @lucene.experimental */ -public final class ByteBuffersDirectory extends BaseDirectory { +public final class ByteBuffersDirectory extends BaseDirectory implements UnloaderCoordinationPoint { public static final BiFunction OUTPUT_AS_MANY_BUFFERS = (fileName, output) -> { ByteBuffersDataInput dataInput = output.toDataInput(); @@ -159,6 +164,9 @@ public ByteBuffersDirectory( super(factory); this.outputToInput = Objects.requireNonNull(outputToInput); this.bbOutputSupplier = Objects.requireNonNull(bbOutputSupplier); + if (exec != null) { + unloadHelperSupplier = () -> new Unloader.AbstractUnloadHelper(exec, InfoStream.NO_OUTPUT) {}; + } } @Override @@ -256,9 +264,34 @@ public IndexInput openInput(String name, IOContext context) throws IOException { @Override public void close() throws IOException { isOpen = false; + if (Unloader.EXECUTOR_PER_DIRECTORY) { + exec.shutdownNow(); + } files.clear(); } + private ScheduledExecutorService exec = + Unloader.EXECUTOR_PER_DIRECTORY + ? Executors.newSingleThreadScheduledExecutor( + new NamedThreadFactory("unload@" + System.identityHashCode(this))) + : null; + + private volatile Supplier unloadHelperSupplier; + + @Override + public void setUnloadHelperSupplier(Supplier supplier) { + if (Unloader.EXECUTOR_PER_DIRECTORY) { + throw new IllegalStateException(); + } + this.unloadHelperSupplier = supplier; + } + + @Override + public Unloader.UnloadHelper getUnloadHelper() { + Supplier supplier = unloadHelperSupplier; + return supplier == null ? null : supplier.get(); + } + @Override public Set getPendingDeletions() { return Collections.emptySet(); diff --git a/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java b/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java index f36a651b16b6..07069436738b 100644 --- a/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java +++ b/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java @@ -36,11 +36,17 @@ import java.util.List; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Supplier; +import org.apache.lucene.index.Unloader; import org.apache.lucene.util.Constants; import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.InfoStream; +import org.apache.lucene.util.NamedThreadFactory; /** * Base class for Directory implementations that store index files in the file system. new Unloader.AbstractUnloadHelper(exec, InfoStream.NO_OUTPUT) {}; + } } /** @@ -283,9 +292,34 @@ public void syncMetaData() throws IOException { @Override public synchronized void close() throws IOException { isOpen = false; + if (Unloader.EXECUTOR_PER_DIRECTORY) { + exec.shutdownNow(); + } deletePendingFiles(); } + private final ScheduledExecutorService exec = + Unloader.EXECUTOR_PER_DIRECTORY + ? Executors.newSingleThreadScheduledExecutor( + new NamedThreadFactory("unload@" + System.identityHashCode(this))) + : null; + + private volatile Supplier unloadHelperSupplier; + + @Override + public void setUnloadHelperSupplier(Supplier supplier) { + if (Unloader.EXECUTOR_PER_DIRECTORY) { + throw new IllegalStateException(); + } + this.unloadHelperSupplier = supplier; + } + + @Override + public Unloader.UnloadHelper getUnloadHelper() { + Supplier supplier = unloadHelperSupplier; + return supplier == null ? null : supplier.get(); + } + /** * @return the underlying filesystem directory */ diff --git a/lucene/core/src/java/org/apache/lucene/store/MappedByteBufferIndexInputProvider.java b/lucene/core/src/java/org/apache/lucene/store/MappedByteBufferIndexInputProvider.java index 0b3612407a20..c0dfa6013a0c 100644 --- a/lucene/core/src/java/org/apache/lucene/store/MappedByteBufferIndexInputProvider.java +++ b/lucene/core/src/java/org/apache/lucene/store/MappedByteBufferIndexInputProvider.java @@ -37,6 +37,7 @@ import org.apache.lucene.util.Constants; import org.apache.lucene.util.SuppressForbidden; +/** Provides {@link IndexInput} over {@link MappedByteBuffer}s. */ public final class MappedByteBufferIndexInputProvider implements MMapDirectory.MMapIndexInputProvider { diff --git a/lucene/core/src/java/org/apache/lucene/store/UnloaderCoordinationPoint.java b/lucene/core/src/java/org/apache/lucene/store/UnloaderCoordinationPoint.java new file mode 100644 index 000000000000..168b4210484e --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/store/UnloaderCoordinationPoint.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.lucene.store; + +import java.util.concurrent.ScheduledExecutorService; +import java.util.function.Supplier; +import org.apache.lucene.index.Unloader; + +/** + * Provides a means of setting and getting a {@link ScheduledExecutorService} for executing + * ancillary tasks. + */ +public interface UnloaderCoordinationPoint { + Unloader.UnloadHelper getUnloadHelper(); + + void setUnloadHelperSupplier(Supplier helper); + + static boolean setUnloadHelperSupplier( + Directory dir, Supplier helperSupplier) { + dir = FilterDirectory.unwrap(dir); + if (dir instanceof UnloaderCoordinationPoint) { + ((UnloaderCoordinationPoint) dir).setUnloadHelperSupplier(helperSupplier); + return true; + } else { + return false; + } + } + + static Unloader.UnloadHelper getUnloadHelper(Directory dir) { + dir = FilterDirectory.unwrap(dir); + if (dir instanceof UnloaderCoordinationPoint) { + return ((UnloaderCoordinationPoint) dir).getUnloadHelper(); + } else { + return null; + } + } +} From e89c1a5734f1bcd0344fe3dd8dc6ee97ec92bb40 Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Fri, 10 Oct 2025 12:59:34 -0400 Subject: [PATCH 02/36] add test --- .github/workflows/gradle-precommit.yml | 4 +- gradle/testing/defaults-tests.gradle | 3 + .../org/apache/lucene/index/TestUnloader.java | 398 ++++++++++++++++++ 3 files changed, 403 insertions(+), 2 deletions(-) create mode 100644 lucene/core/src/test/org/apache/lucene/index/TestUnloader.java diff --git a/.github/workflows/gradle-precommit.yml b/.github/workflows/gradle-precommit.yml index dab5b89c08c2..1b181e4de427 100644 --- a/.github/workflows/gradle-precommit.yml +++ b/.github/workflows/gradle-precommit.yml @@ -43,7 +43,7 @@ jobs: uses: ./.github/actions/gradle-caches - name: Run gradle check (without tests) - run: ./gradlew check -x test -Ptask.times=true --max-workers 2 + run: ./gradlew check -x test -Ptask.times=true --max-workers 2 -x :lucene:analysis.tests:renderJavadoc -x :lucene:distribution.tests:renderJavadoc -x :lucene:analysis:morfologik.tests:renderJavadoc # This runs all tests without any other validation checks. tests: @@ -74,7 +74,7 @@ jobs: uses: ./.github/actions/gradle-caches - name: Run gradle tests - run: ./gradlew test "-Ptask.times=true" --max-workers 2 + run: ./gradlew test "-Ptask.times=true" -Plucene.unload.executorPerDirectory=true -Plucene.unload.ttl=1 -Plucene.unload.initial=0 --max-workers 2 - name: Echo settings run: cat gradle.properties diff --git a/gradle/testing/defaults-tests.gradle b/gradle/testing/defaults-tests.gradle index 25a2291bc182..23cfe629cb73 100644 --- a/gradle/testing/defaults-tests.gradle +++ b/gradle/testing/defaults-tests.gradle @@ -31,6 +31,9 @@ allprojects { // completes. // [propName: 'tests.foo', value: "bar", description: "Sets foo in tests."], testOptions = [ + [propName: 'lucene.unload.executorPerDirectory', value: false, description: "Directory instances supply their own unloading executor"], + [propName: 'lucene.unload.ttl', value: '60m', description: "Time since last use at which a resource becomes eligible for unloading"], + [propName: 'lucene.unload.initial', value: '1m', description: "Extra time allotted for first resource use after load/reload"], // asserts, debug output. [propName: 'tests.verbose', value: false, description: "Enables verbose mode (emits full test outputs immediately)."], [propName: 'tests.workDir', diff --git a/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java b/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java new file mode 100644 index 000000000000..44952a5e781c --- /dev/null +++ b/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java @@ -0,0 +1,398 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.lucene.index; + +import java.io.Closeable; +import java.io.IOException; +import java.lang.ref.ReferenceQueue; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.concurrent.CancellationException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.LongAdder; +import java.util.function.Consumer; +import java.util.function.LongSupplier; +import org.apache.lucene.store.AlreadyClosedException; +import org.apache.lucene.tests.util.LuceneTestCase; +import org.apache.lucene.util.IOFunction; +import org.apache.lucene.util.InfoStream; +import org.apache.lucene.util.NamedThreadFactory; +import org.apache.lucene.util.RamUsageEstimator; + +public class TestUnloader extends LuceneTestCase { + private static final class MyCloseable implements Closeable { + + private final LongAdder tracker; + private final AtomicInteger count; + private final LongAdder closedCt; + private final AtomicInteger exceptionOnClose = new AtomicInteger(); + + private MyCloseable(LongAdder tracker, AtomicInteger count, LongAdder closedCt) { + if (random().nextInt(20) == 0) { + throw new RuntimeException("exception opening"); + } + try { + Thread.sleep(10); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + tracker.increment(); + this.tracker = tracker; + this.count = count; + this.closedCt = closedCt; + } + + @Override + public void close() throws IOException { + int v = random().nextInt(20) == 0 ? 1 : 2; + switch (exceptionOnClose.compareAndExchange(0, v)) { + case 0: + // first time here + break; + case 1: + return; // no exception + case 2: + throw new RuntimeException("exception closing"); + default: + throw new IllegalStateException(); + } + try { + Thread.sleep(10); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + tracker.decrement(); + closedCt.increment(); + if (v == 2) { + throw new RuntimeException("exception closing"); + } + } + } + + private static final IOFunction NO_OP = (c) -> null; + + public void test() throws IOException, InterruptedException { + for (int i = 0; i < 10; i++) { + System.out.println("do " + i); + doTest(); + } + } + + private static Unloader newInstance( + LongAdder tracker, AtomicInteger count, LongAdder createdCt, LongAdder closedCt) { + while (true) { + try { + return new Unloader<>( + new Unloader.AbstractUnloadHelper(null, InfoStream.NO_OUTPUT) {}, + (unloader) -> { + MyCloseable ret = new MyCloseable(tracker, count, closedCt); + createdCt.increment(); + return ret; + }, + 0, + NO_OP); + } catch ( + @SuppressWarnings("unused") + Exception ex) { + // keep trying + } + } + } + + public void doTest() throws IOException, InterruptedException { + final int nThreadsPerOp = 2; + final LongAdder tracker = new LongAdder(); + final AtomicInteger count = new AtomicInteger(); + final LongAdder createdCt = new LongAdder(); + final LongAdder closedCt = new LongAdder(); + Unloader u = newInstance(tracker, count, createdCt, closedCt); + ExecutorService exec = + Executors.newFixedThreadPool(nThreadsPerOp + 1, new NamedThreadFactory("testUnloader")); + AtomicBoolean complete = new AtomicBoolean(); + LongAdder check = new LongAdder(); + try { + List> futures = new ArrayList<>(nThreadsPerOp + 1); + for (int i = nThreadsPerOp; i > 0; i--) { + futures.add( + exec.submit( + () -> { + while (!complete.get()) { + Thread.sleep(20); + @SuppressWarnings("unused") + Object o = + u.execute( + (a, b) -> { + a.count.incrementAndGet(); + return null; + }, + null); + check.increment(); + } + return "execute"; + })); + } + futures.add( + exec.submit( + () -> { + while (!complete.get()) { + try { + u.maybeUnload(); + } catch ( + @SuppressWarnings("unused") + AlreadyClosedException ex) { + break; + } catch (Throwable t) { + if (!"exception closing".equals(t.getMessage())) { + t.printStackTrace(System.err); + } + } + } + return "unload"; + })); + Thread.sleep(1000); // let it run for a while + System.out.println("closing ..."); + try { + u.close(); + } catch (RuntimeException ex) { + if (!"exception closing".equals(ex.getMessage())) { + throw ex; + } + } + int iterations = 0; + AssertionError deferred = null; + boolean eventuallyOk = false; + do { + try { + assertEquals(createdCt.sum(), closedCt.sum()); + assertEquals(0, tracker.sum()); + eventuallyOk = true; + } catch (AssertionError er) { + Thread.sleep(100); + if (deferred == null) { + deferred = er; + } + } + } while (++iterations < 10); + if (deferred != null) { + System.err.println("eventually ok: " + eventuallyOk); + throw deferred; + } + System.out.println( + "closed; tracker=" + + tracker.sum() + + ", count=" + + count.get() + + ", " + + createdCt.sum() + + "?=" + + closedCt.sum()); + long now = System.nanoTime(); + long until = now + TimeUnit.SECONDS.toNanos(30); + for (Future f : futures) { + try { + System.out.println("\t" + f.get(until - now, TimeUnit.NANOSECONDS)); + } catch (Exception ex) { + Throwable cause = ex.getCause(); + String msg = cause == null ? null : cause.getMessage(); + if (!(cause instanceof AlreadyClosedException) + && !"exception opening".equals(msg) + && !"exception closing".equals(msg)) { + System.out.println("\t" + ex); + ex.printStackTrace(System.out); + } + } + now = System.nanoTime(); + } + } finally { + exec.shutdown(); + exec.awaitTermination(5, TimeUnit.SECONDS); + } + } + + private static final int MAX_KB = 1024; + private static final int MIN_KB = 1; + private static final int MAX_KB_BASELINE = MAX_KB - MIN_KB + 1; + + private static final int N_SECONDS = 5; + + public void testRefQueueHandling() throws InterruptedException, ExecutionException { + int nThreads = 20; + final int batchSize = 1024; + @SuppressWarnings({"unchecked", "rawtypes"}) + Consumer[] registerRef = new Consumer[1]; + LongSupplier[] outstandingSizeHolder = new LongSupplier[1]; + @SuppressWarnings({"unchecked", "rawtypes"}) + ReferenceQueue[][] removeOutstandingHolder = new ReferenceQueue[1][]; + AtomicBoolean[] handleRefQueueHolder = new AtomicBoolean[1]; + Unloader.configure( + new Unloader.UnloadHelper() { + @Override + public ScheduledExecutorService onCreation(Unloader u) { + return null; + } + + @Override + public void maybeHandleRefQueues( + ReferenceQueue[] queues, + Consumer handler, + AtomicBoolean handleRefQueue, + LongSupplier outstandingSize) { + handleRefQueue.set(true); + handleRefQueueHolder[0] = handleRefQueue; + registerRef[0] = handler; + outstandingSizeHolder[0] = outstandingSize; + removeOutstandingHolder[0] = queues; + } + }); + ReferenceQueue[] queues = removeOutstandingHolder[0]; + AtomicBoolean handleRefQueue = handleRefQueueHolder[0]; + Consumer handler = registerRef[0]; + LongSupplier outstandingSize = outstandingSizeHolder[0]; + + int PARALLEL_HEAD_FACTOR = queues.length; + ExecutorService exec = + Executors.newFixedThreadPool( + nThreads + PARALLEL_HEAD_FACTOR, new NamedThreadFactory("TestUnloader")); + AtomicBoolean finished = new AtomicBoolean(); + @SuppressWarnings("rawtypes") + Future[] futures = new Future[nThreads]; + LongAdder total = new LongAdder(); + long start = System.nanoTime(); + for (int i = nThreads - 1; i >= 0; i--) { + futures[i] = + exec.submit( + () -> { + Random r = new Random(random().nextLong()); + try { + while (!finished.get()) { + for (int j = batchSize - 1; j >= 0; j--) { + // between 1k and 1m + Unloader.addDummyReference(1024 * (r.nextInt(MAX_KB_BASELINE) + MIN_KB)); + total.increment(); + } + } + } catch (Throwable t) { + t.printStackTrace(System.err); + throw t; + } + }); + } + LongAdder activeRefQueueProcessors = new LongAdder(); + LongAdder collectedRefs = new LongAdder(); + @SuppressWarnings("rawtypes") + Future[] refQueueFutures = new Future[queues.length]; + for (int i = queues.length - 1; i >= 0; i--) { + ReferenceQueue q = queues[i]; + refQueueFutures[i] = + exec.submit( + () -> { + activeRefQueueProcessors.increment(); + try { + while (handleRefQueue.get()) { + handler.accept(q.remove()); + collectedRefs.increment(); + } + } catch (InterruptedException ex) { + if (handleRefQueue.get()) { + // unexpected -- we've been interrupted but are still + // supposed to be handling ref queue? + handleRefQueue.set(false); + System.err.println("unexpected interruption of ref queue processing"); + ex.printStackTrace(System.err); + throw ex; + } + } catch (Throwable t) { + handleRefQueue.set(false); + System.err.println("exception in ref queue processing"); + t.printStackTrace(System.err); + throw t; + } finally { + activeRefQueueProcessors.decrement(); + } + return null; + }); + } + long endNanos = System.nanoTime() + TimeUnit.SECONDS.toNanos(N_SECONDS); + long remainingNanos; + while ((remainingNanos = endNanos - System.nanoTime()) > 0) { + long sz = outstandingSize.getAsLong(); + System.out.println( + "seconds remaining: " + + TimeUnit.NANOSECONDS.toSeconds(remainingNanos) + + ", outstandingSize=" + + sz + + " (" + + RamUsageEstimator.humanReadableUnits(sz * Unloader.RAMBYTES_PER_REF) + + ")"); + Thread.sleep(Math.min(1000, TimeUnit.NANOSECONDS.toMillis(remainingNanos))); + } + finished.set(true); + long sum = total.sum(); + for (int i = nThreads - 1; i >= 0; i--) { + futures[i].get(); + } + System.out.println( + "tasks completed " + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start)); + start = System.nanoTime(); + int gcIterations = 0; + long sz; + while ((sz = outstandingSize.getAsLong()) > 0 || Unloader.nonEmptyRefQueueHeadCount() > 0) { + gcIterations++; + System.gc(); + Thread.sleep(250); + System.err.println( + "gc iteration " + + gcIterations + + ", " + + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start) + + ", outstandingSize=" + + sz + + ", nonEmptyRefQueueHeadCount=" + + Unloader.nonEmptyRefQueueHeadCount()); + if (gcIterations > 40) { + fail("failed to converge"); + } + } + handleRefQueue.set(false); + for (int i = refQueueFutures.length - 1; i >= 0; i--) { + refQueueFutures[i].cancel(true); + } + for (int i = refQueueFutures.length - 1; i >= 0; i--) { + int idx = i; + expectThrows(CancellationException.class, () -> refQueueFutures[idx].get()); + } + exec.shutdown(); + long createdSum = total.sum(); + long collectedSum = collectedRefs.sum(); + assertEquals(createdSum, collectedSum); + System.out.println( + "success! " + + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start) + + " millis; throughput=" + + (sum / N_SECONDS) + + "/s"); + System.out.println("total created=" + createdSum + ", collected=" + collectedSum); + } +} From 7420c744e11384f9aa7a823f4cc0678cfdebd624 Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Fri, 10 Oct 2025 13:00:48 -0400 Subject: [PATCH 03/36] fix tests -- mostly around Directory closing there are still a few tests that fail, but they fail in known ways hard to address, but none are alarming. --- .../apache/lucene/index/TestAddIndexes.java | 2 + .../lucene/index/TestCodecHoldsOpenFiles.java | 53 ++++++++++--------- .../index/TestDirectoryReaderReopen.java | 2 + .../lucene/index/TestForTooMuchCloning.java | 20 +++++++ .../lucene/index/TestIndexWriterCommit.java | 3 ++ .../lucene/index/TestLazyProxSkipping.java | 6 ++- .../lucene/index/TestMultiDocValues.java | 27 ++++++++++ .../lucene/index/TestMultiLevelSkipList.java | 2 + .../apache/lucene/index/TestPointValues.java | 2 + .../lucene/index/TestReadOnlyIndex.java | 1 + .../index/TestSizeBoundedForceMerge.java | 11 ++++ .../apache/lucene/index/TestTryDelete.java | 3 ++ 12 files changed, 106 insertions(+), 26 deletions(-) diff --git a/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java b/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java index 8832f4cfde8a..32cb6e78e972 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java @@ -1053,6 +1053,7 @@ void close(boolean doWait) throws Throwable { void closeDir() throws Throwable { for (int i = 0; i < NUM_COPY; i++) readers[i].close(); dir2.close(); + dir.close(); } abstract void doBody(int j, Directory[] dirs) throws Throwable; @@ -1464,6 +1465,7 @@ public void testNonCFSLeftovers() throws Exception { assertEquals("Only one compound segment should exist", 1, sis.size()); assertTrue(sis.info(0).info.getUseCompoundFile()); dir.close(); + IOUtils.close(dirs); } private static final class UnRegisteredCodec extends FilterCodec { diff --git a/lucene/core/src/test/org/apache/lucene/index/TestCodecHoldsOpenFiles.java b/lucene/core/src/test/org/apache/lucene/index/TestCodecHoldsOpenFiles.java index 5f1a4cc5e365..60d5f1008503 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestCodecHoldsOpenFiles.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestCodecHoldsOpenFiles.java @@ -27,33 +27,38 @@ public class TestCodecHoldsOpenFiles extends LuceneTestCase { public void test() throws Exception { - BaseDirectoryWrapper d = newDirectory(); - d.setCheckIndexOnClose(false); - // we nuke files, but verify the reader still works - RandomIndexWriter w = new RandomIndexWriter(random(), d); - int numDocs = atLeast(100); - for (int i = 0; i < numDocs; i++) { - Document doc = new Document(); - doc.add(newField("foo", "bar", TextField.TYPE_NOT_STORED)); - doc.add(new IntPoint("doc", i)); - doc.add(new IntPoint("doc2d", i, i)); - doc.add(new NumericDocValuesField("dv", i)); - w.addDocument(doc); - } + Unloader.DISABLE = true; + try { + BaseDirectoryWrapper d = newDirectory(); + d.setCheckIndexOnClose(false); + // we nuke files, but verify the reader still works + RandomIndexWriter w = new RandomIndexWriter(random(), d); + int numDocs = atLeast(100); + for (int i = 0; i < numDocs; i++) { + Document doc = new Document(); + doc.add(newField("foo", "bar", TextField.TYPE_NOT_STORED)); + doc.add(new IntPoint("doc", i)); + doc.add(new IntPoint("doc2d", i, i)); + doc.add(new NumericDocValuesField("dv", i)); + w.addDocument(doc); + } - IndexReader r = w.getReader(); - w.commit(); - w.close(); + IndexReader r = w.getReader(); + w.commit(); + w.close(); - for (String name : d.listAll()) { - d.deleteFile(name); - } + for (String name : d.listAll()) { + d.deleteFile(name); + } - for (LeafReaderContext cxt : r.leaves()) { - TestUtil.checkReader(cxt.reader()); - } + for (LeafReaderContext cxt : r.leaves()) { + TestUtil.checkReader(cxt.reader()); + } - r.close(); - d.close(); + r.close(); + d.close(); + } finally { + Unloader.DISABLE = false; + } } } diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReaderReopen.java b/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReaderReopen.java index 6e993a17473e..6ab38e6aeb3c 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReaderReopen.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReaderReopen.java @@ -1068,6 +1068,8 @@ public void testDeleteIndexFilesWhileReaderStillOpen() throws Exception { () -> { DirectoryReader.openIfChanged(r); }); + r.close(); + dir.close(); } public void testReuseUnchangedLeafReaderOnDVUpdate() throws IOException { diff --git a/lucene/core/src/test/org/apache/lucene/index/TestForTooMuchCloning.java b/lucene/core/src/test/org/apache/lucene/index/TestForTooMuchCloning.java index 7c72b3d2e76a..cf2cdec6b990 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestForTooMuchCloning.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestForTooMuchCloning.java @@ -16,6 +16,8 @@ */ package org.apache.lucene.index; +import java.io.IOException; +import java.io.UncheckedIOException; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.TextField; @@ -65,6 +67,23 @@ public void test() throws Exception { dir.getInputCloneCount() < 500); final IndexSearcher s = newSearcher(r); + + // NOTE: this is a workaround -- holding Terms instances in memory prevent them + // from being GC'd, which prevents the backing FieldsProducer from being aggressively + // unloaded (via `Unloader`), which artificially inflates the number of clones. + @SuppressWarnings("unused") + Terms[] holdTermsHack = + s.getLeafContexts().stream() + .map( + (c) -> { + try { + return c.reader().terms("field"); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + }) + .toArray(Terms[]::new); + // important: set this after newSearcher, it might have run checkindex final int cloneCount = dir.getInputCloneCount(); // dir.setVerboseClone(true); @@ -83,5 +102,6 @@ public void test() throws Exception { queryCloneCount < 50); r.close(); dir.close(); + holdTermsHack = null; // ensure it's referenced, so it's not GC'd until now. } } diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java index ff13703eb5d0..4792f26af8e7 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterCommit.java @@ -383,6 +383,9 @@ public void run() { assertTrue(r2 != r); r.close(); r = r2; + // TODO: we sometimes hit `Unloader`-related FileNotFoundExceptions + // here, because there's no connection between IndexWriter and Reader. + // TODO: Confirm this is only a test issue. assertEquals("term=f:" + s + "; r=" + r, 1, r.docFreq(new Term("f", s))); } } while (++iterations < maxIterations); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestLazyProxSkipping.java b/lucene/core/src/test/org/apache/lucene/index/TestLazyProxSkipping.java index 310d8524be91..2ca27cd8df77 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestLazyProxSkipping.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestLazyProxSkipping.java @@ -62,7 +62,7 @@ public IndexInput openInput(String name, IOContext context) throws IOException { } } - private void createIndex(int numHits) throws IOException { + private Directory createIndex(int numHits) throws IOException { int numDocs = 500; final Analyzer analyzer = @@ -106,6 +106,7 @@ public TokenStreamComponents createComponents(String fieldName) { LeafReader reader = getOnlyLeafReader(DirectoryReader.open(directory)); this.searcher = newSearcher(reader); + return directory; } private ScoreDoc[] search() throws IOException { @@ -115,7 +116,7 @@ private ScoreDoc[] search() throws IOException { } private void performTest(int numHits) throws IOException { - createIndex(numHits); + Directory dir = createIndex(numHits); this.seeksCounter = 0; ScoreDoc[] hits = search(); // verify that the right number of docs was found @@ -127,6 +128,7 @@ private void performTest(int numHits) throws IOException { "seeksCounter=" + this.seeksCounter + " numHits=" + numHits, this.seeksCounter <= numHits + 1); searcher.getIndexReader().close(); + dir.close(); } public void testLazySkipping() throws IOException { diff --git a/lucene/core/src/test/org/apache/lucene/index/TestMultiDocValues.java b/lucene/core/src/test/org/apache/lucene/index/TestMultiDocValues.java index e4cb61eca0af..acee0a7d25b6 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestMultiDocValues.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestMultiDocValues.java @@ -55,6 +55,7 @@ public void testNumerics() throws Exception { } } DirectoryReader ir = iw.getReader(); + incRefFiles(iw, ir); iw.forceMerge(1); DirectoryReader ir2 = iw.getReader(); LeafReader merged = getOnlyLeafReader(ir2); @@ -100,6 +101,7 @@ public void testBinary() throws Exception { } } DirectoryReader ir = iw.getReader(); + incRefFiles(iw, ir); iw.forceMerge(1); DirectoryReader ir2 = iw.getReader(); LeafReader merged = getOnlyLeafReader(ir2); @@ -149,6 +151,7 @@ public void testSorted() throws Exception { } } DirectoryReader ir = iw.getReader(); + incRefFiles(iw, ir); iw.forceMerge(1); DirectoryReader ir2 = iw.getReader(); LeafReader merged = getOnlyLeafReader(ir2); @@ -202,6 +205,7 @@ public void testSortedWithLotsOfDups() throws Exception { } } DirectoryReader ir = iw.getReader(); + incRefFiles(iw, ir); iw.forceMerge(1); DirectoryReader ir2 = iw.getReader(); LeafReader merged = getOnlyLeafReader(ir2); @@ -254,6 +258,7 @@ public void testSortedSet() throws Exception { } } DirectoryReader ir = iw.getReader(); + incRefFiles(iw, ir); iw.forceMerge(1); DirectoryReader ir2 = iw.getReader(); LeafReader merged = getOnlyLeafReader(ir2); @@ -320,6 +325,7 @@ public void testSortedSetWithDups() throws Exception { } } DirectoryReader ir = iw.getReader(); + incRefFiles(iw, ir); iw.forceMerge(1); DirectoryReader ir2 = iw.getReader(); LeafReader merged = getOnlyLeafReader(ir2); @@ -385,6 +391,7 @@ public void testSortedNumeric() throws Exception { } } DirectoryReader ir = iw.getReader(); + incRefFiles(iw, ir); iw.forceMerge(1); DirectoryReader ir2 = iw.getReader(); LeafReader merged = getOnlyLeafReader(ir2); @@ -446,4 +453,24 @@ private void testRandomAdvanceExact(DocValuesIterator iter1, DocValuesIterator i assertEquals(exists1, exists2); } } + + /** + * In conjunction with {@link Unloader}, certain configurations/implementations of {@link + * DirectoryReader} do not incRef their associated files. This is a problem here because the files + * get deleted (though presumably still referenced as zombie filehandles from + * unclosed/non-unloading resource objects. + * + *

To work around this, we manually incRef the files. + * + *

TODO: confirm that in practice this will be a "test-only" problem. + */ + private static void incRefFiles(RandomIndexWriter w, DirectoryReader ir) throws IOException { + ir = FilterDirectoryReader.unwrap(ir); + if (ir instanceof StandardDirectoryReader) { + StandardDirectoryReader sdr = (StandardDirectoryReader) ir; + SegmentInfos segInfos = sdr.segmentInfos; + IndexWriter writer = w.w; + writer.incRefDeleter(segInfos); + } + } } diff --git a/lucene/core/src/test/org/apache/lucene/index/TestMultiLevelSkipList.java b/lucene/core/src/test/org/apache/lucene/index/TestMultiLevelSkipList.java index 0b0e6c8759cf..c444574a00b9 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestMultiLevelSkipList.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestMultiLevelSkipList.java @@ -96,6 +96,8 @@ public void testSimpleSkip() throws IOException { // because than more bytes would be read from the freqStream checkSkipTo(tp, 4800, 250); // one skip on level 2 } + reader.close(); + dir.close(); } public void checkSkipTo(PostingsEnum tp, int target, int maxCounter) throws IOException { diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java b/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java index 334db5dccc2c..a24fe10f728b 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java @@ -772,6 +772,7 @@ public void testMergedStatsOneSegmentWithoutPoints() throws IOException { assertNull(PointValues.getMaxPackedValue(reader, "field2")); assertEquals(0, PointValues.getDocCount(reader, "field2")); assertEquals(0, PointValues.size(reader, "field2")); + dir.close(); } public void testMergedStatsAllPointsDeleted() throws IOException { @@ -792,6 +793,7 @@ public void testMergedStatsAllPointsDeleted() throws IOException { assertNull(PointValues.getMaxPackedValue(reader, "field")); assertEquals(0, PointValues.getDocCount(reader, "field")); assertEquals(0, PointValues.size(reader, "field")); + dir.close(); } public void testMergedStats() throws IOException { diff --git a/lucene/core/src/test/org/apache/lucene/index/TestReadOnlyIndex.java b/lucene/core/src/test/org/apache/lucene/index/TestReadOnlyIndex.java index 9ea726f8abff..5de68c6cacdc 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestReadOnlyIndex.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestReadOnlyIndex.java @@ -100,6 +100,7 @@ private Void doTestReadOnlyIndex() throws Exception { assertEquals(1, isearcher.count(phraseQuery)); ireader.close(); + dir.close(); return null; // void } } diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSizeBoundedForceMerge.java b/lucene/core/src/test/org/apache/lucene/index/TestSizeBoundedForceMerge.java index e0fe06167871..9a1d159b373b 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestSizeBoundedForceMerge.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestSizeBoundedForceMerge.java @@ -93,6 +93,7 @@ public void testByteSizeLimit() throws Exception { // Should only be 3 segments in the index, because one of them exceeds the size limit sis = SegmentInfos.readLatestCommit(dir); assertEquals(3, sis.size()); + dir.close(); } public void testNumDocsLimit() throws Exception { @@ -125,6 +126,7 @@ public void testNumDocsLimit() throws Exception { // Should only be 3 segments in the index, because one of them exceeds the size limit SegmentInfos sis = SegmentInfos.readLatestCommit(dir); assertEquals(3, sis.size()); + dir.close(); } public void testLastSegmentTooLarge() throws Exception { @@ -151,6 +153,7 @@ public void testLastSegmentTooLarge() throws Exception { SegmentInfos sis = SegmentInfos.readLatestCommit(dir); assertEquals(2, sis.size()); + dir.close(); } public void testFirstSegmentTooLarge() throws Exception { @@ -177,6 +180,7 @@ public void testFirstSegmentTooLarge() throws Exception { SegmentInfos sis = SegmentInfos.readLatestCommit(dir); assertEquals(2, sis.size()); + dir.close(); } public void testAllSegmentsSmall() throws Exception { @@ -203,6 +207,7 @@ public void testAllSegmentsSmall() throws Exception { SegmentInfos sis = SegmentInfos.readLatestCommit(dir); assertEquals(1, sis.size()); + dir.close(); } public void testAllSegmentsLarge() throws Exception { @@ -228,6 +233,7 @@ public void testAllSegmentsLarge() throws Exception { SegmentInfos sis = SegmentInfos.readLatestCommit(dir); assertEquals(3, sis.size()); + dir.close(); } public void testOneLargeOneSmall() throws Exception { @@ -254,6 +260,7 @@ public void testOneLargeOneSmall() throws Exception { SegmentInfos sis = SegmentInfos.readLatestCommit(dir); assertEquals(4, sis.size()); + dir.close(); } public void testMergeFactor() throws Exception { @@ -286,6 +293,7 @@ public void testMergeFactor() throws Exception { // max merge docs settings. SegmentInfos sis = SegmentInfos.readLatestCommit(dir); assertEquals(4, sis.size()); + dir.close(); } public void testSingleMergeableSegment() throws Exception { @@ -315,6 +323,7 @@ public void testSingleMergeableSegment() throws Exception { SegmentInfos sis = SegmentInfos.readLatestCommit(dir); assertEquals(3, sis.size()); assertFalse(sis.info(2).hasDeletions()); + dir.close(); } public void testSingleNonMergeableSegment() throws Exception { @@ -339,6 +348,7 @@ public void testSingleNonMergeableSegment() throws Exception { // Verify that the last segment does not have deletions. SegmentInfos sis = SegmentInfos.readLatestCommit(dir); assertEquals(1, sis.size()); + dir.close(); } public void testSingleMergeableTooLargeSegment() throws Exception { @@ -367,5 +377,6 @@ public void testSingleMergeableTooLargeSegment() throws Exception { SegmentInfos sis = SegmentInfos.readLatestCommit(dir); assertEquals(1, sis.size()); assertTrue(sis.info(0).hasDeletions()); + dir.close(); } } diff --git a/lucene/core/src/test/org/apache/lucene/index/TestTryDelete.java b/lucene/core/src/test/org/apache/lucene/index/TestTryDelete.java index 78d00a1eec4e..e3e9bdf903a1 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestTryDelete.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestTryDelete.java @@ -100,6 +100,7 @@ public void testTryDeleteDocument() throws IOException { topDocs = searcher.search(new TermQuery(new Term("foo", "0")), 100); assertEquals(0, topDocs.totalHits.value); + directory.close(); } public void testTryDeleteDocumentCloseAndReopen() throws IOException { @@ -137,6 +138,7 @@ public void testTryDeleteDocumentCloseAndReopen() throws IOException { topDocs = searcher.search(new TermQuery(new Term("foo", "0")), 100); assertEquals(0, topDocs.totalHits.value); + directory.close(); } public void testDeleteDocuments() throws IOException { @@ -166,5 +168,6 @@ public void testDeleteDocuments() throws IOException { topDocs = searcher.search(new TermQuery(new Term("foo", "0")), 100); assertEquals(0, topDocs.totalHits.value); + directory.close(); } } From b6fabe61e9cf27a90a616201dee4e0ab2e0c982e Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Tue, 14 Oct 2025 12:10:17 -0400 Subject: [PATCH 04/36] allow backing val to be GC'd --- .../org/apache/lucene/index/Unloader.java | 68 ++++++++++++++++--- 1 file changed, 58 insertions(+), 10 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index 8dc95154170a..d786e5aec6a8 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -73,14 +73,48 @@ public class Unloader implements Closeable { private final UnloadHelper reporter; private final ScheduledExecutorService exec; - private static final class DelegateFuture extends CompletableFuture { + private static final class DelegateFuture extends CompletableFuture> + implements Closeable { private final boolean unloading; private final WeakReference> prev; private final AtomicInteger refCount; + private volatile T strongRef; @SuppressWarnings("unused") private DelegateFuture hardRef; // kept to prevent collection + public boolean completeStrong(T value) { + if (super.complete(new WeakReference<>(value))) { + strongRef = value; + return true; + } else { + return false; + } + } + + public T getNowStrong(T valueIfAbsent) { + T ret; + WeakReference extant = super.getNow(null); + if (extant == null) { + return valueIfAbsent; + } else if ((ret = extant.get()) == null) { + throw new NullPointerException(); + } else { + return ret; + } + } + + public T getStrong(long timeout, TimeUnit unit) + throws InterruptedException, ExecutionException, TimeoutException { + WeakReference ref = super.get(timeout, unit); + T ret = ref.get(); + if (ret == null) { + throw new NullPointerException(); + } else { + return ret; + } + } + private DelegateFuture(boolean unloading, DelegateFuture prev, int initialRefCount) { this.unloading = unloading; this.prev = new WeakReference<>(prev); @@ -118,6 +152,12 @@ private boolean unload(boolean force) { return refCount.updateAndGet(UNLOAD) == UNLOADED_REFCOUNT; } } + + @Override + public void close() { + // release so it can be GC'd + strongRef = null; + } } @SuppressWarnings("unchecked") @@ -161,7 +201,7 @@ public Unloader( try { description = receiveFirstInstance.apply(in); DelegateFuture holder = new DelegateFuture<>(false, null, 0); - holder.complete(in); + holder.completeStrong(in); backing = new AtomicReference<>(holder); this.reopen = reopen; this.keepAliveNanos = keepAliveNanos; @@ -308,7 +348,7 @@ public long maybeUnload() throws IOException { // try to unload try { T weUnloaded = doUnload(holder, unloaded); - holder.complete(weUnloaded); + holder.complete(new WeakReference<>(weUnloaded)); if (weUnloaded != null) { return UNLOADED; } else { @@ -322,12 +362,12 @@ public long maybeUnload() throws IOException { private T doUnload(DelegateFuture holder, boolean[] unloaded) throws IOException { assert injectDelay(unloadRandom, 5, 20); - DelegateFuture active; + final DelegateFuture active; T toClose; try { active = holder.prev.get(); assert active != null; - toClose = active.getNow(null); + toClose = active.getNowStrong(null); } catch ( @SuppressWarnings("unused") Exception ex) { @@ -356,7 +396,7 @@ private T doUnload(DelegateFuture holder, boolean[] unloaded) throws IOExcept // time, because it's a leak at this point if we don't close the // resource we've pulled. try { - toClose = active.get(10, TimeUnit.MINUTES); + toClose = active.getStrong(10, TimeUnit.MINUTES); } catch (InterruptedException ex) { // we're probably shutting down throw new ThreadInterruptedException(ex); @@ -377,7 +417,9 @@ private T doUnload(DelegateFuture holder, boolean[] unloaded) throws IOExcept } } } - toClose.close(); + try (active) { + toClose.close(); + } unloaded[0] = true; return toClose; } @@ -469,6 +511,11 @@ private void closeUnloading(DelegateFuture unloading) throws IOException { * interrupted status before returning. * *

TODO: evaluate whether this behavior is actually desired in non-test context. + * + *

This method may return null! It should only be called (directly or indirectly) from within + * top-level {@link Unloader#close()} code. It is guaranteed to block for the specified amount of + * time, and is thus appropriate for coordination; but it should be considered "best-effort" in + * terms of returning an actual {@link Closeable} value. */ private static T interruptProtectedGet( DelegateFuture future, long longWaitSeconds, TimeUnit timeUnit) @@ -480,7 +527,7 @@ private static T interruptProtectedGet( try { while ((waitNanos = until - now) >= 0) { try { - return future.get(waitNanos, TimeUnit.NANOSECONDS); + return future.get(waitNanos, TimeUnit.NANOSECONDS).get(); } catch ( @SuppressWarnings("unused") InterruptedException ex) { @@ -526,7 +573,8 @@ private CloseableVal backing() throws IOException { long until = now + TOTAL_BLOCK_NANOS; while (!weCompute[0]) { try { - return new CloseableVal<>(holder.get(until - now, TimeUnit.NANOSECONDS), holder.refCount); + return new CloseableVal<>( + holder.getStrong(until - now, TimeUnit.NANOSECONDS), holder.refCount); } catch (ExecutionException e) { Throwable t = e.getCause(); if (t instanceof IOException) { @@ -547,7 +595,7 @@ private CloseableVal backing() throws IOException { T candidate = null; try { candidate = reopen.apply(this); - holder.complete(candidate); + holder.completeStrong(candidate); successfullyComputed = true; return new CloseableVal<>(candidate, holder.refCount); } catch (Throwable t) { From a1d1185d342737d58cc1f5c2be4d3fca47e6752a Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Tue, 14 Oct 2025 18:47:45 -0400 Subject: [PATCH 05/36] safer one-time set of external refqueue handling --- .../java/org/apache/lucene/index/Unloader.java | 18 +++++++++--------- .../org/apache/lucene/index/TestUnloader.java | 12 +++++++----- 2 files changed, 16 insertions(+), 14 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index d786e5aec6a8..8c036193ea65 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -34,7 +34,6 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.LongAdder; @@ -163,7 +162,8 @@ public void close() { @SuppressWarnings("unchecked") private final DelegateFuture closedSentinel = (DelegateFuture) CLOSED; - private static final AtomicBoolean EXTERNAL_REFQUEUE_HANDLING = new AtomicBoolean(); + private static final AtomicReference EXTERNAL_REFQUEUE_HANDLING = + new AtomicReference<>(); private static final LongAdder OUTSTANDING_SIZE = new LongAdder(); @@ -191,7 +191,7 @@ public Unloader( long keepAliveNanos, IOFunction receiveFirstInstance) throws IOException { - if (!EXTERNAL_REFQUEUE_HANDLING.get()) { + if (EXTERNAL_REFQUEUE_HANDLING.get() != Boolean.TRUE) { unloadHelper.maybeHandleRefQueues( removeOutstanding, REF_REMOVER, EXTERNAL_REFQUEUE_HANDLING, OUTSTANDING_SIZE_SUPPLIER); } @@ -334,7 +334,7 @@ private static DelegateFuture retry( * somewhere. */ public long maybeUnload() throws IOException { - if (!EXTERNAL_REFQUEUE_HANDLING.get()) drainRemoveOutstanding(); + if (EXTERNAL_REFQUEUE_HANDLING.get() != Boolean.TRUE) drainRemoveOutstanding(); long nanosSinceLastAccess = System.nanoTime() - lastAccessNanos; if (nanosSinceLastAccess < keepAliveNanos) { // don't unload @@ -670,8 +670,8 @@ private CloseableVal backing() throws IOException { /** * Number of ram bytes per instance of {@link Ref}. This can be used in conjunction with {@link * #OUTSTANDING_SIZE_SUPPLIER} (accessed via the final arg to {@link - * UnloadHelper#maybeHandleRefQueues(ReferenceQueue[], Consumer, AtomicBoolean, LongSupplier)}) to - * determine the point-in-time heap usage associated with refQueue reference tracking. + * UnloadHelper#maybeHandleRefQueues(ReferenceQueue[], Consumer, AtomicReference, LongSupplier)}) + * to determine the point-in-time heap usage associated with refQueue reference tracking. */ public static final long RAMBYTES_PER_REF = RamUsageEstimator.shallowSizeOfInstance(Ref.class) @@ -704,7 +704,7 @@ public Ref( private static void add(final Object o, AtomicInteger refCount) { int parallelIdx = Thread.currentThread().hashCode() & PARALLEL_HEAD_MASK; OUTSTANDING_SIZE.increment(); - if (!EXTERNAL_REFQUEUE_HANDLING.get()) drainRemoveOutstanding(); + if (EXTERNAL_REFQUEUE_HANDLING.get() != Boolean.TRUE) drainRemoveOutstanding(); Ref head = HEAD[parallelIdx]; final Ref ref = new Ref(o, removeOutstanding[parallelIdx], refCount, head); Ref next = reserve(head, RESERVED); @@ -944,7 +944,7 @@ static void addDummyReference(int byteSize) { * the underlying components that handle load/unload according to framework lifecycles. * *

e.g., framework may supply its own {@link ScheduledExecutorService} for running unload - * checks, and may (via {@link #maybeHandleRefQueues(ReferenceQueue[], Consumer, AtomicBoolean, + * checks, and may (via {@link #maybeHandleRefQueues(ReferenceQueue[], Consumer, AtomicReference, * LongSupplier)} manage the handling of reference tracking as well. */ public interface UnloadHelper { @@ -996,7 +996,7 @@ default void onClose() {} default void maybeHandleRefQueues( ReferenceQueue[] queues, Consumer handler, - AtomicBoolean handleRefQueue, + AtomicReference handleRefQueue, LongSupplier outstandingSize) {} ; } diff --git a/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java b/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java index 44952a5e781c..ab4d2fe22f51 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java @@ -31,6 +31,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.LongAdder; import java.util.function.Consumer; import java.util.function.LongSupplier; @@ -245,7 +246,8 @@ public void testRefQueueHandling() throws InterruptedException, ExecutionExcepti LongSupplier[] outstandingSizeHolder = new LongSupplier[1]; @SuppressWarnings({"unchecked", "rawtypes"}) ReferenceQueue[][] removeOutstandingHolder = new ReferenceQueue[1][]; - AtomicBoolean[] handleRefQueueHolder = new AtomicBoolean[1]; + @SuppressWarnings({"unchecked", "rawtypes"}) + AtomicReference[] handleRefQueueHolder = new AtomicReference[1]; Unloader.configure( new Unloader.UnloadHelper() { @Override @@ -257,7 +259,7 @@ public ScheduledExecutorService onCreation(Unloader u) { public void maybeHandleRefQueues( ReferenceQueue[] queues, Consumer handler, - AtomicBoolean handleRefQueue, + AtomicReference handleRefQueue, LongSupplier outstandingSize) { handleRefQueue.set(true); handleRefQueueHolder[0] = handleRefQueue; @@ -267,7 +269,7 @@ public void maybeHandleRefQueues( } }); ReferenceQueue[] queues = removeOutstandingHolder[0]; - AtomicBoolean handleRefQueue = handleRefQueueHolder[0]; + AtomicReference handleRefQueue = handleRefQueueHolder[0]; Consumer handler = registerRef[0]; LongSupplier outstandingSize = outstandingSizeHolder[0]; @@ -310,12 +312,12 @@ public void maybeHandleRefQueues( () -> { activeRefQueueProcessors.increment(); try { - while (handleRefQueue.get()) { + while (handleRefQueue.get() == Boolean.TRUE) { handler.accept(q.remove()); collectedRefs.increment(); } } catch (InterruptedException ex) { - if (handleRefQueue.get()) { + if (handleRefQueue.get() == Boolean.TRUE) { // unexpected -- we've been interrupted but are still // supposed to be handling ref queue? handleRefQueue.set(false); From 4e7629875988f686c9b04de7a73a75a125163071 Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Wed, 15 Oct 2025 10:31:00 -0400 Subject: [PATCH 06/36] reorder field assignments in Unloader ctor (NPE, etc.) --- .../core/src/java/org/apache/lucene/index/Unloader.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index 8c036193ea65..10e7d45eead8 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -196,15 +196,15 @@ public Unloader( removeOutstanding, REF_REMOVER, EXTERNAL_REFQUEUE_HANDLING, OUTSTANDING_SIZE_SUPPLIER); } this.reporter = unloadHelper; + this.reopen = reopen; + this.keepAliveNanos = keepAliveNanos; + DelegateFuture holder = new DelegateFuture<>(false, null, 0); + backing = new AtomicReference<>(holder); this.exec = unloadHelper.onCreation(this); T in = reopen.apply(this); try { description = receiveFirstInstance.apply(in); - DelegateFuture holder = new DelegateFuture<>(false, null, 0); holder.completeStrong(in); - backing = new AtomicReference<>(holder); - this.reopen = reopen; - this.keepAliveNanos = keepAliveNanos; } catch (Throwable t) { try (in) { unloadHelper.onClose(); From 826da66b23d7d8a1f5077e4ca87b5511afff76d8 Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Wed, 15 Oct 2025 11:20:06 -0400 Subject: [PATCH 07/36] Unloader ctor should hold a ref to prevent immediate collection --- .../org/apache/lucene/index/Unloader.java | 24 +++++++++++-------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index 10e7d45eead8..44f69470d4a9 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -198,18 +198,22 @@ public Unloader( this.reporter = unloadHelper; this.reopen = reopen; this.keepAliveNanos = keepAliveNanos; - DelegateFuture holder = new DelegateFuture<>(false, null, 0); - backing = new AtomicReference<>(holder); - this.exec = unloadHelper.onCreation(this); - T in = reopen.apply(this); + DelegateFuture holder = new DelegateFuture<>(false, null, 1); try { - description = receiveFirstInstance.apply(in); - holder.completeStrong(in); - } catch (Throwable t) { - try (in) { - unloadHelper.onClose(); - throw t; + backing = new AtomicReference<>(holder); + this.exec = unloadHelper.onCreation(this); + T in = reopen.apply(this); + try { + description = receiveFirstInstance.apply(in); + holder.completeStrong(in); + } catch (Throwable t) { + try (in) { + unloadHelper.onClose(); + throw t; + } } + } finally { + holder.refCount.updateAndGet(RELEASE); } } From 4946d5ca2fc09cb46855fd505735d7fac013c331 Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Thu, 16 Oct 2025 16:35:40 -0400 Subject: [PATCH 08/36] also unload PointsReader --- .../lucene/index/SegmentCoreReaders.java | 5 +- .../org/apache/lucene/index/Unloader.java | 90 +++++++++++++ .../lucene/index/UnloadingPointsReader.java | 118 ++++++++++++++++++ 3 files changed, 212 insertions(+), 1 deletion(-) create mode 100644 lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java b/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java index 20fdc82701a0..81965b0db5e7 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java +++ b/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java @@ -30,6 +30,7 @@ import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.KnnVectorsReader; import org.apache.lucene.codecs.NormsProducer; +import org.apache.lucene.codecs.PointsFormat; import org.apache.lucene.codecs.PointsReader; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.StoredFieldsReader; @@ -151,7 +152,9 @@ protected TermVectorsReader initialValue() { } if (coreFieldInfos.hasPointValues()) { - pointsReader = codec.pointsFormat().fieldsReader(segmentReadState); + PointsFormat pf = codec.pointsFormat(); + pointsReader = + Unloader.pointsReader(() -> pf.fieldsReader(segmentReadState), dir, segmentReadState); } else { pointsReader = null; } diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index 44f69470d4a9..33ff2ff25e4a 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -43,6 +43,7 @@ import java.util.function.Supplier; import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.codecs.FieldsProducer; +import org.apache.lucene.codecs.PointsReader; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.Directory; import org.apache.lucene.store.UnloaderCoordinationPoint; @@ -764,6 +765,58 @@ private static void remove(final Ref ref) { } } + static PointValues.PointTree wrap(PointValues.PointTree pt, Consumer registerRef) { + PointValues.PointTree ret = + new PointValues.PointTree() { + @Override + public PointValues.PointTree clone() { + return wrap(pt.clone(), registerRef); + } + + @Override + public boolean moveToChild() throws IOException { + return pt.moveToChild(); + } + + @Override + public boolean moveToSibling() throws IOException { + return pt.moveToSibling(); + } + + @Override + public boolean moveToParent() throws IOException { + return pt.moveToParent(); + } + + @Override + public byte[] getMinPackedValue() { + return pt.getMinPackedValue(); + } + + @Override + public byte[] getMaxPackedValue() { + return pt.getMaxPackedValue(); + } + + @Override + public long size() { + return pt.size(); + } + + @Override + public void visitDocIDs(PointValues.IntersectVisitor visitor) throws IOException { + pt.visitDocIDs(visitor); + } + + @Override + public void visitDocValues(PointValues.IntersectVisitor visitor) throws IOException { + pt.visitDocValues(visitor); + } + }; + registerRef.accept(ret); + return ret; + } + static TermsEnum wrap(TermsEnum te, Consumer registerRef) { TermsEnum ret = new FilterLeafReader.FilterTermsEnum(te) { @@ -1042,6 +1095,43 @@ static void configure(UnloadHelper unloadHelper) { removeOutstanding, REF_REMOVER, EXTERNAL_REFQUEUE_HANDLING, OUTSTANDING_SIZE_SUPPLIER); } + public static PointsReader pointsReader( + IOSupplier open, Directory dir, SegmentReadState srs) throws IOException { + UnloadHelper unloadHelper; + if (srs.context.mergeInfo != null + || srs.context.flushInfo != null + || DISABLE + || (unloadHelper = UnloaderCoordinationPoint.getUnloadHelper(dir)) == null) { + return open.get(); + } + String type = PointsReader.class.getSimpleName(); + return new UnloadingPointsReader( + unloadHelper, + (u) -> { + long start = System.nanoTime(); + PointsReader pr = open.get(); + try { + u.exec.schedule( + maybeUnloadTask(u, type, u.reporter), + KEEP_ALIVE_NANOS + INITIAL_NANOS, + TimeUnit.NANOSECONDS); + } catch ( + @SuppressWarnings("unused") + RejectedExecutionException ex) { + // shutting down; log and swallow + if (u.out.isEnabled("UN")) + u.out.message("UN", "WARN: new PointsReader while shutting down"); + } catch (Throwable t) { + try (pr) { + throw t; + } + } + u.reporter.onLoad(start - u.lastAccessNanos, System.nanoTime() - start); + return pr; + }, + KEEP_ALIVE_NANOS); + } + /** * Returns a {@link FieldsProducer} over the specified {@link SegmentReadState}, conditionally * wrapped to allow dynamic unloading and on-demand reloading of the backing resource. diff --git a/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java b/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java new file mode 100644 index 000000000000..645dc86a55bf --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.lucene.index; + +import static org.apache.lucene.index.Unloader.FPIOFunction; + +import java.io.IOException; +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.codecs.PointsReader; +import org.apache.lucene.util.IOFunction; + +/** + * A {@link DocValuesProducer} that conditionally unloads (and subsequently reloads on-demand) + * backing resources (via {@link Unloader}). + */ +public class UnloadingPointsReader extends PointsReader { + + private final Unloader u; + + /** + * Creates a new instance + * + * @param reopen opens/reopens the backing resource + * @param keepAliveNanos time threshold (since last access) at which the backing resource is + * eligible to be unloaded + * @throws IOException e.g., on error opening backing resource + */ + public UnloadingPointsReader( + Unloader.UnloadHelper unloadHelper, + IOFunction, PointsReader> reopen, + long keepAliveNanos) + throws IOException { + u = new Unloader<>(unloadHelper, reopen, keepAliveNanos, Object::toString); + } + + @Override + public void close() throws IOException { + u.close(); + } + + private final FPIOFunction checkIntegrity = + (pr, ignored) -> { + pr.checkIntegrity(); + return null; + }; + + @Override + public void checkIntegrity() throws IOException { + u.execute(checkIntegrity, null); + } + + private final FPIOFunction getValues = PointsReader::getValues; + + @Override + public PointValues getValues(String field) throws IOException { + return u.execute( + getValues, + field, + (rawPointValues, registerRef) -> { + // NOTE: we have to wrap here in order to track derived `PointTree` instances + return new PointValues() { + @Override + public PointTree getPointTree() throws IOException { + return Unloader.wrap(rawPointValues.getPointTree(), registerRef); + } + + @Override + public byte[] getMinPackedValue() throws IOException { + return rawPointValues.getMinPackedValue(); + } + + @Override + public byte[] getMaxPackedValue() throws IOException { + return rawPointValues.getMaxPackedValue(); + } + + @Override + public int getNumDimensions() throws IOException { + return rawPointValues.getNumDimensions(); + } + + @Override + public int getNumIndexDimensions() throws IOException { + return rawPointValues.getNumIndexDimensions(); + } + + @Override + public int getBytesPerDimension() throws IOException { + return rawPointValues.getBytesPerDimension(); + } + + @Override + public long size() { + return rawPointValues.size(); + } + + @Override + public int getDocCount() { + return rawPointValues.getDocCount(); + } + }; + }); + } +} From e77fb9b110a4b9993f64bbb33b3de81512817bb7 Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Fri, 17 Oct 2025 12:43:47 -0400 Subject: [PATCH 09/36] sysprop-configurable totally disable unloading --- lucene/core/src/java/org/apache/lucene/index/Unloader.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index 33ff2ff25e4a..5051fa592215 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -914,7 +914,11 @@ interface FPIOFunction { * but not for Lucene as used by Solr, e.g., where {@link DirectoryReader} instances are * always acquired from (and incRef) {@link IndexWriter}. */ - static boolean DISABLE = false; + static boolean DISABLE; + + static { + DISABLE = "true".equals(System.getProperty("lucene.unload.disable")); + } private static final AtomicReference> DEFERRED_INIT_MESSAGES = new AtomicReference<>(new ArrayList<>()); From dee742d7f3a90f31ff7109a755482e03305d771a Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Mon, 20 Oct 2025 12:27:56 -0400 Subject: [PATCH 10/36] must incRef before derivative refs otherwise dead reference analysis GC parent object may cause decRef, making resource to eligible for unloading before we've incRef'd for the new "child" object Saw this issue during merge on shard split --- .../org/apache/lucene/index/Unloader.java | 153 ++++++++++-------- 1 file changed, 83 insertions(+), 70 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index 5051fa592215..cd375106d8ac 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -20,6 +20,7 @@ import java.io.Closeable; import java.io.IOException; import java.io.PrintStream; +import java.io.UncheckedIOException; import java.lang.ref.ReferenceQueue; import java.lang.ref.WeakReference; import java.nio.charset.StandardCharsets; @@ -765,81 +766,101 @@ private static void remove(final Ref ref) { } } - static PointValues.PointTree wrap(PointValues.PointTree pt, Consumer registerRef) { - PointValues.PointTree ret = - new PointValues.PointTree() { - @Override - public PointValues.PointTree clone() { - return wrap(pt.clone(), registerRef); - } + static PointValues.PointTree wrap(PointValues.PointTree pt, RefTracker registerRef) + throws IOException { + return registerRef.trackedInstance( + () -> + new PointValues.PointTree() { + @Override + public PointValues.PointTree clone() { + try { + return wrap(pt.clone(), registerRef); + } catch (IOException e) { + throw new UncheckedIOException("this should never happen", e); + } + } - @Override - public boolean moveToChild() throws IOException { - return pt.moveToChild(); - } + @Override + public boolean moveToChild() throws IOException { + return pt.moveToChild(); + } - @Override - public boolean moveToSibling() throws IOException { - return pt.moveToSibling(); - } + @Override + public boolean moveToSibling() throws IOException { + return pt.moveToSibling(); + } - @Override - public boolean moveToParent() throws IOException { - return pt.moveToParent(); - } + @Override + public boolean moveToParent() throws IOException { + return pt.moveToParent(); + } - @Override - public byte[] getMinPackedValue() { - return pt.getMinPackedValue(); - } + @Override + public byte[] getMinPackedValue() { + return pt.getMinPackedValue(); + } - @Override - public byte[] getMaxPackedValue() { - return pt.getMaxPackedValue(); - } + @Override + public byte[] getMaxPackedValue() { + return pt.getMaxPackedValue(); + } - @Override - public long size() { - return pt.size(); - } + @Override + public long size() { + return pt.size(); + } - @Override - public void visitDocIDs(PointValues.IntersectVisitor visitor) throws IOException { - pt.visitDocIDs(visitor); - } + @Override + public void visitDocIDs(PointValues.IntersectVisitor visitor) throws IOException { + pt.visitDocIDs(visitor); + } - @Override - public void visitDocValues(PointValues.IntersectVisitor visitor) throws IOException { - pt.visitDocValues(visitor); - } - }; - registerRef.accept(ret); - return ret; + @Override + public void visitDocValues(PointValues.IntersectVisitor visitor) throws IOException { + pt.visitDocValues(visitor); + } + }); } - static TermsEnum wrap(TermsEnum te, Consumer registerRef) { - TermsEnum ret = - new FilterLeafReader.FilterTermsEnum(te) { - @Override - public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException { - PostingsEnum ret = super.postings(reuse, flags); - registerRef.accept(ret); - return ret; - } + static TermsEnum wrap(TermsEnum te, RefTracker registerRef) throws IOException { + return registerRef.trackedInstance( + () -> + new FilterLeafReader.FilterTermsEnum(te) { + @Override + public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException { + return registerRef.trackedInstance(() -> super.postings(reuse, flags)); + } - @Override - public ImpactsEnum impacts(int flags) throws IOException { - ImpactsEnum ret = super.impacts(flags); - registerRef.accept(ret); - return ret; - } - }; - registerRef.accept(ret); - return ret; + @Override + public ImpactsEnum impacts(int flags) throws IOException { + return registerRef.trackedInstance(() -> super.impacts(flags)); + } + }); + } + + static final class RefTracker { + private final AtomicInteger refCount; + + private RefTracker(AtomicInteger refCount) { + this.refCount = refCount; + } + + T trackedInstance(IOSupplier supplier) throws IOException { + refCount.getAndUpdate(ACQUIRE); + T ret; + try { + ret = supplier.get(); + } catch (Throwable t) { + refCount.getAndUpdate(RELEASE); + throw t; + } + add(ret, refCount); + return ret; + } } interface RefTrackShim { - V shim(V in, Consumer registerRef); + V shim(V in, RefTracker refTracker); } V execute(FPIOFunction function, K arg) throws IOException { @@ -854,15 +875,7 @@ V execute(FPIOFunction function, K arg, RefTrackShim shim) th } else { AtomicInteger refCount = active.refCount; refCount.getAndUpdate(ACQUIRE); - ret = - shim == null - ? ret - : shim.shim( - ret, - (v) -> { - refCount.getAndUpdate(ACQUIRE); - add(v, refCount); - }); + ret = shim == null ? ret : shim.shim(ret, new RefTracker(refCount)); add(ret, refCount); return ret; } From b14f3e191471594eeffb6474bc9b6f9593fb02e3 Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Mon, 20 Oct 2025 12:38:30 -0400 Subject: [PATCH 11/36] use explicit method to prevent dead ref GC --- .../org/apache/lucene/index/Unloader.java | 23 +++++++++++-------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index cd375106d8ac..25c89fd33d4d 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.io.PrintStream; import java.io.UncheckedIOException; +import java.lang.ref.Reference; import java.lang.ref.ReferenceQueue; import java.lang.ref.WeakReference; import java.nio.charset.StandardCharsets; @@ -712,17 +713,19 @@ private static void add(final Object o, AtomicInteger refCount) { OUTSTANDING_SIZE.increment(); if (EXTERNAL_REFQUEUE_HANDLING.get() != Boolean.TRUE) drainRemoveOutstanding(); Ref head = HEAD[parallelIdx]; - final Ref ref = new Ref(o, removeOutstanding[parallelIdx], refCount, head); - Ref next = reserve(head, RESERVED); - if (next != null) { - next.prev = ref; - ref.next.set(next); - } - if (!head.next.compareAndSet(RESERVED, ref)) { - throw new IllegalStateException(); + try { + final Ref ref = new Ref(o, removeOutstanding[parallelIdx], refCount, head); + Ref next = reserve(head, RESERVED); + if (next != null) { + next.prev = ref; + ref.next.set(next); + } + if (!head.next.compareAndSet(RESERVED, ref)) { + throw new IllegalStateException(); + } + } finally { + Reference.reachabilityFence(o); } - @SuppressWarnings("unused") - Object dummy = o; } private static Ref reserve(Ref ref, Ref reservation) { From c236ad0516a9a66a0db5939aa62834f0028a0fcf Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Mon, 20 Oct 2025 14:53:48 -0400 Subject: [PATCH 12/36] always track the _raw_ instance this may yet not be enough for callback-oriented methods such as PointTree.visitDocValues(). If we continue to hit errors, we'll simply have to inc/decRef at beginning/end of such methods. But for now we know we have a problem in the way we're wrapping the raw readers, since the wrapped readers are very explicitly unaware of being wrapped, so if we track the wrapper refs, they become GC'able well before the raw instances that do the actual work. In any case we'll proceed iteratively here so that we have a better sense of where the problem is, and where it's been fixed. --- .../org/apache/lucene/index/Unloader.java | 110 +++++++++--------- .../index/UnloadingDocValuesProducer.java | 10 +- .../lucene/index/UnloadingFieldsProducer.java | 6 +- .../lucene/index/UnloadingPointsReader.java | 3 +- 4 files changed, 65 insertions(+), 64 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index 25c89fd33d4d..1bf7e3cde60e 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -771,74 +771,70 @@ private static void remove(final Ref ref) { static PointValues.PointTree wrap(PointValues.PointTree pt, RefTracker registerRef) throws IOException { - return registerRef.trackedInstance( - () -> - new PointValues.PointTree() { - @Override - public PointValues.PointTree clone() { - try { - return wrap(pt.clone(), registerRef); - } catch (IOException e) { - throw new UncheckedIOException("this should never happen", e); - } - } + return new PointValues.PointTree() { + @Override + public PointValues.PointTree clone() { + try { + return wrap(registerRef.trackedInstance(pt::clone), registerRef); + } catch (IOException e) { + throw new UncheckedIOException("this should never happen", e); + } + } - @Override - public boolean moveToChild() throws IOException { - return pt.moveToChild(); - } + @Override + public boolean moveToChild() throws IOException { + return pt.moveToChild(); + } - @Override - public boolean moveToSibling() throws IOException { - return pt.moveToSibling(); - } + @Override + public boolean moveToSibling() throws IOException { + return pt.moveToSibling(); + } - @Override - public boolean moveToParent() throws IOException { - return pt.moveToParent(); - } + @Override + public boolean moveToParent() throws IOException { + return pt.moveToParent(); + } - @Override - public byte[] getMinPackedValue() { - return pt.getMinPackedValue(); - } + @Override + public byte[] getMinPackedValue() { + return pt.getMinPackedValue(); + } - @Override - public byte[] getMaxPackedValue() { - return pt.getMaxPackedValue(); - } + @Override + public byte[] getMaxPackedValue() { + return pt.getMaxPackedValue(); + } - @Override - public long size() { - return pt.size(); - } + @Override + public long size() { + return pt.size(); + } - @Override - public void visitDocIDs(PointValues.IntersectVisitor visitor) throws IOException { - pt.visitDocIDs(visitor); - } + @Override + public void visitDocIDs(PointValues.IntersectVisitor visitor) throws IOException { + pt.visitDocIDs(visitor); + } - @Override - public void visitDocValues(PointValues.IntersectVisitor visitor) throws IOException { - pt.visitDocValues(visitor); - } - }); + @Override + public void visitDocValues(PointValues.IntersectVisitor visitor) throws IOException { + pt.visitDocValues(visitor); + } + }; } static TermsEnum wrap(TermsEnum te, RefTracker registerRef) throws IOException { - return registerRef.trackedInstance( - () -> - new FilterLeafReader.FilterTermsEnum(te) { - @Override - public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException { - return registerRef.trackedInstance(() -> super.postings(reuse, flags)); - } + return new FilterLeafReader.FilterTermsEnum(te) { + @Override + public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException { + return registerRef.trackedInstance(() -> super.postings(reuse, flags)); + } - @Override - public ImpactsEnum impacts(int flags) throws IOException { - return registerRef.trackedInstance(() -> super.impacts(flags)); - } - }); + @Override + public ImpactsEnum impacts(int flags) throws IOException { + return registerRef.trackedInstance(() -> super.impacts(flags)); + } + }; } static final class RefTracker { @@ -878,8 +874,8 @@ V execute(FPIOFunction function, K arg, RefTrackShim shim) th } else { AtomicInteger refCount = active.refCount; refCount.getAndUpdate(ACQUIRE); - ret = shim == null ? ret : shim.shim(ret, new RefTracker(refCount)); add(ret, refCount); + ret = shim == null ? ret : shim.shim(ret, new RefTracker(refCount)); return ret; } } finally { diff --git a/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java index 37a55b40cbea..d3489a029c3b 100644 --- a/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java +++ b/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java @@ -92,12 +92,13 @@ public SortedDocValues getSorted(FieldInfo field) throws IOException { return new FilterSortedDocValues(rawSorted) { @Override public TermsEnum intersect(CompiledAutomaton automaton) throws IOException { - return Unloader.wrap(super.intersect(automaton), registerRef); + return Unloader.wrap( + registerRef.trackedInstance(() -> super.intersect(automaton)), registerRef); } @Override public TermsEnum termsEnum() throws IOException { - return Unloader.wrap(super.termsEnum(), registerRef); + return Unloader.wrap(registerRef.trackedInstance(super::termsEnum), registerRef); } }; }); @@ -124,12 +125,13 @@ public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException { return new FilterSortedSetDocValues(rawSorted) { @Override public TermsEnum intersect(CompiledAutomaton automaton) throws IOException { - return Unloader.wrap(super.intersect(automaton), registerRef); + return Unloader.wrap( + registerRef.trackedInstance(() -> super.intersect(automaton)), registerRef); } @Override public TermsEnum termsEnum() throws IOException { - return Unloader.wrap(super.termsEnum(), registerRef); + return Unloader.wrap(registerRef.trackedInstance(super::termsEnum), registerRef); } }; }); diff --git a/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java b/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java index 6c6a40aaa20f..d5427356c746 100644 --- a/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java +++ b/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java @@ -111,13 +111,15 @@ public Terms terms(String field) throws IOException { return new FilterLeafReader.FilterTerms(rawTerms) { @Override public TermsEnum iterator() throws IOException { - return Unloader.wrap(super.iterator(), registerRef); + return Unloader.wrap(registerRef.trackedInstance(super::iterator), registerRef); } @Override public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) throws IOException { - return Unloader.wrap(super.intersect(compiled, startTerm), registerRef); + return Unloader.wrap( + registerRef.trackedInstance(() -> super.intersect(compiled, startTerm)), + registerRef); } }; }); diff --git a/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java b/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java index 645dc86a55bf..df968ab937bd 100644 --- a/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java +++ b/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java @@ -75,7 +75,8 @@ public PointValues getValues(String field) throws IOException { return new PointValues() { @Override public PointTree getPointTree() throws IOException { - return Unloader.wrap(rawPointValues.getPointTree(), registerRef); + return Unloader.wrap( + registerRef.trackedInstance(rawPointValues::getPointTree), registerRef); } @Override From 8950ded65b679370f4df29a517c96896eb856fe7 Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Thu, 23 Oct 2025 16:34:40 -0400 Subject: [PATCH 13/36] fix regression where shared objects were tracked also fix some javadocs, etc. --- .../org/apache/lucene/index/Unloader.java | 45 ++++++- .../index/UnloadingDocValuesProducer.java | 2 + .../lucene/index/UnloadingFieldsProducer.java | 117 +++++++++++++++++- .../lucene/index/UnloadingPointsReader.java | 52 ++++++-- 4 files changed, 197 insertions(+), 19 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index 1bf7e3cde60e..f102194768ab 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -80,6 +80,8 @@ private static final class DelegateFuture extends CompletableFuture> prev; private final AtomicInteger refCount; + + @SuppressWarnings("unused") private volatile T strongRef; @SuppressWarnings("unused") @@ -863,19 +865,35 @@ interface RefTrackShim { } V execute(FPIOFunction function, K arg) throws IOException { - return execute(function, arg, null); + return execute(function, arg, true, null); } - V execute(FPIOFunction function, K arg, RefTrackShim shim) throws IOException { + /** + * trackRaw deserves special explanation: where possible, we want to track the raw + * instance, not the shimmed instance. There is an exception however (e.g. for {@link + * PointsReader#getValues(String)} and {@link FieldsProducer#terms(String)}), where shared + * instances may be returned. In order to leverage refcounting to determine when a resource is + * eligible for unloading, in such cases we must register/track the reference of our one-off + * shim/wrapper instance. + * + *

When this method is called with trackRaw=false, the shim implementation should + * (out of an abundance of caution) override all methods to wrap with try/finally and + * call {@link Reference#reachabilityFence(Object)} in the finally block; i.e.: + * Reference.reachabilityFence(this). This will prevent dead reference analysis from + * collecting the wrapper and unloading the resource after entering the shim method, but before + * completing the call to the raw/backing method. + */ + V execute(FPIOFunction function, K arg, boolean trackRaw, RefTrackShim shim) + throws IOException { try (CloseableVal active = backing()) { - V ret = function.apply(active.get(), arg); - if (ret == null) { + V raw = function.apply(active.get(), arg); + if (raw == null) { return null; } else { AtomicInteger refCount = active.refCount; refCount.getAndUpdate(ACQUIRE); - add(ret, refCount); - ret = shim == null ? ret : shim.shim(ret, new RefTracker(refCount)); + V ret = shim == null ? raw : shim.shim(raw, new RefTracker(refCount)); + add(trackRaw ? raw : ret, refCount); return ret; } } finally { @@ -1111,6 +1129,21 @@ static void configure(UnloadHelper unloadHelper) { removeOutstanding, REF_REMOVER, EXTERNAL_REFQUEUE_HANDLING, OUTSTANDING_SIZE_SUPPLIER); } + /** + * Returns a {@link PointsReader} over the specified {@link SegmentReadState}, conditionally + * wrapped to allow dynamic unloading and on-demand reloading of the backing resource. + * + *

The backing resource is initially loaded, and will be reloaded if applicable, via the + * provided `open` {@link IOSupplier}. The {@link Directory} is passed only to be used as an + * {@link UnloaderCoordinationPoint}. + * + *

NOTE: the segment files specified by {@link SegmentReadState}, which must be present upon + * initialization, must still be accessible on disk if/when the backing resource is reloaded + * (after having been unloaded). In practice, this means that {@link + * IndexWriter#incRefDeleter(SegmentInfos)} must have been called for the {@link SegmentInfos} + * associated with the specified {@link SegmentReadState}. This happens organically in many + * contexts, but not all -- particularly in tests. + */ public static PointsReader pointsReader( IOSupplier open, Directory dir, SegmentReadState srs) throws IOException { UnloadHelper unloadHelper; diff --git a/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java index d3489a029c3b..f8f929b77bf8 100644 --- a/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java +++ b/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java @@ -87,6 +87,7 @@ public SortedDocValues getSorted(FieldInfo field) throws IOException { return u.execute( getSorted, field, + true, (rawSorted, registerRef) -> { // wrap so that we can track refs for returned `TermsEnum` instances return new FilterSortedDocValues(rawSorted) { @@ -120,6 +121,7 @@ public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException { return u.execute( getSortedSet, field, + true, (rawSorted, registerRef) -> { // wrap so that we can track refs for returned `TermsEnum` instances return new FilterSortedSetDocValues(rawSorted) { diff --git a/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java b/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java index d5427356c746..ec8a1482ad81 100644 --- a/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java +++ b/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.io.UncheckedIOException; +import java.lang.ref.Reference; import java.util.Iterator; import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.codecs.FieldsProducer; @@ -99,6 +100,7 @@ public Terms terms(String field) throws IOException { return u.execute( terms, field, + false, (rawTerms, registerRef) -> { // NOTE: we have to wrap here because a reference to the raw value may be // retained internal to the backing `FieldsProducer`. This can generate a @@ -111,15 +113,122 @@ public Terms terms(String field) throws IOException { return new FilterLeafReader.FilterTerms(rawTerms) { @Override public TermsEnum iterator() throws IOException { - return Unloader.wrap(registerRef.trackedInstance(super::iterator), registerRef); + try { + return Unloader.wrap(registerRef.trackedInstance(super::iterator), registerRef); + } finally { + Reference.reachabilityFence(this); + } } @Override public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) throws IOException { - return Unloader.wrap( - registerRef.trackedInstance(() -> super.intersect(compiled, startTerm)), - registerRef); + try { + return Unloader.wrap( + registerRef.trackedInstance(() -> super.intersect(compiled, startTerm)), + registerRef); + } finally { + Reference.reachabilityFence(this); + } + } + + @Override + public long size() throws IOException { + try { + return super.size(); + } finally { + Reference.reachabilityFence(this); + } + } + + @Override + public long getSumTotalTermFreq() throws IOException { + try { + return super.getSumTotalTermFreq(); + } finally { + Reference.reachabilityFence(this); + } + } + + @Override + public long getSumDocFreq() throws IOException { + try { + return super.getSumDocFreq(); + } finally { + Reference.reachabilityFence(this); + } + } + + @Override + public int getDocCount() throws IOException { + try { + return super.getDocCount(); + } finally { + Reference.reachabilityFence(this); + } + } + + @Override + public boolean hasFreqs() { + try { + return super.hasFreqs(); + } finally { + Reference.reachabilityFence(this); + } + } + + @Override + public boolean hasOffsets() { + try { + return super.hasOffsets(); + } finally { + Reference.reachabilityFence(this); + } + } + + @Override + public boolean hasPositions() { + try { + return super.hasPositions(); + } finally { + Reference.reachabilityFence(this); + } + } + + @Override + public boolean hasPayloads() { + try { + return super.hasPayloads(); + } finally { + Reference.reachabilityFence(this); + } + } + + @Override + public Object getStats() throws IOException { + try { + return super.getStats(); + } finally { + Reference.reachabilityFence(this); + } + } + + @Override + public BytesRef getMin() throws IOException { + try { + return super.getMin(); + } finally { + Reference.reachabilityFence(this); + } + } + + @Override + public BytesRef getMax() throws IOException { + try { + return super.getMax(); + } finally { + Reference.reachabilityFence(this); + } } }; }); diff --git a/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java b/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java index df968ab937bd..0943aa384cd9 100644 --- a/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java +++ b/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java @@ -19,6 +19,7 @@ import static org.apache.lucene.index.Unloader.FPIOFunction; import java.io.IOException; +import java.lang.ref.Reference; import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.codecs.PointsReader; import org.apache.lucene.util.IOFunction; @@ -70,48 +71,81 @@ public PointValues getValues(String field) throws IOException { return u.execute( getValues, field, + false, (rawPointValues, registerRef) -> { // NOTE: we have to wrap here in order to track derived `PointTree` instances return new PointValues() { @Override public PointTree getPointTree() throws IOException { - return Unloader.wrap( - registerRef.trackedInstance(rawPointValues::getPointTree), registerRef); + try { + return Unloader.wrap( + registerRef.trackedInstance(rawPointValues::getPointTree), registerRef); + } finally { + Reference.reachabilityFence(this); + } } @Override public byte[] getMinPackedValue() throws IOException { - return rawPointValues.getMinPackedValue(); + try { + return rawPointValues.getMinPackedValue(); + } finally { + Reference.reachabilityFence(this); + } } @Override public byte[] getMaxPackedValue() throws IOException { - return rawPointValues.getMaxPackedValue(); + try { + return rawPointValues.getMaxPackedValue(); + } finally { + Reference.reachabilityFence(this); + } } @Override public int getNumDimensions() throws IOException { - return rawPointValues.getNumDimensions(); + try { + return rawPointValues.getNumDimensions(); + } finally { + Reference.reachabilityFence(this); + } } @Override public int getNumIndexDimensions() throws IOException { - return rawPointValues.getNumIndexDimensions(); + try { + return rawPointValues.getNumIndexDimensions(); + } finally { + Reference.reachabilityFence(this); + } } @Override public int getBytesPerDimension() throws IOException { - return rawPointValues.getBytesPerDimension(); + try { + return rawPointValues.getBytesPerDimension(); + } finally { + Reference.reachabilityFence(this); + } } @Override public long size() { - return rawPointValues.size(); + try { + return rawPointValues.size(); + } finally { + Reference.reachabilityFence(this); + } } @Override public int getDocCount() { - return rawPointValues.getDocCount(); + try { + return rawPointValues.getDocCount(); + } finally { + Reference.reachabilityFence(this); + } } }; }); From 232b9d5ce918732b8da604862b70e0503f0884da Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Fri, 24 Oct 2025 13:37:39 -0400 Subject: [PATCH 14/36] make lucene.unload.parallelRefQueueCount sysprop-configurable (power of 2) --- .../org/apache/lucene/index/Unloader.java | 33 ++++++++++++++++--- 1 file changed, 29 insertions(+), 4 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index f102194768ab..5668e7c6d67f 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -663,7 +663,35 @@ private CloseableVal backing() throws IOException { } }; - private static final int PARALLEL_HEAD_FACTOR = 32; + private static final AtomicReference> DEFERRED_INIT_MESSAGES = + new AtomicReference<>(new ArrayList<>()); + + private static final int DEFAULT_PARALLEL_HEAD_FACTOR = 32; + private static final int PARALLEL_HEAD_FACTOR; + + static { + List deferred = DEFERRED_INIT_MESSAGES.get(); + String spec = System.getProperty("lucene.unload.parallelRefQueueCount"); + if (spec == null) { + PARALLEL_HEAD_FACTOR = DEFAULT_PARALLEL_HEAD_FACTOR; + } else { + int v; + try { + v = Integer.parseInt(spec); + if (v < 1 || Integer.bitCount(v) != 1) { + deferred.add("WARN: bad lucene.unload.parallelRefQueueCount spec: " + spec); + v = DEFAULT_PARALLEL_HEAD_FACTOR; + } + } catch (Throwable t) { + deferred.add( + "WARN: bad lucene.unload.parallelRefQueueCount spec: " + spec + " (" + t + ")"); + v = DEFAULT_PARALLEL_HEAD_FACTOR; + } + PARALLEL_HEAD_FACTOR = v; + } + deferred.add("INFO: set static property PARALLEL_HEAD_FACTOR=" + PARALLEL_HEAD_FACTOR); + } + private static final int PARALLEL_HEAD_MASK = PARALLEL_HEAD_FACTOR - 1; @SuppressWarnings({"unchecked", "rawtypes"}) @@ -950,9 +978,6 @@ interface FPIOFunction { DISABLE = "true".equals(System.getProperty("lucene.unload.disable")); } - private static final AtomicReference> DEFERRED_INIT_MESSAGES = - new AtomicReference<>(new ArrayList<>()); - static { List deferred = DEFERRED_INIT_MESSAGES.get(); EXECUTOR_PER_DIRECTORY = From 1f9eef359ed8d69edf1e83210ed858a6ac5bd364 Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Fri, 24 Oct 2025 14:55:20 -0400 Subject: [PATCH 15/36] make lucene.unload.assignRefQueueByThread sysprop-configurable (default true) --- .../org/apache/lucene/index/Unloader.java | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index 5668e7c6d67f..824fea35ceb5 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -669,6 +669,14 @@ private CloseableVal backing() throws IOException { private static final int DEFAULT_PARALLEL_HEAD_FACTOR = 32; private static final int PARALLEL_HEAD_FACTOR; + /** + * Setting this to false ensures even (round-robin) utilization of refqueues. Assigning by thread + * is fine at the time of refqueue assignment, but can yield hotspots that could increase + * thread contention at time of ref collection (by GC threads). + */ + private static final boolean ASSIGN_REFQUEUE_BY_THREAD = + !"false".equals(System.getProperty("lucene.unload.assignRefQueueByThread")); + static { List deferred = DEFERRED_INIT_MESSAGES.get(); String spec = System.getProperty("lucene.unload.parallelRefQueueCount"); @@ -690,6 +698,8 @@ private CloseableVal backing() throws IOException { PARALLEL_HEAD_FACTOR = v; } deferred.add("INFO: set static property PARALLEL_HEAD_FACTOR=" + PARALLEL_HEAD_FACTOR); + deferred.add( + "INFO: set static property ASSIGN_REFQUEUE_BY_THREAD=" + ASSIGN_REFQUEUE_BY_THREAD); } private static final int PARALLEL_HEAD_MASK = PARALLEL_HEAD_FACTOR - 1; @@ -738,8 +748,15 @@ public Ref( private static final Ref RESERVED = new Ref(null, null, null, null); private static final Ref REMOVED = new Ref(null, null, null, null); + private static final AtomicInteger ARBITRARY_REFQUEUE = new AtomicInteger(); + private static void add(final Object o, AtomicInteger refCount) { - int parallelIdx = Thread.currentThread().hashCode() & PARALLEL_HEAD_MASK; + int parallelIdx; + if (ASSIGN_REFQUEUE_BY_THREAD) { + parallelIdx = Thread.currentThread().hashCode() & PARALLEL_HEAD_MASK; + } else { + parallelIdx = ARBITRARY_REFQUEUE.getAndIncrement() & PARALLEL_HEAD_MASK; + } OUTSTANDING_SIZE.increment(); if (EXTERNAL_REFQUEUE_HANDLING.get() != Boolean.TRUE) drainRemoveOutstanding(); Ref head = HEAD[parallelIdx]; From 2becf0b5914731c8a8ca4c3423f20b3801d86d3e Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Tue, 28 Oct 2025 16:09:00 -0400 Subject: [PATCH 16/36] place refs in a pre-tracking "holding area" This avoids the need for GC "Reference Handler" thread to do full-on ReferenceQueue tracking, and allows us to more efficiently handle the majority of refs, which live only very briefly. --- gradle/testing/defaults-tests.gradle | 2 + .../org/apache/lucene/index/Unloader.java | 310 +++++++++++++++++- .../org/apache/lucene/index/TestUnloader.java | 127 +++++-- 3 files changed, 400 insertions(+), 39 deletions(-) diff --git a/gradle/testing/defaults-tests.gradle b/gradle/testing/defaults-tests.gradle index 23cfe629cb73..42de0adb3439 100644 --- a/gradle/testing/defaults-tests.gradle +++ b/gradle/testing/defaults-tests.gradle @@ -34,6 +34,8 @@ allprojects { [propName: 'lucene.unload.executorPerDirectory', value: false, description: "Directory instances supply their own unloading executor"], [propName: 'lucene.unload.ttl', value: '60m', description: "Time since last use at which a resource becomes eligible for unloading"], [propName: 'lucene.unload.initial', value: '1m', description: "Extra time allotted for first resource use after load/reload"], + [propName: 'lucene.unload.holding', value: 'true', description: "Place refs in a \"holding area\" before fully tracking"], + [propName: 'lucene.unload.holdTargetMegabytes', value: '10', description: "Target size of pre-tracked ref \"holding area\""], // asserts, debug output. [propName: 'tests.verbose', value: false, description: "Enables verbose mode (emits full test outputs immediately)."], [propName: 'tests.workDir', diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index 824fea35ceb5..04a0d842a4f1 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -36,9 +36,13 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.LongAdder; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; import java.util.function.IntUnaryOperator; import java.util.function.LongSupplier; @@ -176,6 +180,36 @@ public void close() { private static final Consumer REF_REMOVER = (r) -> remove((Ref) r); + private static final HoldingFlusher FLUSH_HOLDING = Unloader::drainHolding; + + /** + * Flushes any eligible references from the "holding" area (where they are held for a period of + * time before being formally tracked by {@link ReferenceQueue}. + */ + public interface HoldingFlusher { + /** + * Flushes any eligible refs from the "holding" pool into the actual {@link ReferenceQueue} + * tracked phase. This method may block according to the value of `expectHoldUntil`; it may also + * execute early if the "holding area" capacity is exceeded (as determined by {@link + * #HIGH_WATERMARK}). + * + * @param expectHoldUntil initial call should pass {@link System#nanoTime()}; subsequent calls + * should pass the value placed in the `nextHoldUntil` array on the preceding invocation of + * this method. + * @param parallelIdx the index of the parallel segment of reference tracking that should be + * operated on + * @param nextHoldUntil this holds the next time (in nanos) when subsequent calls to this method + * should re-execute. The value placed in this holder should be passed into subsequent calls + * as the `expectHoldUntil` parameter. + * @return the number of refs reclaimed (i.e. removed from `holding` area, but already + * collected, so released (instead of being transferred to formal tracking + * @throws InterruptedException if interrupted while waiting to execute (e.g., if shutting down + * and this reference processor is being closed). + */ + int flush(long expectHoldUntil, int parallelIdx, long[] nextHoldUntil) + throws InterruptedException; + } + /** * Creates a new unloader to handle unloading and on-demand reloading a backing resource * @@ -198,7 +232,12 @@ public Unloader( throws IOException { if (EXTERNAL_REFQUEUE_HANDLING.get() != Boolean.TRUE) { unloadHelper.maybeHandleRefQueues( - removeOutstanding, REF_REMOVER, EXTERNAL_REFQUEUE_HANDLING, OUTSTANDING_SIZE_SUPPLIER); + removeOutstanding, + REF_REMOVER, + FLUSH_HOLDING, + EXTERNAL_REFQUEUE_HANDLING, + HOLDING_SIZE_SUPPLIER, + OUTSTANDING_SIZE_SUPPLIER); } this.reporter = unloadHelper; this.reopen = reopen; @@ -714,16 +753,233 @@ private CloseableVal backing() throws IOException { } } + private static final class HoldingRef extends WeakReference { + private final long holdUntilNanos; + private final AtomicInteger refCount; + private volatile HoldingRef prev; + + public HoldingRef(Object referent, AtomicInteger refCount, long holdNanos) { + super(referent); + this.refCount = refCount; + this.holdUntilNanos = System.nanoTime() + holdNanos; + } + } + + private static final class HoldingState { + private final HoldingRef tail; + private final HoldingRef head; + + private HoldingState(HoldingRef tail, HoldingRef head) { + this.tail = tail; + this.head = head; + } + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + private static final AtomicReference[] HOLDING = + new AtomicReference[PARALLEL_HEAD_FACTOR]; + + private static final AtomicBoolean[] HOLDING_DRAIN = new AtomicBoolean[PARALLEL_HEAD_FACTOR]; + + private static final HoldingState INITIAL = new HoldingState(null, null); + + static { + for (int i = PARALLEL_HEAD_FACTOR - 1; i >= 0; i--) { + HOLDING[i] = new AtomicReference<>(INITIAL); + HOLDING_DRAIN[i] = new AtomicBoolean(); + } + } + + private static final long[] DUMMY = new long[1]; + + private static void addHolding(Object o, AtomicInteger refCount, int parallelIdx) { + int sz = HOLD_SIZES[parallelIdx].incrementAndGet(); + if (sz == HIGH_WATERMARK || sz > FALLBACK_HIGH_WATERMARK) { + Lock lock = HOLD_LOCKS[parallelIdx]; + lock.lock(); + try { + HOLD_MUST_FLUSH[parallelIdx].signal(); + } finally { + lock.unlock(); + } + } + AtomicReference state = HOLDING[parallelIdx]; + if (EXTERNAL_REFQUEUE_HANDLING.get() != Boolean.TRUE) drainHolding(10, parallelIdx, DUMMY, 0); + HoldingRef add = new HoldingRef(o, refCount, HOLDING_NANOS); + HoldingState extant = + state.getAndUpdate((e) -> new HoldingState(add, e.head == null ? add : e.head)); + HoldingRef replaced = extant.tail; + if (replaced != null) { + replaced.prev = add; + } + } + + private static final Lock[] HOLD_LOCKS = new Lock[PARALLEL_HEAD_FACTOR]; + private static final Condition[] HOLD_MUST_FLUSH = new Condition[PARALLEL_HEAD_FACTOR]; + private static final AtomicInteger[] HOLD_SIZES = new AtomicInteger[PARALLEL_HEAD_FACTOR]; + + private static final LongSupplier HOLDING_SIZE_SUPPLIER = + () -> Arrays.stream(HOLD_SIZES).mapToLong(AtomicInteger::get).sum(); + + static { + for (int i = PARALLEL_HEAD_MASK; i >= 0; i--) { + Lock lock = new ReentrantLock(); + HOLD_LOCKS[i] = lock; + HOLD_MUST_FLUSH[i] = lock.newCondition(); + HOLD_SIZES[i] = new AtomicInteger(); + } + } + + private static final long MIN_DELAY_NANOS = TimeUnit.MILLISECONDS.toNanos(250); + + private static int drainHolding(long expectHoldUntil, int parallelIdx, long[] nextHoldUntil) + throws InterruptedException { + Lock lock = HOLD_LOCKS[parallelIdx]; + lock.lock(); + int mustTransfer; + int snapshot; + try { + if (HOLD_MUST_FLUSH[parallelIdx].awaitNanos( + Math.max(expectHoldUntil - System.nanoTime(), MIN_DELAY_NANOS)) + > 0) { + snapshot = HOLD_SIZES[parallelIdx].getAndSet(0); + mustTransfer = snapshot - LOW_WATERMARK; + } else { + snapshot = 0; + mustTransfer = 0; + } + } finally { + lock.unlock(); + } + try { + return drainHolding(Integer.MAX_VALUE, parallelIdx, nextHoldUntil, mustTransfer); + } finally { + if (snapshot != 0) { + HOLD_SIZES[parallelIdx].addAndGet(snapshot); + } + } + } + + private static boolean accept(long now, HoldingRef candidate, Object o) { + return now - candidate.holdUntilNanos > 0 || o == null; + } + + private static int drainHolding( + int limit, int parallelIdx, long[] nextHoldUntil, int mustTransfer) { + AtomicBoolean draining = HOLDING_DRAIN[parallelIdx]; + if (!draining.compareAndSet(false, true)) { + nextHoldUntil[0] = + System.nanoTime() + HOLDING_NANOS; // in normal usage this should be unusual + return 0; + } + try { + AtomicReference head = HOLDING[parallelIdx]; + long now = System.nanoTime(); + final HoldingState state = head.get(); + HoldingRef candidate = state.head; + if (candidate == null) { + nextHoldUntil[0] = now + HOLDING_NANOS; + return 0; + } else if (now - candidate.holdUntilNanos <= 0) { + nextHoldUntil[0] = candidate.holdUntilNanos; + return 0; + } + HoldingRef last; + int releaseCount = 0; + Object o = candidate.get(); + int sizeAdjust = 0; + do { + if (o == null) { + // collected; we can just release + releaseCount++; + candidate.refCount.getAndUpdate(RELEASE); + } else { + // not yet collected; we need to create a fully-tracked ref + add(o, candidate.refCount, parallelIdx); + } + last = candidate; + candidate = candidate.prev; + sizeAdjust--; + mustTransfer--; + } while (--limit > 0 + && candidate != null + && (accept(now, candidate, o = candidate.get()) || mustTransfer > 0)); + if (candidate != null) { + // easiest case; set unconditionally + HoldingRef candidateF = candidate; + head.getAndUpdate((e) -> new HoldingState(e.tail, candidateF)); + nextHoldUntil[0] = candidateF.holdUntilNanos; + } else if (state.tail == last && head.compareAndSet(state, INITIAL)) { + // we cleared the queue; nothing more to do + nextHoldUntil[0] = now + HOLDING_NANOS; + } else { + // we know that we'll have a non-null candidate + while ((candidate = last.prev) == null) { + Thread.yield(); + } + HoldingRef candidateF = candidate; + head.getAndUpdate((e) -> new HoldingState(e.tail, candidateF)); + nextHoldUntil[0] = candidateF.holdUntilNanos; + } + HOLD_SIZES[parallelIdx].addAndGet(sizeAdjust); + return releaseCount; + } finally { + draining.set(false); + } + } + /** * Number of ram bytes per instance of {@link Ref}. This can be used in conjunction with {@link * #OUTSTANDING_SIZE_SUPPLIER} (accessed via the final arg to {@link - * UnloadHelper#maybeHandleRefQueues(ReferenceQueue[], Consumer, AtomicReference, LongSupplier)}) - * to determine the point-in-time heap usage associated with refQueue reference tracking. + * UnloadHelper#maybeHandleRefQueues(ReferenceQueue[], Consumer, HoldingFlusher, AtomicReference, + * LongSupplier, LongSupplier)}) to determine the point-in-time heap usage associated with + * refQueue reference tracking. */ public static final long RAMBYTES_PER_REF = RamUsageEstimator.shallowSizeOfInstance(Ref.class) + RamUsageEstimator.shallowSizeOfInstance(AtomicReference.class); + /** + * Number of ram bytes per instance of {@link HoldingRef}. This can be used in conjunction with + * {@link #HOLDING_SIZE_SUPPLIER} (accessed via the second-to-last arg of {@link + * UnloadHelper#maybeHandleRefQueues(ReferenceQueue[], Consumer, HoldingFlusher, AtomicReference, + * LongSupplier, LongSupplier)}) to determine the point-in-time heap usage associated with + * "holding area" refQueue reference tracking. + */ + public static final long RAMBYTES_PER_HOLDINGREF = + RamUsageEstimator.shallowSizeOfInstance(HoldingRef.class); + + private static final int DEFAULT_HOLD_TARGET_MEGABYTES = 10; + private static final int HOLD_TARGET_MEGABYTES; + + static { + List deferred = DEFERRED_INIT_MESSAGES.get(); + String spec = System.getProperty("lucene.unload.holdTargetMegabytes"); + int tmp; + if (spec == null || spec.isEmpty()) { + tmp = DEFAULT_HOLD_TARGET_MEGABYTES; + } else { + try { + tmp = Integer.parseInt(spec); + } catch ( + @SuppressWarnings("unused") + Exception e) { + tmp = DEFAULT_HOLD_TARGET_MEGABYTES; + } + } + HOLD_TARGET_MEGABYTES = tmp; + deferred.add("INFO: set static property HOLD_TARGET_MEGABYTES=" + HOLD_TARGET_MEGABYTES); + } + + private static final int HIGH_WATERMARK = + HOLD_TARGET_MEGABYTES + * 1024 + * 1024 + / Math.toIntExact(RAMBYTES_PER_HOLDINGREF) + / PARALLEL_HEAD_FACTOR; + private static final int LOW_WATERMARK = HIGH_WATERMARK >> 1; + private static final int FALLBACK_HIGH_WATERMARK = HIGH_WATERMARK + LOW_WATERMARK; + private static final class Ref extends WeakReference { private final AtomicInteger refCount; private final AtomicReference next = new AtomicReference<>(); @@ -757,6 +1013,14 @@ private static void add(final Object o, AtomicInteger refCount) { } else { parallelIdx = ARBITRARY_REFQUEUE.getAndIncrement() & PARALLEL_HEAD_MASK; } + if (HOLDING_NANOS == -1) { + add(o, refCount, parallelIdx); + } else { + addHolding(o, refCount, parallelIdx); + } + } + + private static void add(final Object o, AtomicInteger refCount, int parallelIdx) { OUTSTANDING_SIZE.increment(); if (EXTERNAL_REFQUEUE_HANDLING.get() != Boolean.TRUE) drainRemoveOutstanding(); Ref head = HEAD[parallelIdx]; @@ -796,7 +1060,7 @@ private static Ref reserve(Ref ref, Ref reservation) { private static void remove(final Ref ref) { Ref next = reserve(ref, REMOVED); OUTSTANDING_SIZE.decrement(); - ref.refCount.updateAndGet(RELEASE); + ref.refCount.getAndUpdate(RELEASE); // now we have a lock on the link to next Ref prev; for (; ; ) { @@ -1012,6 +1276,9 @@ interface FPIOFunction { + TimeUnit.NANOSECONDS.toMillis(INITIAL_NANOS)); } + private static final long HOLDING_NANOS = + !"false".equals(System.getProperty("lucene.unload.holding")) ? KEEP_ALIVE_NANOS : -1; + private static long getNanos( String syspropName, String defaultSpec, long defaultNanos, List deferred) { try { @@ -1065,6 +1332,18 @@ static int nonEmptyRefQueueHeadCount() { return Math.toIntExact(Arrays.stream(HEAD).filter((r) -> r.next.get() != null).count()); } + // visible for testing + static int nonEmptyHoldingHeadCount() { + return Math.toIntExact( + Arrays.stream(HOLDING) + .filter( + (r) -> { + HoldingState s = r.get(); + return s.tail != null || s.head != null; + }) + .count()); + } + // visible for testing static void addDummyReference(int byteSize) { add(new byte[byteSize], new AtomicInteger(1)); @@ -1077,8 +1356,9 @@ static void addDummyReference(int byteSize) { * the underlying components that handle load/unload according to framework lifecycles. * *

e.g., framework may supply its own {@link ScheduledExecutorService} for running unload - * checks, and may (via {@link #maybeHandleRefQueues(ReferenceQueue[], Consumer, AtomicReference, - * LongSupplier)} manage the handling of reference tracking as well. + * checks, and may (via {@link #maybeHandleRefQueues(ReferenceQueue[], Consumer, HoldingFlusher, + * AtomicReference, LongSupplier, LongSupplier)} manage the handling of reference tracking as + * well. */ public interface UnloadHelper { /** @@ -1129,7 +1409,9 @@ default void onClose() {} default void maybeHandleRefQueues( ReferenceQueue[] queues, Consumer handler, + HoldingFlusher flushHolding, AtomicReference handleRefQueue, + LongSupplier holdingSize, LongSupplier outstandingSize) {} ; } @@ -1168,7 +1450,12 @@ public ScheduledExecutorService onCreation(Unloader u) { */ static void configure(UnloadHelper unloadHelper) { unloadHelper.maybeHandleRefQueues( - removeOutstanding, REF_REMOVER, EXTERNAL_REFQUEUE_HANDLING, OUTSTANDING_SIZE_SUPPLIER); + removeOutstanding, + REF_REMOVER, + FLUSH_HOLDING, + EXTERNAL_REFQUEUE_HANDLING, + HOLDING_SIZE_SUPPLIER, + OUTSTANDING_SIZE_SUPPLIER); } /** @@ -1371,4 +1658,13 @@ private static Callable maybeUnloadTask(Unloader u, String type, UnloadHel return null; }; } + + /** + * Analogous to {@link Runnable}, but the {@link #run()} method throws {@link + * InterruptedException}. + */ + public interface BlockingRunnable { + /** Executes this task, possibly throwing {@link InterruptedException}. */ + void run() throws InterruptedException; + } } diff --git a/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java b/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java index ab4d2fe22f51..2903e2264a22 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java @@ -22,6 +22,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Random; +import java.util.concurrent.Callable; import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -35,6 +36,8 @@ import java.util.concurrent.atomic.LongAdder; import java.util.function.Consumer; import java.util.function.LongSupplier; +import org.apache.lucene.index.Unloader.BlockingRunnable; +import org.apache.lucene.index.Unloader.HoldingFlusher; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.tests.util.LuceneTestCase; import org.apache.lucene.util.IOFunction; @@ -243,6 +246,8 @@ public void testRefQueueHandling() throws InterruptedException, ExecutionExcepti final int batchSize = 1024; @SuppressWarnings({"unchecked", "rawtypes"}) Consumer[] registerRef = new Consumer[1]; + HoldingFlusher[] flushHolder = new HoldingFlusher[1]; + LongSupplier[] holdingSizeHolder = new LongSupplier[1]; LongSupplier[] outstandingSizeHolder = new LongSupplier[1]; @SuppressWarnings({"unchecked", "rawtypes"}) ReferenceQueue[][] removeOutstandingHolder = new ReferenceQueue[1][]; @@ -259,24 +264,30 @@ public ScheduledExecutorService onCreation(Unloader u) { public void maybeHandleRefQueues( ReferenceQueue[] queues, Consumer handler, + HoldingFlusher flushHolding, AtomicReference handleRefQueue, + LongSupplier holdingSize, LongSupplier outstandingSize) { handleRefQueue.set(true); handleRefQueueHolder[0] = handleRefQueue; registerRef[0] = handler; + flushHolder[0] = flushHolding; + holdingSizeHolder[0] = holdingSize; outstandingSizeHolder[0] = outstandingSize; removeOutstandingHolder[0] = queues; } }); ReferenceQueue[] queues = removeOutstandingHolder[0]; + HoldingFlusher flush = flushHolder[0]; AtomicReference handleRefQueue = handleRefQueueHolder[0]; Consumer handler = registerRef[0]; + LongSupplier holdingSize = holdingSizeHolder[0]; LongSupplier outstandingSize = outstandingSizeHolder[0]; int PARALLEL_HEAD_FACTOR = queues.length; ExecutorService exec = Executors.newFixedThreadPool( - nThreads + PARALLEL_HEAD_FACTOR, new NamedThreadFactory("TestUnloader")); + nThreads + PARALLEL_HEAD_FACTOR << 1, new NamedThreadFactory("TestUnloader")); AtomicBoolean finished = new AtomicBoolean(); @SuppressWarnings("rawtypes") Future[] futures = new Future[nThreads]; @@ -302,49 +313,46 @@ public void maybeHandleRefQueues( }); } LongAdder activeRefQueueProcessors = new LongAdder(); + LongAdder collectedHoldingRefs = new LongAdder(); LongAdder collectedRefs = new LongAdder(); @SuppressWarnings("rawtypes") - Future[] refQueueFutures = new Future[queues.length]; + Future[] refQueueFutures = new Future[queues.length << 1]; for (int i = queues.length - 1; i >= 0; i--) { ReferenceQueue q = queues[i]; refQueueFutures[i] = exec.submit( - () -> { - activeRefQueueProcessors.increment(); - try { - while (handleRefQueue.get() == Boolean.TRUE) { + wrapTask( + handleRefQueue, + activeRefQueueProcessors, + () -> { handler.accept(q.remove()); collectedRefs.increment(); - } - } catch (InterruptedException ex) { - if (handleRefQueue.get() == Boolean.TRUE) { - // unexpected -- we've been interrupted but are still - // supposed to be handling ref queue? - handleRefQueue.set(false); - System.err.println("unexpected interruption of ref queue processing"); - ex.printStackTrace(System.err); - throw ex; - } - } catch (Throwable t) { - handleRefQueue.set(false); - System.err.println("exception in ref queue processing"); - t.printStackTrace(System.err); - throw t; - } finally { - activeRefQueueProcessors.decrement(); - } - return null; - }); + })); + long[] nextHoldUntil = new long[] {System.nanoTime()}; + int idx = i; + refQueueFutures[queues.length + i] = + exec.submit( + wrapTask( + handleRefQueue, + activeRefQueueProcessors, + () -> { + collectedHoldingRefs.add(flush.flush(nextHoldUntil[0], idx, nextHoldUntil)); + })); } long endNanos = System.nanoTime() + TimeUnit.SECONDS.toNanos(N_SECONDS); long remainingNanos; while ((remainingNanos = endNanos - System.nanoTime()) > 0) { long sz = outstandingSize.getAsLong(); + long hSz = holdingSize.getAsLong(); System.out.println( "seconds remaining: " + TimeUnit.NANOSECONDS.toSeconds(remainingNanos) + + ", holdingSize=" + + RamUsageEstimator.humanReadableUnits(hSz * Unloader.RAMBYTES_PER_HOLDINGREF) + ", outstandingSize=" + sz + + ", active=" + + activeRefQueueProcessors.sum() + " (" + RamUsageEstimator.humanReadableUnits(sz * Unloader.RAMBYTES_PER_REF) + ")"); @@ -360,7 +368,10 @@ public void maybeHandleRefQueues( start = System.nanoTime(); int gcIterations = 0; long sz; - while ((sz = outstandingSize.getAsLong()) > 0 || Unloader.nonEmptyRefQueueHeadCount() > 0) { + long hSz; + while ((hSz = holdingSize.getAsLong()) + (sz = outstandingSize.getAsLong()) > 0 + || Unloader.nonEmptyRefQueueHeadCount() > 0 + || Unloader.nonEmptyHoldingHeadCount() > 0) { gcIterations++; System.gc(); Thread.sleep(250); @@ -369,10 +380,16 @@ public void maybeHandleRefQueues( + gcIterations + ", " + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start) + + ", holdingSize=" + + RamUsageEstimator.humanReadableUnits(hSz * Unloader.RAMBYTES_PER_HOLDINGREF) + ", outstandingSize=" + sz + + ", active=" + + activeRefQueueProcessors.sum() + ", nonEmptyRefQueueHeadCount=" - + Unloader.nonEmptyRefQueueHeadCount()); + + Unloader.nonEmptyRefQueueHeadCount() + + ", nonEmptyHoldingHeadCount=" + + Unloader.nonEmptyHoldingHeadCount()); if (gcIterations > 40) { fail("failed to converge"); } @@ -382,19 +399,65 @@ public void maybeHandleRefQueues( refQueueFutures[i].cancel(true); } for (int i = refQueueFutures.length - 1; i >= 0; i--) { - int idx = i; - expectThrows(CancellationException.class, () -> refQueueFutures[idx].get()); + try { + refQueueFutures[i].get(); + } catch ( + @SuppressWarnings("unused") + CancellationException ex) { + // this is ok. + // NOTE: we can't do `expectThrows()` because depending on where + // the task is in the loop, it might exit _without_ `CancellationException`. + } } exec.shutdown(); + assertTrue(exec.awaitTermination(60, TimeUnit.SECONDS)); long createdSum = total.sum(); long collectedSum = collectedRefs.sum(); - assertEquals(createdSum, collectedSum); + long collectedHoldingSum = collectedHoldingRefs.sum(); + assertEquals(createdSum, collectedSum + collectedHoldingSum); System.out.println( "success! " + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start) + " millis; throughput=" + (sum / N_SECONDS) + "/s"); - System.out.println("total created=" + createdSum + ", collected=" + collectedSum); + System.out.println( + "total created=" + + createdSum + + ", collectedHolding=" + + collectedHoldingSum + + ", collected=" + + collectedSum); + } + + private static Callable wrapTask( + AtomicReference handleRefQueue, + LongAdder activeRefQueueProcessors, + BlockingRunnable r) { + return () -> { + activeRefQueueProcessors.increment(); + try { + while (handleRefQueue.get() == Boolean.TRUE) { + r.run(); + } + } catch (InterruptedException ex) { + if (handleRefQueue.get() == Boolean.TRUE) { + // unexpected -- we've been interrupted but are still + // supposed to be handling ref queue? + handleRefQueue.set(false); + System.err.println("unexpected interruption of ref queue processing"); + ex.printStackTrace(System.err); + throw ex; + } + } catch (Throwable t) { + handleRefQueue.set(false); + System.err.println("exception in ref queue processing"); + t.printStackTrace(System.err); + throw t; + } finally { + activeRefQueueProcessors.decrement(); + } + return null; + }; } } From c0fd64365c6e8d0e711fbaf1762ecb1f5873f552 Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Tue, 28 Oct 2025 17:35:40 -0400 Subject: [PATCH 17/36] add all available sysprops to defaults-tests.gradle --- gradle/testing/defaults-tests.gradle | 3 +++ 1 file changed, 3 insertions(+) diff --git a/gradle/testing/defaults-tests.gradle b/gradle/testing/defaults-tests.gradle index 42de0adb3439..b50bf5a911ef 100644 --- a/gradle/testing/defaults-tests.gradle +++ b/gradle/testing/defaults-tests.gradle @@ -36,6 +36,9 @@ allprojects { [propName: 'lucene.unload.initial', value: '1m', description: "Extra time allotted for first resource use after load/reload"], [propName: 'lucene.unload.holding', value: 'true', description: "Place refs in a \"holding area\" before fully tracking"], [propName: 'lucene.unload.holdTargetMegabytes', value: '10', description: "Target size of pre-tracked ref \"holding area\""], + [propName: 'lucene.unload.assignRefQueueByThread', value: 'true', description: "true assigns deterministically by thread; false is round-robin"], + [propName: 'lucene.unload.disable', value: 'false', description: "Disables resource unloading"], + [propName: 'lucene.unload.parallelRefQueueCount', value: '32', description: "Parallelization factor of refqueues"], // asserts, debug output. [propName: 'tests.verbose', value: false, description: "Enables verbose mode (emits full test outputs immediately)."], [propName: 'tests.workDir', From bbd92fc39bc9df5e3fac257174c17bb896d0027a Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Wed, 29 Oct 2025 14:05:05 -0400 Subject: [PATCH 18/36] Ref should extent PhantomReference, not WeakReference --- .../core/src/java/org/apache/lucene/index/Unloader.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index 04a0d842a4f1..3340c78692d5 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.io.PrintStream; import java.io.UncheckedIOException; +import java.lang.ref.PhantomReference; import java.lang.ref.Reference; import java.lang.ref.ReferenceQueue; import java.lang.ref.WeakReference; @@ -980,7 +981,7 @@ private static int drainHolding( private static final int LOW_WATERMARK = HIGH_WATERMARK >> 1; private static final int FALLBACK_HIGH_WATERMARK = HIGH_WATERMARK + LOW_WATERMARK; - private static final class Ref extends WeakReference { + private static final class Ref extends PhantomReference { private final AtomicInteger refCount; private final AtomicReference next = new AtomicReference<>(); private volatile Ref prev; @@ -991,6 +992,11 @@ public Ref( this.refCount = refCount; this.prev = prev; } + + @Override + public Object get() { + throw new UnsupportedOperationException("not supported"); + } } private static final Ref[] HEAD = new Ref[PARALLEL_HEAD_FACTOR]; From 3518392b4c77fe9e6345b7a9096fe198da8b353a Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Thu, 30 Oct 2025 12:02:00 -0400 Subject: [PATCH 19/36] Only directly track top-level objects from derivative objects, indirectly ensure reachability of the top-level object to prevent collection while there are outstanding derivative refs --- gradle/testing/defaults-tests.gradle | 1 + .../org/apache/lucene/index/Unloader.java | 157 ++++++++++++++++-- .../index/UnloadingDocValuesProducer.java | 8 +- .../lucene/index/UnloadingFieldsProducer.java | 4 +- .../lucene/index/UnloadingPointsReader.java | 4 +- .../org/apache/lucene/index/TestUnloader.java | 1 + 6 files changed, 159 insertions(+), 16 deletions(-) diff --git a/gradle/testing/defaults-tests.gradle b/gradle/testing/defaults-tests.gradle index b50bf5a911ef..d5903a35445e 100644 --- a/gradle/testing/defaults-tests.gradle +++ b/gradle/testing/defaults-tests.gradle @@ -39,6 +39,7 @@ allprojects { [propName: 'lucene.unload.assignRefQueueByThread', value: 'true', description: "true assigns deterministically by thread; false is round-robin"], [propName: 'lucene.unload.disable', value: 'false', description: "Disables resource unloading"], [propName: 'lucene.unload.parallelRefQueueCount', value: '32', description: "Parallelization factor of refqueues"], + [propName: 'lucene.unload.trackAllRefsDirectly', value: 'false', description: "if true, forces direct tracking of all derivative refs"], // asserts, debug output. [propName: 'tests.verbose', value: false, description: "Enables verbose mode (emits full test outputs immediately)."], [propName: 'tests.workDir', diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index 3340c78692d5..2fa4894398a7 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -54,6 +54,7 @@ import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.Directory; import org.apache.lucene.store.UnloaderCoordinationPoint; +import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.IOFunction; import org.apache.lucene.util.IOSupplier; import org.apache.lucene.util.InfoStream; @@ -237,6 +238,7 @@ public Unloader( REF_REMOVER, FLUSH_HOLDING, EXTERNAL_REFQUEUE_HANDLING, + INDIRECT_TRACK_COUNT_SUPPLIER, HOLDING_SIZE_SUPPLIER, OUTSTANDING_SIZE_SUPPLIER); } @@ -815,6 +817,9 @@ private static void addHolding(Object o, AtomicInteger refCount, int parallelIdx } } + private static final LongAdder INDIRECT_TRACK_COUNT = new LongAdder(); + private static final LongSupplier INDIRECT_TRACK_COUNT_SUPPLIER = INDIRECT_TRACK_COUNT::sum; + private static final Lock[] HOLD_LOCKS = new Lock[PARALLEL_HEAD_FACTOR]; private static final Condition[] HOLD_MUST_FLUSH = new Condition[PARALLEL_HEAD_FACTOR]; private static final AtomicInteger[] HOLD_SIZES = new AtomicInteger[PARALLEL_HEAD_FACTOR]; @@ -933,8 +938,8 @@ private static int drainHolding( * Number of ram bytes per instance of {@link Ref}. This can be used in conjunction with {@link * #OUTSTANDING_SIZE_SUPPLIER} (accessed via the final arg to {@link * UnloadHelper#maybeHandleRefQueues(ReferenceQueue[], Consumer, HoldingFlusher, AtomicReference, - * LongSupplier, LongSupplier)}) to determine the point-in-time heap usage associated with - * refQueue reference tracking. + * LongSupplier, LongSupplier, LongSupplier)}) to determine the point-in-time heap usage + * associated with refQueue reference tracking. */ public static final long RAMBYTES_PER_REF = RamUsageEstimator.shallowSizeOfInstance(Ref.class) @@ -944,8 +949,8 @@ private static int drainHolding( * Number of ram bytes per instance of {@link HoldingRef}. This can be used in conjunction with * {@link #HOLDING_SIZE_SUPPLIER} (accessed via the second-to-last arg of {@link * UnloadHelper#maybeHandleRefQueues(ReferenceQueue[], Consumer, HoldingFlusher, AtomicReference, - * LongSupplier, LongSupplier)}) to determine the point-in-time heap usage associated with - * "holding area" refQueue reference tracking. + * LongSupplier, LongSupplier, LongSupplier)}) to determine the point-in-time heap usage + * associated with "holding area" refQueue reference tracking. */ public static final long RAMBYTES_PER_HOLDINGREF = RamUsageEstimator.shallowSizeOfInstance(HoldingRef.class); @@ -1125,7 +1130,11 @@ public byte[] getMaxPackedValue() { @Override public long size() { - return pt.size(); + try { + return pt.size(); + } finally { + registerRef.ensureTopLevelReachability(); + } } @Override @@ -1140,28 +1149,135 @@ public void visitDocValues(PointValues.IntersectVisitor visitor) throws IOExcept }; } + private static final class TrackingPostingsEnum extends FilterLeafReader.FilterPostingsEnum { + private final RefTracker registerRef; + + private TrackingPostingsEnum(PostingsEnum in, RefTracker registerRef) { + super(in); + this.registerRef = registerRef; + } + + @Override + public long cost() { + try { + return super.cost(); + } finally { + registerRef.ensureTopLevelReachability(); + } + } + } + static TermsEnum wrap(TermsEnum te, RefTracker registerRef) throws IOException { return new FilterLeafReader.FilterTermsEnum(te) { @Override public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException { - return registerRef.trackedInstance(() -> super.postings(reuse, flags)); + if (TRACK_ALL_REFS_DIRECTLY) { + return registerRef.trackedInstance(() -> super.postings(reuse, flags)); + } + INDIRECT_TRACK_COUNT.increment(); + if (reuse instanceof TrackingPostingsEnum) { + PostingsEnum extantIn = ((TrackingPostingsEnum) reuse).in; + PostingsEnum check = super.postings(extantIn, flags); + if (check == extantIn) { + // same backing, with updated state + return reuse; + } else { + return new TrackingPostingsEnum(check, registerRef); + } + } else { + return new TrackingPostingsEnum(super.postings(reuse, flags), registerRef); + } } @Override public ImpactsEnum impacts(int flags) throws IOException { - return registerRef.trackedInstance(() -> super.impacts(flags)); + if (TRACK_ALL_REFS_DIRECTLY) { + return registerRef.trackedInstance(() -> super.impacts(flags)); + } else { + INDIRECT_TRACK_COUNT.increment(); + ImpactsEnum raw = super.impacts(flags); + return new ImpactsEnum() { + @Override + public void advanceShallow(int target) throws IOException { + raw.advanceShallow(target); + } + + @Override + public Impacts getImpacts() throws IOException { + return raw.getImpacts(); + } + + @Override + public int freq() throws IOException { + return raw.freq(); + } + + @Override + public int nextPosition() throws IOException { + return raw.nextPosition(); + } + + @Override + public int startOffset() throws IOException { + return raw.startOffset(); + } + + @Override + public int endOffset() throws IOException { + return raw.endOffset(); + } + + @Override + public BytesRef getPayload() throws IOException { + return raw.getPayload(); + } + + @Override + public int docID() { + return raw.docID(); + } + + @Override + public int nextDoc() throws IOException { + return raw.nextDoc(); + } + + @Override + public int advance(int target) throws IOException { + return raw.advance(target); + } + + @Override + public long cost() { + try { + return raw.cost(); + } finally { + registerRef.ensureTopLevelReachability(); + } + } + }; + } } }; } + static final boolean TRACK_ALL_REFS_DIRECTLY = + "true".equals(System.getProperty("lucene.unload.trackAllRefsDirectly")); + static final class RefTracker { + private final Object topLevel; private final AtomicInteger refCount; - private RefTracker(AtomicInteger refCount) { + RefTracker(Object topLevel, AtomicInteger refCount) { + this.topLevel = TRACK_ALL_REFS_DIRECTLY ? null : topLevel; this.refCount = refCount; } T trackedInstance(IOSupplier supplier) throws IOException { + if (!TRACK_ALL_REFS_DIRECTLY) { + INDIRECT_TRACK_COUNT.increment(); + return supplier.get(); + } refCount.getAndUpdate(ACQUIRE); T ret; try { @@ -1173,10 +1289,21 @@ T trackedInstance(IOSupplier supplier) throws IOException { add(ret, refCount); return ret; } + + /** + * Ensures that the top-level object (if present) is reachable. + * + *

Suppress warning; this method should only be called within a finally + * block. + */ + @SuppressWarnings("ReachabilityFenceUsage") + private void ensureTopLevelReachability() { + Reference.reachabilityFence(topLevel); + } } interface RefTrackShim { - V shim(V in, RefTracker refTracker); + V shim(V in, AtomicInteger refCount); } V execute(FPIOFunction function, K arg) throws IOException { @@ -1207,7 +1334,7 @@ V execute(FPIOFunction function, K arg, boolean trackRaw, RefTra } else { AtomicInteger refCount = active.refCount; refCount.getAndUpdate(ACQUIRE); - V ret = shim == null ? raw : shim.shim(raw, new RefTracker(refCount)); + V ret = shim == null ? raw : shim.shim(raw, refCount); add(trackRaw ? raw : ret, refCount); return ret; } @@ -1363,8 +1490,8 @@ static void addDummyReference(int byteSize) { * *

e.g., framework may supply its own {@link ScheduledExecutorService} for running unload * checks, and may (via {@link #maybeHandleRefQueues(ReferenceQueue[], Consumer, HoldingFlusher, - * AtomicReference, LongSupplier, LongSupplier)} manage the handling of reference tracking as - * well. + * AtomicReference, LongSupplier, LongSupplier, LongSupplier)} manage the handling of reference + * tracking as well. */ public interface UnloadHelper { /** @@ -1409,6 +1536,10 @@ default void onClose() {} * @param handleRefQueue implementations should update this to true if they plan to * handle the refQueues, and should set it back to false if/when they stop * handling any of the provided refQueues. + * @param indirectTrackedCount for metrics; the number of references not tracked directly, but + * tracked via reference to top-level tracked object. + * @param holdingSize for metrics; the number of references not formally tracked by GC, but in a + * "holding area" before being tracked if necessary. * @param outstandingSize for metrics; the number of references tracked but not yet collected * off a refQueue. */ @@ -1417,6 +1548,7 @@ default void maybeHandleRefQueues( Consumer handler, HoldingFlusher flushHolding, AtomicReference handleRefQueue, + LongSupplier indirectTrackedCount, LongSupplier holdingSize, LongSupplier outstandingSize) {} ; @@ -1460,6 +1592,7 @@ static void configure(UnloadHelper unloadHelper) { REF_REMOVER, FLUSH_HOLDING, EXTERNAL_REFQUEUE_HANDLING, + INDIRECT_TRACK_COUNT_SUPPLIER, HOLDING_SIZE_SUPPLIER, OUTSTANDING_SIZE_SUPPLIER); } diff --git a/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java index f8f929b77bf8..258822ab8673 100644 --- a/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java +++ b/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java @@ -88,9 +88,11 @@ public SortedDocValues getSorted(FieldInfo field) throws IOException { getSorted, field, true, - (rawSorted, registerRef) -> { + (rawSorted, refCount) -> { // wrap so that we can track refs for returned `TermsEnum` instances return new FilterSortedDocValues(rawSorted) { + private final Unloader.RefTracker registerRef = new Unloader.RefTracker(in, refCount); + @Override public TermsEnum intersect(CompiledAutomaton automaton) throws IOException { return Unloader.wrap( @@ -122,9 +124,11 @@ public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException { getSortedSet, field, true, - (rawSorted, registerRef) -> { + (rawSorted, refCount) -> { // wrap so that we can track refs for returned `TermsEnum` instances return new FilterSortedSetDocValues(rawSorted) { + private final Unloader.RefTracker registerRef = new Unloader.RefTracker(in, refCount); + @Override public TermsEnum intersect(CompiledAutomaton automaton) throws IOException { return Unloader.wrap( diff --git a/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java b/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java index ec8a1482ad81..4eedbb1b6505 100644 --- a/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java +++ b/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java @@ -101,7 +101,7 @@ public Terms terms(String field) throws IOException { terms, field, false, - (rawTerms, registerRef) -> { + (rawTerms, refCount) -> { // NOTE: we have to wrap here because a reference to the raw value may be // retained internal to the backing `FieldsProducer`. This can generate a // profusion of redundant references that never get collected. This is a @@ -111,6 +111,8 @@ public Terms terms(String field) throws IOException { // This particular rationale for wrapping only applies to `Terms` -- other // resources are already created as one-offs. return new FilterLeafReader.FilterTerms(rawTerms) { + private final Unloader.RefTracker registerRef = new Unloader.RefTracker(this, refCount); + @Override public TermsEnum iterator() throws IOException { try { diff --git a/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java b/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java index 0943aa384cd9..3f92a9c05c84 100644 --- a/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java +++ b/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java @@ -72,9 +72,11 @@ public PointValues getValues(String field) throws IOException { getValues, field, false, - (rawPointValues, registerRef) -> { + (rawPointValues, refCount) -> { // NOTE: we have to wrap here in order to track derived `PointTree` instances return new PointValues() { + private final Unloader.RefTracker registerRef = new Unloader.RefTracker(this, refCount); + @Override public PointTree getPointTree() throws IOException { try { diff --git a/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java b/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java index 2903e2264a22..bdb7217a448f 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java @@ -266,6 +266,7 @@ public void maybeHandleRefQueues( Consumer handler, HoldingFlusher flushHolding, AtomicReference handleRefQueue, + LongSupplier indirectTrackCount, LongSupplier holdingSize, LongSupplier outstandingSize) { handleRefQueue.set(true); From 7c76dfcea1fe080f72e233687506f0aa69425be8 Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Thu, 30 Oct 2025 15:57:55 -0400 Subject: [PATCH 20/36] do ref-tracking on an arbitrary resource-associated sentinel object --- .../org/apache/lucene/index/Unloader.java | 61 ++++++++++++---- .../index/UnloadingDocValuesProducer.java | 73 ++++++++++++++++--- .../lucene/index/UnloadingFieldsProducer.java | 53 +++++++++----- .../lucene/index/UnloadingPointsReader.java | 20 +++-- 4 files changed, 153 insertions(+), 54 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index 2fa4894398a7..636c48eb5a15 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -39,6 +39,7 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.LongAdder; import java.util.concurrent.locks.Condition; @@ -86,6 +87,7 @@ private static final class DelegateFuture extends CompletableFuture> prev; private final AtomicInteger refCount; + private final AtomicReference> sentinel; @SuppressWarnings("unused") private volatile T strongRef; @@ -130,6 +132,7 @@ private DelegateFuture(boolean unloading, DelegateFuture prev, int initialRef this.prev = new WeakReference<>(prev); if (unloading) { this.refCount = null; + this.sentinel = null; hardRef = prev; whenComplete( (r, e) -> { @@ -142,6 +145,7 @@ private DelegateFuture(boolean unloading, DelegateFuture prev, int initialRef }); } else { refCount = new AtomicInteger(initialRefCount); + sentinel = new AtomicReference<>(INITIAL_SENTINEL); } } @@ -170,6 +174,8 @@ public void close() { } } + private static final WeakReference INITIAL_SENTINEL = new WeakReference<>(null); + @SuppressWarnings("unchecked") private final DelegateFuture closedSentinel = (DelegateFuture) CLOSED; @@ -598,10 +604,13 @@ private static final class CloseableVal implements Supplier, Closeable { private final T val; private final AtomicInteger refCount; + private final AtomicReference> sentinel; - private CloseableVal(T val, AtomicInteger refCount) { + private CloseableVal( + T val, AtomicInteger refCount, AtomicReference> sentinel) { this.val = val; this.refCount = refCount; + this.sentinel = sentinel; } @Override @@ -625,7 +634,7 @@ private CloseableVal backing() throws IOException { while (!weCompute[0]) { try { return new CloseableVal<>( - holder.getStrong(until - now, TimeUnit.NANOSECONDS), holder.refCount); + holder.getStrong(until - now, TimeUnit.NANOSECONDS), holder.refCount, holder.sentinel); } catch (ExecutionException e) { Throwable t = e.getCause(); if (t instanceof IOException) { @@ -648,7 +657,7 @@ private CloseableVal backing() throws IOException { candidate = reopen.apply(this); holder.completeStrong(candidate); successfullyComputed = true; - return new CloseableVal<>(candidate, holder.refCount); + return new CloseableVal<>(candidate, holder.refCount, holder.sentinel); } catch (Throwable t) { holder.completeExceptionally(t); throw t; @@ -1133,7 +1142,7 @@ public long size() { try { return pt.size(); } finally { - registerRef.ensureTopLevelReachability(); + registerRef.ensureReachability(); } } @@ -1162,7 +1171,7 @@ public long cost() { try { return super.cost(); } finally { - registerRef.ensureTopLevelReachability(); + registerRef.ensureReachability(); } } } @@ -1252,7 +1261,7 @@ public long cost() { try { return raw.cost(); } finally { - registerRef.ensureTopLevelReachability(); + registerRef.ensureReachability(); } } }; @@ -1297,13 +1306,13 @@ T trackedInstance(IOSupplier supplier) throws IOException { * block. */ @SuppressWarnings("ReachabilityFenceUsage") - private void ensureTopLevelReachability() { + void ensureReachability() { Reference.reachabilityFence(topLevel); } } interface RefTrackShim { - V shim(V in, AtomicInteger refCount); + V shim(V in, RefTracker refTracker); } V execute(FPIOFunction function, K arg) throws IOException { @@ -1325,7 +1334,8 @@ V execute(FPIOFunction function, K arg) throws IOException { * collecting the wrapper and unloading the resource after entering the shim method, but before * completing the call to the raw/backing method. */ - V execute(FPIOFunction function, K arg, boolean trackRaw, RefTrackShim shim) + V execute( + FPIOFunction function, K arg, boolean trackRawUnused, RefTrackShim shim) throws IOException { try (CloseableVal active = backing()) { V raw = function.apply(active.get(), arg); @@ -1333,16 +1343,41 @@ V execute(FPIOFunction function, K arg, boolean trackRaw, RefTra return null; } else { AtomicInteger refCount = active.refCount; - refCount.getAndUpdate(ACQUIRE); - V ret = shim == null ? raw : shim.shim(raw, refCount); - add(trackRaw ? raw : ret, refCount); - return ret; + AtomicReference> sentinel = active.sentinel; + Object tracked; + // TODO: here we assume indirect tracking, so direct tracking won't really work anymore. + // fix to make this consistent + boolean reusedSentinel = true; + final WeakReference initial = sentinel.get(); + WeakReference weak = initial; + while ((tracked = weak.get()) == null) { + tracked = new Object(); + WeakReference extant = + sentinel.compareAndExchange(weak, new WeakReference<>(tracked)); + if (extant == weak) { + refCount.getAndUpdate(ACQUIRE); + add(tracked, refCount); + reusedSentinel = false; + break; + } else { + weak = extant; + } + } + if (reusedSentinel) { + INDIRECT_TRACK_COUNT.increment(); + } else if (initial != INITIAL_SENTINEL && out.isEnabled("UN")) { + long extra = EXTRA_SENTINELS_CREATED.incrementAndGet(); + out.message("UN", "INFO: total additional sentinels created: " + extra); + } + return shim.shim(raw, new RefTracker(tracked, refCount)); } } finally { lastAccessNanos = System.nanoTime(); } } + private static final AtomicLong EXTRA_SENTINELS_CREATED = new AtomicLong(); + interface FPIOFunction { V apply(T fp, K arg) throws IOException; } diff --git a/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java index 258822ab8673..ab85c82b768b 100644 --- a/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java +++ b/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java @@ -68,7 +68,22 @@ public void checkIntegrity() throws IOException { @Override public NumericDocValues getNumeric(FieldInfo field) throws IOException { - return u.execute(getNumeric, field); + return u.execute( + getNumeric, + field, + true, + (raw, refTracker) -> { + return new FilterNumericDocValues(raw) { + @Override + public long cost() { + try { + return super.cost(); + } finally { + refTracker.ensureReachability(); + } + } + }; + }); } private final FPIOFunction getBinary = @@ -76,7 +91,22 @@ public NumericDocValues getNumeric(FieldInfo field) throws IOException { @Override public BinaryDocValues getBinary(FieldInfo field) throws IOException { - return u.execute(getBinary, field); + return u.execute( + getBinary, + field, + true, + (raw, refTracker) -> { + return new FilterBinaryDocValues(raw) { + @Override + public long cost() { + try { + return super.cost(); + } finally { + refTracker.ensureReachability(); + } + } + }; + }); } private final FPIOFunction getSorted = @@ -88,11 +118,9 @@ public SortedDocValues getSorted(FieldInfo field) throws IOException { getSorted, field, true, - (rawSorted, refCount) -> { + (rawSorted, registerRef) -> { // wrap so that we can track refs for returned `TermsEnum` instances return new FilterSortedDocValues(rawSorted) { - private final Unloader.RefTracker registerRef = new Unloader.RefTracker(in, refCount); - @Override public TermsEnum intersect(CompiledAutomaton automaton) throws IOException { return Unloader.wrap( @@ -112,7 +140,22 @@ public TermsEnum termsEnum() throws IOException { @Override public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException { - return u.execute(getSortedNumeric, field); + return u.execute( + getSortedNumeric, + field, + true, + (raw, refTracker) -> { + return new FilterSortedNumericDocValues(raw) { + @Override + public long cost() { + try { + return super.cost(); + } finally { + refTracker.ensureReachability(); + } + } + }; + }); } private final FPIOFunction getSortedSet = @@ -124,20 +167,26 @@ public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException { getSortedSet, field, true, - (rawSorted, refCount) -> { + (rawSorted, registerRef) -> { // wrap so that we can track refs for returned `TermsEnum` instances return new FilterSortedSetDocValues(rawSorted) { - private final Unloader.RefTracker registerRef = new Unloader.RefTracker(in, refCount); - @Override public TermsEnum intersect(CompiledAutomaton automaton) throws IOException { - return Unloader.wrap( - registerRef.trackedInstance(() -> super.intersect(automaton)), registerRef); + try { + return Unloader.wrap( + registerRef.trackedInstance(() -> super.intersect(automaton)), registerRef); + } finally { + registerRef.ensureReachability(); + } } @Override public TermsEnum termsEnum() throws IOException { - return Unloader.wrap(registerRef.trackedInstance(super::termsEnum), registerRef); + try { + return Unloader.wrap(registerRef.trackedInstance(super::termsEnum), registerRef); + } finally { + registerRef.ensureReachability(); + } } }; }); diff --git a/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java b/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java index 4eedbb1b6505..82a416841a4f 100644 --- a/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java +++ b/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.io.UncheckedIOException; -import java.lang.ref.Reference; import java.util.Iterator; import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.codecs.FieldsProducer; @@ -86,7 +85,27 @@ public void checkIntegrity() throws IOException { @Override public Iterator iterator() { try { - return u.execute(iterator, null); + return u.execute( + iterator, + null, + true, + (raw, refTracker) -> { + return new Iterator() { + @Override + public boolean hasNext() { + return raw.hasNext(); + } + + @Override + public String next() { + try { + return raw.next(); + } finally { + refTracker.ensureReachability(); + } + } + }; + }); } catch (IOException e) { // this should never happen throw new UncheckedIOException(e); @@ -101,7 +120,7 @@ public Terms terms(String field) throws IOException { terms, field, false, - (rawTerms, refCount) -> { + (rawTerms, registerRef) -> { // NOTE: we have to wrap here because a reference to the raw value may be // retained internal to the backing `FieldsProducer`. This can generate a // profusion of redundant references that never get collected. This is a @@ -111,14 +130,12 @@ public Terms terms(String field) throws IOException { // This particular rationale for wrapping only applies to `Terms` -- other // resources are already created as one-offs. return new FilterLeafReader.FilterTerms(rawTerms) { - private final Unloader.RefTracker registerRef = new Unloader.RefTracker(this, refCount); - @Override public TermsEnum iterator() throws IOException { try { return Unloader.wrap(registerRef.trackedInstance(super::iterator), registerRef); } finally { - Reference.reachabilityFence(this); + registerRef.ensureReachability(); } } @@ -130,7 +147,7 @@ public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) registerRef.trackedInstance(() -> super.intersect(compiled, startTerm)), registerRef); } finally { - Reference.reachabilityFence(this); + registerRef.ensureReachability(); } } @@ -139,7 +156,7 @@ public long size() throws IOException { try { return super.size(); } finally { - Reference.reachabilityFence(this); + registerRef.ensureReachability(); } } @@ -148,7 +165,7 @@ public long getSumTotalTermFreq() throws IOException { try { return super.getSumTotalTermFreq(); } finally { - Reference.reachabilityFence(this); + registerRef.ensureReachability(); } } @@ -157,7 +174,7 @@ public long getSumDocFreq() throws IOException { try { return super.getSumDocFreq(); } finally { - Reference.reachabilityFence(this); + registerRef.ensureReachability(); } } @@ -166,7 +183,7 @@ public int getDocCount() throws IOException { try { return super.getDocCount(); } finally { - Reference.reachabilityFence(this); + registerRef.ensureReachability(); } } @@ -175,7 +192,7 @@ public boolean hasFreqs() { try { return super.hasFreqs(); } finally { - Reference.reachabilityFence(this); + registerRef.ensureReachability(); } } @@ -184,7 +201,7 @@ public boolean hasOffsets() { try { return super.hasOffsets(); } finally { - Reference.reachabilityFence(this); + registerRef.ensureReachability(); } } @@ -193,7 +210,7 @@ public boolean hasPositions() { try { return super.hasPositions(); } finally { - Reference.reachabilityFence(this); + registerRef.ensureReachability(); } } @@ -202,7 +219,7 @@ public boolean hasPayloads() { try { return super.hasPayloads(); } finally { - Reference.reachabilityFence(this); + registerRef.ensureReachability(); } } @@ -211,7 +228,7 @@ public Object getStats() throws IOException { try { return super.getStats(); } finally { - Reference.reachabilityFence(this); + registerRef.ensureReachability(); } } @@ -220,7 +237,7 @@ public BytesRef getMin() throws IOException { try { return super.getMin(); } finally { - Reference.reachabilityFence(this); + registerRef.ensureReachability(); } } @@ -229,7 +246,7 @@ public BytesRef getMax() throws IOException { try { return super.getMax(); } finally { - Reference.reachabilityFence(this); + registerRef.ensureReachability(); } } }; diff --git a/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java b/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java index 3f92a9c05c84..0edb600e8fc8 100644 --- a/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java +++ b/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java @@ -19,7 +19,6 @@ import static org.apache.lucene.index.Unloader.FPIOFunction; import java.io.IOException; -import java.lang.ref.Reference; import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.codecs.PointsReader; import org.apache.lucene.util.IOFunction; @@ -72,10 +71,9 @@ public PointValues getValues(String field) throws IOException { getValues, field, false, - (rawPointValues, refCount) -> { + (rawPointValues, registerRef) -> { // NOTE: we have to wrap here in order to track derived `PointTree` instances return new PointValues() { - private final Unloader.RefTracker registerRef = new Unloader.RefTracker(this, refCount); @Override public PointTree getPointTree() throws IOException { @@ -83,7 +81,7 @@ public PointTree getPointTree() throws IOException { return Unloader.wrap( registerRef.trackedInstance(rawPointValues::getPointTree), registerRef); } finally { - Reference.reachabilityFence(this); + registerRef.ensureReachability(); } } @@ -92,7 +90,7 @@ public byte[] getMinPackedValue() throws IOException { try { return rawPointValues.getMinPackedValue(); } finally { - Reference.reachabilityFence(this); + registerRef.ensureReachability(); } } @@ -101,7 +99,7 @@ public byte[] getMaxPackedValue() throws IOException { try { return rawPointValues.getMaxPackedValue(); } finally { - Reference.reachabilityFence(this); + registerRef.ensureReachability(); } } @@ -110,7 +108,7 @@ public int getNumDimensions() throws IOException { try { return rawPointValues.getNumDimensions(); } finally { - Reference.reachabilityFence(this); + registerRef.ensureReachability(); } } @@ -119,7 +117,7 @@ public int getNumIndexDimensions() throws IOException { try { return rawPointValues.getNumIndexDimensions(); } finally { - Reference.reachabilityFence(this); + registerRef.ensureReachability(); } } @@ -128,7 +126,7 @@ public int getBytesPerDimension() throws IOException { try { return rawPointValues.getBytesPerDimension(); } finally { - Reference.reachabilityFence(this); + registerRef.ensureReachability(); } } @@ -137,7 +135,7 @@ public long size() { try { return rawPointValues.size(); } finally { - Reference.reachabilityFence(this); + registerRef.ensureReachability(); } } @@ -146,7 +144,7 @@ public int getDocCount() { try { return rawPointValues.getDocCount(); } finally { - Reference.reachabilityFence(this); + registerRef.ensureReachability(); } } }; From 33842e1a414c684e947125c4db9d2ea433dc6e2b Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Thu, 30 Oct 2025 23:22:27 -0400 Subject: [PATCH 21/36] remove the "holding" concept --- gradle/testing/defaults-tests.gradle | 2 - .../org/apache/lucene/index/Unloader.java | 292 +----------------- .../org/apache/lucene/index/TestUnloader.java | 37 +-- 3 files changed, 9 insertions(+), 322 deletions(-) diff --git a/gradle/testing/defaults-tests.gradle b/gradle/testing/defaults-tests.gradle index d5903a35445e..401f037d805d 100644 --- a/gradle/testing/defaults-tests.gradle +++ b/gradle/testing/defaults-tests.gradle @@ -34,8 +34,6 @@ allprojects { [propName: 'lucene.unload.executorPerDirectory', value: false, description: "Directory instances supply their own unloading executor"], [propName: 'lucene.unload.ttl', value: '60m', description: "Time since last use at which a resource becomes eligible for unloading"], [propName: 'lucene.unload.initial', value: '1m', description: "Extra time allotted for first resource use after load/reload"], - [propName: 'lucene.unload.holding', value: 'true', description: "Place refs in a \"holding area\" before fully tracking"], - [propName: 'lucene.unload.holdTargetMegabytes', value: '10', description: "Target size of pre-tracked ref \"holding area\""], [propName: 'lucene.unload.assignRefQueueByThread', value: 'true', description: "true assigns deterministically by thread; false is round-robin"], [propName: 'lucene.unload.disable', value: 'false', description: "Disables resource unloading"], [propName: 'lucene.unload.parallelRefQueueCount', value: '32', description: "Parallelization factor of refqueues"], diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index 636c48eb5a15..da6dd7f9f897 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -37,14 +37,10 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.LongAdder; -import java.util.concurrent.locks.Condition; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; import java.util.function.IntUnaryOperator; import java.util.function.LongSupplier; @@ -188,36 +184,6 @@ public void close() { private static final Consumer REF_REMOVER = (r) -> remove((Ref) r); - private static final HoldingFlusher FLUSH_HOLDING = Unloader::drainHolding; - - /** - * Flushes any eligible references from the "holding" area (where they are held for a period of - * time before being formally tracked by {@link ReferenceQueue}. - */ - public interface HoldingFlusher { - /** - * Flushes any eligible refs from the "holding" pool into the actual {@link ReferenceQueue} - * tracked phase. This method may block according to the value of `expectHoldUntil`; it may also - * execute early if the "holding area" capacity is exceeded (as determined by {@link - * #HIGH_WATERMARK}). - * - * @param expectHoldUntil initial call should pass {@link System#nanoTime()}; subsequent calls - * should pass the value placed in the `nextHoldUntil` array on the preceding invocation of - * this method. - * @param parallelIdx the index of the parallel segment of reference tracking that should be - * operated on - * @param nextHoldUntil this holds the next time (in nanos) when subsequent calls to this method - * should re-execute. The value placed in this holder should be passed into subsequent calls - * as the `expectHoldUntil` parameter. - * @return the number of refs reclaimed (i.e. removed from `holding` area, but already - * collected, so released (instead of being transferred to formal tracking - * @throws InterruptedException if interrupted while waiting to execute (e.g., if shutting down - * and this reference processor is being closed). - */ - int flush(long expectHoldUntil, int parallelIdx, long[] nextHoldUntil) - throws InterruptedException; - } - /** * Creates a new unloader to handle unloading and on-demand reloading a backing resource * @@ -242,10 +208,8 @@ public Unloader( unloadHelper.maybeHandleRefQueues( removeOutstanding, REF_REMOVER, - FLUSH_HOLDING, EXTERNAL_REFQUEUE_HANDLING, INDIRECT_TRACK_COUNT_SUPPLIER, - HOLDING_SIZE_SUPPLIER, OUTSTANDING_SIZE_SUPPLIER); } this.reporter = unloadHelper; @@ -765,236 +729,20 @@ private CloseableVal backing() throws IOException { } } - private static final class HoldingRef extends WeakReference { - private final long holdUntilNanos; - private final AtomicInteger refCount; - private volatile HoldingRef prev; - - public HoldingRef(Object referent, AtomicInteger refCount, long holdNanos) { - super(referent); - this.refCount = refCount; - this.holdUntilNanos = System.nanoTime() + holdNanos; - } - } - - private static final class HoldingState { - private final HoldingRef tail; - private final HoldingRef head; - - private HoldingState(HoldingRef tail, HoldingRef head) { - this.tail = tail; - this.head = head; - } - } - - @SuppressWarnings({"unchecked", "rawtypes"}) - private static final AtomicReference[] HOLDING = - new AtomicReference[PARALLEL_HEAD_FACTOR]; - - private static final AtomicBoolean[] HOLDING_DRAIN = new AtomicBoolean[PARALLEL_HEAD_FACTOR]; - - private static final HoldingState INITIAL = new HoldingState(null, null); - - static { - for (int i = PARALLEL_HEAD_FACTOR - 1; i >= 0; i--) { - HOLDING[i] = new AtomicReference<>(INITIAL); - HOLDING_DRAIN[i] = new AtomicBoolean(); - } - } - - private static final long[] DUMMY = new long[1]; - - private static void addHolding(Object o, AtomicInteger refCount, int parallelIdx) { - int sz = HOLD_SIZES[parallelIdx].incrementAndGet(); - if (sz == HIGH_WATERMARK || sz > FALLBACK_HIGH_WATERMARK) { - Lock lock = HOLD_LOCKS[parallelIdx]; - lock.lock(); - try { - HOLD_MUST_FLUSH[parallelIdx].signal(); - } finally { - lock.unlock(); - } - } - AtomicReference state = HOLDING[parallelIdx]; - if (EXTERNAL_REFQUEUE_HANDLING.get() != Boolean.TRUE) drainHolding(10, parallelIdx, DUMMY, 0); - HoldingRef add = new HoldingRef(o, refCount, HOLDING_NANOS); - HoldingState extant = - state.getAndUpdate((e) -> new HoldingState(add, e.head == null ? add : e.head)); - HoldingRef replaced = extant.tail; - if (replaced != null) { - replaced.prev = add; - } - } - private static final LongAdder INDIRECT_TRACK_COUNT = new LongAdder(); private static final LongSupplier INDIRECT_TRACK_COUNT_SUPPLIER = INDIRECT_TRACK_COUNT::sum; - private static final Lock[] HOLD_LOCKS = new Lock[PARALLEL_HEAD_FACTOR]; - private static final Condition[] HOLD_MUST_FLUSH = new Condition[PARALLEL_HEAD_FACTOR]; - private static final AtomicInteger[] HOLD_SIZES = new AtomicInteger[PARALLEL_HEAD_FACTOR]; - - private static final LongSupplier HOLDING_SIZE_SUPPLIER = - () -> Arrays.stream(HOLD_SIZES).mapToLong(AtomicInteger::get).sum(); - - static { - for (int i = PARALLEL_HEAD_MASK; i >= 0; i--) { - Lock lock = new ReentrantLock(); - HOLD_LOCKS[i] = lock; - HOLD_MUST_FLUSH[i] = lock.newCondition(); - HOLD_SIZES[i] = new AtomicInteger(); - } - } - - private static final long MIN_DELAY_NANOS = TimeUnit.MILLISECONDS.toNanos(250); - - private static int drainHolding(long expectHoldUntil, int parallelIdx, long[] nextHoldUntil) - throws InterruptedException { - Lock lock = HOLD_LOCKS[parallelIdx]; - lock.lock(); - int mustTransfer; - int snapshot; - try { - if (HOLD_MUST_FLUSH[parallelIdx].awaitNanos( - Math.max(expectHoldUntil - System.nanoTime(), MIN_DELAY_NANOS)) - > 0) { - snapshot = HOLD_SIZES[parallelIdx].getAndSet(0); - mustTransfer = snapshot - LOW_WATERMARK; - } else { - snapshot = 0; - mustTransfer = 0; - } - } finally { - lock.unlock(); - } - try { - return drainHolding(Integer.MAX_VALUE, parallelIdx, nextHoldUntil, mustTransfer); - } finally { - if (snapshot != 0) { - HOLD_SIZES[parallelIdx].addAndGet(snapshot); - } - } - } - - private static boolean accept(long now, HoldingRef candidate, Object o) { - return now - candidate.holdUntilNanos > 0 || o == null; - } - - private static int drainHolding( - int limit, int parallelIdx, long[] nextHoldUntil, int mustTransfer) { - AtomicBoolean draining = HOLDING_DRAIN[parallelIdx]; - if (!draining.compareAndSet(false, true)) { - nextHoldUntil[0] = - System.nanoTime() + HOLDING_NANOS; // in normal usage this should be unusual - return 0; - } - try { - AtomicReference head = HOLDING[parallelIdx]; - long now = System.nanoTime(); - final HoldingState state = head.get(); - HoldingRef candidate = state.head; - if (candidate == null) { - nextHoldUntil[0] = now + HOLDING_NANOS; - return 0; - } else if (now - candidate.holdUntilNanos <= 0) { - nextHoldUntil[0] = candidate.holdUntilNanos; - return 0; - } - HoldingRef last; - int releaseCount = 0; - Object o = candidate.get(); - int sizeAdjust = 0; - do { - if (o == null) { - // collected; we can just release - releaseCount++; - candidate.refCount.getAndUpdate(RELEASE); - } else { - // not yet collected; we need to create a fully-tracked ref - add(o, candidate.refCount, parallelIdx); - } - last = candidate; - candidate = candidate.prev; - sizeAdjust--; - mustTransfer--; - } while (--limit > 0 - && candidate != null - && (accept(now, candidate, o = candidate.get()) || mustTransfer > 0)); - if (candidate != null) { - // easiest case; set unconditionally - HoldingRef candidateF = candidate; - head.getAndUpdate((e) -> new HoldingState(e.tail, candidateF)); - nextHoldUntil[0] = candidateF.holdUntilNanos; - } else if (state.tail == last && head.compareAndSet(state, INITIAL)) { - // we cleared the queue; nothing more to do - nextHoldUntil[0] = now + HOLDING_NANOS; - } else { - // we know that we'll have a non-null candidate - while ((candidate = last.prev) == null) { - Thread.yield(); - } - HoldingRef candidateF = candidate; - head.getAndUpdate((e) -> new HoldingState(e.tail, candidateF)); - nextHoldUntil[0] = candidateF.holdUntilNanos; - } - HOLD_SIZES[parallelIdx].addAndGet(sizeAdjust); - return releaseCount; - } finally { - draining.set(false); - } - } - /** * Number of ram bytes per instance of {@link Ref}. This can be used in conjunction with {@link * #OUTSTANDING_SIZE_SUPPLIER} (accessed via the final arg to {@link - * UnloadHelper#maybeHandleRefQueues(ReferenceQueue[], Consumer, HoldingFlusher, AtomicReference, - * LongSupplier, LongSupplier, LongSupplier)}) to determine the point-in-time heap usage - * associated with refQueue reference tracking. + * UnloadHelper#maybeHandleRefQueues(ReferenceQueue[], Consumer, AtomicReference, LongSupplier, + * LongSupplier)}) to determine the point-in-time heap usage associated with refQueue reference + * tracking. */ public static final long RAMBYTES_PER_REF = RamUsageEstimator.shallowSizeOfInstance(Ref.class) + RamUsageEstimator.shallowSizeOfInstance(AtomicReference.class); - /** - * Number of ram bytes per instance of {@link HoldingRef}. This can be used in conjunction with - * {@link #HOLDING_SIZE_SUPPLIER} (accessed via the second-to-last arg of {@link - * UnloadHelper#maybeHandleRefQueues(ReferenceQueue[], Consumer, HoldingFlusher, AtomicReference, - * LongSupplier, LongSupplier, LongSupplier)}) to determine the point-in-time heap usage - * associated with "holding area" refQueue reference tracking. - */ - public static final long RAMBYTES_PER_HOLDINGREF = - RamUsageEstimator.shallowSizeOfInstance(HoldingRef.class); - - private static final int DEFAULT_HOLD_TARGET_MEGABYTES = 10; - private static final int HOLD_TARGET_MEGABYTES; - - static { - List deferred = DEFERRED_INIT_MESSAGES.get(); - String spec = System.getProperty("lucene.unload.holdTargetMegabytes"); - int tmp; - if (spec == null || spec.isEmpty()) { - tmp = DEFAULT_HOLD_TARGET_MEGABYTES; - } else { - try { - tmp = Integer.parseInt(spec); - } catch ( - @SuppressWarnings("unused") - Exception e) { - tmp = DEFAULT_HOLD_TARGET_MEGABYTES; - } - } - HOLD_TARGET_MEGABYTES = tmp; - deferred.add("INFO: set static property HOLD_TARGET_MEGABYTES=" + HOLD_TARGET_MEGABYTES); - } - - private static final int HIGH_WATERMARK = - HOLD_TARGET_MEGABYTES - * 1024 - * 1024 - / Math.toIntExact(RAMBYTES_PER_HOLDINGREF) - / PARALLEL_HEAD_FACTOR; - private static final int LOW_WATERMARK = HIGH_WATERMARK >> 1; - private static final int FALLBACK_HIGH_WATERMARK = HIGH_WATERMARK + LOW_WATERMARK; - private static final class Ref extends PhantomReference { private final AtomicInteger refCount; private final AtomicReference next = new AtomicReference<>(); @@ -1033,14 +781,6 @@ private static void add(final Object o, AtomicInteger refCount) { } else { parallelIdx = ARBITRARY_REFQUEUE.getAndIncrement() & PARALLEL_HEAD_MASK; } - if (HOLDING_NANOS == -1) { - add(o, refCount, parallelIdx); - } else { - addHolding(o, refCount, parallelIdx); - } - } - - private static void add(final Object o, AtomicInteger refCount, int parallelIdx) { OUTSTANDING_SIZE.increment(); if (EXTERNAL_REFQUEUE_HANDLING.get() != Boolean.TRUE) drainRemoveOutstanding(); Ref head = HEAD[parallelIdx]; @@ -1444,9 +1184,6 @@ interface FPIOFunction { + TimeUnit.NANOSECONDS.toMillis(INITIAL_NANOS)); } - private static final long HOLDING_NANOS = - !"false".equals(System.getProperty("lucene.unload.holding")) ? KEEP_ALIVE_NANOS : -1; - private static long getNanos( String syspropName, String defaultSpec, long defaultNanos, List deferred) { try { @@ -1500,18 +1237,6 @@ static int nonEmptyRefQueueHeadCount() { return Math.toIntExact(Arrays.stream(HEAD).filter((r) -> r.next.get() != null).count()); } - // visible for testing - static int nonEmptyHoldingHeadCount() { - return Math.toIntExact( - Arrays.stream(HOLDING) - .filter( - (r) -> { - HoldingState s = r.get(); - return s.tail != null || s.head != null; - }) - .count()); - } - // visible for testing static void addDummyReference(int byteSize) { add(new byte[byteSize], new AtomicInteger(1)); @@ -1524,9 +1249,8 @@ static void addDummyReference(int byteSize) { * the underlying components that handle load/unload according to framework lifecycles. * *

e.g., framework may supply its own {@link ScheduledExecutorService} for running unload - * checks, and may (via {@link #maybeHandleRefQueues(ReferenceQueue[], Consumer, HoldingFlusher, - * AtomicReference, LongSupplier, LongSupplier, LongSupplier)} manage the handling of reference - * tracking as well. + * checks, and may (via {@link #maybeHandleRefQueues(ReferenceQueue[], Consumer, AtomicReference, + * LongSupplier, LongSupplier)} manage the handling of reference tracking as well. */ public interface UnloadHelper { /** @@ -1573,18 +1297,14 @@ default void onClose() {} * handling any of the provided refQueues. * @param indirectTrackedCount for metrics; the number of references not tracked directly, but * tracked via reference to top-level tracked object. - * @param holdingSize for metrics; the number of references not formally tracked by GC, but in a - * "holding area" before being tracked if necessary. * @param outstandingSize for metrics; the number of references tracked but not yet collected * off a refQueue. */ default void maybeHandleRefQueues( ReferenceQueue[] queues, Consumer handler, - HoldingFlusher flushHolding, AtomicReference handleRefQueue, LongSupplier indirectTrackedCount, - LongSupplier holdingSize, LongSupplier outstandingSize) {} ; } @@ -1625,10 +1345,8 @@ static void configure(UnloadHelper unloadHelper) { unloadHelper.maybeHandleRefQueues( removeOutstanding, REF_REMOVER, - FLUSH_HOLDING, EXTERNAL_REFQUEUE_HANDLING, INDIRECT_TRACK_COUNT_SUPPLIER, - HOLDING_SIZE_SUPPLIER, OUTSTANDING_SIZE_SUPPLIER); } diff --git a/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java b/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java index bdb7217a448f..3b17e61e22d7 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java @@ -37,7 +37,6 @@ import java.util.function.Consumer; import java.util.function.LongSupplier; import org.apache.lucene.index.Unloader.BlockingRunnable; -import org.apache.lucene.index.Unloader.HoldingFlusher; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.tests.util.LuceneTestCase; import org.apache.lucene.util.IOFunction; @@ -246,8 +245,6 @@ public void testRefQueueHandling() throws InterruptedException, ExecutionExcepti final int batchSize = 1024; @SuppressWarnings({"unchecked", "rawtypes"}) Consumer[] registerRef = new Consumer[1]; - HoldingFlusher[] flushHolder = new HoldingFlusher[1]; - LongSupplier[] holdingSizeHolder = new LongSupplier[1]; LongSupplier[] outstandingSizeHolder = new LongSupplier[1]; @SuppressWarnings({"unchecked", "rawtypes"}) ReferenceQueue[][] removeOutstandingHolder = new ReferenceQueue[1][]; @@ -264,31 +261,25 @@ public ScheduledExecutorService onCreation(Unloader u) { public void maybeHandleRefQueues( ReferenceQueue[] queues, Consumer handler, - HoldingFlusher flushHolding, AtomicReference handleRefQueue, LongSupplier indirectTrackCount, - LongSupplier holdingSize, LongSupplier outstandingSize) { handleRefQueue.set(true); handleRefQueueHolder[0] = handleRefQueue; registerRef[0] = handler; - flushHolder[0] = flushHolding; - holdingSizeHolder[0] = holdingSize; outstandingSizeHolder[0] = outstandingSize; removeOutstandingHolder[0] = queues; } }); ReferenceQueue[] queues = removeOutstandingHolder[0]; - HoldingFlusher flush = flushHolder[0]; AtomicReference handleRefQueue = handleRefQueueHolder[0]; Consumer handler = registerRef[0]; - LongSupplier holdingSize = holdingSizeHolder[0]; LongSupplier outstandingSize = outstandingSizeHolder[0]; int PARALLEL_HEAD_FACTOR = queues.length; ExecutorService exec = Executors.newFixedThreadPool( - nThreads + PARALLEL_HEAD_FACTOR << 1, new NamedThreadFactory("TestUnloader")); + nThreads + PARALLEL_HEAD_FACTOR, new NamedThreadFactory("TestUnloader")); AtomicBoolean finished = new AtomicBoolean(); @SuppressWarnings("rawtypes") Future[] futures = new Future[nThreads]; @@ -317,7 +308,7 @@ public void maybeHandleRefQueues( LongAdder collectedHoldingRefs = new LongAdder(); LongAdder collectedRefs = new LongAdder(); @SuppressWarnings("rawtypes") - Future[] refQueueFutures = new Future[queues.length << 1]; + Future[] refQueueFutures = new Future[queues.length]; for (int i = queues.length - 1; i >= 0; i--) { ReferenceQueue q = queues[i]; refQueueFutures[i] = @@ -329,27 +320,14 @@ public void maybeHandleRefQueues( handler.accept(q.remove()); collectedRefs.increment(); })); - long[] nextHoldUntil = new long[] {System.nanoTime()}; - int idx = i; - refQueueFutures[queues.length + i] = - exec.submit( - wrapTask( - handleRefQueue, - activeRefQueueProcessors, - () -> { - collectedHoldingRefs.add(flush.flush(nextHoldUntil[0], idx, nextHoldUntil)); - })); } long endNanos = System.nanoTime() + TimeUnit.SECONDS.toNanos(N_SECONDS); long remainingNanos; while ((remainingNanos = endNanos - System.nanoTime()) > 0) { long sz = outstandingSize.getAsLong(); - long hSz = holdingSize.getAsLong(); System.out.println( "seconds remaining: " + TimeUnit.NANOSECONDS.toSeconds(remainingNanos) - + ", holdingSize=" - + RamUsageEstimator.humanReadableUnits(hSz * Unloader.RAMBYTES_PER_HOLDINGREF) + ", outstandingSize=" + sz + ", active=" @@ -369,10 +347,7 @@ public void maybeHandleRefQueues( start = System.nanoTime(); int gcIterations = 0; long sz; - long hSz; - while ((hSz = holdingSize.getAsLong()) + (sz = outstandingSize.getAsLong()) > 0 - || Unloader.nonEmptyRefQueueHeadCount() > 0 - || Unloader.nonEmptyHoldingHeadCount() > 0) { + while ((sz = outstandingSize.getAsLong()) > 0 || Unloader.nonEmptyRefQueueHeadCount() > 0) { gcIterations++; System.gc(); Thread.sleep(250); @@ -381,16 +356,12 @@ public void maybeHandleRefQueues( + gcIterations + ", " + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start) - + ", holdingSize=" - + RamUsageEstimator.humanReadableUnits(hSz * Unloader.RAMBYTES_PER_HOLDINGREF) + ", outstandingSize=" + sz + ", active=" + activeRefQueueProcessors.sum() + ", nonEmptyRefQueueHeadCount=" - + Unloader.nonEmptyRefQueueHeadCount() - + ", nonEmptyHoldingHeadCount=" - + Unloader.nonEmptyHoldingHeadCount()); + + Unloader.nonEmptyRefQueueHeadCount()); if (gcIterations > 40) { fail("failed to converge"); } From 42f945d7e20a7ddacd958940745d6da4afedec0c Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Fri, 31 Oct 2025 00:30:46 -0400 Subject: [PATCH 22/36] reduce num of refs? --- .../org/apache/lucene/index/Unloader.java | 33 ++++++++----------- 1 file changed, 14 insertions(+), 19 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index da6dd7f9f897..60e6985ebe6d 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.io.PrintStream; import java.io.UncheckedIOException; -import java.lang.ref.PhantomReference; import java.lang.ref.Reference; import java.lang.ref.ReferenceQueue; import java.lang.ref.WeakReference; @@ -83,7 +82,7 @@ private static final class DelegateFuture extends CompletableFuture> prev; private final AtomicInteger refCount; - private final AtomicReference> sentinel; + private final AtomicReference sentinel; @SuppressWarnings("unused") private volatile T strongRef; @@ -170,7 +169,7 @@ public void close() { } } - private static final WeakReference INITIAL_SENTINEL = new WeakReference<>(null); + private static final Ref INITIAL_SENTINEL = new Ref(null, null, null, null); @SuppressWarnings("unchecked") private final DelegateFuture closedSentinel = (DelegateFuture) CLOSED; @@ -568,10 +567,9 @@ private static final class CloseableVal implements Supplier, Closeable { private final T val; private final AtomicInteger refCount; - private final AtomicReference> sentinel; + private final AtomicReference sentinel; - private CloseableVal( - T val, AtomicInteger refCount, AtomicReference> sentinel) { + private CloseableVal(T val, AtomicInteger refCount, AtomicReference sentinel) { this.val = val; this.refCount = refCount; this.sentinel = sentinel; @@ -743,7 +741,7 @@ private CloseableVal backing() throws IOException { RamUsageEstimator.shallowSizeOfInstance(Ref.class) + RamUsageEstimator.shallowSizeOfInstance(AtomicReference.class); - private static final class Ref extends PhantomReference { + private static final class Ref extends WeakReference { private final AtomicInteger refCount; private final AtomicReference next = new AtomicReference<>(); private volatile Ref prev; @@ -754,11 +752,6 @@ public Ref( this.refCount = refCount; this.prev = prev; } - - @Override - public Object get() { - throw new UnsupportedOperationException("not supported"); - } } private static final Ref[] HEAD = new Ref[PARALLEL_HEAD_FACTOR]; @@ -774,7 +767,7 @@ public Object get() { private static final AtomicInteger ARBITRARY_REFQUEUE = new AtomicInteger(); - private static void add(final Object o, AtomicInteger refCount) { + private static Ref add(final Object o, AtomicInteger refCount) { int parallelIdx; if (ASSIGN_REFQUEUE_BY_THREAD) { parallelIdx = Thread.currentThread().hashCode() & PARALLEL_HEAD_MASK; @@ -794,6 +787,7 @@ private static void add(final Object o, AtomicInteger refCount) { if (!head.next.compareAndSet(RESERVED, ref)) { throw new IllegalStateException(); } + return ref; } finally { Reference.reachabilityFence(o); } @@ -1083,23 +1077,24 @@ V execute( return null; } else { AtomicInteger refCount = active.refCount; - AtomicReference> sentinel = active.sentinel; + AtomicReference sentinel = active.sentinel; Object tracked; // TODO: here we assume indirect tracking, so direct tracking won't really work anymore. // fix to make this consistent boolean reusedSentinel = true; - final WeakReference initial = sentinel.get(); - WeakReference weak = initial; + final Ref initial = sentinel.get(); + Ref weak = initial; while ((tracked = weak.get()) == null) { + remove(weak); tracked = new Object(); - WeakReference extant = - sentinel.compareAndExchange(weak, new WeakReference<>(tracked)); + Ref ref = add(tracked, refCount); + Ref extant = sentinel.compareAndExchange(weak, ref); if (extant == weak) { refCount.getAndUpdate(ACQUIRE); - add(tracked, refCount); reusedSentinel = false; break; } else { + remove(ref); weak = extant; } } From 7c3a368c286e6d791179ee2c56d6bc4fb4913a60 Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Fri, 31 Oct 2025 09:26:18 -0400 Subject: [PATCH 23/36] checkpoint before excising refqueue stuff --- lucene/core/src/java/org/apache/lucene/index/Unloader.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index 60e6985ebe6d..2a67825fa5bb 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -1085,7 +1085,12 @@ V execute( final Ref initial = sentinel.get(); Ref weak = initial; while ((tracked = weak.get()) == null) { - remove(weak); + if (weak != INITIAL_SENTINEL) { + // TODO: this double-remove can violate assumptions and cause IllegalStateException + // upon release. Maybe not really a practical problem, but we won't fix this b/c + // I think we can excise all the refqueue stuff. + remove(weak); + } tracked = new Object(); Ref ref = add(tracked, refCount); Ref extant = sentinel.compareAndExchange(weak, ref); From 9f9aeda39ab8b0900c9baae8a342a521307c41f3 Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Fri, 31 Oct 2025 16:02:17 -0400 Subject: [PATCH 24/36] we don't actually need refcounts -- much simpler --- gradle/testing/defaults-tests.gradle | 3 - .../org/apache/lucene/index/Unloader.java | 657 +++++------------- .../index/UnloadingDocValuesProducer.java | 36 +- .../lucene/index/UnloadingFieldsProducer.java | 41 +- .../lucene/index/UnloadingPointsReader.java | 23 +- .../org/apache/lucene/index/TestUnloader.java | 210 ------ 6 files changed, 232 insertions(+), 738 deletions(-) diff --git a/gradle/testing/defaults-tests.gradle b/gradle/testing/defaults-tests.gradle index 401f037d805d..136140e1d6a2 100644 --- a/gradle/testing/defaults-tests.gradle +++ b/gradle/testing/defaults-tests.gradle @@ -34,10 +34,7 @@ allprojects { [propName: 'lucene.unload.executorPerDirectory', value: false, description: "Directory instances supply their own unloading executor"], [propName: 'lucene.unload.ttl', value: '60m', description: "Time since last use at which a resource becomes eligible for unloading"], [propName: 'lucene.unload.initial', value: '1m', description: "Extra time allotted for first resource use after load/reload"], - [propName: 'lucene.unload.assignRefQueueByThread', value: 'true', description: "true assigns deterministically by thread; false is round-robin"], [propName: 'lucene.unload.disable', value: 'false', description: "Disables resource unloading"], - [propName: 'lucene.unload.parallelRefQueueCount', value: '32', description: "Parallelization factor of refqueues"], - [propName: 'lucene.unload.trackAllRefsDirectly', value: 'false', description: "if true, forces direct tracking of all derivative refs"], // asserts, debug output. [propName: 'tests.verbose', value: false, description: "Enables verbose mode (emits full test outputs immediately)."], [propName: 'tests.workDir', diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index 2a67825fa5bb..60c6d3bd0b58 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -22,12 +22,12 @@ import java.io.PrintStream; import java.io.UncheckedIOException; import java.lang.ref.Reference; -import java.lang.ref.ReferenceQueue; import java.lang.ref.WeakReference; import java.nio.charset.StandardCharsets; +import java.util.AbstractMap; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.Random; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; @@ -36,12 +36,9 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.LongAdder; -import java.util.function.Consumer; -import java.util.function.IntUnaryOperator; import java.util.function.LongSupplier; import java.util.function.Supplier; import org.apache.lucene.codecs.DocValuesProducer; @@ -54,7 +51,6 @@ import org.apache.lucene.util.IOFunction; import org.apache.lucene.util.IOSupplier; import org.apache.lucene.util.InfoStream; -import org.apache.lucene.util.RamUsageEstimator; import org.apache.lucene.util.ThreadInterruptedException; /** Handles thread-safe dynamic unloading and on-demand reloading of backing resource. */ @@ -62,7 +58,7 @@ public class Unloader implements Closeable { private InfoStream out = InfoStream.getDefault(); - private static final DelegateFuture CLOSED = new DelegateFuture<>(true, null, 0); + private static final DelegateFuture CLOSED = new DelegateFuture<>(null, null); static { CLOSED.complete(null); @@ -81,8 +77,7 @@ private static final class DelegateFuture extends CompletableFuture> prev; - private final AtomicInteger refCount; - private final AtomicReference sentinel; + private final AtomicReference> sentinel; @SuppressWarnings("unused") private volatile T strongRef; @@ -122,11 +117,10 @@ public T getStrong(long timeout, TimeUnit unit) } } - private DelegateFuture(boolean unloading, DelegateFuture prev, int initialRefCount) { - this.unloading = unloading; + private DelegateFuture(Object initialSentinel, DelegateFuture prev) { + this.unloading = initialSentinel == null; this.prev = new WeakReference<>(prev); if (unloading) { - this.refCount = null; this.sentinel = null; hardRef = prev; whenComplete( @@ -139,8 +133,7 @@ private DelegateFuture(boolean unloading, DelegateFuture prev, int initialRef } }); } else { - refCount = new AtomicInteger(initialRefCount); - sentinel = new AtomicReference<>(INITIAL_SENTINEL); + sentinel = new AtomicReference<>(new WeakReference<>(initialSentinel)); } } @@ -148,17 +141,51 @@ private DelegateFuture(boolean unloading, DelegateFuture prev, int initialRef * true if a reservation was acquired for this instance. Reservation release must be handled * elsewhere. */ - private boolean acquire() { - return refCount.updateAndGet(ACQUIRE) > 0; + private Object acquire() { + Object ret; + boolean reusedSentinel = true; + WeakReference candidate = sentinel.get(); + while ((ret = candidate.get()) == null) { + if (candidate == UNLOADED_REF) { + // unloaded while we're trying to acquire + return null; + } + ret = new Object(); + WeakReference extant = + sentinel.compareAndExchange(candidate, new WeakReference<>(ret)); + if (extant == candidate) { + reusedSentinel = false; + break; + } else { + candidate = extant; + } + } + if (reusedSentinel) { + INDIRECT_TRACK_COUNT.increment(); + } else { + REFS_COLLECTED.increment(); + } + return ret; } private boolean unload(boolean force) { assert !unloading; if (force) { - refCount.set(UNLOADED_REFCOUNT); + sentinel.set(UNLOADED_REF); return true; } else { - return refCount.updateAndGet(UNLOAD) == UNLOADED_REFCOUNT; + WeakReference candidate = sentinel.get(); + for (; ; ) { + WeakReference extant; + if (candidate == UNLOADED_REF || candidate.get() != null) { + // either already unloaded, or still referenced + return false; + } else if ((extant = sentinel.compareAndExchange(candidate, UNLOADED_REF)) == candidate) { + return true; + } else { + candidate = extant; + } + } } } @@ -169,19 +196,14 @@ public void close() { } } - private static final Ref INITIAL_SENTINEL = new Ref(null, null, null, null); + private static final WeakReference UNLOADED_REF = new WeakReference<>(null); @SuppressWarnings("unchecked") private final DelegateFuture closedSentinel = (DelegateFuture) CLOSED; - private static final AtomicReference EXTERNAL_REFQUEUE_HANDLING = - new AtomicReference<>(); - - private static final LongAdder OUTSTANDING_SIZE = new LongAdder(); - - private static final LongSupplier OUTSTANDING_SIZE_SUPPLIER = OUTSTANDING_SIZE::sum; + private static final LongAdder REFS_COLLECTED = new LongAdder(); - private static final Consumer REF_REMOVER = (r) -> remove((Ref) r); + private static final LongSupplier REFS_COLLECTED_COUNT_SUPPLIER = REFS_COLLECTED::sum; /** * Creates a new unloader to handle unloading and on-demand reloading a backing resource @@ -203,36 +225,33 @@ public Unloader( long keepAliveNanos, IOFunction receiveFirstInstance) throws IOException { - if (EXTERNAL_REFQUEUE_HANDLING.get() != Boolean.TRUE) { + if (INITIALIZED_EXTERNAL.compareAndSet(false, true)) { unloadHelper.maybeHandleRefQueues( - removeOutstanding, - REF_REMOVER, - EXTERNAL_REFQUEUE_HANDLING, - INDIRECT_TRACK_COUNT_SUPPLIER, - OUTSTANDING_SIZE_SUPPLIER); + INITIALIZED_EXTERNAL, INDIRECT_TRACK_COUNT_SUPPLIER, REFS_COLLECTED_COUNT_SUPPLIER); } this.reporter = unloadHelper; this.reopen = reopen; this.keepAliveNanos = keepAliveNanos; - DelegateFuture holder = new DelegateFuture<>(false, null, 1); + Object sentinel = new Object(); + DelegateFuture holder = new DelegateFuture<>(sentinel, null); + backing = new AtomicReference<>(holder); + this.exec = unloadHelper.onCreation(this); + T in = reopen.apply(this); try { - backing = new AtomicReference<>(holder); - this.exec = unloadHelper.onCreation(this); - T in = reopen.apply(this); - try { - description = receiveFirstInstance.apply(in); - holder.completeStrong(in); - } catch (Throwable t) { - try (in) { - unloadHelper.onClose(); - throw t; - } + description = receiveFirstInstance.apply(in); + holder.completeStrong(in); + } catch (Throwable t) { + try (in) { + unloadHelper.onClose(); + throw t; } } finally { - holder.refCount.updateAndGet(RELEASE); + Reference.reachabilityFence(sentinel); } } + private static final AtomicBoolean INITIALIZED_EXTERNAL = new AtomicBoolean(false); + /** Sets the infostream for {@link Unloader}. */ public void setInfoStream(InfoStream out) { this.out = out; @@ -257,15 +276,6 @@ public void setInfoStream(InfoStream out) { /** This resource is still referenced, so was not unloaded. */ public static final long STILL_REFERENCED = 0; - private static void drainRemoveOutstanding() { - for (ReferenceQueue q : removeOutstanding) { - Ref collected; - while ((collected = (Ref) q.poll()) != null) { - remove(collected); - } - } - } - private final Random unloadRandom = new Random(); // single-threaded access private static boolean injectDelay(Random r, int oneIn, int millis) { @@ -289,7 +299,7 @@ private static DelegateFuture unloadRef( // still referenced return null; } - DelegateFuture candidate = new DelegateFuture<>(true, extant, 0); + DelegateFuture candidate = new DelegateFuture<>(null, extant); if (ref.compareAndSet(extant, candidate)) { return candidate; } @@ -302,43 +312,48 @@ private static DelegateFuture unloadRef( return null; } - private static DelegateFuture loadRef( + private static Map.Entry, Object> loadRef( AtomicReference> ref, boolean[] weCompute) { DelegateFuture extant; - while ((extant = ref.get()).unloading || !extant.acquire()) { + Object sentinel; + while ((extant = ref.get()).unloading || (sentinel = extant.acquire()) == null) { if (extant.unloading) { if (extant == CLOSED) { throw new AlreadyClosedException(""); } - DelegateFuture candidate = new DelegateFuture<>(false, extant, 1); + Object initialSentinel = new Object(); + DelegateFuture candidate = new DelegateFuture<>(initialSentinel, extant); if (ref.compareAndSet(extant, candidate)) { weCompute[0] = true; - return candidate; + return new AbstractMap.SimpleImmutableEntry<>(candidate, initialSentinel); } } } assert !extant.unloading; - return extant; + return new AbstractMap.SimpleImmutableEntry<>(extant, sentinel); } - private static DelegateFuture retry( - AtomicReference> ref, DelegateFuture replace, boolean[] weCompute) { + private static Map.Entry, Object> retry( + AtomicReference> ref, + final DelegateFuture replace, + boolean[] weCompute) { DelegateFuture prev = replace.prev.get(); if (prev == null) { replace.unload(true); - ref.compareAndSet(replace, new DelegateFuture<>(true, replace, 0)); + ref.compareAndSet(replace, new DelegateFuture<>(null, replace)); return loadRef(ref, weCompute); } - DelegateFuture candidate = new DelegateFuture<>(false, prev, 1); + Object sentinel = new Object(); + DelegateFuture candidate = new DelegateFuture<>(sentinel, prev); DelegateFuture extant = ref.compareAndExchange(replace, candidate); if (extant == replace) { weCompute[0] = true; - return candidate; - } else if (!extant.unloading && extant.acquire()) { - return extant; + return new AbstractMap.SimpleImmutableEntry<>(candidate, sentinel); + } else if (!extant.unloading && (sentinel = extant.acquire()) != null) { + return new AbstractMap.SimpleImmutableEntry<>(extant, sentinel); } else { replace.unload(true); - ref.compareAndSet(replace, new DelegateFuture<>(true, replace, 0)); + ref.compareAndSet(replace, new DelegateFuture<>(null, replace)); return loadRef(ref, weCompute); } } @@ -354,7 +369,6 @@ private static DelegateFuture retry( * somewhere. */ public long maybeUnload() throws IOException { - if (EXTERNAL_REFQUEUE_HANDLING.get() != Boolean.TRUE) drainRemoveOutstanding(); long nanosSinceLastAccess = System.nanoTime() - lastAccessNanos; if (nanosSinceLastAccess < keepAliveNanos) { // don't unload @@ -566,12 +580,10 @@ private static T interruptProtectedGet( private static final class CloseableVal implements Supplier, Closeable { private final T val; - private final AtomicInteger refCount; - private final AtomicReference sentinel; + private Object sentinel; - private CloseableVal(T val, AtomicInteger refCount, AtomicReference sentinel) { + private CloseableVal(T val, Object sentinel) { this.val = val; - this.refCount = refCount; this.sentinel = sentinel; } @@ -582,7 +594,12 @@ public T get() { @Override public void close() throws IOException { - refCount.updateAndGet(RELEASE); + Object toRemove = sentinel; + try { + sentinel = null; + } finally { + Reference.reachabilityFence(toRemove); + } } } @@ -590,19 +607,19 @@ public void close() throws IOException { private CloseableVal backing() throws IOException { boolean[] weCompute = new boolean[1]; - DelegateFuture holder = loadRef(backing, weCompute); + Map.Entry, Object> holder = loadRef(backing, weCompute); long now = System.nanoTime(); long until = now + TOTAL_BLOCK_NANOS; while (!weCompute[0]) { try { return new CloseableVal<>( - holder.getStrong(until - now, TimeUnit.NANOSECONDS), holder.refCount, holder.sentinel); + holder.getKey().getStrong(until - now, TimeUnit.NANOSECONDS), holder.getValue()); } catch (ExecutionException e) { Throwable t = e.getCause(); if (t instanceof IOException) { throw (IOException) t; } - holder = retry(backing, holder, weCompute); + holder = retry(backing, holder.getKey(), weCompute); } catch (InterruptedException e) { throw new ThreadInterruptedException(e); } catch ( @@ -617,11 +634,11 @@ private CloseableVal backing() throws IOException { T candidate = null; try { candidate = reopen.apply(this); - holder.completeStrong(candidate); + holder.getKey().completeStrong(candidate); successfullyComputed = true; - return new CloseableVal<>(candidate, holder.refCount, holder.sentinel); + return new CloseableVal<>(candidate, holder.getValue()); } catch (Throwable t) { - holder.completeExceptionally(t); + holder.getKey().completeExceptionally(t); throw t; } finally { if (candidate != null && !successfullyComputed) { @@ -630,217 +647,18 @@ private CloseableVal backing() throws IOException { } } - // Arbitrary negative values that we won't hit accidentally - private static final int UNLOADED_REFCOUNT = ~(Integer.MAX_VALUE >> 1); - private static final int FORCE_UNLOADED_REFCOUNT = UNLOADED_REFCOUNT + 10; - - private static final IntUnaryOperator ACQUIRE = - (extant) -> { - switch (extant) { - case FORCE_UNLOADED_REFCOUNT: - return FORCE_UNLOADED_REFCOUNT; - case UNLOADED_REFCOUNT: - return UNLOADED_REFCOUNT; - default: - assert extant >= 0; - return extant + 1; - } - }; - - private static final IntUnaryOperator RELEASE = - (extant) -> { - switch (extant) { - case FORCE_UNLOADED_REFCOUNT: - return FORCE_UNLOADED_REFCOUNT; - case UNLOADED_REFCOUNT: - case 0: - throw new IllegalStateException(); - default: - assert extant > 0; - return extant - 1; - } - }; - - private static final IntUnaryOperator UNLOAD = - (extant) -> { - switch (extant) { - case FORCE_UNLOADED_REFCOUNT: - return FORCE_UNLOADED_REFCOUNT; - case UNLOADED_REFCOUNT: - throw new IllegalStateException("already unloaded"); - case 0: - return UNLOADED_REFCOUNT; - default: - assert extant > 0; - return extant; - } - }; - private static final AtomicReference> DEFERRED_INIT_MESSAGES = new AtomicReference<>(new ArrayList<>()); - private static final int DEFAULT_PARALLEL_HEAD_FACTOR = 32; - private static final int PARALLEL_HEAD_FACTOR; - - /** - * Setting this to false ensures even (round-robin) utilization of refqueues. Assigning by thread - * is fine at the time of refqueue assignment, but can yield hotspots that could increase - * thread contention at time of ref collection (by GC threads). - */ - private static final boolean ASSIGN_REFQUEUE_BY_THREAD = - !"false".equals(System.getProperty("lucene.unload.assignRefQueueByThread")); - - static { - List deferred = DEFERRED_INIT_MESSAGES.get(); - String spec = System.getProperty("lucene.unload.parallelRefQueueCount"); - if (spec == null) { - PARALLEL_HEAD_FACTOR = DEFAULT_PARALLEL_HEAD_FACTOR; - } else { - int v; - try { - v = Integer.parseInt(spec); - if (v < 1 || Integer.bitCount(v) != 1) { - deferred.add("WARN: bad lucene.unload.parallelRefQueueCount spec: " + spec); - v = DEFAULT_PARALLEL_HEAD_FACTOR; - } - } catch (Throwable t) { - deferred.add( - "WARN: bad lucene.unload.parallelRefQueueCount spec: " + spec + " (" + t + ")"); - v = DEFAULT_PARALLEL_HEAD_FACTOR; - } - PARALLEL_HEAD_FACTOR = v; - } - deferred.add("INFO: set static property PARALLEL_HEAD_FACTOR=" + PARALLEL_HEAD_FACTOR); - deferred.add( - "INFO: set static property ASSIGN_REFQUEUE_BY_THREAD=" + ASSIGN_REFQUEUE_BY_THREAD); - } - - private static final int PARALLEL_HEAD_MASK = PARALLEL_HEAD_FACTOR - 1; - - @SuppressWarnings({"unchecked", "rawtypes"}) - private static final ReferenceQueue[] removeOutstanding = - new ReferenceQueue[PARALLEL_HEAD_FACTOR]; - - static { - for (int i = PARALLEL_HEAD_FACTOR - 1; i >= 0; i--) { - removeOutstanding[i] = new ReferenceQueue<>(); - } - } - private static final LongAdder INDIRECT_TRACK_COUNT = new LongAdder(); private static final LongSupplier INDIRECT_TRACK_COUNT_SUPPLIER = INDIRECT_TRACK_COUNT::sum; - /** - * Number of ram bytes per instance of {@link Ref}. This can be used in conjunction with {@link - * #OUTSTANDING_SIZE_SUPPLIER} (accessed via the final arg to {@link - * UnloadHelper#maybeHandleRefQueues(ReferenceQueue[], Consumer, AtomicReference, LongSupplier, - * LongSupplier)}) to determine the point-in-time heap usage associated with refQueue reference - * tracking. - */ - public static final long RAMBYTES_PER_REF = - RamUsageEstimator.shallowSizeOfInstance(Ref.class) - + RamUsageEstimator.shallowSizeOfInstance(AtomicReference.class); - - private static final class Ref extends WeakReference { - private final AtomicInteger refCount; - private final AtomicReference next = new AtomicReference<>(); - private volatile Ref prev; - - public Ref( - Object referent, ReferenceQueue q, AtomicInteger refCount, Ref prev) { - super(referent, q); - this.refCount = refCount; - this.prev = prev; - } - } - - private static final Ref[] HEAD = new Ref[PARALLEL_HEAD_FACTOR]; - - static { - for (int i = PARALLEL_HEAD_FACTOR - 1; i >= 0; i--) { - HEAD[i] = new Ref(null, null, null, null); - } - } - - private static final Ref RESERVED = new Ref(null, null, null, null); - private static final Ref REMOVED = new Ref(null, null, null, null); - - private static final AtomicInteger ARBITRARY_REFQUEUE = new AtomicInteger(); - - private static Ref add(final Object o, AtomicInteger refCount) { - int parallelIdx; - if (ASSIGN_REFQUEUE_BY_THREAD) { - parallelIdx = Thread.currentThread().hashCode() & PARALLEL_HEAD_MASK; - } else { - parallelIdx = ARBITRARY_REFQUEUE.getAndIncrement() & PARALLEL_HEAD_MASK; - } - OUTSTANDING_SIZE.increment(); - if (EXTERNAL_REFQUEUE_HANDLING.get() != Boolean.TRUE) drainRemoveOutstanding(); - Ref head = HEAD[parallelIdx]; - try { - final Ref ref = new Ref(o, removeOutstanding[parallelIdx], refCount, head); - Ref next = reserve(head, RESERVED); - if (next != null) { - next.prev = ref; - ref.next.set(next); - } - if (!head.next.compareAndSet(RESERVED, ref)) { - throw new IllegalStateException(); - } - return ref; - } finally { - Reference.reachabilityFence(o); - } - } - - private static Ref reserve(Ref ref, Ref reservation) { - Ref next = ref.next.get(); - for (; ; ) { - while (next == RESERVED) { - if (reservation == REMOVED) { - Thread.yield(); - } - next = ref.next.get(); - } - Ref extant = ref.next.compareAndExchange(next, reservation); - if (extant == next) { - return next; - } else { - next = extant; - } - } - } - - private static void remove(final Ref ref) { - Ref next = reserve(ref, REMOVED); - OUTSTANDING_SIZE.decrement(); - ref.refCount.getAndUpdate(RELEASE); - // now we have a lock on the link to next - Ref prev; - for (; ; ) { - prev = ref.prev; - if (prev.next.compareAndSet(ref, RESERVED)) { - break; - } else { - Thread.yield(); - } - } - // now we have a lock on the link from prev - if (next != null) { - next.prev = prev; - } - if (!prev.next.compareAndSet(RESERVED, next)) { - throw new IllegalStateException(); - } - } - - static PointValues.PointTree wrap(PointValues.PointTree pt, RefTracker registerRef) - throws IOException { + static PointValues.PointTree wrap(PointValues.PointTree pt, Object sentinel) throws IOException { return new PointValues.PointTree() { @Override public PointValues.PointTree clone() { try { - return wrap(registerRef.trackedInstance(pt::clone), registerRef); + return wrap(pt.clone(), sentinel); } catch (IOException e) { throw new UncheckedIOException("this should never happen", e); } @@ -873,31 +691,35 @@ public byte[] getMaxPackedValue() { @Override public long size() { - try { - return pt.size(); - } finally { - registerRef.ensureReachability(); - } + return pt.size(); } @Override public void visitDocIDs(PointValues.IntersectVisitor visitor) throws IOException { - pt.visitDocIDs(visitor); + try { + pt.visitDocIDs(visitor); + } finally { + Reference.reachabilityFence(sentinel); + } } @Override public void visitDocValues(PointValues.IntersectVisitor visitor) throws IOException { - pt.visitDocValues(visitor); + try { + pt.visitDocValues(visitor); + } finally { + Reference.reachabilityFence(sentinel); + } } }; } private static final class TrackingPostingsEnum extends FilterLeafReader.FilterPostingsEnum { - private final RefTracker registerRef; + private final Object sentinel; - private TrackingPostingsEnum(PostingsEnum in, RefTracker registerRef) { + private TrackingPostingsEnum(PostingsEnum in, Object sentinel) { super(in); - this.registerRef = registerRef; + this.sentinel = sentinel; } @Override @@ -905,18 +727,15 @@ public long cost() { try { return super.cost(); } finally { - registerRef.ensureReachability(); + Reference.reachabilityFence(sentinel); } } } - static TermsEnum wrap(TermsEnum te, RefTracker registerRef) throws IOException { + static TermsEnum wrap(TermsEnum te, Object sentinel) throws IOException { return new FilterLeafReader.FilterTermsEnum(te) { @Override public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException { - if (TRACK_ALL_REFS_DIRECTLY) { - return registerRef.trackedInstance(() -> super.postings(reuse, flags)); - } INDIRECT_TRACK_COUNT.increment(); if (reuse instanceof TrackingPostingsEnum) { PostingsEnum extantIn = ((TrackingPostingsEnum) reuse).in; @@ -925,128 +744,83 @@ public PostingsEnum postings(PostingsEnum reuse, int flags) throws IOException { // same backing, with updated state return reuse; } else { - return new TrackingPostingsEnum(check, registerRef); + return new TrackingPostingsEnum(check, sentinel); } } else { - return new TrackingPostingsEnum(super.postings(reuse, flags), registerRef); + return new TrackingPostingsEnum(super.postings(reuse, flags), sentinel); } } @Override public ImpactsEnum impacts(int flags) throws IOException { - if (TRACK_ALL_REFS_DIRECTLY) { - return registerRef.trackedInstance(() -> super.impacts(flags)); - } else { - INDIRECT_TRACK_COUNT.increment(); - ImpactsEnum raw = super.impacts(flags); - return new ImpactsEnum() { - @Override - public void advanceShallow(int target) throws IOException { - raw.advanceShallow(target); - } + INDIRECT_TRACK_COUNT.increment(); + ImpactsEnum raw = super.impacts(flags); + return new ImpactsEnum() { + @Override + public void advanceShallow(int target) throws IOException { + raw.advanceShallow(target); + } - @Override - public Impacts getImpacts() throws IOException { - return raw.getImpacts(); - } + @Override + public Impacts getImpacts() throws IOException { + return raw.getImpacts(); + } - @Override - public int freq() throws IOException { - return raw.freq(); - } + @Override + public int freq() throws IOException { + return raw.freq(); + } - @Override - public int nextPosition() throws IOException { - return raw.nextPosition(); - } + @Override + public int nextPosition() throws IOException { + return raw.nextPosition(); + } - @Override - public int startOffset() throws IOException { - return raw.startOffset(); - } + @Override + public int startOffset() throws IOException { + return raw.startOffset(); + } - @Override - public int endOffset() throws IOException { - return raw.endOffset(); - } + @Override + public int endOffset() throws IOException { + return raw.endOffset(); + } - @Override - public BytesRef getPayload() throws IOException { - return raw.getPayload(); - } + @Override + public BytesRef getPayload() throws IOException { + return raw.getPayload(); + } - @Override - public int docID() { - return raw.docID(); - } + @Override + public int docID() { + return raw.docID(); + } - @Override - public int nextDoc() throws IOException { - return raw.nextDoc(); - } + @Override + public int nextDoc() throws IOException { + return raw.nextDoc(); + } - @Override - public int advance(int target) throws IOException { - return raw.advance(target); - } + @Override + public int advance(int target) throws IOException { + return raw.advance(target); + } - @Override - public long cost() { - try { - return raw.cost(); - } finally { - registerRef.ensureReachability(); - } + @Override + public long cost() { + try { + return raw.cost(); + } finally { + Reference.reachabilityFence(sentinel); } - }; - } + } + }; } }; } - static final boolean TRACK_ALL_REFS_DIRECTLY = - "true".equals(System.getProperty("lucene.unload.trackAllRefsDirectly")); - - static final class RefTracker { - private final Object topLevel; - private final AtomicInteger refCount; - - RefTracker(Object topLevel, AtomicInteger refCount) { - this.topLevel = TRACK_ALL_REFS_DIRECTLY ? null : topLevel; - this.refCount = refCount; - } - - T trackedInstance(IOSupplier supplier) throws IOException { - if (!TRACK_ALL_REFS_DIRECTLY) { - INDIRECT_TRACK_COUNT.increment(); - return supplier.get(); - } - refCount.getAndUpdate(ACQUIRE); - T ret; - try { - ret = supplier.get(); - } catch (Throwable t) { - refCount.getAndUpdate(RELEASE); - throw t; - } - add(ret, refCount); - return ret; - } - - /** - * Ensures that the top-level object (if present) is reachable. - * - *

Suppress warning; this method should only be called within a finally - * block. - */ - @SuppressWarnings("ReachabilityFenceUsage") - void ensureReachability() { - Reference.reachabilityFence(topLevel); - } - } - interface RefTrackShim { - V shim(V in, RefTracker refTracker); + V shim(V in, Object sentinel); } V execute(FPIOFunction function, K arg) throws IOException { @@ -1076,48 +850,13 @@ V execute( if (raw == null) { return null; } else { - AtomicInteger refCount = active.refCount; - AtomicReference sentinel = active.sentinel; - Object tracked; - // TODO: here we assume indirect tracking, so direct tracking won't really work anymore. - // fix to make this consistent - boolean reusedSentinel = true; - final Ref initial = sentinel.get(); - Ref weak = initial; - while ((tracked = weak.get()) == null) { - if (weak != INITIAL_SENTINEL) { - // TODO: this double-remove can violate assumptions and cause IllegalStateException - // upon release. Maybe not really a practical problem, but we won't fix this b/c - // I think we can excise all the refqueue stuff. - remove(weak); - } - tracked = new Object(); - Ref ref = add(tracked, refCount); - Ref extant = sentinel.compareAndExchange(weak, ref); - if (extant == weak) { - refCount.getAndUpdate(ACQUIRE); - reusedSentinel = false; - break; - } else { - remove(ref); - weak = extant; - } - } - if (reusedSentinel) { - INDIRECT_TRACK_COUNT.increment(); - } else if (initial != INITIAL_SENTINEL && out.isEnabled("UN")) { - long extra = EXTRA_SENTINELS_CREATED.incrementAndGet(); - out.message("UN", "INFO: total additional sentinels created: " + extra); - } - return shim.shim(raw, new RefTracker(tracked, refCount)); + return shim.shim(raw, active.sentinel); } } finally { lastAccessNanos = System.nanoTime(); } } - private static final AtomicLong EXTRA_SENTINELS_CREATED = new AtomicLong(); - interface FPIOFunction { V apply(T fp, K arg) throws IOException; } @@ -1232,16 +971,6 @@ private static long getNanos( } } - // visible for testing - static int nonEmptyRefQueueHeadCount() { - return Math.toIntExact(Arrays.stream(HEAD).filter((r) -> r.next.get() != null).count()); - } - - // visible for testing - static void addDummyReference(int byteSize) { - add(new byte[byteSize], new AtomicInteger(1)); - } - /** * Passed to {@link Unloader#Unloader(UnloadHelper, IOFunction, long, IOFunction)} ctor. This * provides a means for reporting unload/reload lifecycle events to higher-level components. This @@ -1249,8 +978,8 @@ static void addDummyReference(int byteSize) { * the underlying components that handle load/unload according to framework lifecycles. * *

e.g., framework may supply its own {@link ScheduledExecutorService} for running unload - * checks, and may (via {@link #maybeHandleRefQueues(ReferenceQueue[], Consumer, AtomicReference, - * LongSupplier, LongSupplier)} manage the handling of reference tracking as well. + * checks, and may (via {@link #maybeHandleRefQueues(AtomicBoolean, LongSupplier, LongSupplier)} + * manage the handling of reference tracking as well. */ public interface UnloadHelper { /** @@ -1290,22 +1019,16 @@ default void onClose() {} * A callback that allows a framework to handle refQueue management (and provides a window into * the size of the refQueue(s) for metrics purposes. * - * @param queues refQueue instances - * @param handler to be called for each {@link java.lang.ref.Reference} removed from a refQueue - * @param handleRefQueue implementations should update this to true if they plan to - * handle the refQueues, and should set it back to false if/when they stop - * handling any of the provided refQueues. + *

handle the refQueues, and should set it back to false if/when they stop + * handling any of the provided refQueues. + * * @param indirectTrackedCount for metrics; the number of references not tracked directly, but * tracked via reference to top-level tracked object. - * @param outstandingSize for metrics; the number of references tracked but not yet collected - * off a refQueue. */ default void maybeHandleRefQueues( - ReferenceQueue[] queues, - Consumer handler, - AtomicReference handleRefQueue, - LongSupplier indirectTrackedCount, - LongSupplier outstandingSize) {} + AtomicBoolean initialized, LongSupplier indirectTrackedCount, LongSupplier refsCollected) { + // no-op default impl + } ; } @@ -1337,19 +1060,6 @@ public ScheduledExecutorService onCreation(Unloader u) { } } - /** - * For testing; provides a hook so that tests of refQueue functionality may directly manage - * refQueue handling. - */ - static void configure(UnloadHelper unloadHelper) { - unloadHelper.maybeHandleRefQueues( - removeOutstanding, - REF_REMOVER, - EXTERNAL_REFQUEUE_HANDLING, - INDIRECT_TRACK_COUNT_SUPPLIER, - OUTSTANDING_SIZE_SUPPLIER); - } - /** * Returns a {@link PointsReader} over the specified {@link SegmentReadState}, conditionally * wrapped to allow dynamic unloading and on-demand reloading of the backing resource. @@ -1550,13 +1260,4 @@ private static Callable maybeUnloadTask(Unloader u, String type, UnloadHel return null; }; } - - /** - * Analogous to {@link Runnable}, but the {@link #run()} method throws {@link - * InterruptedException}. - */ - public interface BlockingRunnable { - /** Executes this task, possibly throwing {@link InterruptedException}. */ - void run() throws InterruptedException; - } } diff --git a/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java index ab85c82b768b..badfe20d3381 100644 --- a/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java +++ b/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java @@ -19,6 +19,7 @@ import static org.apache.lucene.index.Unloader.FPIOFunction; import java.io.IOException; +import java.lang.ref.Reference; import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.util.IOFunction; import org.apache.lucene.util.automaton.CompiledAutomaton; @@ -72,14 +73,15 @@ public NumericDocValues getNumeric(FieldInfo field) throws IOException { getNumeric, field, true, - (raw, refTracker) -> { + (raw, sentinel) -> { + assert sentinel != null : "sentinel must not be null"; return new FilterNumericDocValues(raw) { @Override public long cost() { try { return super.cost(); } finally { - refTracker.ensureReachability(); + Reference.reachabilityFence(sentinel); } } }; @@ -95,14 +97,15 @@ public BinaryDocValues getBinary(FieldInfo field) throws IOException { getBinary, field, true, - (raw, refTracker) -> { + (raw, sentinel) -> { + assert sentinel != null : "sentinel must not be null"; return new FilterBinaryDocValues(raw) { @Override public long cost() { try { return super.cost(); } finally { - refTracker.ensureReachability(); + Reference.reachabilityFence(sentinel); } } }; @@ -118,18 +121,18 @@ public SortedDocValues getSorted(FieldInfo field) throws IOException { getSorted, field, true, - (rawSorted, registerRef) -> { + (rawSorted, sentinel) -> { + assert sentinel != null : "sentinel must not be null"; // wrap so that we can track refs for returned `TermsEnum` instances return new FilterSortedDocValues(rawSorted) { @Override public TermsEnum intersect(CompiledAutomaton automaton) throws IOException { - return Unloader.wrap( - registerRef.trackedInstance(() -> super.intersect(automaton)), registerRef); + return Unloader.wrap(super.intersect(automaton), sentinel); } @Override public TermsEnum termsEnum() throws IOException { - return Unloader.wrap(registerRef.trackedInstance(super::termsEnum), registerRef); + return Unloader.wrap(super.termsEnum(), sentinel); } }; }); @@ -144,14 +147,15 @@ public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOExcepti getSortedNumeric, field, true, - (raw, refTracker) -> { + (raw, sentinel) -> { + assert sentinel != null : "sentinel must not be null"; return new FilterSortedNumericDocValues(raw) { @Override public long cost() { try { return super.cost(); } finally { - refTracker.ensureReachability(); + Reference.reachabilityFence(sentinel); } } }; @@ -167,25 +171,25 @@ public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException { getSortedSet, field, true, - (rawSorted, registerRef) -> { + (rawSorted, sentinel) -> { + assert sentinel != null : "sentinel must not be null"; // wrap so that we can track refs for returned `TermsEnum` instances return new FilterSortedSetDocValues(rawSorted) { @Override public TermsEnum intersect(CompiledAutomaton automaton) throws IOException { try { - return Unloader.wrap( - registerRef.trackedInstance(() -> super.intersect(automaton)), registerRef); + return Unloader.wrap(super.intersect(automaton), sentinel); } finally { - registerRef.ensureReachability(); + Reference.reachabilityFence(sentinel); } } @Override public TermsEnum termsEnum() throws IOException { try { - return Unloader.wrap(registerRef.trackedInstance(super::termsEnum), registerRef); + return Unloader.wrap(super.termsEnum(), sentinel); } finally { - registerRef.ensureReachability(); + Reference.reachabilityFence(sentinel); } } }; diff --git a/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java b/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java index 82a416841a4f..138917d26ccf 100644 --- a/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java +++ b/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.io.UncheckedIOException; +import java.lang.ref.Reference; import java.util.Iterator; import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.codecs.FieldsProducer; @@ -89,7 +90,8 @@ public Iterator iterator() { iterator, null, true, - (raw, refTracker) -> { + (raw, sentinel) -> { + assert sentinel != null : "sentinel must not be null"; return new Iterator() { @Override public boolean hasNext() { @@ -101,7 +103,7 @@ public String next() { try { return raw.next(); } finally { - refTracker.ensureReachability(); + Reference.reachabilityFence(sentinel); } } }; @@ -120,7 +122,8 @@ public Terms terms(String field) throws IOException { terms, field, false, - (rawTerms, registerRef) -> { + (rawTerms, sentinel) -> { + assert sentinel != null : "sentinel must not be null"; // NOTE: we have to wrap here because a reference to the raw value may be // retained internal to the backing `FieldsProducer`. This can generate a // profusion of redundant references that never get collected. This is a @@ -133,9 +136,9 @@ public Terms terms(String field) throws IOException { @Override public TermsEnum iterator() throws IOException { try { - return Unloader.wrap(registerRef.trackedInstance(super::iterator), registerRef); + return Unloader.wrap(super.iterator(), sentinel); } finally { - registerRef.ensureReachability(); + Reference.reachabilityFence(sentinel); } } @@ -143,11 +146,9 @@ public TermsEnum iterator() throws IOException { public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) throws IOException { try { - return Unloader.wrap( - registerRef.trackedInstance(() -> super.intersect(compiled, startTerm)), - registerRef); + return Unloader.wrap(super.intersect(compiled, startTerm), sentinel); } finally { - registerRef.ensureReachability(); + Reference.reachabilityFence(sentinel); } } @@ -156,7 +157,7 @@ public long size() throws IOException { try { return super.size(); } finally { - registerRef.ensureReachability(); + Reference.reachabilityFence(sentinel); } } @@ -165,7 +166,7 @@ public long getSumTotalTermFreq() throws IOException { try { return super.getSumTotalTermFreq(); } finally { - registerRef.ensureReachability(); + Reference.reachabilityFence(sentinel); } } @@ -174,7 +175,7 @@ public long getSumDocFreq() throws IOException { try { return super.getSumDocFreq(); } finally { - registerRef.ensureReachability(); + Reference.reachabilityFence(sentinel); } } @@ -183,7 +184,7 @@ public int getDocCount() throws IOException { try { return super.getDocCount(); } finally { - registerRef.ensureReachability(); + Reference.reachabilityFence(sentinel); } } @@ -192,7 +193,7 @@ public boolean hasFreqs() { try { return super.hasFreqs(); } finally { - registerRef.ensureReachability(); + Reference.reachabilityFence(sentinel); } } @@ -201,7 +202,7 @@ public boolean hasOffsets() { try { return super.hasOffsets(); } finally { - registerRef.ensureReachability(); + Reference.reachabilityFence(sentinel); } } @@ -210,7 +211,7 @@ public boolean hasPositions() { try { return super.hasPositions(); } finally { - registerRef.ensureReachability(); + Reference.reachabilityFence(sentinel); } } @@ -219,7 +220,7 @@ public boolean hasPayloads() { try { return super.hasPayloads(); } finally { - registerRef.ensureReachability(); + Reference.reachabilityFence(sentinel); } } @@ -228,7 +229,7 @@ public Object getStats() throws IOException { try { return super.getStats(); } finally { - registerRef.ensureReachability(); + Reference.reachabilityFence(sentinel); } } @@ -237,7 +238,7 @@ public BytesRef getMin() throws IOException { try { return super.getMin(); } finally { - registerRef.ensureReachability(); + Reference.reachabilityFence(sentinel); } } @@ -246,7 +247,7 @@ public BytesRef getMax() throws IOException { try { return super.getMax(); } finally { - registerRef.ensureReachability(); + Reference.reachabilityFence(sentinel); } } }; diff --git a/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java b/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java index 0edb600e8fc8..f7ea08192f89 100644 --- a/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java +++ b/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java @@ -19,6 +19,7 @@ import static org.apache.lucene.index.Unloader.FPIOFunction; import java.io.IOException; +import java.lang.ref.Reference; import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.codecs.PointsReader; import org.apache.lucene.util.IOFunction; @@ -71,17 +72,17 @@ public PointValues getValues(String field) throws IOException { getValues, field, false, - (rawPointValues, registerRef) -> { + (rawPointValues, sentinel) -> { + assert sentinel != null : "sentinel must not be null!"; // NOTE: we have to wrap here in order to track derived `PointTree` instances return new PointValues() { @Override public PointTree getPointTree() throws IOException { try { - return Unloader.wrap( - registerRef.trackedInstance(rawPointValues::getPointTree), registerRef); + return Unloader.wrap(rawPointValues.getPointTree(), sentinel); } finally { - registerRef.ensureReachability(); + Reference.reachabilityFence(sentinel); } } @@ -90,7 +91,7 @@ public byte[] getMinPackedValue() throws IOException { try { return rawPointValues.getMinPackedValue(); } finally { - registerRef.ensureReachability(); + Reference.reachabilityFence(sentinel); } } @@ -99,7 +100,7 @@ public byte[] getMaxPackedValue() throws IOException { try { return rawPointValues.getMaxPackedValue(); } finally { - registerRef.ensureReachability(); + Reference.reachabilityFence(sentinel); } } @@ -108,7 +109,7 @@ public int getNumDimensions() throws IOException { try { return rawPointValues.getNumDimensions(); } finally { - registerRef.ensureReachability(); + Reference.reachabilityFence(sentinel); } } @@ -117,7 +118,7 @@ public int getNumIndexDimensions() throws IOException { try { return rawPointValues.getNumIndexDimensions(); } finally { - registerRef.ensureReachability(); + Reference.reachabilityFence(sentinel); } } @@ -126,7 +127,7 @@ public int getBytesPerDimension() throws IOException { try { return rawPointValues.getBytesPerDimension(); } finally { - registerRef.ensureReachability(); + Reference.reachabilityFence(sentinel); } } @@ -135,7 +136,7 @@ public long size() { try { return rawPointValues.size(); } finally { - registerRef.ensureReachability(); + Reference.reachabilityFence(sentinel); } } @@ -144,7 +145,7 @@ public int getDocCount() { try { return rawPointValues.getDocCount(); } finally { - registerRef.ensureReachability(); + Reference.reachabilityFence(sentinel); } } }; diff --git a/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java b/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java index 3b17e61e22d7..09a6d4379011 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java @@ -18,31 +18,20 @@ import java.io.Closeable; import java.io.IOException; -import java.lang.ref.ReferenceQueue; import java.util.ArrayList; import java.util.List; -import java.util.Random; -import java.util.concurrent.Callable; -import java.util.concurrent.CancellationException; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; -import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.LongAdder; -import java.util.function.Consumer; -import java.util.function.LongSupplier; -import org.apache.lucene.index.Unloader.BlockingRunnable; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.tests.util.LuceneTestCase; import org.apache.lucene.util.IOFunction; import org.apache.lucene.util.InfoStream; import org.apache.lucene.util.NamedThreadFactory; -import org.apache.lucene.util.RamUsageEstimator; public class TestUnloader extends LuceneTestCase { private static final class MyCloseable implements Closeable { @@ -233,203 +222,4 @@ public void doTest() throws IOException, InterruptedException { exec.awaitTermination(5, TimeUnit.SECONDS); } } - - private static final int MAX_KB = 1024; - private static final int MIN_KB = 1; - private static final int MAX_KB_BASELINE = MAX_KB - MIN_KB + 1; - - private static final int N_SECONDS = 5; - - public void testRefQueueHandling() throws InterruptedException, ExecutionException { - int nThreads = 20; - final int batchSize = 1024; - @SuppressWarnings({"unchecked", "rawtypes"}) - Consumer[] registerRef = new Consumer[1]; - LongSupplier[] outstandingSizeHolder = new LongSupplier[1]; - @SuppressWarnings({"unchecked", "rawtypes"}) - ReferenceQueue[][] removeOutstandingHolder = new ReferenceQueue[1][]; - @SuppressWarnings({"unchecked", "rawtypes"}) - AtomicReference[] handleRefQueueHolder = new AtomicReference[1]; - Unloader.configure( - new Unloader.UnloadHelper() { - @Override - public ScheduledExecutorService onCreation(Unloader u) { - return null; - } - - @Override - public void maybeHandleRefQueues( - ReferenceQueue[] queues, - Consumer handler, - AtomicReference handleRefQueue, - LongSupplier indirectTrackCount, - LongSupplier outstandingSize) { - handleRefQueue.set(true); - handleRefQueueHolder[0] = handleRefQueue; - registerRef[0] = handler; - outstandingSizeHolder[0] = outstandingSize; - removeOutstandingHolder[0] = queues; - } - }); - ReferenceQueue[] queues = removeOutstandingHolder[0]; - AtomicReference handleRefQueue = handleRefQueueHolder[0]; - Consumer handler = registerRef[0]; - LongSupplier outstandingSize = outstandingSizeHolder[0]; - - int PARALLEL_HEAD_FACTOR = queues.length; - ExecutorService exec = - Executors.newFixedThreadPool( - nThreads + PARALLEL_HEAD_FACTOR, new NamedThreadFactory("TestUnloader")); - AtomicBoolean finished = new AtomicBoolean(); - @SuppressWarnings("rawtypes") - Future[] futures = new Future[nThreads]; - LongAdder total = new LongAdder(); - long start = System.nanoTime(); - for (int i = nThreads - 1; i >= 0; i--) { - futures[i] = - exec.submit( - () -> { - Random r = new Random(random().nextLong()); - try { - while (!finished.get()) { - for (int j = batchSize - 1; j >= 0; j--) { - // between 1k and 1m - Unloader.addDummyReference(1024 * (r.nextInt(MAX_KB_BASELINE) + MIN_KB)); - total.increment(); - } - } - } catch (Throwable t) { - t.printStackTrace(System.err); - throw t; - } - }); - } - LongAdder activeRefQueueProcessors = new LongAdder(); - LongAdder collectedHoldingRefs = new LongAdder(); - LongAdder collectedRefs = new LongAdder(); - @SuppressWarnings("rawtypes") - Future[] refQueueFutures = new Future[queues.length]; - for (int i = queues.length - 1; i >= 0; i--) { - ReferenceQueue q = queues[i]; - refQueueFutures[i] = - exec.submit( - wrapTask( - handleRefQueue, - activeRefQueueProcessors, - () -> { - handler.accept(q.remove()); - collectedRefs.increment(); - })); - } - long endNanos = System.nanoTime() + TimeUnit.SECONDS.toNanos(N_SECONDS); - long remainingNanos; - while ((remainingNanos = endNanos - System.nanoTime()) > 0) { - long sz = outstandingSize.getAsLong(); - System.out.println( - "seconds remaining: " - + TimeUnit.NANOSECONDS.toSeconds(remainingNanos) - + ", outstandingSize=" - + sz - + ", active=" - + activeRefQueueProcessors.sum() - + " (" - + RamUsageEstimator.humanReadableUnits(sz * Unloader.RAMBYTES_PER_REF) - + ")"); - Thread.sleep(Math.min(1000, TimeUnit.NANOSECONDS.toMillis(remainingNanos))); - } - finished.set(true); - long sum = total.sum(); - for (int i = nThreads - 1; i >= 0; i--) { - futures[i].get(); - } - System.out.println( - "tasks completed " + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start)); - start = System.nanoTime(); - int gcIterations = 0; - long sz; - while ((sz = outstandingSize.getAsLong()) > 0 || Unloader.nonEmptyRefQueueHeadCount() > 0) { - gcIterations++; - System.gc(); - Thread.sleep(250); - System.err.println( - "gc iteration " - + gcIterations - + ", " - + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start) - + ", outstandingSize=" - + sz - + ", active=" - + activeRefQueueProcessors.sum() - + ", nonEmptyRefQueueHeadCount=" - + Unloader.nonEmptyRefQueueHeadCount()); - if (gcIterations > 40) { - fail("failed to converge"); - } - } - handleRefQueue.set(false); - for (int i = refQueueFutures.length - 1; i >= 0; i--) { - refQueueFutures[i].cancel(true); - } - for (int i = refQueueFutures.length - 1; i >= 0; i--) { - try { - refQueueFutures[i].get(); - } catch ( - @SuppressWarnings("unused") - CancellationException ex) { - // this is ok. - // NOTE: we can't do `expectThrows()` because depending on where - // the task is in the loop, it might exit _without_ `CancellationException`. - } - } - exec.shutdown(); - assertTrue(exec.awaitTermination(60, TimeUnit.SECONDS)); - long createdSum = total.sum(); - long collectedSum = collectedRefs.sum(); - long collectedHoldingSum = collectedHoldingRefs.sum(); - assertEquals(createdSum, collectedSum + collectedHoldingSum); - System.out.println( - "success! " - + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start) - + " millis; throughput=" - + (sum / N_SECONDS) - + "/s"); - System.out.println( - "total created=" - + createdSum - + ", collectedHolding=" - + collectedHoldingSum - + ", collected=" - + collectedSum); - } - - private static Callable wrapTask( - AtomicReference handleRefQueue, - LongAdder activeRefQueueProcessors, - BlockingRunnable r) { - return () -> { - activeRefQueueProcessors.increment(); - try { - while (handleRefQueue.get() == Boolean.TRUE) { - r.run(); - } - } catch (InterruptedException ex) { - if (handleRefQueue.get() == Boolean.TRUE) { - // unexpected -- we've been interrupted but are still - // supposed to be handling ref queue? - handleRefQueue.set(false); - System.err.println("unexpected interruption of ref queue processing"); - ex.printStackTrace(System.err); - throw ex; - } - } catch (Throwable t) { - handleRefQueue.set(false); - System.err.println("exception in ref queue processing"); - t.printStackTrace(System.err); - throw t; - } finally { - activeRefQueueProcessors.decrement(); - } - return null; - }; - } } From f5723d4718a53cd36d5356e2af116c78ccd0c13a Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Mon, 3 Nov 2025 17:33:39 -0500 Subject: [PATCH 25/36] remove unused args --- .../core/src/java/org/apache/lucene/index/Unloader.java | 8 +++----- .../apache/lucene/index/UnloadingDocValuesProducer.java | 7 +------ .../org/apache/lucene/index/UnloadingFieldsProducer.java | 4 +--- .../org/apache/lucene/index/UnloadingPointsReader.java | 3 +-- .../src/test/org/apache/lucene/index/TestUnloader.java | 3 +-- 5 files changed, 7 insertions(+), 18 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index 60c6d3bd0b58..799f8cb5b8ef 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -823,8 +823,8 @@ interface RefTrackShim { V shim(V in, Object sentinel); } - V execute(FPIOFunction function, K arg) throws IOException { - return execute(function, arg, true, null); + V execute(FPIOFunction function) throws IOException { + return execute(function, null, null); } /** @@ -842,9 +842,7 @@ V execute(FPIOFunction function, K arg) throws IOException { * collecting the wrapper and unloading the resource after entering the shim method, but before * completing the call to the raw/backing method. */ - V execute( - FPIOFunction function, K arg, boolean trackRawUnused, RefTrackShim shim) - throws IOException { + V execute(FPIOFunction function, K arg, RefTrackShim shim) throws IOException { try (CloseableVal active = backing()) { V raw = function.apply(active.get(), arg); if (raw == null) { diff --git a/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java index badfe20d3381..e6e8421fafa7 100644 --- a/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java +++ b/lucene/core/src/java/org/apache/lucene/index/UnloadingDocValuesProducer.java @@ -61,7 +61,7 @@ public void close() throws IOException { @Override public void checkIntegrity() throws IOException { - u.execute(checkIntegrity, null); + u.execute(checkIntegrity); } private final FPIOFunction getNumeric = @@ -72,7 +72,6 @@ public NumericDocValues getNumeric(FieldInfo field) throws IOException { return u.execute( getNumeric, field, - true, (raw, sentinel) -> { assert sentinel != null : "sentinel must not be null"; return new FilterNumericDocValues(raw) { @@ -96,7 +95,6 @@ public BinaryDocValues getBinary(FieldInfo field) throws IOException { return u.execute( getBinary, field, - true, (raw, sentinel) -> { assert sentinel != null : "sentinel must not be null"; return new FilterBinaryDocValues(raw) { @@ -120,7 +118,6 @@ public SortedDocValues getSorted(FieldInfo field) throws IOException { return u.execute( getSorted, field, - true, (rawSorted, sentinel) -> { assert sentinel != null : "sentinel must not be null"; // wrap so that we can track refs for returned `TermsEnum` instances @@ -146,7 +143,6 @@ public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOExcepti return u.execute( getSortedNumeric, field, - true, (raw, sentinel) -> { assert sentinel != null : "sentinel must not be null"; return new FilterSortedNumericDocValues(raw) { @@ -170,7 +166,6 @@ public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException { return u.execute( getSortedSet, field, - true, (rawSorted, sentinel) -> { assert sentinel != null : "sentinel must not be null"; // wrap so that we can track refs for returned `TermsEnum` instances diff --git a/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java b/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java index 138917d26ccf..e4af2e093e9d 100644 --- a/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java +++ b/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java @@ -77,7 +77,7 @@ public void close() throws IOException { @Override public void checkIntegrity() throws IOException { - u.execute(checkIntegrity, null); + u.execute(checkIntegrity); } private final FPIOFunction> iterator = @@ -89,7 +89,6 @@ public Iterator iterator() { return u.execute( iterator, null, - true, (raw, sentinel) -> { assert sentinel != null : "sentinel must not be null"; return new Iterator() { @@ -121,7 +120,6 @@ public Terms terms(String field) throws IOException { return u.execute( terms, field, - false, (rawTerms, sentinel) -> { assert sentinel != null : "sentinel must not be null"; // NOTE: we have to wrap here because a reference to the raw value may be diff --git a/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java b/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java index f7ea08192f89..8d6d00fd8258 100644 --- a/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java +++ b/lucene/core/src/java/org/apache/lucene/index/UnloadingPointsReader.java @@ -61,7 +61,7 @@ public void close() throws IOException { @Override public void checkIntegrity() throws IOException { - u.execute(checkIntegrity, null); + u.execute(checkIntegrity); } private final FPIOFunction getValues = PointsReader::getValues; @@ -71,7 +71,6 @@ public PointValues getValues(String field) throws IOException { return u.execute( getValues, field, - false, (rawPointValues, sentinel) -> { assert sentinel != null : "sentinel must not be null!"; // NOTE: we have to wrap here in order to track derived `PointTree` instances diff --git a/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java b/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java index 09a6d4379011..62388850b25e 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java @@ -138,8 +138,7 @@ public void doTest() throws IOException, InterruptedException { (a, b) -> { a.count.incrementAndGet(); return null; - }, - null); + }); check.increment(); } return "execute"; From 63d4ae0b069272a5d8dd695c8e6dc376af730959 Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Tue, 4 Nov 2025 08:18:50 -0500 Subject: [PATCH 26/36] allow helper-controlled deferral of unloading --- .../src/java/org/apache/lucene/index/Unloader.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index 799f8cb5b8ef..3ca9080b99d8 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -370,9 +370,12 @@ private static Map.Entry, Object> retry( */ public long maybeUnload() throws IOException { long nanosSinceLastAccess = System.nanoTime() - lastAccessNanos; + long deferNanos; if (nanosSinceLastAccess < keepAliveNanos) { // don't unload return keepAliveNanos - nanosSinceLastAccess; + } else if ((deferNanos = reporter.deferUnload(nanosSinceLastAccess)) > 0) { + return deferNanos; } final boolean[] unloaded = new boolean[1]; DelegateFuture holder = unloadRef(backing, unloaded); @@ -871,7 +874,7 @@ interface FPIOFunction { * Time threshold at which a resource becomes eligible for unloading. Set this very low (0 or 1) * for stress testing. */ - private static final long KEEP_ALIVE_NANOS; + public static final long KEEP_ALIVE_NANOS; private static final String DEFAULT_KEEP_ALIVE_SPEC = "60m"; private static final long DEFAULT_KEEP_ALIVE_NANOS = TimeUnit.MINUTES.toNanos(60); @@ -1027,7 +1030,10 @@ default void maybeHandleRefQueues( AtomicBoolean initialized, LongSupplier indirectTrackedCount, LongSupplier refsCollected) { // no-op default impl } - ; + + default long deferUnload(long nanosSinceLastAccess) { + return -1; + } } /** From aa4f3c8a23c0cf9a9f606ab0f878a3623a35b650 Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Tue, 4 Nov 2025 10:41:19 -0500 Subject: [PATCH 27/36] support disabling "adaptive defer" --- .../src/java/org/apache/lucene/index/Unloader.java | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index 3ca9080b99d8..12b1a0e4c327 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -358,6 +358,14 @@ private static Map.Entry, Object> retry( } } + /** + * Defaults to true; if true, allows {@link #reporter} to defer unloading (e.g., based on + * historical context, to avoid thrashing) via {@link UnloadHelper#deferUnload(long)}. This + * behavior may be disabled by setting sysprop to false. + */ + public static final boolean ADAPTIVE_DEFER = + !"false".equals(System.getProperty("lucene.ttl.adaptiveDefer")); + /** * Conditionally unloads (closes) the delegate {@link FieldsProducer}. Returns {@link #UNLOADED} * if resources were unloaded, otherwise returns the number of nanos remaining until the resources @@ -374,7 +382,7 @@ public long maybeUnload() throws IOException { if (nanosSinceLastAccess < keepAliveNanos) { // don't unload return keepAliveNanos - nanosSinceLastAccess; - } else if ((deferNanos = reporter.deferUnload(nanosSinceLastAccess)) > 0) { + } else if (ADAPTIVE_DEFER && (deferNanos = reporter.deferUnload(nanosSinceLastAccess)) > 0) { return deferNanos; } final boolean[] unloaded = new boolean[1]; From eb89ad8ac081ae19e004c0bf32a3e3331cdc3888 Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Tue, 4 Nov 2025 11:28:32 -0500 Subject: [PATCH 28/36] measuring time since last access makes no sense on initial load (messes up metrics/state-tracking) --- .../org/apache/lucene/index/Unloader.java | 20 +++++++++++++++---- .../org/apache/lucene/index/TestUnloader.java | 2 +- 2 files changed, 17 insertions(+), 5 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index 12b1a0e4c327..169b8c804931 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -1007,7 +1007,7 @@ public interface UnloadHelper { * reload * @param loadTime how long did it take to load this resource (nanos) */ - default void onLoad(long nanosSincePriorAccess, long loadTime) {} + default void onLoad(long nanosSincePriorAccess, long loadTime, boolean initial) {} /** * Called for each unload of backing resource @@ -1039,6 +1039,15 @@ default void maybeHandleRefQueues( // no-op default impl } + /** + * Allows this {@link UnloadHelper} to defer the unloading of the associated resource. This is + * usually used in conjunction with tracking historical behavior of the resource, and may be + * used to avoid thrashing. + * + * @param nanosSinceLastAccess number of nanos since last access of the associated resource + * @return If positive (> 0), represents the number of nanos that unloading + * should be deferred (if <= 0, unloading will not be deferred). + */ default long deferUnload(long nanosSinceLastAccess) { return -1; } @@ -1118,7 +1127,8 @@ public static PointsReader pointsReader( throw t; } } - u.reporter.onLoad(start - u.lastAccessNanos, System.nanoTime() - start); + u.reporter.onLoad( + start - u.lastAccessNanos, System.nanoTime() - start, u.description == null); return pr; }, KEEP_ALIVE_NANOS); @@ -1170,7 +1180,8 @@ public static FieldsProducer fieldsProducer( throw t; } } - u.reporter.onLoad(start - u.lastAccessNanos, System.nanoTime() - start); + u.reporter.onLoad( + start - u.lastAccessNanos, System.nanoTime() - start, u.description == null); return fp; }, KEEP_ALIVE_NANOS); @@ -1222,7 +1233,8 @@ public static DocValuesProducer docValuesProducer( throw t; } } - u.reporter.onLoad(start - u.lastAccessNanos, System.nanoTime() - start); + u.reporter.onLoad( + start - u.lastAccessNanos, System.nanoTime() - start, u.description == null); return dvp; }, KEEP_ALIVE_NANOS); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java b/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java index 62388850b25e..cd64a10cf9f0 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestUnloader.java @@ -83,7 +83,7 @@ public void close() throws IOException { } } - private static final IOFunction NO_OP = (c) -> null; + private static final IOFunction NO_OP = Object::toString; public void test() throws IOException, InterruptedException { for (int i = 0; i < 10; i++) { From b21c1341922f4c2ce041cc0a63061bac16c76e0b Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Tue, 4 Nov 2025 15:21:12 -0500 Subject: [PATCH 29/36] fix typo in sysprop name --- lucene/core/src/java/org/apache/lucene/index/Unloader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index 169b8c804931..edf4fbe29b68 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -364,7 +364,7 @@ private static Map.Entry, Object> retry( * behavior may be disabled by setting sysprop to false. */ public static final boolean ADAPTIVE_DEFER = - !"false".equals(System.getProperty("lucene.ttl.adaptiveDefer")); + !"false".equals(System.getProperty("lucene.unload.adaptiveDefer")); /** * Conditionally unloads (closes) the delegate {@link FieldsProducer}. Returns {@link #UNLOADED} From 454a430b26a5e9177c2cad93fa9a5d848792963f Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Mon, 10 Nov 2025 10:58:31 -0500 Subject: [PATCH 30/36] expose time parsing method publicly --- .../org/apache/lucene/index/Unloader.java | 71 ++++++++++--------- 1 file changed, 36 insertions(+), 35 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index edf4fbe29b68..5960a1f8bf88 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -936,42 +936,11 @@ private static long getNanos( String syspropName, String defaultSpec, long defaultNanos, List deferred) { try { String unloadSpec = System.getProperty(syspropName, defaultSpec); - int endIdx = unloadSpec.length() - 1; - if (unloadSpec.isEmpty()) { - deferred.add("WARN: empty " + syspropName + " spec"); + try { + return getNanos(unloadSpec); + } catch (IllegalArgumentException ex) { + deferred.add("WARN: bad " + syspropName + " spec: " + unloadSpec + " " + ex); return defaultNanos; - } else { - TimeUnit t; - char c = unloadSpec.charAt(endIdx); - switch (c) { - case 's': - t = TimeUnit.SECONDS; - break; - case 'm': - t = TimeUnit.MINUTES; - break; - case 'h': - t = TimeUnit.HOURS; - break; - case 'd': - t = TimeUnit.DAYS; - break; - default: - if (c >= '0' && c <= '9') { - endIdx++; - t = TimeUnit.MILLISECONDS; - } else { - deferred.add("WARN: bad " + syspropName + " spec: " + unloadSpec); - return defaultNanos; - } - } - try { - int v = Integer.parseInt(unloadSpec, 0, endIdx, 10); - return t.toNanos(v); - } catch (NumberFormatException ex) { - deferred.add("WARN: bad " + syspropName + " spec: " + unloadSpec + " " + ex); - throw ex; - } } } catch ( @SuppressWarnings("unused") @@ -980,6 +949,38 @@ private static long getNanos( } } + public static long getNanos(String unloadSpec) { + if (unloadSpec.isEmpty()) { + throw new IllegalArgumentException("empty unloadSpec"); + } + int endIdx = unloadSpec.length() - 1; + TimeUnit t; + char c = unloadSpec.charAt(endIdx); + switch (c) { + case 's': + t = TimeUnit.SECONDS; + break; + case 'm': + t = TimeUnit.MINUTES; + break; + case 'h': + t = TimeUnit.HOURS; + break; + case 'd': + t = TimeUnit.DAYS; + break; + default: + if (c >= '0' && c <= '9') { + endIdx++; + t = TimeUnit.MILLISECONDS; + } else { + throw new IllegalArgumentException("expected plain numeric arg (millis)"); + } + } + int v = Integer.parseInt(unloadSpec, 0, endIdx, 10); + return t.toNanos(v); + } + /** * Passed to {@link Unloader#Unloader(UnloadHelper, IOFunction, long, IOFunction)} ctor. This * provides a means for reporting unload/reload lifecycle events to higher-level components. This From 46e1d721b185c2955635c7b66b74ab6aa6794608 Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Tue, 18 Nov 2025 14:10:49 -0500 Subject: [PATCH 31/36] add public method javaodocs --- lucene/core/src/java/org/apache/lucene/index/Unloader.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index 5960a1f8bf88..3257576744aa 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -949,6 +949,12 @@ private static long getNanos( } } + /** + * Returns the number of nanoseconds represented by the specified string spec. + * + *

The spec may be suffixed with "s" (seconds), "m" (minutes), "h" (hours), or "d" (days). If + * unsuffixed (a straight numeric value), the spec is interpreted as milliseconds. + */ public static long getNanos(String unloadSpec) { if (unloadSpec.isEmpty()) { throw new IllegalArgumentException("empty unloadSpec"); From d37e17bb5382fe376c85c9d7f099f4ce77d7d9f7 Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Tue, 18 Nov 2025 09:16:54 -0500 Subject: [PATCH 32/36] support per-format disabling of unloading --- .../lucene/index/SegmentCoreReaders.java | 20 ++++++++++++++----- .../apache/lucene/index/SegmentDocValues.java | 10 ++++++++-- .../org/apache/lucene/index/Unloader.java | 15 ++++++++++++++ 3 files changed, 38 insertions(+), 7 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java b/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java index 81965b0db5e7..247f3144ae18 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java +++ b/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java @@ -117,9 +117,14 @@ protected TermVectorsReader initialValue() { if (coreFieldInfos.hasPostings()) { final PostingsFormat format = codec.postingsFormat(); // Ask codec for its Fields - fields = - Unloader.fieldsProducer( - () -> format.fieldsProducer(segmentReadState), dir, segmentReadState); + if (format instanceof Unloader.UnloadAware + && ((Unloader.UnloadAware) format).disableUnload(segmentReadState)) { + fields = format.fieldsProducer(segmentReadState); + } else { + fields = + Unloader.fieldsProducer( + () -> format.fieldsProducer(segmentReadState), dir, segmentReadState); + } assert fields != null; } else { fields = null; @@ -153,8 +158,13 @@ protected TermVectorsReader initialValue() { if (coreFieldInfos.hasPointValues()) { PointsFormat pf = codec.pointsFormat(); - pointsReader = - Unloader.pointsReader(() -> pf.fieldsReader(segmentReadState), dir, segmentReadState); + if (pf instanceof Unloader.UnloadAware + && ((Unloader.UnloadAware) pf).disableUnload(segmentReadState)) { + pointsReader = pf.fieldsReader(segmentReadState); + } else { + pointsReader = + Unloader.pointsReader(() -> pf.fieldsReader(segmentReadState), dir, segmentReadState); + } } else { pointsReader = null; } diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java b/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java index c3dd49b0ce3d..0b3c1a6cab18 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java +++ b/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java @@ -49,8 +49,14 @@ private RefCount newDocValuesProducer( SegmentReadState srs = new SegmentReadState(dvDir, si.info, infos, IOContext.READ, segmentSuffix); DocValuesFormat dvFormat = si.info.getCodec().docValuesFormat(); - return new RefCount( - Unloader.docValuesProducer(() -> dvFormat.fieldsProducer(srs), si.info.dir, srs)) { + DocValuesProducer dvp; + if (dvFormat instanceof Unloader.UnloadAware + && ((Unloader.UnloadAware) dvFormat).disableUnload(srs)) { + dvp = dvFormat.fieldsProducer(srs); + } else { + dvp = Unloader.docValuesProducer(() -> dvFormat.fieldsProducer(srs), si.info.dir, srs); + } + return new RefCount(dvp) { @SuppressWarnings("synthetic-access") @Override protected void release() throws IOException { diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index 3257576744aa..cefc51c9e046 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -1141,6 +1141,21 @@ public static PointsReader pointsReader( KEEP_ALIVE_NANOS); } + /** + * To be implemented by components ({@link org.apache.lucene.codecs.PostingsFormat}, {@link + * org.apache.lucene.codecs.DocValuesFormat} or {@link org.apache.lucene.codecs.PointsFormat}) + * that are aware of resource unloading and need to be able to prevent unloading of associated + * resources. + */ + public interface UnloadAware { + /** + * If this method returns {@code true}, unloading of associated resources should be disabled. + */ + default boolean disableUnload(SegmentReadState state) { + return true; + } + } + /** * Returns a {@link FieldsProducer} over the specified {@link SegmentReadState}, conditionally * wrapped to allow dynamic unloading and on-demand reloading of the backing resource. From f16d5091ab1adff6b3212d685493ae796bf8cf89 Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Tue, 18 Nov 2025 21:08:54 -0500 Subject: [PATCH 33/36] move Unloader invocation to PerFieldPostingsFormat and PerFieldDocValuesFormat --- .../codecs/perfield/PerFieldDocValuesFormat.java | 15 ++++++++++++--- .../codecs/perfield/PerFieldPostingsFormat.java | 14 +++++++++++--- .../apache/lucene/index/SegmentCoreReaders.java | 9 +-------- .../org/apache/lucene/index/SegmentDocValues.java | 9 +-------- 4 files changed, 25 insertions(+), 22 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java index 0e8352da4451..43a74fc0ea04 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java @@ -37,6 +37,7 @@ import org.apache.lucene.index.SortedDocValues; import org.apache.lucene.index.SortedNumericDocValues; import org.apache.lucene.index.SortedSetDocValues; +import org.apache.lucene.index.Unloader; import org.apache.lucene.util.IOUtils; /** @@ -300,9 +301,17 @@ public FieldsReader(final SegmentReadState readState) throws IOException { String segmentSuffix = getFullSegmentSuffix(readState.segmentSuffix, getSuffix(formatName, suffix)); if (!formats.containsKey(segmentSuffix)) { - formats.put( - segmentSuffix, - format.fieldsProducer(new SegmentReadState(readState, segmentSuffix))); + SegmentReadState srs = new SegmentReadState(readState, segmentSuffix); + DocValuesProducer dvp; + if (format instanceof Unloader.UnloadAware + && ((Unloader.UnloadAware) format).disableUnload(srs)) { + dvp = format.fieldsProducer(srs); + } else { + dvp = + Unloader.docValuesProducer( + () -> format.fieldsProducer(srs), srs.segmentInfo.dir, srs); + } + formats.put(segmentSuffix, dvp); } fields.put(fieldName, formats.get(segmentSuffix)); } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java index 61749a09284e..fb8607e15059 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java @@ -43,6 +43,7 @@ import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.Terms; +import org.apache.lucene.index.Unloader; import org.apache.lucene.util.CollectionUtil; import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.MergedIterator; @@ -323,9 +324,16 @@ public FieldsReader(final SegmentReadState readState) throws IOException { PostingsFormat format = PostingsFormat.forName(formatName); String segmentSuffix = getSuffix(formatName, suffix); if (!formats.containsKey(segmentSuffix)) { - formats.put( - segmentSuffix, - format.fieldsProducer(new SegmentReadState(readState, segmentSuffix))); + SegmentReadState srs = new SegmentReadState(readState, segmentSuffix); + FieldsProducer fp; + if (format instanceof Unloader.UnloadAware + && ((Unloader.UnloadAware) format).disableUnload(srs)) { + fp = format.fieldsProducer(srs); + } else { + fp = + Unloader.fieldsProducer(() -> format.fieldsProducer(srs), srs.directory, srs); + } + formats.put(segmentSuffix, fp); } fields.put(fieldName, formats.get(segmentSuffix)); } diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java b/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java index 247f3144ae18..8cc0d747a4d4 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java +++ b/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java @@ -117,14 +117,7 @@ protected TermVectorsReader initialValue() { if (coreFieldInfos.hasPostings()) { final PostingsFormat format = codec.postingsFormat(); // Ask codec for its Fields - if (format instanceof Unloader.UnloadAware - && ((Unloader.UnloadAware) format).disableUnload(segmentReadState)) { - fields = format.fieldsProducer(segmentReadState); - } else { - fields = - Unloader.fieldsProducer( - () -> format.fieldsProducer(segmentReadState), dir, segmentReadState); - } + fields = format.fieldsProducer(segmentReadState); assert fields != null; } else { fields = null; diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java b/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java index 0b3c1a6cab18..1d90783da393 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java +++ b/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java @@ -49,14 +49,7 @@ private RefCount newDocValuesProducer( SegmentReadState srs = new SegmentReadState(dvDir, si.info, infos, IOContext.READ, segmentSuffix); DocValuesFormat dvFormat = si.info.getCodec().docValuesFormat(); - DocValuesProducer dvp; - if (dvFormat instanceof Unloader.UnloadAware - && ((Unloader.UnloadAware) dvFormat).disableUnload(srs)) { - dvp = dvFormat.fieldsProducer(srs); - } else { - dvp = Unloader.docValuesProducer(() -> dvFormat.fieldsProducer(srs), si.info.dir, srs); - } - return new RefCount(dvp) { + return new RefCount(dvFormat.fieldsProducer(srs)) { @SuppressWarnings("synthetic-access") @Override protected void release() throws IOException { From c55f0718810b44a833d5875ff27a95771383acbe Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Wed, 19 Nov 2025 14:29:50 -0500 Subject: [PATCH 34/36] temporary commit for evaluting reload paths --- .../apache/lucene/index/UnloadingFieldsProducer.java | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java b/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java index e4af2e093e9d..f24a0383ddea 100644 --- a/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java +++ b/lucene/core/src/java/org/apache/lucene/index/UnloadingFieldsProducer.java @@ -115,8 +115,19 @@ public String next() { private final FPIOFunction terms = Fields::terms; + public static final ThreadLocal FIELD_REQUESTED = new ThreadLocal<>(); + @Override public Terms terms(String field) throws IOException { + try { + FIELD_REQUESTED.set(field); + return terms0(field); + } finally { + FIELD_REQUESTED.remove(); + } + } + + private Terms terms0(String field) throws IOException { return u.execute( terms, field, From a18002092ce86a7e10626b4f507a92bda1870dd6 Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Fri, 21 Nov 2025 12:22:30 -0500 Subject: [PATCH 35/36] more nuanced disabling of unloading --- .../org/apache/lucene/index/Unloader.java | 27 ++++++++++++------- 1 file changed, 18 insertions(+), 9 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index cefc51c9e046..249c507634c4 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -46,6 +46,7 @@ import org.apache.lucene.codecs.PointsReader; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; import org.apache.lucene.store.UnloaderCoordinationPoint; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.IOFunction; @@ -1058,6 +1059,10 @@ default void maybeHandleRefQueues( default long deferUnload(long nanosSinceLastAccess) { return -1; } + + default boolean disableUnload(Class resourceClass, SegmentReadState srs) { + return false; + } } /** @@ -1088,6 +1093,10 @@ public ScheduledExecutorService onCreation(Unloader u) { } } + private static boolean disableUnload(SegmentReadState srs) { + return srs.context.context == IOContext.Context.MERGE || !srs.segmentInfo.getUseCompoundFile(); + } + /** * Returns a {@link PointsReader} over the specified {@link SegmentReadState}, conditionally * wrapped to allow dynamic unloading and on-demand reloading of the backing resource. @@ -1106,10 +1115,10 @@ public ScheduledExecutorService onCreation(Unloader u) { public static PointsReader pointsReader( IOSupplier open, Directory dir, SegmentReadState srs) throws IOException { UnloadHelper unloadHelper; - if (srs.context.mergeInfo != null - || srs.context.flushInfo != null + if (disableUnload(srs) || DISABLE - || (unloadHelper = UnloaderCoordinationPoint.getUnloadHelper(dir)) == null) { + || (unloadHelper = UnloaderCoordinationPoint.getUnloadHelper(dir)) == null + || unloadHelper.disableUnload(PointsReader.class, srs)) { return open.get(); } String type = PointsReader.class.getSimpleName(); @@ -1174,10 +1183,10 @@ default boolean disableUnload(SegmentReadState state) { public static FieldsProducer fieldsProducer( IOSupplier open, Directory dir, SegmentReadState srs) throws IOException { UnloadHelper unloadHelper; - if (srs.context.mergeInfo != null - || srs.context.flushInfo != null + if (disableUnload(srs) || DISABLE - || (unloadHelper = UnloaderCoordinationPoint.getUnloadHelper(dir)) == null) { + || (unloadHelper = UnloaderCoordinationPoint.getUnloadHelper(dir)) == null + || unloadHelper.disableUnload(FieldsProducer.class, srs)) { return open.get(); } String type = FieldsProducer.class.getSimpleName(); @@ -1227,10 +1236,10 @@ public static FieldsProducer fieldsProducer( public static DocValuesProducer docValuesProducer( IOSupplier open, Directory dir, SegmentReadState srs) throws IOException { UnloadHelper unloadHelper; - if (srs.context.mergeInfo != null - || srs.context.flushInfo != null + if (disableUnload(srs) || DISABLE - || (unloadHelper = UnloaderCoordinationPoint.getUnloadHelper(dir)) == null) { + || (unloadHelper = UnloaderCoordinationPoint.getUnloadHelper(dir)) == null + || unloadHelper.disableUnload(DocValuesProducer.class, srs)) { return open.get(); } String type = DocValuesProducer.class.getSimpleName(); From 0c2523899c0cca5245c349e97eefbd4e1dbd66cf Mon Sep 17 00:00:00 2001 From: Michael Gibney Date: Fri, 21 Nov 2025 18:11:11 -0500 Subject: [PATCH 36/36] CFS is the wrong heuristic for unload enable --- lucene/core/src/java/org/apache/lucene/index/Unloader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/Unloader.java b/lucene/core/src/java/org/apache/lucene/index/Unloader.java index 249c507634c4..9f5a8c72e7fc 100644 --- a/lucene/core/src/java/org/apache/lucene/index/Unloader.java +++ b/lucene/core/src/java/org/apache/lucene/index/Unloader.java @@ -1094,7 +1094,7 @@ public ScheduledExecutorService onCreation(Unloader u) { } private static boolean disableUnload(SegmentReadState srs) { - return srs.context.context == IOContext.Context.MERGE || !srs.segmentInfo.getUseCompoundFile(); + return srs.context.context == IOContext.Context.MERGE; } /**