LUCENE-7876 avoid leafReader.fields

This commit is contained in:
David Smiley 2017-06-13 22:44:04 -04:00
parent a18a4ce245
commit f470bbcbdc
41 changed files with 141 additions and 138 deletions

View File

@ -145,6 +145,10 @@ Other
from methods that don't declare them ("sneaky throw" hack). (Robert Muir,
Uwe Schindler, Dawid Weiss)
* LUCENE-7876: Avoid calls to LeafReader.fields() and MultiFields.getFields()
that are trivially replaced by LeafReader.terms() and MultiFields.getTerms()
(David Smiley)
Improvements
* LUCENE-7841: Normalize ґ to г in Ukrainian analyzer. (Andriy Rysin via Dawid Weiss)

View File

@ -1192,7 +1192,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
for (String name : oldNames) {
Directory dir = oldIndexDirs.get(name);
IndexReader r = DirectoryReader.open(dir);
TermsEnum terms = MultiFields.getFields(r).terms("content").iterator();
TermsEnum terms = MultiFields.getTerms(r, "content").iterator();
BytesRef t = terms.next();
assertNotNull(t);

View File

@ -501,9 +501,8 @@ class BufferedUpdatesStream implements Accountable {
queue = new SegmentQueue(numReaders);
long segTermCount = 0;
for(int i=0;i<numReaders;i++) {
SegmentState state = segStates[i];
Terms terms = state.reader.fields().terms(field);
for (SegmentState state : segStates) {
Terms terms = state.reader.terms(field);
if (terms != null) {
segTermCount += terms.size();
state.termsEnum = terms.iterator();
@ -617,7 +616,6 @@ class BufferedUpdatesStream implements Accountable {
// DocValues updates
private synchronized void applyDocValuesUpdates(List<DocValuesUpdate> updates,
SegmentState segState, DocValuesFieldUpdates.Container dvUpdatesContainer) throws IOException {
Fields fields = segState.reader.fields();
// TODO: we can process the updates per DV field, from last to first so that
// if multiple terms affect same document for the same field, we add an update
@ -651,7 +649,7 @@ class BufferedUpdatesStream implements Accountable {
// if we change the code to process updates in terms order, enable this assert
// assert currentField == null || currentField.compareTo(term.field()) < 0;
currentField = term.field();
Terms terms = fields.terms(currentField);
Terms terms = segState.reader.terms(currentField);
if (terms != null) {
termsEnum = terms.iterator();
} else {

View File

@ -26,7 +26,6 @@ import java.util.Objects;
import java.util.Set;
import java.util.SortedSet;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
@ -224,8 +223,7 @@ public class TermInSetQuery extends Query implements Accountable {
private WeightOrDocIdSet rewrite(LeafReaderContext context) throws IOException {
final LeafReader reader = context.reader();
final Fields fields = reader.fields();
Terms terms = fields.terms(field);
Terms terms = reader.terms(field);
if (terms == null) {
return null;
}

View File

@ -55,7 +55,7 @@ public class TestBlockPostingsFormat extends BasePostingsFormatTestCase {
DirectoryReader r = DirectoryReader.open(w);
assertEquals(1, r.leaves().size());
FieldReader field = (FieldReader) r.leaves().get(0).reader().fields().terms("field");
FieldReader field = (FieldReader) r.leaves().get(0).reader().terms("field");
// We should see exactly two blocks: one root block (prefix empty string) and one block for z* terms (prefix z):
Stats stats = field.getStats();
assertEquals(0, stats.floorBlockCount);

View File

@ -91,7 +91,7 @@ public class Test2BDocs extends LuceneTestCase {
LeafReader reader = context.reader();
int lim = context.reader().maxDoc();
Terms terms = reader.fields().terms("f1");
Terms terms = reader.terms("f1");
for (int i=0; i<10000; i++) {
TermsEnum te = terms.iterator();
assertTrue( te.seekExact(term) );

View File

@ -249,14 +249,16 @@ public class TestDoc extends LuceneTestCase {
for (int i = 0; i < reader.numDocs(); i++)
out.println(reader.document(i));
Fields fields = reader.fields();
for (String field : fields) {
Terms terms = fields.terms(field);
for (FieldInfo fieldInfo : reader.getFieldInfos()) {
if (fieldInfo.getIndexOptions() == IndexOptions.NONE) {
continue;
}
Terms terms = reader.terms(fieldInfo.name);
assertNotNull(terms);
TermsEnum tis = terms.iterator();
while(tis.next() != null) {
out.print(" term=" + field + ":" + tis.term());
out.print(" term=" + fieldInfo.name + ":" + tis.term());
out.println(" DF=" + tis.docFreq());
PostingsEnum positions = tis.postings(null, PostingsEnum.POSITIONS);

View File

@ -17,10 +17,13 @@
package org.apache.lucene.index;
import org.apache.lucene.store.*;
import org.apache.lucene.analysis.*;
import org.apache.lucene.document.*;
import org.apache.lucene.util.*;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
public class TestFlex extends LuceneTestCase {
@ -70,7 +73,7 @@ public class TestFlex extends LuceneTestCase {
w.addDocument(doc);
w.forceMerge(1);
DirectoryReader r = w.getReader();
TermsEnum terms = getOnlyLeafReader(r).fields().terms("f").iterator();
TermsEnum terms = getOnlyLeafReader(r).terms("f").iterator();
assertTrue(terms.next() != null);
try {
assertEquals(0, terms.ord());

View File

@ -80,9 +80,7 @@ public class TestIndexReaderClose extends LuceneTestCase {
reader.getReaderCacheHelper().addClosedListener(new FaultyListener());
}
IllegalStateException expected = expectThrows(IllegalStateException.class, () -> {
reader.close();
});
IllegalStateException expected = expectThrows(IllegalStateException.class, () -> reader.close());
if (throwOnClose) {
assertEquals("BOOM!", expected.getMessage());
@ -90,9 +88,7 @@ public class TestIndexReaderClose extends LuceneTestCase {
assertEquals("GRRRRRRRRRRRR!", expected.getMessage());
}
expectThrows(AlreadyClosedException.class, () -> {
reader.fields();
});
expectThrows(AlreadyClosedException.class, () -> reader.terms("someField"));
if (random().nextBoolean()) {
reader.close(); // call it again

View File

@ -694,7 +694,7 @@ public class TestIndexWriter extends LuceneTestCase {
writer.close();
DirectoryReader reader = DirectoryReader.open(dir);
LeafReader subreader = getOnlyLeafReader(reader);
TermsEnum te = subreader.fields().terms("").iterator();
TermsEnum te = subreader.terms("").iterator();
assertEquals(new BytesRef("a"), te.next());
assertEquals(new BytesRef("b"), te.next());
assertEquals(new BytesRef("c"), te.next());
@ -715,7 +715,7 @@ public class TestIndexWriter extends LuceneTestCase {
writer.close();
DirectoryReader reader = DirectoryReader.open(dir);
LeafReader subreader = getOnlyLeafReader(reader);
TermsEnum te = subreader.fields().terms("").iterator();
TermsEnum te = subreader.terms("").iterator();
assertEquals(new BytesRef(""), te.next());
assertEquals(new BytesRef("a"), te.next());
assertEquals(new BytesRef("b"), te.next());

View File

@ -136,7 +136,7 @@ public class TestIndexWriterUnicode extends LuceneTestCase {
}
private void checkTermsOrder(IndexReader r, Set<String> allTerms, boolean isTop) throws IOException {
TermsEnum terms = MultiFields.getFields(r).terms("f").iterator();
TermsEnum terms = MultiFields.getTerms(r, "f").iterator();
BytesRefBuilder last = new BytesRefBuilder();

View File

@ -479,7 +479,7 @@ public class TestPayloads extends LuceneTestCase {
}
writer.close();
IndexReader reader = DirectoryReader.open(dir);
TermsEnum terms = MultiFields.getFields(reader).terms(field).iterator();
TermsEnum terms = MultiFields.getTerms(reader, field).iterator();
PostingsEnum tp = null;
while (terms.next() != null) {
String termText = terms.term().utf8ToString();
@ -602,7 +602,7 @@ public class TestPayloads extends LuceneTestCase {
field.setTokenStream(ts);
writer.addDocument(doc);
DirectoryReader reader = writer.getReader();
TermsEnum te = MultiFields.getFields(reader).terms("field").iterator();
TermsEnum te = MultiFields.getTerms(reader, "field").iterator();
assertTrue(te.seekExact(new BytesRef("withPayload")));
PostingsEnum de = te.postings(null, PostingsEnum.PAYLOADS);
de.nextDoc();

View File

@ -221,9 +221,7 @@ public class TestPerSegmentDeletes extends LuceneTestCase {
public int[] toDocsArray(Term term, Bits bits, IndexReader reader)
throws IOException {
Fields fields = MultiFields.getFields(reader);
Terms cterms = fields.terms(term.field);
TermsEnum ctermsEnum = cterms.iterator();
TermsEnum ctermsEnum = MultiFields.getTerms(reader, term.field).iterator();
if (ctermsEnum.seekExact(new BytesRef(term.text()))) {
PostingsEnum postingsEnum = TestUtil.docs(random(), ctermsEnum, null, PostingsEnum.NONE);
return toArray(postingsEnum);

View File

@ -291,7 +291,7 @@ public class TestPostingsOffsets extends LuceneTestCase {
// TODO: improve this
LeafReader sub = ctx.reader();
//System.out.println("\nsub=" + sub);
final TermsEnum termsEnum = sub.fields().terms("content").iterator();
final TermsEnum termsEnum = sub.terms("content").iterator();
PostingsEnum docs = null;
PostingsEnum docsAndPositions = null;
PostingsEnum docsAndPositionsAndOffsets = null;

View File

@ -57,7 +57,7 @@ public class TestSegmentTermDocs extends LuceneTestCase {
SegmentReader reader = new SegmentReader(info, Version.LATEST.major, newIOContext(random()));
assertTrue(reader != null);
TermsEnum terms = reader.fields().terms(DocHelper.TEXT_FIELD_2_KEY).iterator();
TermsEnum terms = reader.terms(DocHelper.TEXT_FIELD_2_KEY).iterator();
terms.seekCeil(new BytesRef("field"));
PostingsEnum termDocs = TestUtil.docs(random(), terms, null, PostingsEnum.FREQS);
if (termDocs.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {

View File

@ -19,14 +19,14 @@ package org.apache.lucene.index;
import java.io.IOException;
import org.apache.lucene.document.Field;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
public class TestSegmentTermEnum extends LuceneTestCase {
@ -80,7 +80,7 @@ public class TestSegmentTermEnum extends LuceneTestCase {
addDoc(writer, "aaa bbb");
writer.close();
LeafReader reader = getOnlyLeafReader(DirectoryReader.open(dir));
TermsEnum terms = reader.fields().terms("content").iterator();
TermsEnum terms = reader.terms("content").iterator();
assertNotNull(terms.next());
assertEquals("aaa", terms.term().utf8ToString());
assertNotNull(terms.next());

View File

@ -21,10 +21,13 @@ import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.lucene.util.*;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.*;
import org.apache.lucene.document.*;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
public class TestStressAdvance extends LuceneTestCase {
@ -74,7 +77,7 @@ public class TestStressAdvance extends LuceneTestCase {
bDocIDs.add(docID);
}
}
final TermsEnum te = getOnlyLeafReader(r).fields().terms("field").iterator();
final TermsEnum te = getOnlyLeafReader(r).terms("field").iterator();
PostingsEnum de = null;
for(int iter2=0;iter2<10;iter2++) {

View File

@ -18,7 +18,17 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.*;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.TreeSet;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
@ -738,7 +748,7 @@ public class TestTermsEnum extends LuceneTestCase {
DirectoryReader r = w.getReader();
w.close();
LeafReader sub = getOnlyLeafReader(r);
Terms terms = sub.fields().terms("field");
Terms terms = sub.terms("field");
Automaton automaton = new RegExp(".*", RegExp.NONE).toAutomaton();
CompiledAutomaton ca = new CompiledAutomaton(automaton, false, false);
TermsEnum te = terms.intersect(ca, null);
@ -792,7 +802,7 @@ public class TestTermsEnum extends LuceneTestCase {
DirectoryReader r = w.getReader();
w.close();
LeafReader sub = getOnlyLeafReader(r);
Terms terms = sub.fields().terms("field");
Terms terms = sub.terms("field");
Automaton automaton = new RegExp(".*d", RegExp.NONE).toAutomaton();
CompiledAutomaton ca = new CompiledAutomaton(automaton, false, false);
@ -846,7 +856,7 @@ public class TestTermsEnum extends LuceneTestCase {
DirectoryReader r = w.getReader();
w.close();
LeafReader sub = getOnlyLeafReader(r);
Terms terms = sub.fields().terms("field");
Terms terms = sub.terms("field");
Automaton automaton = new RegExp(".*", RegExp.NONE).toAutomaton(); // accept ALL
CompiledAutomaton ca = new CompiledAutomaton(automaton, false, false);
@ -986,7 +996,7 @@ public class TestTermsEnum extends LuceneTestCase {
w.addDocument(doc);
IndexReader r = w.getReader();
assertEquals(1, r.leaves().size());
TermsEnum te = r.leaves().get(0).reader().fields().terms("field").iterator();
TermsEnum te = r.leaves().get(0).reader().terms("field").iterator();
for(int i=0;i<=termCount;i++) {
assertTrue("term '" + termsList.get(i).utf8ToString() + "' should exist but doesn't", te.seekExact(termsList.get(i)));
}
@ -1007,9 +1017,8 @@ public class TestTermsEnum extends LuceneTestCase {
doc.add(newStringField("field", "foobar", Field.Store.NO));
w.addDocument(doc);
IndexReader r = w.getReader();
Fields fields = MultiFields.getFields(r);
Terms terms = MultiFields.getTerms(r, "field");
CompiledAutomaton automaton = new CompiledAutomaton(new RegExp("do_not_match_anything").toAutomaton());
Terms terms = fields.terms("field");
String message = expectThrows(IllegalArgumentException.class, () -> {terms.intersect(automaton, null);}).getMessage();
assertEquals("please use CompiledAutomaton.getTermsEnum instead", message);
r.close();

View File

@ -71,7 +71,7 @@ public class TestMultiPhraseQuery extends LuceneTestCase {
// this TermEnum gives "piccadilly", "pie" and "pizza".
String prefix = "pi";
TermsEnum te = MultiFields.getFields(reader).terms("body").iterator();
TermsEnum te = MultiFields.getTerms(reader,"body").iterator();
te.seekCeil(new BytesRef(prefix));
do {
String s = te.term().utf8ToString();

View File

@ -73,7 +73,7 @@ public class TestPhrasePrefixQuery extends LuceneTestCase {
// this TermEnum gives "piccadilly", "pie" and "pizza".
String prefix = "pi";
TermsEnum te = MultiFields.getFields(reader).terms("body").iterator();
TermsEnum te = MultiFields.getTerms(reader, "body").iterator();
te.seekCeil(new BytesRef(prefix));
do {
String s = te.term().utf8ToString();

View File

@ -61,7 +61,7 @@ public class TestSameScoresWithThreads extends LuceneTestCase {
w.close();
final IndexSearcher s = newSearcher(r);
Terms terms = MultiFields.getFields(r).terms("body");
Terms terms = MultiFields.getTerms(r, "body");
int termCount = 0;
TermsEnum termsEnum = terms.iterator();
while(termsEnum.next() != null) {

View File

@ -131,7 +131,7 @@ public class TestMemoryIndex extends LuceneTestCase {
mi.addField("field", "some terms be here", analyzer);
IndexSearcher searcher = mi.createSearcher();
LeafReader reader = (LeafReader) searcher.getIndexReader();
TermsEnum terms = reader.fields().terms("field").iterator();
TermsEnum terms = reader.terms("field").iterator();
terms.seekExact(0);
assertEquals("be", terms.term().utf8ToString());
TestUtil.checkReader(reader);

View File

@ -53,7 +53,6 @@ import org.apache.lucene.document.SortedSetDocValuesField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.*;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.queryparser.classic.QueryParser;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.IndexSearcher;
@ -67,8 +66,8 @@ import org.apache.lucene.search.spans.SpanOrQuery;
import org.apache.lucene.search.spans.SpanQuery;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.ByteBlockPool.Allocator;
import org.apache.lucene.util.ByteBlockPool;
import org.apache.lucene.util.ByteBlockPool.Allocator;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LineFileDocs;
@ -171,7 +170,7 @@ public class TestMemoryIndexAgainstRAMDir extends BaseTokenStreamTestCase {
private void duellReaders(CompositeReader other, LeafReader memIndexReader)
throws IOException {
Fields memFields = memIndexReader.fields();
Fields memFields = memIndexReader.getTermVectors(0);
for (String field : MultiFields.getFields(other)) {
Terms memTerms = memFields.terms(field);
Terms iwTerms = memIndexReader.terms(field);

View File

@ -22,7 +22,6 @@ import java.util.Collections;
import java.util.List;
import java.util.Objects;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.Term;
@ -214,11 +213,10 @@ public class CommonTermsQuery extends Query {
Term[] queryTerms) throws IOException {
TermsEnum termsEnum = null;
for (LeafReaderContext context : leaves) {
final Fields fields = context.reader().fields();
for (int i = 0; i < queryTerms.length; i++) {
Term term = queryTerms[i];
TermContext termContext = contextArray[i];
final Terms terms = fields.terms(term.field());
final Terms terms = context.reader().terms(term.field());
if (terms == null) {
// field does not exist
continue;

View File

@ -19,9 +19,8 @@ package org.apache.lucene.queries.function.valuesource;
import java.io.IOException;
import java.util.Map;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.queries.function.FunctionValues;
@ -50,8 +49,7 @@ public class TFValueSource extends TermFreqValueSource {
@Override
public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
Fields fields = readerContext.reader().fields();
final Terms terms = fields.terms(indexedField);
final Terms terms = readerContext.reader().terms(indexedField);
IndexSearcher searcher = (IndexSearcher)context.get("searcher");
final TFIDFSimilarity similarity = IDFValueSource.asTFIDF(searcher.getSimilarity(true), indexedField);
if (similarity == null) {

View File

@ -19,9 +19,8 @@ package org.apache.lucene.queries.function.valuesource;
import java.io.IOException;
import java.util.Map;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.queries.function.FunctionValues;
@ -48,8 +47,7 @@ public class TermFreqValueSource extends DocFreqValueSource {
@Override
public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
Fields fields = readerContext.reader().fields();
final Terms terms = fields.terms(indexedField);
final Terms terms = readerContext.reader().terms(indexedField);
return new IntDocValues(this) {
PostingsEnum docs ;

View File

@ -37,12 +37,12 @@ import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.index.ConcurrentMergeScheduler;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.MergeScheduler;
import org.apache.lucene.index.PerThreadPKLookup;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.TieredMergePolicy;
@ -75,7 +75,7 @@ public class TestIDVersionPostingsFormat extends LuceneTestCase {
doc.add(makeIDField("id1", 110));
w.addDocument(doc);
IndexReader r = w.getReader();
IDVersionSegmentTermsEnum termsEnum = (IDVersionSegmentTermsEnum) r.leaves().get(0).reader().fields().terms("id").iterator();
IDVersionSegmentTermsEnum termsEnum = (IDVersionSegmentTermsEnum) r.leaves().get(0).reader().terms("id").iterator();
assertTrue(termsEnum.seekExact(new BytesRef("id0"), 50));
assertTrue(termsEnum.seekExact(new BytesRef("id0"), 100));
assertFalse(termsEnum.seekExact(new BytesRef("id0"), 101));

View File

@ -18,7 +18,14 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.nio.file.Path;
import java.util.*;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.WeakHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
@ -349,8 +356,7 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas
}
if (s.getIndexReader().numDocs() > 0) {
smokeTestSearcher(s);
Fields fields = MultiFields.getFields(s.getIndexReader());
Terms terms = fields.terms("body");
Terms terms = MultiFields.getTerms(s.getIndexReader(), "body");
if (terms == null) {
continue;
}

View File

@ -16,10 +16,6 @@
*/
package org.apache.solr.handler.admin;
import static org.apache.lucene.index.IndexOptions.DOCS;
import static org.apache.lucene.index.IndexOptions.DOCS_AND_FREQS;
import static org.apache.lucene.index.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
@ -83,6 +79,10 @@ import org.apache.solr.update.SolrIndexWriter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.lucene.index.IndexOptions.DOCS;
import static org.apache.lucene.index.IndexOptions.DOCS_AND_FREQS;
import static org.apache.lucene.index.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS;
/**
* This handler exposes the internal lucene index. It is inspired by and
* modeled on Luke, the Lucene Index Browser by Andrzej Bialecki.
@ -366,7 +366,7 @@ public class LukeRequestHandler extends RequestHandlerBase
if (sfield != null && schema.isDynamicField(sfield.getName()) && schema.getDynamicPattern(sfield.getName()) != null) {
fieldMap.add("dynamicBase", schema.getDynamicPattern(sfield.getName()));
}
Terms terms = reader.fields().terms(fieldName);
Terms terms = reader.terms(fieldName);
if (terms == null) { // Not indexed, so we need to report what we can (it made it through the fl param if specified)
finfo.add( fieldName, fieldMap );
continue;

View File

@ -38,7 +38,6 @@ import com.carrotsearch.hppc.IntIntHashMap;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
@ -665,9 +664,7 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
protected void doSetNextReader(LeafReaderContext context) throws IOException {
//convert the ids to Lucene doc ids, the ordSet and termValues needs to be the same size as the number of elevation docs we have
ordSet.clear();
Fields fields = context.reader().fields();
if (fields == null) return;
Terms terms = fields.terms(idField);
Terms terms = context.reader().terms(idField);
if (terms == null) return;
TermsEnum termsEnum = terms.iterator();
BytesRefBuilder term = new BytesRefBuilder();

View File

@ -15,22 +15,6 @@
* limitations under the License.
*/
package org.apache.solr.handler.component;
import org.apache.lucene.index.*;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.CharsRefBuilder;
import org.apache.lucene.util.StringHelper;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.*;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.schema.FieldType;
import org.apache.solr.schema.StrField;
import org.apache.solr.request.SimpleFacets.CountPair;
import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.util.BoundedTreeSet;
import org.apache.solr.client.solrj.response.TermsResponse;
import java.io.IOException;
import java.util.Arrays;
@ -39,6 +23,32 @@ import java.util.List;
import java.util.Locale;
import java.util.regex.Pattern;
import org.apache.lucene.index.IndexReaderContext;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.TermContext;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.CharsRefBuilder;
import org.apache.lucene.util.StringHelper;
import org.apache.solr.client.solrj.response.TermsResponse;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.ShardParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.params.TermsParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.common.util.StrUtils;
import org.apache.solr.request.SimpleFacets.CountPair;
import org.apache.solr.schema.FieldType;
import org.apache.solr.schema.StrField;
import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.util.BoundedTreeSet;
/**
* Return TermEnum information, useful for things like auto suggest.
*
@ -138,13 +148,12 @@ public class TermsComponent extends SearchComponent {
final LeafReader indexReader = rb.req.getSearcher().getSlowAtomicReader();
Fields lfields = indexReader.fields();
for (String field : fields) {
NamedList<Integer> fieldTerms = new NamedList<>();
termsResult.add(field, fieldTerms);
Terms terms = lfields.terms(field);
Terms terms = indexReader.terms(field);
if (terms == null) {
// field does not exist
continue;
@ -562,10 +571,9 @@ public class TermsComponent extends SearchComponent {
Term[] queryTerms) throws IOException {
TermsEnum termsEnum = null;
for (LeafReaderContext context : topReaderContext.leaves()) {
final Fields fields = context.reader().fields();
for (int i = 0; i < queryTerms.length; i++) {
Term term = queryTerms[i];
final Terms terms = fields.terms(term.field());
final Terms terms = context.reader().terms(term.field());
if (terms == null) {
// field does not exist
continue;

View File

@ -245,7 +245,7 @@ public class DefaultSolrHighlighter extends SolrHighlighter implements PluginInf
try {
// It'd be nice to know if payloads are on the tokenStream but the presence of the attribute isn't a good
// indicator.
final Terms terms = request.getSearcher().getSlowAtomicReader().fields().terms(fieldName);
final Terms terms = request.getSearcher().getSlowAtomicReader().terms(fieldName);
if (terms != null) {
defaultPayloads = terms.hasPayloads();
}

View File

@ -37,7 +37,6 @@ import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.TimeUnit;
import java.util.function.Predicate;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.MultiPostingsEnum;
@ -935,8 +934,7 @@ public class SimpleFacets {
prefixTermBytes = new BytesRef(indexedPrefix);
}
Fields fields = r.fields();
Terms terms = fields==null ? null : fields.terms(field);
Terms terms = r.terms(field);
TermsEnum termsEnum = null;
SolrIndexSearcher.DocsEnumState deState = null;
BytesRef term = null;

View File

@ -21,7 +21,6 @@ import java.util.Iterator;
import java.util.List;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
@ -169,8 +168,7 @@ public class DocSetUtil {
for (LeafReaderContext ctx : leaves) {
assert leaves.get(ctx.ord) == ctx;
LeafReader r = ctx.reader();
Fields f = r.fields();
Terms t = f.terms(field);
Terms t = r.terms(field);
if (t == null) continue; // field is missing
TermsEnum te = t.iterator();
if (te.seekExact(termVal)) {

View File

@ -69,7 +69,7 @@ public abstract class Filter extends Query {
* represent the whole underlying index i.e. if the index has more than
* one segment the given reader only represents a single segment.
* The provided context is always an atomic context, so you can call
* {@link org.apache.lucene.index.LeafReader#fields()}
* {@link org.apache.lucene.index.LeafReader#terms(String)}
* on the context's reader, for example.
*
* @param acceptDocs

View File

@ -20,7 +20,6 @@ package org.apache.solr.search;
import java.io.IOException;
import java.util.Map;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.Terms;
@ -56,8 +55,7 @@ public class FloatPayloadValueSource extends ValueSource {
@Override
public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
Fields fields = readerContext.reader().fields();
final Terms terms = fields.terms(indexedField);
final Terms terms = readerContext.reader().terms(indexedField);
FunctionValues defaultValues = defaultValueSource.getValues(context, readerContext);

View File

@ -18,15 +18,19 @@
package org.apache.solr.search;
import org.apache.lucene.index.Fields;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.TermContext;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.BulkScorer;
import org.apache.lucene.search.ConstantScoreQuery;
@ -49,11 +53,6 @@ import org.apache.solr.common.params.SolrParams;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.schema.FieldType;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
/**
* The GraphTermsQuery builds a disjunction query from a list of terms. The terms are first filtered by the maxDocFreq parameter.
* This allows graph traversals to skip traversing high frequency nodes which is often desirable from a performance standpoint.
@ -220,8 +219,7 @@ public class GraphTermsQParserPlugin extends QParserPlugin {
private WeightOrDocIdSet rewrite(LeafReaderContext context) throws IOException {
final LeafReader reader = context.reader();
final Fields fields = reader.fields();
Terms terms = fields.terms(field);
Terms terms = reader.terms(field);
if(terms == null) {
return new WeightOrDocIdSet(new BitDocIdSet(new FixedBitSet(reader.maxDoc()), 0));
}

View File

@ -23,8 +23,8 @@ import java.util.Arrays;
import java.util.List;
import java.util.Objects;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.MultiPostingsEnum;
import org.apache.lucene.index.PostingsEnum;
@ -316,11 +316,11 @@ class JoinQuery extends Query {
fastForRandomSet = new HashDocSet(sset.getDocs(), 0, sset.size());
}
Fields fromFields = fromSearcher.getSlowAtomicReader().fields();
Fields toFields = fromSearcher==toSearcher ? fromFields : toSearcher.getSlowAtomicReader().fields();
if (fromFields == null) return DocSet.EMPTY;
Terms terms = fromFields.terms(fromField);
Terms toTerms = toFields.terms(toField);
LeafReader fromReader = fromSearcher.getSlowAtomicReader();
LeafReader toReader = fromSearcher==toSearcher ? fromReader : toSearcher.getSlowAtomicReader();
Terms terms = fromReader.terms(fromField);
Terms toTerms = toReader.terms(toField);
if (terms == null || toTerms==null) return DocSet.EMPTY;
String prefixStr = TrieField.getMainValuePrefix(fromSearcher.getSchema().getFieldType(fromField));
BytesRef prefix = prefixStr == null ? null : new BytesRef(prefixStr);

View File

@ -22,7 +22,6 @@ import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.MultiPostingsEnum;
import org.apache.lucene.index.PostingsEnum;
@ -156,8 +155,7 @@ class FacetFieldProcessorByEnumTermsStream extends FacetFieldProcessor implement
}
}
Fields fields = fcontext.searcher.getSlowAtomicReader().fields();
Terms terms = fields == null ? null : fields.terms(sf.getName());
Terms terms = fcontext.searcher.getSlowAtomicReader().terms(sf.getName());
termsEnum = null;
deState = null;

View File

@ -22,7 +22,7 @@ import java.util.Arrays;
import java.util.List;
import java.util.concurrent.TimeUnit;
import org.apache.lucene.codecs.PostingsFormat; // javadocs
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.DocValuesType;
import org.apache.lucene.index.FieldInfo;
@ -595,7 +595,7 @@ public class DocTermOrds implements Accountable {
public OrdWrappedTermsEnum(LeafReader reader) throws IOException {
assert indexedTermsArray != null;
assert 0 != indexedTermsArray.length;
termsEnum = reader.fields().terms(field).iterator();
termsEnum = reader.terms(field).iterator();
}
@Override

View File

@ -22,7 +22,6 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.index.CodecReader;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.FilterCodecReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.LeafReader;
@ -165,8 +164,7 @@ public class SolrIndexSplitter {
}
Bits liveDocs = reader.getLiveDocs();
Fields fields = reader.fields();
Terms terms = fields==null ? null : fields.terms(field.getName());
Terms terms = reader.terms(field.getName());
TermsEnum termsEnum = terms==null ? null : terms.iterator();
if (termsEnum == null) return docSets;