mirror of https://github.com/apache/lucene.git
LUCENE-6315: simplify the private iterator Lucene uses when resolving deleted terms -> docids
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1672710 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
de61fa6f3c
commit
6f1b9bb79a
|
@ -84,6 +84,10 @@ Other
|
||||||
|
|
||||||
* LUCENE-5439: Add 'ant jacoco' build target. (Robert Muir)
|
* LUCENE-5439: Add 'ant jacoco' build target. (Robert Muir)
|
||||||
|
|
||||||
|
* LUCENE-6315: Simplify the private iterator Lucene uses internally
|
||||||
|
when resolving deleted terms to matched docids. (Robert Muir, Adrien
|
||||||
|
Grand, Mike McCandless)
|
||||||
|
|
||||||
======================= Lucene 5.1.0 =======================
|
======================= Lucene 5.1.0 =======================
|
||||||
|
|
||||||
New Features
|
New Features
|
||||||
|
|
|
@ -328,12 +328,6 @@ class BufferedUpdatesStream implements Accountable {
|
||||||
}
|
}
|
||||||
|
|
||||||
if (infoStream.isEnabled("BD")) {
|
if (infoStream.isEnabled("BD")) {
|
||||||
Directory dir;
|
|
||||||
if (segmentInfos.size() > 0) {
|
|
||||||
dir = segmentInfos.info(0).info.dir;
|
|
||||||
} else {
|
|
||||||
dir = null;
|
|
||||||
}
|
|
||||||
infoStream.message("BD", "prune sis=" + segmentInfos + " minGen=" + minGen + " packetCount=" + updates.size());
|
infoStream.message("BD", "prune sis=" + segmentInfos + " minGen=" + minGen + " packetCount=" + updates.size());
|
||||||
}
|
}
|
||||||
final int limit = updates.size();
|
final int limit = updates.size();
|
||||||
|
@ -489,18 +483,13 @@ class BufferedUpdatesStream implements Accountable {
|
||||||
String field = null;
|
String field = null;
|
||||||
SegmentQueue queue = null;
|
SegmentQueue queue = null;
|
||||||
|
|
||||||
while (true) {
|
BytesRef term;
|
||||||
|
|
||||||
boolean newField;
|
while ((term = iter.next()) != null) {
|
||||||
|
|
||||||
newField = iter.next();
|
if (iter.field() != field) {
|
||||||
|
// field changed
|
||||||
if (newField) {
|
|
||||||
field = iter.field();
|
field = iter.field();
|
||||||
if (field == null) {
|
|
||||||
// No more terms:
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
|
|
||||||
queue = new SegmentQueue(numReaders);
|
queue = new SegmentQueue(numReaders);
|
||||||
|
|
||||||
|
@ -521,9 +510,8 @@ class BufferedUpdatesStream implements Accountable {
|
||||||
assert checkDeleteTerm(null);
|
assert checkDeleteTerm(null);
|
||||||
}
|
}
|
||||||
|
|
||||||
// Get next term to delete
|
|
||||||
BytesRef term = iter.term();
|
|
||||||
assert checkDeleteTerm(term);
|
assert checkDeleteTerm(term);
|
||||||
|
|
||||||
delTermVisitedCount++;
|
delTermVisitedCount++;
|
||||||
|
|
||||||
long delGen = iter.delGen();
|
long delGen = iter.delGen();
|
||||||
|
|
|
@ -17,24 +17,24 @@ package org.apache.lucene.index;
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRefIterator;
|
||||||
|
|
||||||
// TODO: maybe TermsFilter could use this?
|
// TODO: maybe TermsFilter could use this?
|
||||||
|
|
||||||
/** Iterates over terms in multiple fields, notifying the caller when a new field is started. */
|
/** Iterates over terms in across multiple fields. The caller must
|
||||||
interface FieldTermIterator {
|
* check {@link #field} after each {@link #next} to see if the field
|
||||||
/** Advances to the next term, returning true if it's in a new field or there are no more terms. Call {@link #field} to see which
|
* changed, but {@code ==} can be used since the iterator
|
||||||
* field; if that returns null then the iteration ended. */
|
* implementation ensures it will use the same String instance for
|
||||||
boolean next();
|
* a given field. */
|
||||||
|
|
||||||
/** Returns current field, or null if the iteration ended. */
|
abstract class FieldTermIterator implements BytesRefIterator {
|
||||||
String field();
|
/** Returns current field. This method should not be called
|
||||||
|
* after iteration is done. Note that you may use == to
|
||||||
/** Returns current term. */
|
* detect a change in field. */
|
||||||
BytesRef term();
|
abstract String field();
|
||||||
|
|
||||||
/** Del gen of the current term. */
|
/** Del gen of the current term. */
|
||||||
// TODO: this is really per-iterator not per term, but when we use MergedPrefixCodedTermsIterator we need to know which iterator we are on
|
// TODO: this is really per-iterator not per term, but when we use MergedPrefixCodedTermsIterator we need to know which iterator we are on
|
||||||
long delGen();
|
abstract long delGen();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -24,7 +24,7 @@ import org.apache.lucene.util.BytesRef;
|
||||||
import org.apache.lucene.util.PriorityQueue;
|
import org.apache.lucene.util.PriorityQueue;
|
||||||
|
|
||||||
/** Merges multiple {@link FieldTermIterator}s */
|
/** Merges multiple {@link FieldTermIterator}s */
|
||||||
class MergedPrefixCodedTermsIterator implements FieldTermIterator {
|
class MergedPrefixCodedTermsIterator extends FieldTermIterator {
|
||||||
|
|
||||||
private static class TermMergeQueue extends PriorityQueue<TermIterator> {
|
private static class TermMergeQueue extends PriorityQueue<TermIterator> {
|
||||||
TermMergeQueue(int size) {
|
TermMergeQueue(int size) {
|
||||||
|
@ -74,53 +74,51 @@ class MergedPrefixCodedTermsIterator implements FieldTermIterator {
|
||||||
String field;
|
String field;
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean next() {
|
public BytesRef next() {
|
||||||
if (termQueue.size() == 0) {
|
if (termQueue.size() == 0) {
|
||||||
// Current field is done:
|
// No more terms in current field:
|
||||||
if (fieldQueue.size() == 0) {
|
if (fieldQueue.size() == 0) {
|
||||||
// No more fields:
|
// No more fields:
|
||||||
field = null;
|
field = null;
|
||||||
return true;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
// Transfer all iterators on the next field into the term queue:
|
// Transfer all iterators on the next field into the term queue:
|
||||||
TermIterator top = fieldQueue.pop();
|
TermIterator top = fieldQueue.pop();
|
||||||
termQueue.add(top);
|
termQueue.add(top);
|
||||||
assert top.field() != null;
|
field = top.field;
|
||||||
|
assert field != null;
|
||||||
|
|
||||||
while (fieldQueue.size() != 0 && fieldQueue.top().field.equals(top.field)) {
|
while (fieldQueue.size() != 0 && fieldQueue.top().field.equals(top.field)) {
|
||||||
termQueue.add(fieldQueue.pop());
|
TermIterator iter = fieldQueue.pop();
|
||||||
|
assert iter.field.equals(field);
|
||||||
|
// TODO: a little bit evil; we do this so we can == on field down below:
|
||||||
|
iter.field = field;
|
||||||
|
termQueue.add(iter);
|
||||||
}
|
}
|
||||||
|
|
||||||
field = top.field;
|
return termQueue.top().bytes;
|
||||||
return true;
|
|
||||||
} else {
|
} else {
|
||||||
TermIterator top = termQueue.top();
|
TermIterator top = termQueue.top();
|
||||||
if (top.next()) {
|
if (top.next() == null) {
|
||||||
// New field
|
|
||||||
termQueue.pop();
|
termQueue.pop();
|
||||||
if (top.field() != null) {
|
} else if (top.field() != field) {
|
||||||
fieldQueue.add(top);
|
// Field changed
|
||||||
}
|
termQueue.pop();
|
||||||
|
fieldQueue.add(top);
|
||||||
} else {
|
} else {
|
||||||
termQueue.updateTop();
|
termQueue.updateTop();
|
||||||
}
|
}
|
||||||
|
if (termQueue.size() == 0) {
|
||||||
if (termQueue.size() != 0) {
|
|
||||||
// Still terms left in this field
|
|
||||||
return false;
|
|
||||||
} else {
|
|
||||||
// Recurse (just once) to go to next field:
|
// Recurse (just once) to go to next field:
|
||||||
return next();
|
return next();
|
||||||
|
} else {
|
||||||
|
// Still terms left in this field
|
||||||
|
return termQueue.top().bytes;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public BytesRef term() {
|
|
||||||
return termQueue.top().bytes;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String field() {
|
public String field() {
|
||||||
return field;
|
return field;
|
||||||
|
|
|
@ -104,7 +104,7 @@ class PrefixCodedTerms implements Accountable {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class TermIterator implements FieldTermIterator {
|
public static class TermIterator extends FieldTermIterator {
|
||||||
final IndexInput input;
|
final IndexInput input;
|
||||||
final BytesRefBuilder builder = new BytesRefBuilder();
|
final BytesRefBuilder builder = new BytesRefBuilder();
|
||||||
final BytesRef bytes = builder.get();
|
final BytesRef bytes = builder.get();
|
||||||
|
@ -123,7 +123,7 @@ class PrefixCodedTerms implements Accountable {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean next() {
|
public BytesRef next() {
|
||||||
if (input.getFilePointer() < end) {
|
if (input.getFilePointer() < end) {
|
||||||
try {
|
try {
|
||||||
int code = input.readVInt();
|
int code = input.readVInt();
|
||||||
|
@ -134,13 +134,13 @@ class PrefixCodedTerms implements Accountable {
|
||||||
int prefix = code >>> 1;
|
int prefix = code >>> 1;
|
||||||
int suffix = input.readVInt();
|
int suffix = input.readVInt();
|
||||||
readTermBytes(prefix, suffix);
|
readTermBytes(prefix, suffix);
|
||||||
return newField;
|
return bytes;
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new RuntimeException(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
field = null;
|
field = null;
|
||||||
return true;
|
return null;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -151,11 +151,6 @@ class PrefixCodedTerms implements Accountable {
|
||||||
builder.setLength(prefix + suffix);
|
builder.setLength(prefix + suffix);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public BytesRef term() {
|
|
||||||
return bytes;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String field() {
|
public String field() {
|
||||||
return field;
|
return field;
|
||||||
|
|
|
@ -79,17 +79,9 @@ public class TestDocumentsWriterDeleteQueue extends LuceneTestCase {
|
||||||
HashSet<Term> frozenSet = new HashSet<>();
|
HashSet<Term> frozenSet = new HashSet<>();
|
||||||
BytesRefBuilder bytesRef = new BytesRefBuilder();
|
BytesRefBuilder bytesRef = new BytesRefBuilder();
|
||||||
TermIterator iter = queue.freezeGlobalBuffer(null).termIterator();
|
TermIterator iter = queue.freezeGlobalBuffer(null).termIterator();
|
||||||
String field = null;
|
while (iter.next() != null) {
|
||||||
while (true) {
|
|
||||||
boolean newField = iter.next();
|
|
||||||
if (newField) {
|
|
||||||
field = iter.field;
|
|
||||||
if (field == null) {
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
bytesRef.copyBytes(iter.bytes);
|
bytesRef.copyBytes(iter.bytes);
|
||||||
frozenSet.add(new Term(field, bytesRef.toBytesRef()));
|
frozenSet.add(new Term(iter.field(), bytesRef.toBytesRef()));
|
||||||
}
|
}
|
||||||
assertEquals(uniqueValues, frozenSet);
|
assertEquals(uniqueValues, frozenSet);
|
||||||
assertEquals("num deletes must be 0 after freeze", 0, queue
|
assertEquals("num deletes must be 0 after freeze", 0, queue
|
||||||
|
@ -110,16 +102,12 @@ public class TestDocumentsWriterDeleteQueue extends LuceneTestCase {
|
||||||
queue.clear();
|
queue.clear();
|
||||||
assertFalse(queue.anyChanges());
|
assertFalse(queue.anyChanges());
|
||||||
final int size = 200 + random().nextInt(500) * RANDOM_MULTIPLIER;
|
final int size = 200 + random().nextInt(500) * RANDOM_MULTIPLIER;
|
||||||
int termsSinceFreeze = 0;
|
|
||||||
int queriesSinceFreeze = 0;
|
|
||||||
for (int i = 0; i < size; i++) {
|
for (int i = 0; i < size; i++) {
|
||||||
Term term = new Term("id", "" + i);
|
Term term = new Term("id", "" + i);
|
||||||
if (random().nextInt(10) == 0) {
|
if (random().nextInt(10) == 0) {
|
||||||
queue.addDelete(new TermQuery(term));
|
queue.addDelete(new TermQuery(term));
|
||||||
queriesSinceFreeze++;
|
|
||||||
} else {
|
} else {
|
||||||
queue.addDelete(term);
|
queue.addDelete(term);
|
||||||
termsSinceFreeze++;
|
|
||||||
}
|
}
|
||||||
assertTrue(queue.anyChanges());
|
assertTrue(queue.anyChanges());
|
||||||
if (random().nextInt(10) == 0) {
|
if (random().nextInt(10) == 0) {
|
||||||
|
@ -218,17 +206,9 @@ public class TestDocumentsWriterDeleteQueue extends LuceneTestCase {
|
||||||
BytesRefBuilder builder = new BytesRefBuilder();
|
BytesRefBuilder builder = new BytesRefBuilder();
|
||||||
|
|
||||||
TermIterator iter = queue.freezeGlobalBuffer(null).termIterator();
|
TermIterator iter = queue.freezeGlobalBuffer(null).termIterator();
|
||||||
String field = null;
|
while (iter.next() != null) {
|
||||||
while (true) {
|
|
||||||
boolean newField = iter.next();
|
|
||||||
if (newField) {
|
|
||||||
field = iter.field;
|
|
||||||
if (field == null) {
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
builder.copyBytes(iter.bytes);
|
builder.copyBytes(iter.bytes);
|
||||||
frozenSet.add(new Term(field, builder.toBytesRef()));
|
frozenSet.add(new Term(iter.field(), builder.toBytesRef()));
|
||||||
}
|
}
|
||||||
|
|
||||||
assertEquals("num deletes must be 0 after freeze", 0, queue
|
assertEquals("num deletes must be 0 after freeze", 0, queue
|
||||||
|
|
|
@ -33,8 +33,7 @@ public class TestPrefixCodedTerms extends LuceneTestCase {
|
||||||
PrefixCodedTerms.Builder b = new PrefixCodedTerms.Builder();
|
PrefixCodedTerms.Builder b = new PrefixCodedTerms.Builder();
|
||||||
PrefixCodedTerms pb = b.finish();
|
PrefixCodedTerms pb = b.finish();
|
||||||
TermIterator iter = pb.iterator();
|
TermIterator iter = pb.iterator();
|
||||||
assertTrue(iter.next());
|
assertNull(iter.next());
|
||||||
assertNull(iter.field);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testOne() {
|
public void testOne() {
|
||||||
|
@ -43,11 +42,10 @@ public class TestPrefixCodedTerms extends LuceneTestCase {
|
||||||
b.add(term);
|
b.add(term);
|
||||||
PrefixCodedTerms pb = b.finish();
|
PrefixCodedTerms pb = b.finish();
|
||||||
TermIterator iter = pb.iterator();
|
TermIterator iter = pb.iterator();
|
||||||
assertTrue(iter.next());
|
assertNotNull(iter.next());
|
||||||
assertEquals("foo", iter.field);
|
assertEquals("foo", iter.field());
|
||||||
assertEquals("bogus", iter.bytes.utf8ToString());
|
assertEquals("bogus", iter.bytes.utf8ToString());
|
||||||
assertTrue(iter.next());
|
assertNull(iter.next());
|
||||||
assertNull(iter.field);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testRandom() {
|
public void testRandom() {
|
||||||
|
@ -66,19 +64,10 @@ public class TestPrefixCodedTerms extends LuceneTestCase {
|
||||||
|
|
||||||
TermIterator iter = pb.iterator();
|
TermIterator iter = pb.iterator();
|
||||||
Iterator<Term> expected = terms.iterator();
|
Iterator<Term> expected = terms.iterator();
|
||||||
String field = "";
|
|
||||||
//System.out.println("TEST: now iter");
|
//System.out.println("TEST: now iter");
|
||||||
while (true) {
|
while (iter.next() != null) {
|
||||||
boolean newField = iter.next();
|
|
||||||
//System.out.println(" newField=" + newField);
|
|
||||||
if (newField) {
|
|
||||||
field = iter.field;
|
|
||||||
if (field == null) {
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
assertTrue(expected.hasNext());
|
assertTrue(expected.hasNext());
|
||||||
assertEquals(expected.next(), new Term(field, iter.bytes));
|
assertEquals(expected.next(), new Term(iter.field(), iter.bytes));
|
||||||
}
|
}
|
||||||
|
|
||||||
assertFalse(expected.hasNext());
|
assertFalse(expected.hasNext());
|
||||||
|
@ -97,13 +86,14 @@ public class TestPrefixCodedTerms extends LuceneTestCase {
|
||||||
PrefixCodedTerms pb2 = b2.finish();
|
PrefixCodedTerms pb2 = b2.finish();
|
||||||
|
|
||||||
MergedPrefixCodedTermsIterator merged = new MergedPrefixCodedTermsIterator(Arrays.asList(new PrefixCodedTerms[] {pb1, pb2}));
|
MergedPrefixCodedTermsIterator merged = new MergedPrefixCodedTermsIterator(Arrays.asList(new PrefixCodedTerms[] {pb1, pb2}));
|
||||||
assertTrue(merged.next());
|
BytesRef term = merged.next();
|
||||||
|
assertNotNull(term);
|
||||||
assertEquals("foo", merged.field());
|
assertEquals("foo", merged.field());
|
||||||
assertEquals("a", merged.term().utf8ToString());
|
assertEquals("a", term.utf8ToString());
|
||||||
assertFalse(merged.next());
|
term = merged.next();
|
||||||
assertEquals("b", merged.term().utf8ToString());
|
assertNotNull(term);
|
||||||
assertTrue(merged.next());
|
assertEquals("b", term.utf8ToString());
|
||||||
assertNull(merged.field());
|
assertNull(merged.next());
|
||||||
}
|
}
|
||||||
|
|
||||||
@SuppressWarnings({"unchecked","rawtypes"})
|
@SuppressWarnings({"unchecked","rawtypes"})
|
||||||
|
@ -137,25 +127,20 @@ public class TestPrefixCodedTerms extends LuceneTestCase {
|
||||||
String field = "";
|
String field = "";
|
||||||
|
|
||||||
BytesRef lastTerm = null;
|
BytesRef lastTerm = null;
|
||||||
|
BytesRef term;
|
||||||
while (true) {
|
while ((term = actual.next()) != null) {
|
||||||
if (actual.next()) {
|
if (field != actual.field()) {
|
||||||
field = actual.field();
|
field = actual.field();
|
||||||
if (field == null) {
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
lastTerm = null;
|
lastTerm = null;
|
||||||
//System.out.println("\nTEST: new field: " + field);
|
|
||||||
}
|
}
|
||||||
if (lastTerm != null && lastTerm.equals(actual.term())) {
|
if (lastTerm != null && lastTerm.equals(term)) {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
//System.out.println("TEST: iter: field=" + field + " term=" + actual.term());
|
lastTerm = BytesRef.deepCopyOf(term);
|
||||||
lastTerm = BytesRef.deepCopyOf(actual.term());
|
|
||||||
assertTrue(expected.hasNext());
|
assertTrue(expected.hasNext());
|
||||||
|
|
||||||
Term expectedTerm = expected.next();
|
Term expectedTerm = expected.next();
|
||||||
assertEquals(expectedTerm, new Term(field, actual.term()));
|
assertEquals(expectedTerm, new Term(field, term));
|
||||||
}
|
}
|
||||||
|
|
||||||
assertFalse(expected.hasNext());
|
assertFalse(expected.hasNext());
|
||||||
|
|
Loading…
Reference in New Issue