LUCENE-8513: Remove MultiFields.getFields

SlowCompositeReaderWrapper now works with MultiTerms directly
This commit is contained in:
David Smiley 2018-10-01 10:39:12 -04:00
parent 86e00405b0
commit fe844c739b
17 changed files with 179 additions and 184 deletions

View File

@ -84,6 +84,9 @@ API Changes
* LUCENE-8498: LowerCaseTokenizer has been removed, and CharTokenizer no longer
takes a normalizer function. (Alan Woodward)
* LUCENE-8513: MultiFields.getFields is now removed. Please avoid this class,
and Fields in general, when possible. (David Smiley)
Changes in Runtime Behavior
* LUCENE-8333: Switch MoreLikeThis.setMaxDocFreqPct to use maxDoc instead of

View File

@ -21,6 +21,7 @@ import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Path;
import java.text.Collator;
import java.util.Collection;
import java.util.List;
import java.util.Locale;
@ -37,15 +38,15 @@ import org.apache.lucene.benchmark.byTask.tasks.WriteLineDocTask;
import org.apache.lucene.collation.CollationKeyAnalyzer;
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.index.LogDocMergePolicy;
import org.apache.lucene.index.LogMergePolicy;
import org.apache.lucene.index.MultiFields;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.index.SerialMergeScheduler;
import org.apache.lucene.index.Terms;
@ -373,13 +374,13 @@ public class TestPerfTasksLogic extends BenchmarkTestCase {
int totalTokenCount2 = 0;
Fields fields = MultiFields.getFields(reader);
Collection<String> fields = MultiFields.getIndexedFields(reader);
for (String fieldName : fields) {
if (fieldName.equals(DocMaker.ID_FIELD) || fieldName.equals(DocMaker.DATE_MSEC_FIELD) || fieldName.equals(DocMaker.TIME_SEC_FIELD)) {
continue;
}
Terms terms = fields.terms(fieldName);
Terms terms = MultiFields.getTerms(reader, fieldName);
if (terms == null) {
continue;
}

View File

@ -20,6 +20,7 @@ package org.apache.lucene.codecs.perfield;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
@ -27,7 +28,7 @@ import java.util.IdentityHashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.ServiceLoader; // javadocs
import java.util.ServiceLoader;
import java.util.Set;
import java.util.TreeMap;
import java.util.TreeSet;
@ -41,13 +42,13 @@ import org.apache.lucene.index.Fields;
import org.apache.lucene.index.FilterLeafReader.FilterFields;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.MultiFields;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.MergedIterator;
import org.apache.lucene.util.RamUsageEstimator;
/**
@ -150,7 +151,10 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
@Override
public void merge(MergeState mergeState, NormsProducer norms) throws IOException {
Map<PostingsFormat, FieldsGroup> formatToGroups = buildFieldsGroupMapping(new MultiFields(mergeState.fieldsProducers, null));
@SuppressWarnings("unchecked") Iterable<String> indexedFieldNames = () ->
new MergedIterator<>(true,
Arrays.stream(mergeState.fieldsProducers).map(FieldsProducer::iterator).toArray(Iterator[]::new));
Map<PostingsFormat, FieldsGroup> formatToGroups = buildFieldsGroupMapping(indexedFieldNames);
// Merge postings
PerFieldMergeState pfMergeState = new PerFieldMergeState(mergeState);
@ -173,7 +177,7 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
}
}
private Map<PostingsFormat, FieldsGroup> buildFieldsGroupMapping(Fields fields) {
private Map<PostingsFormat, FieldsGroup> buildFieldsGroupMapping(Iterable<String> indexedFieldNames) {
// Maps a PostingsFormat instance to the suffix it
// should use
Map<PostingsFormat,FieldsGroup> formatToGroups = new HashMap<>();
@ -182,7 +186,7 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
Map<String,Integer> suffixes = new HashMap<>();
// Assign field -> PostingsFormat
for(String field : fields) {
for(String field : indexedFieldNames) {
FieldInfo fieldInfo = writeState.fieldInfos.fieldInfo(field);
final PostingsFormat format = getPostingsFormatForField(field);

View File

@ -20,7 +20,6 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@ -47,45 +46,13 @@ import org.apache.lucene.util.MergedIterator;
* atomic leaves and then operate per-LeafReader,
* instead of using this class.
*
* @lucene.experimental
* @lucene.internal
*/
public final class MultiFields extends Fields {
private final Fields[] subs;
private final ReaderSlice[] subSlices;
private final Map<String,Terms> terms = new ConcurrentHashMap<>();
/** Returns a single {@link Fields} instance for this
* reader, merging fields/terms/docs/positions on the
* fly. This method will return null if the reader
* has no postings.
*
* <p><b>NOTE</b>: this is a slow way to access postings.
* It's better to get the sub-readers and iterate through them
* yourself. */
public static Fields getFields(IndexReader reader) throws IOException {
final List<LeafReaderContext> leaves = reader.leaves();
switch (leaves.size()) {
case 1:
// already an atomic reader / reader with one leave
return new LeafReaderFields(leaves.get(0).reader());
default:
final List<Fields> fields = new ArrayList<>(leaves.size());
final List<ReaderSlice> slices = new ArrayList<>(leaves.size());
for (final LeafReaderContext ctx : leaves) {
final LeafReader r = ctx.reader();
final Fields f = new LeafReaderFields(r);
fields.add(f);
slices.add(new ReaderSlice(ctx.docBase, r.maxDoc(), fields.size()-1));
}
if (fields.size() == 1) {
return fields.get(0);
} else {
return new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
slices.toArray(ReaderSlice.EMPTY_ARRAY));
}
}
}
/** Returns a single {@link Bits} instance for this
* reader, merging live Documents on the
* fly. This method will return null if the reader
@ -133,7 +100,7 @@ public final class MultiFields extends Fields {
Terms subTerms = ctx.reader().terms(field);
if (subTerms != null) {
termsPerLeaf.add(subTerms);
slicePerLeaf.add(new ReaderSlice(ctx.docBase, r.maxDoc(), leafIdx - 1));
slicePerLeaf.add(new ReaderSlice(ctx.docBase, r.maxDoc(), leafIdx));
}
}
@ -285,36 +252,5 @@ public final class MultiFields extends Fields {
.collect(Collectors.toSet());
}
private static class LeafReaderFields extends Fields {
private final LeafReader leafReader;
private final List<String> indexedFields;
LeafReaderFields(LeafReader leafReader) {
this.leafReader = leafReader;
this.indexedFields = new ArrayList<>();
for (FieldInfo fieldInfo : leafReader.getFieldInfos()) {
if (fieldInfo.getIndexOptions() != IndexOptions.NONE) {
indexedFields.add(fieldInfo.name);
}
}
Collections.sort(indexedFields);
}
@Override
public Iterator<String> iterator() {
return Collections.unmodifiableList(indexedFields).iterator();
}
@Override
public int size() {
return indexedFields.size();
}
@Override
public Terms terms(String field) throws IOException {
return leafReader.terms(field);
}
}
}

View File

@ -27,6 +27,7 @@ import java.util.Iterator;
import java.util.List;
import java.util.Random;
import java.util.Set;
import java.util.TreeSet;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
@ -618,19 +619,19 @@ public class TestDirectoryReader extends LuceneTestCase {
}
// check dictionary and posting lists
Fields fields1 = MultiFields.getFields(index1);
Fields fields2 = MultiFields.getFields(index2);
TreeSet<String> fields1 = new TreeSet<>(MultiFields.getIndexedFields(index1));
TreeSet<String> fields2 = new TreeSet<>(MultiFields.getIndexedFields(index2));
Iterator<String> fenum2 = fields2.iterator();
for (String field1 : fields1) {
assertEquals("Different fields", field1, fenum2.next());
Terms terms1 = fields1.terms(field1);
Terms terms1 = MultiFields.getTerms(index1, field1);
if (terms1 == null) {
assertNull(fields2.terms(field1));
assertNull(MultiFields.getTerms(index2, field1));
continue;
}
TermsEnum enum1 = terms1.iterator();
Terms terms2 = fields2.terms(field1);
Terms terms2 = MultiFields.getTerms(index2, field1);
assertNotNull(terms2);
TermsEnum enum2 = terms2.iterator();

View File

@ -17,6 +17,8 @@
package org.apache.lucene.index;
import java.util.Collection;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.search.DocIdSetIterator;
@ -57,9 +59,9 @@ public class TestDocCount extends LuceneTestCase {
}
private void verifyCount(IndexReader ir) throws Exception {
Fields fields = MultiFields.getFields(ir);
final Collection<String> fields = MultiFields.getIndexedFields(ir);
for (String field : fields) {
Terms terms = fields.terms(field);
Terms terms = MultiFields.getTerms(ir, field);
if (terms == null) {
continue;
}

View File

@ -280,11 +280,10 @@ public class TestParallelCompositeReader extends LuceneTestCase {
assertNull(pr.document(0).get("f3"));
assertNull(pr.document(0).get("f4"));
// check that fields are there
Fields slow = MultiFields.getFields(pr);
assertNotNull(slow.terms("f1"));
assertNotNull(slow.terms("f2"));
assertNotNull(slow.terms("f3"));
assertNotNull(slow.terms("f4"));
assertNotNull(MultiFields.getTerms(pr, "f1"));
assertNotNull(MultiFields.getTerms(pr, "f2"));
assertNotNull(MultiFields.getTerms(pr, "f3"));
assertNotNull(MultiFields.getTerms(pr, "f4"));
pr.close();
// no stored fields at all
@ -296,11 +295,10 @@ public class TestParallelCompositeReader extends LuceneTestCase {
assertNull(pr.document(0).get("f3"));
assertNull(pr.document(0).get("f4"));
// check that fields are there
slow = MultiFields.getFields(pr);
assertNull(slow.terms("f1"));
assertNull(slow.terms("f2"));
assertNotNull(slow.terms("f3"));
assertNotNull(slow.terms("f4"));
assertNull(MultiFields.getTerms(pr, "f1"));
assertNull(MultiFields.getTerms(pr, "f2"));
assertNotNull(MultiFields.getTerms(pr, "f3"));
assertNotNull(MultiFields.getTerms(pr, "f4"));
pr.close();
// without overlapping
@ -312,11 +310,10 @@ public class TestParallelCompositeReader extends LuceneTestCase {
assertNull(pr.document(0).get("f3"));
assertNull(pr.document(0).get("f4"));
// check that fields are there
slow = MultiFields.getFields(pr);
assertNull(slow.terms("f1"));
assertNull(slow.terms("f2"));
assertNotNull(slow.terms("f3"));
assertNotNull(slow.terms("f4"));
assertNull(MultiFields.getTerms(pr, "f1"));
assertNull(MultiFields.getTerms(pr, "f2"));
assertNotNull(MultiFields.getTerms(pr, "f3"));
assertNotNull(MultiFields.getTerms(pr, "f4"));
pr.close();
// no main readers

View File

@ -115,9 +115,9 @@ public class TestSegmentReader extends LuceneTestCase {
}
public void testTerms() throws IOException {
Fields fields = MultiFields.getFields(reader);
final Collection<String> fields = MultiFields.getIndexedFields(reader);
for (String field : fields) {
Terms terms = fields.terms(field);
Terms terms = MultiFields.getTerms(reader, field);
assertNotNull(terms);
TermsEnum termsEnum = terms.iterator();
while(termsEnum.next() != null) {

View File

@ -310,16 +310,13 @@ public class TestStressIndexing2 extends LuceneTestCase {
int[] r2r1 = new int[r2.maxDoc()]; // r2 id to r1 id mapping
// create mapping from id2 space to id2 based on idField
final Fields f1 = MultiFields.getFields(r1);
if (f1 == null) {
// make sure r2 is empty
assertNull(MultiFields.getFields(r2));
if (MultiFields.getIndexedFields(r1).isEmpty()) {
assertTrue(MultiFields.getIndexedFields(r2).isEmpty());
return;
}
final Terms terms1 = f1.terms(idField);
final Terms terms1 = MultiFields.getTerms(r1, idField);
if (terms1 == null) {
assertTrue(MultiFields.getFields(r2) == null ||
MultiFields.getFields(r2).terms(idField) == null);
assertTrue(MultiFields.getTerms(r2, idField) == null);
return;
}
final TermsEnum termsEnum = terms1.iterator();
@ -327,9 +324,8 @@ public class TestStressIndexing2 extends LuceneTestCase {
final Bits liveDocs1 = MultiFields.getLiveDocs(r1);
final Bits liveDocs2 = MultiFields.getLiveDocs(r2);
Fields fields = MultiFields.getFields(r2);
Terms terms2 = fields.terms(idField);
if (fields.size() == 0 || terms2 == null) {
Terms terms2 = MultiFields.getTerms(r2, idField);
if (terms2 == null) {
// make sure r1 is in fact empty (eg has only all
// deleted docs):
Bits liveDocs = MultiFields.getLiveDocs(r1);
@ -463,10 +459,8 @@ public class TestStressIndexing2 extends LuceneTestCase {
// Verify postings
//System.out.println("TEST: create te1");
final Fields fields1 = MultiFields.getFields(r1);
final Iterator<String> fields1Enum = fields1.iterator();
final Fields fields2 = MultiFields.getFields(r2);
final Iterator<String> fields2Enum = fields2.iterator();
final Iterator<String> fields1Enum = MultiFields.getIndexedFields(r1).stream().sorted().iterator();
final Iterator<String> fields2Enum = MultiFields.getIndexedFields(r2).stream().sorted().iterator();
String field1=null, field2=null;
@ -490,7 +484,7 @@ public class TestStressIndexing2 extends LuceneTestCase {
break;
}
field1 = fields1Enum.next();
Terms terms = fields1.terms(field1);
Terms terms = MultiFields.getTerms(r1, field1);
if (terms == null) {
continue;
}
@ -526,7 +520,7 @@ public class TestStressIndexing2 extends LuceneTestCase {
break;
}
field2 = fields2Enum.next();
Terms terms = fields2.terms(field2);
Terms terms = MultiFields.getTerms(r2, field2);
if (terms == null) {
continue;
}

View File

@ -17,6 +17,8 @@
package org.apache.lucene.index;
import java.util.Collection;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.store.Directory;
@ -73,10 +75,9 @@ public class TestSumDocFreq extends LuceneTestCase {
private void assertSumDocFreq(IndexReader ir) throws Exception {
// compute sumDocFreq across all fields
Fields fields = MultiFields.getFields(ir);
final Collection<String> fields = MultiFields.getIndexedFields(ir);
for (String f : fields) {
Terms terms = fields.terms(f);
Terms terms = MultiFields.getTerms(ir, f);
long sumDocFreq = terms.getSumDocFreq();
if (sumDocFreq == -1) {
if (VERBOSE) {

View File

@ -171,7 +171,7 @@ public class TestMemoryIndexAgainstRAMDir extends BaseTokenStreamTestCase {
private void duellReaders(CompositeReader other, LeafReader memIndexReader)
throws IOException {
Fields memFields = memIndexReader.getTermVectors(0);
for (String field : MultiFields.getFields(other)) {
for (String field : MultiFields.getIndexedFields(other)) {
Terms memTerms = memFields.terms(field);
Terms iwTerms = memIndexReader.terms(field);
if (iwTerms == null) {

View File

@ -16,23 +16,23 @@
*/
package org.apache.lucene.misc;
import java.io.IOException;
import java.nio.file.Paths;
import java.util.Collection;
import java.util.Comparator;
import java.util.Locale;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.MultiFields;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.util.PriorityQueue;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.PriorityQueue;
import org.apache.lucene.util.SuppressForbidden;
import java.io.IOException;
import java.nio.file.Paths;
import java.util.Comparator;
import java.util.Locale;
/**
* <code>HighFreqTerms</code> class extracts the top n most frequent terms
* (by document frequency) from an existing Lucene index and reports their
@ -107,13 +107,13 @@ public class HighFreqTerms {
tiq = new TermStatsQueue(numTerms, comparator);
tiq.fill(field, termsEnum);
} else {
Fields fields = MultiFields.getFields(reader);
Collection<String> fields = MultiFields.getIndexedFields(reader);
if (fields.size() == 0) {
throw new RuntimeException("no fields found for this index");
}
tiq = new TermStatsQueue(numTerms, comparator);
for (String fieldName : fields) {
Terms terms = fields.terms(fieldName);
Terms terms = MultiFields.getTerms(reader, fieldName);
if (terms != null) {
tiq.fill(fieldName, terms.iterator());
}

View File

@ -25,9 +25,11 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.IdentityHashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeSet;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.MockAnalyzer;
@ -357,7 +359,25 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
};
try (FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(writeState)) {
consumer.write(MultiFields.getFields(oneDocReader), fakeNorms);
final Fields fields = new Fields() {
TreeSet<String> indexedFields = new TreeSet<>(MultiFields.getIndexedFields(oneDocReader));
@Override
public Iterator<String> iterator() {
return indexedFields.iterator();
}
@Override
public Terms terms(String field) throws IOException {
return oneDocReader.terms(field);
}
@Override
public int size() {
return indexedFields.size();
}
};
consumer.write(fields, fakeNorms);
IOUtils.close(consumer);
IOUtils.close(consumer);
}

View File

@ -1950,7 +1950,7 @@ public abstract class LuceneTestCase extends Assert {
public void assertReaderEquals(String info, IndexReader leftReader, IndexReader rightReader) throws IOException {
assertReaderStatisticsEquals(info, leftReader, rightReader);
assertFieldsEquals(info, leftReader, MultiFields.getFields(leftReader), MultiFields.getFields(rightReader), true);
assertTermsEquals(info, leftReader, rightReader, true);
assertNormsEquals(info, leftReader, rightReader);
assertStoredFieldsEquals(info, leftReader, rightReader);
assertTermVectorsEquals(info, leftReader, rightReader);
@ -1974,33 +1974,13 @@ public abstract class LuceneTestCase extends Assert {
/**
* Fields api equivalency
*/
public void assertFieldsEquals(String info, IndexReader leftReader, Fields leftFields, Fields rightFields, boolean deep) throws IOException {
// Fields could be null if there are no postings,
// but then it must be null for both
if (leftFields == null || rightFields == null) {
assertNull(info, leftFields);
assertNull(info, rightFields);
return;
}
assertFieldStatisticsEquals(info, leftFields, rightFields);
Iterator<String> leftEnum = leftFields.iterator();
Iterator<String> rightEnum = rightFields.iterator();
while (leftEnum.hasNext()) {
String field = leftEnum.next();
assertEquals(info, field, rightEnum.next());
assertTermsEquals(info, leftReader, leftFields.terms(field), rightFields.terms(field), deep);
}
assertFalse(rightEnum.hasNext());
}
public void assertTermsEquals(String info, IndexReader leftReader, IndexReader rightReader, boolean deep) throws IOException {
Set<String> leftFields = new HashSet<>(MultiFields.getIndexedFields(leftReader));
Set<String> rightFields = new HashSet<>(MultiFields.getIndexedFields(rightReader));
assertEquals(info, leftFields, rightFields);
/**
* checks that top-level statistics on Fields are the same
*/
public void assertFieldStatisticsEquals(String info, Fields leftFields, Fields rightFields) throws IOException {
if (leftFields.size() != -1 && rightFields.size() != -1) {
assertEquals(info, leftFields.size(), rightFields.size());
for (String field : leftFields) {
assertTermsEquals(info, leftReader, MultiFields.getTerms(leftReader, field), MultiFields.getTerms(rightReader, field), deep);
}
}
@ -2331,15 +2311,9 @@ public abstract class LuceneTestCase extends Assert {
* checks that norms are the same across all fields
*/
public void assertNormsEquals(String info, IndexReader leftReader, IndexReader rightReader) throws IOException {
Fields leftFields = MultiFields.getFields(leftReader);
Fields rightFields = MultiFields.getFields(rightReader);
// Fields could be null if there are no postings,
// but then it must be null for both
if (leftFields == null || rightFields == null) {
assertNull(info, leftFields);
assertNull(info, rightFields);
return;
}
Set<String> leftFields = new HashSet<>(MultiFields.getIndexedFields(leftReader));
Set<String> rightFields = new HashSet<>(MultiFields.getIndexedFields(rightReader));
assertEquals(info, leftFields, rightFields);
for (String field : leftFields) {
NumericDocValues leftNorms = MultiDocValues.getNormValues(leftReader, field);
@ -2407,7 +2381,26 @@ public abstract class LuceneTestCase extends Assert {
for (int i = 0; i < leftReader.maxDoc(); i++) {
Fields leftFields = leftReader.getTermVectors(i);
Fields rightFields = rightReader.getTermVectors(i);
assertFieldsEquals(info, leftReader, leftFields, rightFields, rarely());
// Fields could be null if there are no postings,
// but then it must be null for both
if (leftFields == null || rightFields == null) {
assertNull(info, leftFields);
assertNull(info, rightFields);
return;
}
if (leftFields.size() != -1 && rightFields.size() != -1) {
assertEquals(info, leftFields.size(), rightFields.size());
}
Iterator<String> leftEnum = leftFields.iterator();
Iterator<String> rightEnum = rightFields.iterator();
while (leftEnum.hasNext()) {
String field = leftEnum.next();
assertEquals(info, field, rightEnum.next());
assertTermsEquals(info, leftReader, leftFields.terms(field), rightFields.terms(field), rarely());
}
assertFalse(rightEnum.hasNext());
}
}

View File

@ -82,6 +82,8 @@ Other Changes
* SOLR-12652: Remove SolrMetricManager.overridableRegistryName method (Peter Somogyi via David Smiley)
* LUCENE-8513: SlowCompositeReaderWrapper now uses MultiTerms directly instead of MultiFields (David Smiley)
================== 7.6.0 ==================
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

View File

@ -19,6 +19,7 @@ package org.apache.solr.index;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.lucene.index.*;
import org.apache.lucene.index.MultiDocValues.MultiSortedDocValues;
@ -45,9 +46,15 @@ import org.apache.lucene.util.Version;
public final class SlowCompositeReaderWrapper extends LeafReader {
private final CompositeReader in;
private final Fields fields;
private final LeafMetaData metaData;
final Map<String,Terms> cachedTerms = new ConcurrentHashMap<>();
// TODO: consider ConcurrentHashMap ?
// TODO: this could really be a weak map somewhere else on the coreCacheKey,
// but do we really need to optimize slow-wrapper any more?
final Map<String,OrdinalMap> cachedOrdMaps = new HashMap<>();
/** This method is sugar for getting an {@link LeafReader} from
* an {@link IndexReader} of any kind. If the reader is already atomic,
* it is returned unchanged, otherwise wrapped by this class.
@ -62,9 +69,7 @@ public final class SlowCompositeReaderWrapper extends LeafReader {
}
SlowCompositeReaderWrapper(CompositeReader reader) throws IOException {
super();
in = reader;
fields = MultiFields.getFields(in);
in.registerParentReader(this);
if (reader.leaves().isEmpty()) {
metaData = new LeafMetaData(Version.LATEST.major, Version.LATEST, null);
@ -104,25 +109,38 @@ public final class SlowCompositeReaderWrapper extends LeafReader {
@Override
public Terms terms(String field) throws IOException {
ensureOpen();
return fields.terms(field);
try {
return cachedTerms.computeIfAbsent(field, f -> {
try {
return MultiFields.getTerms(in, f);
} catch (IOException e) { // yuck! ...sigh... checked exceptions with built-in lambdas are a pain
throw new RuntimeException("unwrapMe", e);
}
});
} catch (RuntimeException e) {
if (e.getMessage().equals("unwrapMe") && e.getCause() instanceof IOException) {
throw (IOException) e.getCause();
}
throw e;
}
}
@Override
public NumericDocValues getNumericDocValues(String field) throws IOException {
ensureOpen();
return MultiDocValues.getNumericValues(in, field);
return MultiDocValues.getNumericValues(in, field); // TODO cache?
}
@Override
public BinaryDocValues getBinaryDocValues(String field) throws IOException {
ensureOpen();
return MultiDocValues.getBinaryValues(in, field);
return MultiDocValues.getBinaryValues(in, field); // TODO cache?
}
@Override
public SortedNumericDocValues getSortedNumericDocValues(String field) throws IOException {
ensureOpen();
return MultiDocValues.getSortedNumericValues(in, field);
return MultiDocValues.getSortedNumericValues(in, field); // TODO cache?
}
@Override
@ -210,15 +228,11 @@ public final class SlowCompositeReaderWrapper extends LeafReader {
starts[size] = maxDoc();
return new MultiDocValues.MultiSortedSetDocValues(values, starts, map, cost);
}
// TODO: this could really be a weak map somewhere else on the coreCacheKey,
// but do we really need to optimize slow-wrapper any more?
final Map<String,OrdinalMap> cachedOrdMaps = new HashMap<>();
@Override
public NumericDocValues getNormValues(String field) throws IOException {
ensureOpen();
return MultiDocValues.getNormValues(in, field);
return MultiDocValues.getNormValues(in, field); // TODO cache?
}
@Override
@ -248,19 +262,19 @@ public final class SlowCompositeReaderWrapper extends LeafReader {
@Override
public Bits getLiveDocs() {
ensureOpen();
return MultiFields.getLiveDocs(in);
return MultiFields.getLiveDocs(in); // TODO cache?
}
@Override
public PointValues getPointValues(String field) {
ensureOpen();
return null;
return null; // because not supported. Throw UOE?
}
@Override
public FieldInfos getFieldInfos() {
ensureOpen();
return MultiFields.getMergedFieldInfos(in);
return MultiFields.getMergedFieldInfos(in); // TODO cache?
}
@Override

View File

@ -22,15 +22,18 @@ import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.SortedDocValuesField;
import org.apache.lucene.document.SortedSetDocValuesField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.MultiDocValues.MultiSortedDocValues;
import org.apache.lucene.index.MultiDocValues.MultiSortedSetDocValues;
import org.apache.lucene.index.MultiTerms;
import org.apache.lucene.index.NoMergePolicy;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
@ -120,4 +123,28 @@ public class TestSlowCompositeReaderWrapper extends LuceneTestCase {
w.close();
dir.close();
}
public void testTermsAreCached() throws IOException {
Directory dir = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), dir, newIndexWriterConfig().setMergePolicy(NoMergePolicy.INSTANCE));
Document doc = new Document();
doc.add(new TextField("text", "hello world", Field.Store.NO));
w.addDocument(doc);
w.getReader().close();
doc = new Document();
doc.add(new TextField("text", "cruel world", Field.Store.NO));
w.addDocument(doc);
IndexReader reader = w.getReader();
assertTrue(reader.leaves().size() > 1);
SlowCompositeReaderWrapper slowWrapper = (SlowCompositeReaderWrapper) SlowCompositeReaderWrapper.wrap(reader);
assertEquals(0, slowWrapper.cachedTerms.size());
assertEquals(MultiTerms.class, slowWrapper.terms("text").getClass());
assertEquals(1, slowWrapper.cachedTerms.size());
assertNull(slowWrapper.terms("bogusField"));
assertEquals(1, slowWrapper.cachedTerms.size());//bogus field isn't cached
reader.close();
w.close();
dir.close();
}
}