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:
Michael McCandless 2015-04-10 17:33:33 +00:00
parent de61fa6f3c
commit 6f1b9bb79a
7 changed files with 69 additions and 119 deletions

View File

@ -84,6 +84,10 @@ Other
* 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 =======================
New Features

View File

@ -328,12 +328,6 @@ class BufferedUpdatesStream implements Accountable {
}
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());
}
final int limit = updates.size();
@ -489,18 +483,13 @@ class BufferedUpdatesStream implements Accountable {
String field = null;
SegmentQueue queue = null;
while (true) {
BytesRef term;
boolean newField;
while ((term = iter.next()) != null) {
newField = iter.next();
if (newField) {
if (iter.field() != field) {
// field changed
field = iter.field();
if (field == null) {
// No more terms:
break;
}
queue = new SegmentQueue(numReaders);
@ -521,9 +510,8 @@ class BufferedUpdatesStream implements Accountable {
assert checkDeleteTerm(null);
}
// Get next term to delete
BytesRef term = iter.term();
assert checkDeleteTerm(term);
delTermVisitedCount++;
long delGen = iter.delGen();

View File

@ -17,24 +17,24 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefIterator;
// TODO: maybe TermsFilter could use this?
/** Iterates over terms in multiple fields, notifying the caller when a new field is started. */
interface FieldTermIterator {
/** 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
* field; if that returns null then the iteration ended. */
boolean next();
/** Iterates over terms in across multiple fields. The caller must
* check {@link #field} after each {@link #next} to see if the field
* changed, but {@code ==} can be used since the iterator
* implementation ensures it will use the same String instance for
* a given field. */
/** Returns current field, or null if the iteration ended. */
String field();
/** Returns current term. */
BytesRef term();
abstract class FieldTermIterator implements BytesRefIterator {
/** Returns current field. This method should not be called
* after iteration is done. Note that you may use == to
* detect a change in field. */
abstract String field();
/** 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
long delGen();
abstract long delGen();
}

View File

@ -24,7 +24,7 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.PriorityQueue;
/** Merges multiple {@link FieldTermIterator}s */
class MergedPrefixCodedTermsIterator implements FieldTermIterator {
class MergedPrefixCodedTermsIterator extends FieldTermIterator {
private static class TermMergeQueue extends PriorityQueue<TermIterator> {
TermMergeQueue(int size) {
@ -74,52 +74,50 @@ class MergedPrefixCodedTermsIterator implements FieldTermIterator {
String field;
@Override
public boolean next() {
public BytesRef next() {
if (termQueue.size() == 0) {
// Current field is done:
// No more terms in current field:
if (fieldQueue.size() == 0) {
// No more fields:
field = null;
return true;
return null;
}
// Transfer all iterators on the next field into the term queue:
TermIterator top = fieldQueue.pop();
termQueue.add(top);
assert top.field() != null;
field = top.field;
assert field != null;
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 true;
return termQueue.top().bytes;
} else {
TermIterator top = termQueue.top();
if (top.next()) {
// New field
if (top.next() == null) {
termQueue.pop();
} else if (top.field() != field) {
// Field changed
termQueue.pop();
if (top.field() != null) {
fieldQueue.add(top);
}
} else {
termQueue.updateTop();
}
if (termQueue.size() != 0) {
// Still terms left in this field
return false;
} else {
if (termQueue.size() == 0) {
// Recurse (just once) to go to next field:
return next();
}
}
}
@Override
public BytesRef term() {
} else {
// Still terms left in this field
return termQueue.top().bytes;
}
}
}
@Override
public String field() {

View File

@ -104,7 +104,7 @@ class PrefixCodedTerms implements Accountable {
}
}
public static class TermIterator implements FieldTermIterator {
public static class TermIterator extends FieldTermIterator {
final IndexInput input;
final BytesRefBuilder builder = new BytesRefBuilder();
final BytesRef bytes = builder.get();
@ -123,7 +123,7 @@ class PrefixCodedTerms implements Accountable {
}
@Override
public boolean next() {
public BytesRef next() {
if (input.getFilePointer() < end) {
try {
int code = input.readVInt();
@ -134,13 +134,13 @@ class PrefixCodedTerms implements Accountable {
int prefix = code >>> 1;
int suffix = input.readVInt();
readTermBytes(prefix, suffix);
return newField;
return bytes;
} catch (IOException e) {
throw new RuntimeException(e);
}
} else {
field = null;
return true;
return null;
}
}
@ -151,11 +151,6 @@ class PrefixCodedTerms implements Accountable {
builder.setLength(prefix + suffix);
}
@Override
public BytesRef term() {
return bytes;
}
@Override
public String field() {
return field;

View File

@ -79,17 +79,9 @@ public class TestDocumentsWriterDeleteQueue extends LuceneTestCase {
HashSet<Term> frozenSet = new HashSet<>();
BytesRefBuilder bytesRef = new BytesRefBuilder();
TermIterator iter = queue.freezeGlobalBuffer(null).termIterator();
String field = null;
while (true) {
boolean newField = iter.next();
if (newField) {
field = iter.field;
if (field == null) {
break;
}
}
while (iter.next() != null) {
bytesRef.copyBytes(iter.bytes);
frozenSet.add(new Term(field, bytesRef.toBytesRef()));
frozenSet.add(new Term(iter.field(), bytesRef.toBytesRef()));
}
assertEquals(uniqueValues, frozenSet);
assertEquals("num deletes must be 0 after freeze", 0, queue
@ -110,16 +102,12 @@ public class TestDocumentsWriterDeleteQueue extends LuceneTestCase {
queue.clear();
assertFalse(queue.anyChanges());
final int size = 200 + random().nextInt(500) * RANDOM_MULTIPLIER;
int termsSinceFreeze = 0;
int queriesSinceFreeze = 0;
for (int i = 0; i < size; i++) {
Term term = new Term("id", "" + i);
if (random().nextInt(10) == 0) {
queue.addDelete(new TermQuery(term));
queriesSinceFreeze++;
} else {
queue.addDelete(term);
termsSinceFreeze++;
}
assertTrue(queue.anyChanges());
if (random().nextInt(10) == 0) {
@ -218,17 +206,9 @@ public class TestDocumentsWriterDeleteQueue extends LuceneTestCase {
BytesRefBuilder builder = new BytesRefBuilder();
TermIterator iter = queue.freezeGlobalBuffer(null).termIterator();
String field = null;
while (true) {
boolean newField = iter.next();
if (newField) {
field = iter.field;
if (field == null) {
break;
}
}
while (iter.next() != null) {
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

View File

@ -33,8 +33,7 @@ public class TestPrefixCodedTerms extends LuceneTestCase {
PrefixCodedTerms.Builder b = new PrefixCodedTerms.Builder();
PrefixCodedTerms pb = b.finish();
TermIterator iter = pb.iterator();
assertTrue(iter.next());
assertNull(iter.field);
assertNull(iter.next());
}
public void testOne() {
@ -43,11 +42,10 @@ public class TestPrefixCodedTerms extends LuceneTestCase {
b.add(term);
PrefixCodedTerms pb = b.finish();
TermIterator iter = pb.iterator();
assertTrue(iter.next());
assertEquals("foo", iter.field);
assertNotNull(iter.next());
assertEquals("foo", iter.field());
assertEquals("bogus", iter.bytes.utf8ToString());
assertTrue(iter.next());
assertNull(iter.field);
assertNull(iter.next());
}
public void testRandom() {
@ -66,19 +64,10 @@ public class TestPrefixCodedTerms extends LuceneTestCase {
TermIterator iter = pb.iterator();
Iterator<Term> expected = terms.iterator();
String field = "";
//System.out.println("TEST: now iter");
while (true) {
boolean newField = iter.next();
//System.out.println(" newField=" + newField);
if (newField) {
field = iter.field;
if (field == null) {
break;
}
}
while (iter.next() != null) {
assertTrue(expected.hasNext());
assertEquals(expected.next(), new Term(field, iter.bytes));
assertEquals(expected.next(), new Term(iter.field(), iter.bytes));
}
assertFalse(expected.hasNext());
@ -97,13 +86,14 @@ public class TestPrefixCodedTerms extends LuceneTestCase {
PrefixCodedTerms pb2 = b2.finish();
MergedPrefixCodedTermsIterator merged = new MergedPrefixCodedTermsIterator(Arrays.asList(new PrefixCodedTerms[] {pb1, pb2}));
assertTrue(merged.next());
BytesRef term = merged.next();
assertNotNull(term);
assertEquals("foo", merged.field());
assertEquals("a", merged.term().utf8ToString());
assertFalse(merged.next());
assertEquals("b", merged.term().utf8ToString());
assertTrue(merged.next());
assertNull(merged.field());
assertEquals("a", term.utf8ToString());
term = merged.next();
assertNotNull(term);
assertEquals("b", term.utf8ToString());
assertNull(merged.next());
}
@SuppressWarnings({"unchecked","rawtypes"})
@ -137,25 +127,20 @@ public class TestPrefixCodedTerms extends LuceneTestCase {
String field = "";
BytesRef lastTerm = null;
while (true) {
if (actual.next()) {
BytesRef term;
while ((term = actual.next()) != null) {
if (field != actual.field()) {
field = actual.field();
if (field == null) {
break;
}
lastTerm = null;
//System.out.println("\nTEST: new field: " + field);
}
if (lastTerm != null && lastTerm.equals(actual.term())) {
if (lastTerm != null && lastTerm.equals(term)) {
continue;
}
//System.out.println("TEST: iter: field=" + field + " term=" + actual.term());
lastTerm = BytesRef.deepCopyOf(actual.term());
lastTerm = BytesRef.deepCopyOf(term);
assertTrue(expected.hasNext());
Term expectedTerm = expected.next();
assertEquals(expectedTerm, new Term(field, actual.term()));
assertEquals(expectedTerm, new Term(field, term));
}
assertFalse(expected.hasNext());