forked from apache/lucene
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Make sure
DocumentsWriterPerThread#getAndLock
never returns null
…
…on a non-empty queue. (apache#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 apache#12649 apache#12916
- Loading branch information
Showing
4 changed files
with
203 additions
and
25 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
70 changes: 70 additions & 0 deletions
70
lucene/core/src/java/org/apache/lucene/index/LockableConcurrentApproximatePriorityQueue.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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(); | ||
} | ||
} |
104 changes: 104 additions & 0 deletions
104
...core/src/test/org/apache/lucene/index/TestLockableConcurrentApproximatePriorityQueue.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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(); | ||
} | ||
} | ||
} | ||
} |