Make sure `DocumentsWriterPerThread#getAndLock` never returns `null` on a non-empty queue. (#12959)

Before this change, `DocumentsWriterPerThread#getAndLock` could sometimes
return `null` even though the queue was empty at no point in time. The
practical implication is that we can end up with more DWPTs in memory than
indexing threads, which, while not strictly a bug, may require doing more
merging than we'd like later on.

I ran luceneutil's `IndexGeonames` with this change, and
`DocumentsWriterPerThread#getAndLock` was not the main source of
contention.

Closes #12649 #12916
This commit is contained in:
Adrien Grand 2024-01-12 16:21:01 +01:00 committed by GitHub
parent 7dfef017e3
commit e0daca1eb4
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 203 additions and 25 deletions

View File

@ -29,6 +29,8 @@ import java.util.Objects;
import java.util.Set; import java.util.Set;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantLock;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.document.NumericDocValuesField;
@ -47,7 +49,7 @@ import org.apache.lucene.util.SetOnce;
import org.apache.lucene.util.StringHelper; import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.Version; import org.apache.lucene.util.Version;
final class DocumentsWriterPerThread implements Accountable { final class DocumentsWriterPerThread implements Accountable, Lock {
private Throwable abortingException; private Throwable abortingException;
@ -752,25 +754,26 @@ final class DocumentsWriterPerThread implements Accountable {
return delta; return delta;
} }
/** @Override
* Locks this DWPT for exclusive access. public void lock() {
*
* @see ReentrantLock#lock()
*/
void lock() {
lock.lock(); lock.lock();
} }
/** @Override
* Acquires the DWPT's lock only if it is not held by another thread at the time of invocation. public void lockInterruptibly() throws InterruptedException {
* lock.lockInterruptibly();
* @return true if the lock was acquired. }
* @see ReentrantLock#tryLock()
*/ @Override
boolean tryLock() { public boolean tryLock() {
return lock.tryLock(); return lock.tryLock();
} }
@Override
public boolean tryLock(long time, TimeUnit unit) throws InterruptedException {
return lock.tryLock(time, unit);
}
/** /**
* Returns true if the DWPT's lock is held by the current thread * Returns true if the DWPT's lock is held by the current thread
* *
@ -780,15 +783,16 @@ final class DocumentsWriterPerThread implements Accountable {
return lock.isHeldByCurrentThread(); return lock.isHeldByCurrentThread();
} }
/** @Override
* Unlocks the DWPT's lock public void unlock() {
*
* @see ReentrantLock#unlock()
*/
void unlock() {
lock.unlock(); lock.unlock();
} }
@Override
public Condition newCondition() {
throw new UnsupportedOperationException();
}
/** Returns <code>true</code> iff this DWPT has been flushed */ /** Returns <code>true</code> iff this DWPT has been flushed */
boolean hasFlushed() { boolean hasFlushed() {
return hasFlushed.get() == Boolean.TRUE; return hasFlushed.get() == Boolean.TRUE;

View File

@ -44,8 +44,8 @@ final class DocumentsWriterPerThreadPool implements Iterable<DocumentsWriterPerT
private final Set<DocumentsWriterPerThread> dwpts = private final Set<DocumentsWriterPerThread> dwpts =
Collections.newSetFromMap(new IdentityHashMap<>()); Collections.newSetFromMap(new IdentityHashMap<>());
private final ConcurrentApproximatePriorityQueue<DocumentsWriterPerThread> freeList = private final LockableConcurrentApproximatePriorityQueue<DocumentsWriterPerThread> freeList =
new ConcurrentApproximatePriorityQueue<>(); new LockableConcurrentApproximatePriorityQueue<>();
private final Supplier<DocumentsWriterPerThread> dwptFactory; private final Supplier<DocumentsWriterPerThread> dwptFactory;
private int takenWriterPermits = 0; private int takenWriterPermits = 0;
private volatile boolean closed; private volatile boolean closed;
@ -114,10 +114,11 @@ final class DocumentsWriterPerThreadPool implements Iterable<DocumentsWriterPerT
*/ */
DocumentsWriterPerThread getAndLock() { DocumentsWriterPerThread getAndLock() {
ensureOpen(); ensureOpen();
DocumentsWriterPerThread dwpt = freeList.poll(DocumentsWriterPerThread::tryLock); DocumentsWriterPerThread dwpt = freeList.lockAndPoll();
if (dwpt != null) { if (dwpt != null) {
return dwpt; return dwpt;
} }
// newWriter() adds the DWPT to the `dwpts` set as a side-effect. However it is not added to // newWriter() adds the DWPT to the `dwpts` set as a side-effect. However it is not added to
// `freeList` at this point, it will be added later on once DocumentsWriter has indexed a // `freeList` at this point, it will be added later on once DocumentsWriter has indexed a
// document into this DWPT and then gives it back to the pool by calling // document into this DWPT and then gives it back to the pool by calling
@ -139,8 +140,7 @@ final class DocumentsWriterPerThreadPool implements Iterable<DocumentsWriterPerT
final long ramBytesUsed = state.ramBytesUsed(); final long ramBytesUsed = state.ramBytesUsed();
assert contains(state) assert contains(state)
: "we tried to add a DWPT back to the pool but the pool doesn't know about this DWPT"; : "we tried to add a DWPT back to the pool but the pool doesn't know about this DWPT";
freeList.add(state, ramBytesUsed); freeList.addAndUnlock(state, ramBytesUsed);
state.unlock();
} }
@Override @Override

View File

@ -0,0 +1,70 @@
/*
* 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.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.Lock;
/** A {@link ConcurrentApproximatePriorityQueue} of {@link Lock} objects. */
final class LockableConcurrentApproximatePriorityQueue<T extends Lock> {
private final ConcurrentApproximatePriorityQueue<T> queue;
private final AtomicInteger addAndUnlockCounter = new AtomicInteger();
LockableConcurrentApproximatePriorityQueue(int concurrency) {
this.queue = new ConcurrentApproximatePriorityQueue<>(concurrency);
}
LockableConcurrentApproximatePriorityQueue() {
this.queue = new ConcurrentApproximatePriorityQueue<>();
}
/**
* Lock an entry, and poll it from the queue, in that order. If no entry can be found and locked,
* {@code null} is returned.
*/
T lockAndPoll() {
int addAndUnlockCount;
do {
addAndUnlockCount = addAndUnlockCounter.get();
T entry = queue.poll(Lock::tryLock);
if (entry != null) {
return entry;
}
// If an entry has been added to the queue in the meantime, try again.
} while (addAndUnlockCount != addAndUnlockCounter.get());
return null;
}
/** Remove an entry from the queue. */
boolean remove(Object o) {
return queue.remove(o);
}
// Only used for assertions
boolean contains(Object o) {
return queue.contains(o);
}
/** Add an entry to the queue and unlock it, in that order. */
void addAndUnlock(T entry, long weight) {
queue.add(entry, weight);
entry.unlock();
addAndUnlockCounter.incrementAndGet();
}
}

View File

@ -0,0 +1,104 @@
/*
* 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.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.lucene.tests.util.LuceneTestCase;
import org.apache.lucene.tests.util.TestUtil;
import org.apache.lucene.util.ThreadInterruptedException;
public class TestLockableConcurrentApproximatePriorityQueue extends LuceneTestCase {
private static class WeightedLock implements Lock {
private final Lock lock = new ReentrantLock();
long weight;
@Override
public void lock() {
lock.lock();
}
@Override
public void lockInterruptibly() throws InterruptedException {
throw new UnsupportedOperationException();
}
@Override
public boolean tryLock() {
return lock.tryLock();
}
@Override
public boolean tryLock(long time, TimeUnit unit) throws InterruptedException {
throw new UnsupportedOperationException();
}
@Override
public void unlock() {
lock.unlock();
}
@Override
public Condition newCondition() {
throw new UnsupportedOperationException();
}
}
public void testNeverReturnNullOnNonEmptyQueue() throws Exception {
final int iters = atLeast(10);
for (int iter = 0; iter < iters; ++iter) {
final int concurrency = TestUtil.nextInt(random(), 1, 16);
final LockableConcurrentApproximatePriorityQueue<WeightedLock> queue =
new LockableConcurrentApproximatePriorityQueue<>(concurrency);
final int numThreads = TestUtil.nextInt(random(), 2, 16);
final Thread[] threads = new Thread[numThreads];
final CountDownLatch startingGun = new CountDownLatch(1);
for (int t = 0; t < threads.length; ++t) {
threads[t] =
new Thread(
() -> {
try {
startingGun.await();
} catch (InterruptedException e) {
throw new ThreadInterruptedException(e);
}
WeightedLock lock = new WeightedLock();
lock.lock();
lock.weight++; // Simulate a DWPT whose RAM usage increases
queue.addAndUnlock(lock, lock.weight);
for (int i = 0; i < 10_000; ++i) {
lock = queue.lockAndPoll();
assertNotNull(lock);
queue.addAndUnlock(lock, lock.hashCode());
}
});
}
for (Thread thread : threads) {
thread.start();
}
startingGun.countDown();
for (Thread thread : threads) {
thread.join();
}
}
}
}