LUCENE-10292: Suggest: Fix FreeTextSuggester so that getCount() returned results consistent with lookup() during concurrent build()

Fix SuggestRebuildTestUtil to reliably surfice this kind of failure that was previously sporadic
This commit is contained in:
Chris Hostetter 2022-04-27 18:14:01 -07:00
parent 8d9a333fac
commit a8d86ea6e8
2 changed files with 57 additions and 28 deletions

View File

@ -140,7 +140,7 @@ public class FreeTextSuggester extends Lookup {
private final byte separator;
/** Number of entries the lookup was built with */
private long count = 0;
private volatile long count = 0;
/**
* The default character used to join multiple tokens into a single ngram token. The input tokens
@ -273,7 +273,7 @@ public class FreeTextSuggester extends Lookup {
IndexReader reader = null;
boolean success = false;
count = 0;
long newCount = 0;
try {
while (true) {
BytesRef surfaceForm = iterator.next();
@ -282,7 +282,7 @@ public class FreeTextSuggester extends Lookup {
}
field.setStringValue(surfaceForm.utf8ToString());
writer.addDocument(doc);
count++;
newCount++;
}
reader = DirectoryReader.open(writer);
@ -320,10 +320,13 @@ public class FreeTextSuggester extends Lookup {
fstCompiler.add(Util.toIntsRef(term, scratchInts), encodeWeight(termsEnum.totalTermFreq()));
}
fst = fstCompiler.compile();
if (fst == null) {
final FST<Long> newFst = fstCompiler.compile();
if (newFst == null) {
throw new IllegalArgumentException("need at least one suggestion");
}
fst = newFst;
count = newCount;
// System.out.println("FST: " + fst.getNodeCount() + " nodes");
/*

View File

@ -18,11 +18,13 @@ package org.apache.lucene.search.suggest;
import static org.junit.Assert.assertNull;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Set;
import java.util.concurrent.Semaphore;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.lucene.util.BytesRef;
/** Reusable Logic for confirming that Lookup impls can return suggestions during a 'rebuild' */
public final class SuggestRebuildTestUtil {
@ -57,25 +59,29 @@ public final class SuggestRebuildTestUtil {
// modify source data we're going to build from, and spin up background thread that
// will rebuild (slowly)
data.addAll(extraData);
final Semaphore rebuildGate = new Semaphore(0);
final Semaphore readyToCheck = new Semaphore(0);
final Semaphore readyToAdvance = new Semaphore(0);
final AtomicReference<Throwable> buildError = new AtomicReference<>();
final Thread rebuilder =
new Thread(
() -> {
try {
suggester.build(
new InputArrayIterator(new DelayedIterator<>(rebuildGate, data.iterator())));
new DelayedInputIterator(
readyToCheck, readyToAdvance, new InputArrayIterator(data.iterator())));
} catch (Throwable t) {
buildError.set(t);
}
});
rebuilder.start();
// at every stage of the slow rebuild, we should still be able to get our original suggestions
for (int i = 0; i < data.size(); i++) {
// (+1 iteration to ensure final next() call can return null)
for (int i = 0; i < data.size() + 1; i++) {
readyToCheck.acquire();
initialChecks.check(suggester);
rebuildGate.release();
readyToAdvance.release();
}
// once all the data is releasedfrom the iterator, the background rebuild should finish, and
// once all the data is released from the iterator, the background rebuild should finish, and
// suggest results
// should change
rebuilder.join();
@ -92,34 +98,54 @@ public final class SuggestRebuildTestUtil {
}
/**
* An iterator wrapper whose {@link Iterator#next} method will only return when a Semaphore permit
* is acquirable
* An InputArrayIterator wrapper whose {@link InputIterator#next} method releases on a Semaphore,
* and then acquires from a differnet Semaphore.
*/
private static final class DelayedIterator<E> implements Iterator<E> {
final Iterator<E> inner;
final Semaphore gate;
private static final class DelayedInputIterator implements InputIterator {
final Semaphore releaseOnNext;
final Semaphore acquireOnNext;
final InputIterator inner;
public DelayedIterator(final Semaphore gate, final Iterator<E> inner) {
assert null != gate;
public DelayedInputIterator(
final Semaphore releaseOnNext, final Semaphore acquireOnNext, final InputIterator inner) {
assert null != releaseOnNext;
assert null != acquireOnNext;
assert null != inner;
this.gate = gate;
this.releaseOnNext = releaseOnNext;
this.acquireOnNext = acquireOnNext;
this.inner = inner;
}
@Override
public boolean hasNext() {
return inner.hasNext();
}
@Override
public E next() {
gate.acquireUninterruptibly();
public BytesRef next() throws IOException {
releaseOnNext.release();
acquireOnNext.acquireUninterruptibly();
return inner.next();
}
@Override
public void remove() {
inner.remove();
public long weight() {
return inner.weight();
}
@Override
public BytesRef payload() {
return inner.payload();
}
@Override
public boolean hasPayloads() {
return inner.hasPayloads();
}
@Override
public Set<BytesRef> contexts() {
return inner.contexts();
}
@Override
public boolean hasContexts() {
return inner.hasContexts();
}
}
}