From a9721702e1c11127e05d5c7836292438b053dcc5 Mon Sep 17 00:00:00 2001 From: Robert Muir Date: Fri, 18 May 2012 15:42:40 +0000 Subject: [PATCH] LUCENE-4055: make fieldinfos always readonly once built git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene4055@1340134 13f79535-47bb-0310-9956-ffa450edef68 --- .../lucene3x/Lucene3xFieldInfosReader.java | 3 +- .../lucene40/Lucene40FieldInfosReader.java | 3 +- .../SimpleTextFieldInfosReader.java | 3 +- .../org/apache/lucene/index/DocConsumer.java | 2 +- .../lucene/index/DocFieldProcessor.java | 2 +- .../apache/lucene/index/DocumentsWriter.java | 2 +- .../index/DocumentsWriterPerThread.java | 10 +- .../index/DocumentsWriterPerThreadPool.java | 6 +- .../org/apache/lucene/index/FieldInfos.java | 365 +++++++++++++++--- .../org/apache/lucene/index/IndexWriter.java | 6 +- .../org/apache/lucene/index/MultiFields.java | 6 +- .../lucene/index/MutableFieldInfos.java | 316 --------------- .../lucene/index/ParallelAtomicReader.java | 2 +- .../lucene/index/ReadOnlyFieldInfos.java | 115 ------ .../org/apache/lucene/index/SegmentInfos.java | 1 - .../apache/lucene/index/SegmentMerger.java | 4 +- .../org/apache/lucene/index/TestCodecs.java | 6 +- .../test/org/apache/lucene/index/TestDoc.java | 2 +- .../apache/lucene/index/TestFieldInfos.java | 2 +- .../apache/lucene/index/TestFieldsReader.java | 4 +- .../lucene/index/TestSegmentMerger.java | 2 +- .../lucene/index/TestTermVectorsReader.java | 2 +- .../lucene/index/memory/MemoryIndex.java | 55 +-- .../lucene3x/PreFlexRWFieldInfosReader.java | 3 +- .../lucene/index/FieldFilterAtomicReader.java | 2 +- .../org/apache/solr/search/TestDocSet.java | 3 +- 26 files changed, 345 insertions(+), 582 deletions(-) delete mode 100644 lucene/core/src/java/org/apache/lucene/index/MutableFieldInfos.java delete mode 100644 lucene/core/src/java/org/apache/lucene/index/ReadOnlyFieldInfos.java diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xFieldInfosReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xFieldInfosReader.java index fb775c9d239..d1d7dc8be75 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xFieldInfosReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xFieldInfosReader.java @@ -26,7 +26,6 @@ import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexFormatTooNewException; import org.apache.lucene.index.IndexFormatTooOldException; -import org.apache.lucene.index.ReadOnlyFieldInfos; import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.FieldInfo.IndexOptions; import org.apache.lucene.store.Directory; @@ -107,7 +106,7 @@ class Lucene3xFieldInfosReader extends FieldInfosReader { if (input.getFilePointer() != input.length()) { throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")"); } - return new ReadOnlyFieldInfos(infos); + return new FieldInfos(infos); } finally { input.close(); } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosReader.java index d659ef61cda..4e981884ce9 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosReader.java @@ -10,7 +10,6 @@ import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexFormatTooNewException; import org.apache.lucene.index.IndexFormatTooOldException; -import org.apache.lucene.index.ReadOnlyFieldInfos; import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.FieldInfo.IndexOptions; import org.apache.lucene.index.DocValues; @@ -100,7 +99,7 @@ public class Lucene40FieldInfosReader extends FieldInfosReader { throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")"); } - return new ReadOnlyFieldInfos(infos); + return new FieldInfos(infos); } finally { input.close(); } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java b/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java index 838c8859329..f54e7164308 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java @@ -25,7 +25,6 @@ import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.IndexFileNames; -import org.apache.lucene.index.ReadOnlyFieldInfos; import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.FieldInfo.IndexOptions; import org.apache.lucene.index.DocValues; @@ -108,7 +107,7 @@ public class SimpleTextFieldInfosReader extends FieldInfosReader { throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")"); } - return new ReadOnlyFieldInfos(infos); + return new FieldInfos(infos); } finally { input.close(); } diff --git a/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java b/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java index 62a93bce7f3..87b732f6dfb 100644 --- a/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java +++ b/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java @@ -20,7 +20,7 @@ package org.apache.lucene.index; import java.io.IOException; abstract class DocConsumer { - abstract void processDocument(MutableFieldInfos fieldInfos) throws IOException; + abstract void processDocument(FieldInfos.Builder fieldInfos) throws IOException; abstract void finishDocument() throws IOException; abstract void flush(final SegmentWriteState state) throws IOException; abstract void abort(); diff --git a/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java b/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java index ed7e65bbe09..e539e08ca07 100644 --- a/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java +++ b/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java @@ -203,7 +203,7 @@ final class DocFieldProcessor extends DocConsumer { } @Override - public void processDocument(MutableFieldInfos fieldInfos) throws IOException { + public void processDocument(FieldInfos.Builder fieldInfos) throws IOException { consumer.startDocument(); fieldsWriter.startDocument(); diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java index 652bb8dc5dd..9d444bbe2d9 100644 --- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java +++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java @@ -28,7 +28,7 @@ import org.apache.lucene.index.DocumentsWriterFlushQueue.SegmentFlushTicket; import org.apache.lucene.index.DocumentsWriterPerThread.FlushedSegment; import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain; import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState; -import org.apache.lucene.index.MutableFieldInfos.FieldNumberBiMap; +import org.apache.lucene.index.FieldInfos.FieldNumberBiMap; import org.apache.lucene.search.Query; import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.store.AlreadyClosedException; diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java index bc8d5d6f0f5..76d68e73786 100644 --- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java +++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java @@ -170,7 +170,7 @@ class DocumentsWriterPerThread { boolean aborting = false; // True if an abort is pending boolean hasAborted = false; // True if the last exception throws by #updateDocument was aborting - private MutableFieldInfos fieldInfos; + private FieldInfos.Builder fieldInfos; private final InfoStream infoStream; private int numDocsInRAM; private int flushedDocCount; @@ -181,7 +181,7 @@ class DocumentsWriterPerThread { public DocumentsWriterPerThread(Directory directory, DocumentsWriter parent, - MutableFieldInfos fieldInfos, IndexingChain indexingChain) { + FieldInfos.Builder fieldInfos, IndexingChain indexingChain) { this.directory = directory; this.parent = parent; this.fieldInfos = fieldInfos; @@ -197,7 +197,7 @@ class DocumentsWriterPerThread { initialize(); } - public DocumentsWriterPerThread(DocumentsWriterPerThread other, MutableFieldInfos fieldInfos) { + public DocumentsWriterPerThread(DocumentsWriterPerThread other, FieldInfos.Builder fieldInfos) { this(other.directory, other.parent, fieldInfos, other.parent.chain); } @@ -414,7 +414,7 @@ class DocumentsWriterPerThread { private void doAfterFlush() throws IOException { segment = null; consumer.doAfterFlush(); - fieldInfos = MutableFieldInfos.from(fieldInfos); + fieldInfos = FieldInfos.Builder.from(fieldInfos); parent.subtractFlushedNumDocs(numDocsInRAM); numDocsInRAM = 0; } @@ -442,7 +442,7 @@ class DocumentsWriterPerThread { FlushedSegment flush() throws IOException { assert numDocsInRAM > 0; assert deleteSlice == null : "all deletes must be applied in prepareFlush"; - flushState = new SegmentWriteState(infoStream, directory, segment, fieldInfos, + flushState = new SegmentWriteState(infoStream, directory, segment, fieldInfos.finish(), numDocsInRAM, writer.getConfig().getTermIndexInterval(), codec, pendingDeletes, new IOContext(new FlushInfo(numDocsInRAM, bytesUsed()))); final double startMBUsed = parent.flushControl.netBytes() / 1024. / 1024.; diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java index f52a6090a05..71877792df6 100644 --- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java +++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java @@ -18,7 +18,7 @@ package org.apache.lucene.index; import java.util.concurrent.locks.ReentrantLock; -import org.apache.lucene.index.MutableFieldInfos.FieldNumberBiMap; +import org.apache.lucene.index.FieldInfos.FieldNumberBiMap; import org.apache.lucene.util.SetOnce; /** @@ -139,7 +139,7 @@ abstract class DocumentsWriterPerThreadPool { this.documentsWriter.set(documentsWriter); // thread pool is bound to DW this.globalFieldMap.set(globalFieldMap); for (int i = 0; i < threadStates.length; i++) { - final MutableFieldInfos infos = new MutableFieldInfos(globalFieldMap); + final FieldInfos.Builder infos = new FieldInfos.Builder(globalFieldMap); threadStates[i] = new ThreadState(new DocumentsWriterPerThread(documentsWriter.directory, documentsWriter, infos, documentsWriter.chain)); } } @@ -228,7 +228,7 @@ abstract class DocumentsWriterPerThreadPool { assert globalFieldMap.get() != null; final DocumentsWriterPerThread dwpt = threadState.dwpt; if (!closed) { - final MutableFieldInfos infos = new MutableFieldInfos(globalFieldMap.get()); + final FieldInfos.Builder infos = new FieldInfos.Builder(globalFieldMap.get()); final DocumentsWriterPerThread newDwpt = new DocumentsWriterPerThread(dwpt, infos); newDwpt.initialize(); threadState.resetWriter(newDwpt); diff --git a/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java b/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java index e617935625e..5c0b9eea790 100644 --- a/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java +++ b/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java @@ -17,7 +17,13 @@ package org.apache.lucene.index; * limitations under the License. */ +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; import java.util.Iterator; +import java.util.Map; +import java.util.SortedMap; +import java.util.TreeMap; import org.apache.lucene.index.FieldInfo.IndexOptions; @@ -25,14 +31,101 @@ import org.apache.lucene.index.FieldInfo.IndexOptions; * Collection of {@link FieldInfo}s (accessible by number or by name). * @lucene.experimental */ -public abstract class FieldInfos implements Iterable { - +public class FieldInfos implements Iterable { + private final boolean hasFreq; + private final boolean hasProx; + private final boolean hasVectors; + private final boolean hasNorms; + private final boolean hasDocValues; + + private final SortedMap byNumber = new TreeMap(); + private final HashMap byName = new HashMap(); + private final Collection values; // for an unmodifiable iterator + + public FieldInfos(FieldInfo[] infos) { + boolean hasVectors = false; + boolean hasProx = false; + boolean hasFreq = false; + boolean hasNorms = false; + boolean hasDocValues = false; + + for (FieldInfo info : infos) { + assert !byNumber.containsKey(info.number); + byNumber.put(info.number, info); + assert !byName.containsKey(info.name); + byName.put(info.name, info); + + hasVectors |= info.hasVectors(); + hasProx |= info.isIndexed() && info.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; + hasFreq |= info.isIndexed() && info.getIndexOptions() != IndexOptions.DOCS_ONLY; + hasNorms |= info.hasNorms(); + hasDocValues |= info.hasDocValues(); + } + + this.hasVectors = hasVectors; + this.hasProx = hasProx; + this.hasFreq = hasFreq; + this.hasNorms = hasNorms; + this.hasDocValues = hasDocValues; + this.values = Collections.unmodifiableCollection(byNumber.values()); + } + + /** Returns true if any fields have freqs */ + public boolean hasFreq() { + return hasFreq; + } + + /** Returns true if any fields have positions */ + public boolean hasProx() { + return hasProx; + } + + /** + * @return true if at least one field has any vectors + */ + public boolean hasVectors() { + return hasVectors; + } + + /** + * @return true if at least one field has any norms + */ + public boolean hasNorms() { + return hasNorms; + } + + /** + * @return true if at least one field has doc values + */ + public boolean hasDocValues() { + return hasDocValues; + } + + /** + * @return number of fields + */ + public int size() { + assert byNumber.size() == byName.size(); + return byNumber.size(); + } + + /** + * Returns an iterator over all the fieldinfo objects present, + * ordered by ascending field number + */ + // TODO: what happens if in fact a different order is used? + public Iterator iterator() { + return values.iterator(); + } + /** * Return the fieldinfo object referenced by the field name * @return the FieldInfo object or null when the given fieldName * doesn't exist. */ - public abstract FieldInfo fieldInfo(String fieldName); + public FieldInfo fieldInfo(String fieldName) { + return byName.get(fieldName); + } /** * Return the fieldinfo object referenced by the fieldNumber. @@ -40,73 +133,223 @@ public abstract class FieldInfos implements Iterable { * @return the FieldInfo object or null when the given fieldNumber * doesn't exist. */ - public abstract FieldInfo fieldInfo(int fieldNumber); - - /** - * Returns an iterator over all the fieldinfo objects present, - * ordered by ascending field number - */ - // TODO: what happens if in fact a different order is used? - public abstract Iterator iterator(); - - /** - * @return number of fields - */ - public abstract int size(); - - /** Returns true if any fields have positions */ - public boolean hasProx() { - for (FieldInfo fi : this) { - if (fi.isIndexed() && fi.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0) { - return true; - } - } - return false; + public FieldInfo fieldInfo(int fieldNumber) { + return (fieldNumber >= 0) ? byNumber.get(fieldNumber) : null; } - /** Returns true if any fields have freqs */ - public boolean hasFreq() { - for (FieldInfo fi : this) { - if (fi.isIndexed() && fi.getIndexOptions() != IndexOptions.DOCS_ONLY) { - return true; - } + static final class FieldNumberBiMap { + + private final Map numberToName; + private final Map nameToNumber; + private int lowestUnassignedFieldNumber = -1; + + FieldNumberBiMap() { + this.nameToNumber = new HashMap(); + this.numberToName = new HashMap(); } - return false; - } - - /** - * @return true if at least one field has any vectors - */ - public boolean hasVectors() { - for (FieldInfo fi : this) { - if (fi.hasVectors()) { - return true; - } - } - return false; - } + + /** + * Returns the global field number for the given field name. If the name + * does not exist yet it tries to add it with the given preferred field + * number assigned if possible otherwise the first unassigned field number + * is used as the field number. + */ + synchronized int addOrGet(String fieldName, int preferredFieldNumber) { + Integer fieldNumber = nameToNumber.get(fieldName); + if (fieldNumber == null) { + final Integer preferredBoxed = Integer.valueOf(preferredFieldNumber); - /** - * @return true if at least one field has doc values - */ - public boolean hasDocValues() { - for (FieldInfo fi : this) { - if (fi.hasDocValues()) { - return true; + if (preferredFieldNumber != -1 && !numberToName.containsKey(preferredBoxed)) { + // cool - we can use this number globally + fieldNumber = preferredBoxed; + } else { + // find a new FieldNumber + while (numberToName.containsKey(++lowestUnassignedFieldNumber)) { + // might not be up to date - lets do the work once needed + } + fieldNumber = lowestUnassignedFieldNumber; + } + + numberToName.put(fieldNumber, fieldName); + nameToNumber.put(fieldName, fieldNumber); + } + + return fieldNumber.intValue(); + } + + /** + * Sets the given field number and name if not yet set. + */ + synchronized void setIfNotSet(int fieldNumber, String fieldName) { + final Integer boxedFieldNumber = Integer.valueOf(fieldNumber); + if (!numberToName.containsKey(boxedFieldNumber) + && !nameToNumber.containsKey(fieldName)) { + numberToName.put(boxedFieldNumber, fieldName); + nameToNumber.put(fieldName, boxedFieldNumber); + } else { + assert containsConsistent(boxedFieldNumber, fieldName); } } - return false; + + // used by assert + synchronized boolean containsConsistent(Integer number, String name) { + return name.equals(numberToName.get(number)) + && number.equals(nameToNumber.get(name)); + } } - /** - * @return true if at least one field has any norms - */ - public boolean hasNorms() { - for (FieldInfo fi : this) { - if (fi.hasNorms()) { - return true; + static final class Builder { + private final SortedMap byNumber = new TreeMap(); + private final HashMap byName = new HashMap(); + private final FieldNumberBiMap globalFieldNumbers; + + private long version; // internal use to track changes + + public Builder() { + this(new FieldNumberBiMap()); + } + + public void add(FieldInfos other) { + for(FieldInfo fieldInfo : other){ + add(fieldInfo); } } - return false; + + /** + * Creates a new instance with the given {@link FieldNumberBiMap}. + */ + Builder(FieldNumberBiMap globalFieldNumbers) { + assert globalFieldNumbers != null; + this.globalFieldNumbers = globalFieldNumbers; + } + + /** + * adds the given field to this FieldInfos name / number mapping. The given FI + * must be present in the global field number mapping before this method it + * called + */ + private void putInternal(FieldInfo fi) { + assert !byNumber.containsKey(fi.number); + assert !byName.containsKey(fi.name); + assert globalFieldNumbers == null || globalFieldNumbers.containsConsistent(Integer.valueOf(fi.number), fi.name); + byNumber.put(fi.number, fi); + byName.put(fi.name, fi); + } + + private int nextFieldNumber(String name, int preferredFieldNumber) { + // get a global number for this field + final int fieldNumber = globalFieldNumbers.addOrGet(name, + preferredFieldNumber); + assert byNumber.get(fieldNumber) == null : "field number " + fieldNumber + + " already taken"; + return fieldNumber; + } + + /** If the field is not yet known, adds it. If it is known, checks to make + * sure that the isIndexed flag is the same as was given previously for this + * field. If not - marks it as being indexed. Same goes for the TermVector + * parameters. + * + * @param name The name of the field + * @param isIndexed true if the field is indexed + * @param storeTermVector true if the term vector should be stored + * @param omitNorms true if the norms for the indexed field should be omitted + * @param storePayloads true if payloads should be stored for this field + * @param indexOptions if term freqs should be omitted for this field + */ + // nocommit: fix testCodecs to do this another way, its the only user of this + synchronized FieldInfo addOrUpdate(String name, boolean isIndexed, boolean storeTermVector, + boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValues, DocValues.Type normType) { + return addOrUpdateInternal(name, -1, isIndexed, storeTermVector, omitNorms, storePayloads, indexOptions, docValues, normType); + } + + // NOTE: this method does not carry over termVector + // booleans nor docValuesType; the indexer chain + // (TermVectorsConsumerPerField, DocFieldProcessor) must + // set these fields when they succeed in consuming + // the document: + public FieldInfo addOrUpdate(String name, IndexableFieldType fieldType) { + // TODO: really, indexer shouldn't even call this + // method (it's only called from DocFieldProcessor); + // rather, each component in the chain should update + // what it "owns". EG fieldType.indexOptions() should + // be updated by maybe FreqProxTermsWriterPerField: + return addOrUpdateInternal(name, -1, fieldType.indexed(), false, + fieldType.omitNorms(), false, + fieldType.indexOptions(), null, null); + } + + synchronized private FieldInfo addOrUpdateInternal(String name, int preferredFieldNumber, boolean isIndexed, + boolean storeTermVector, + boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValues, DocValues.Type normType) { + if (globalFieldNumbers == null) { + throw new IllegalStateException("FieldInfos are read-only, create a new instance with a global field map to make modifications to FieldInfos"); + } + FieldInfo fi = fieldInfo(name); + if (fi == null) { + final int fieldNumber = nextFieldNumber(name, preferredFieldNumber); + fi = addInternal(name, fieldNumber, isIndexed, storeTermVector, omitNorms, storePayloads, indexOptions, docValues, normType); + } else { + fi.update(isIndexed, storeTermVector, omitNorms, storePayloads, indexOptions); + if (docValues != null) { + fi.setDocValuesType(docValues); + } + if (normType != null) { + fi.setNormValueType(normType); + } + } + version++; + return fi; + } + + synchronized public FieldInfo add(FieldInfo fi) { + // IMPORTANT - reuse the field number if possible for consistent field numbers across segments + return addOrUpdateInternal(fi.name, fi.number, fi.isIndexed(), fi.hasVectors(), + fi.omitsNorms(), fi.hasPayloads(), + fi.getIndexOptions(), fi.getDocValuesType(), fi.getNormType()); + } + + private FieldInfo addInternal(String name, int fieldNumber, boolean isIndexed, + boolean storeTermVector, boolean omitNorms, boolean storePayloads, + IndexOptions indexOptions, DocValues.Type docValuesType, DocValues.Type normType) { + // don't check modifiable here since we use that to initially build up FIs + if (globalFieldNumbers != null) { + globalFieldNumbers.setIfNotSet(fieldNumber, name); + } + final FieldInfo fi = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, omitNorms, storePayloads, indexOptions, docValuesType, normType); + putInternal(fi); + return fi; + } + + public FieldInfo fieldInfo(String fieldName) { + return byName.get(fieldName); + } + + /** + * Return the fieldinfo object referenced by the fieldNumber. + * @param fieldNumber + * @return the FieldInfo object or null when the given fieldNumber + * doesn't exist. + */ + public FieldInfo fieldInfo(int fieldNumber) { + return (fieldNumber >= 0) ? byNumber.get(fieldNumber) : null; + } + + synchronized final long getVersion() { + return version; + } + + final FieldInfos finish() { + // nocommit: bogus we don't clone each FI + return new FieldInfos(byNumber.values().toArray(new FieldInfo[byNumber.size()])); + } + + /** + * Creates a new instance from the given instance. + */ + // nocommit + static Builder from(Builder other) { + return new Builder(other.globalFieldNumbers); + } } } diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java index 0f81ece1660..784a6ac3b64 100644 --- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java +++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java @@ -34,7 +34,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.codecs.Codec; import org.apache.lucene.index.DocumentsWriterPerThread.FlushedSegment; -import org.apache.lucene.index.MutableFieldInfos.FieldNumberBiMap; +import org.apache.lucene.index.FieldInfos.FieldNumberBiMap; import org.apache.lucene.index.IndexWriterConfig.OpenMode; import org.apache.lucene.index.MergeState.CheckAbort; import org.apache.lucene.search.Query; @@ -2280,7 +2280,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit { // abortable so that IW.close(false) is able to stop it SegmentMerger merger = new SegmentMerger(infoStream, directory, config.getTermIndexInterval(), mergedName, MergeState.CheckAbort.NONE, payloadProcessorProvider, - new MutableFieldInfos(globalFieldNumberMap), codec, context); + new FieldInfos.Builder(globalFieldNumberMap), codec, context); for (IndexReader reader : readers) { // add new indexes merger.add(reader); @@ -3372,7 +3372,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit { final MergeState.CheckAbort checkAbort = new MergeState.CheckAbort(merge, directory); SegmentMerger merger = new SegmentMerger(infoStream, directory, config.getTermIndexInterval(), mergedName, checkAbort, // nocommit - payloadProcessorProvider, new MutableFieldInfos(globalFieldNumberMap), codec, context); + payloadProcessorProvider, new FieldInfos.Builder(globalFieldNumberMap), codec, context); if (infoStream.isEnabled("IW")) { infoStream.message("IW", "merging " + segString(merge.segments)); diff --git a/lucene/core/src/java/org/apache/lucene/index/MultiFields.java b/lucene/core/src/java/org/apache/lucene/index/MultiFields.java index c933c4aaf97..d9a9b5bef4f 100644 --- a/lucene/core/src/java/org/apache/lucene/index/MultiFields.java +++ b/lucene/core/src/java/org/apache/lucene/index/MultiFields.java @@ -248,11 +248,11 @@ public final class MultiFields extends Fields { public static FieldInfos getMergedFieldInfos(IndexReader reader) { final List subReaders = new ArrayList(); ReaderUtil.gatherSubReaders(subReaders, reader); - final MutableFieldInfos fieldInfos = new MutableFieldInfos(); + final FieldInfos.Builder builder = new FieldInfos.Builder(); for(AtomicReader subReader : subReaders) { - fieldInfos.add(subReader.getFieldInfos()); + builder.add(subReader.getFieldInfos()); } - return fieldInfos.finish(); + return builder.finish(); } public static Collection getIndexedFields(IndexReader reader) { diff --git a/lucene/core/src/java/org/apache/lucene/index/MutableFieldInfos.java b/lucene/core/src/java/org/apache/lucene/index/MutableFieldInfos.java deleted file mode 100644 index 8202bd04c89..00000000000 --- a/lucene/core/src/java/org/apache/lucene/index/MutableFieldInfos.java +++ /dev/null @@ -1,316 +0,0 @@ -package org.apache.lucene.index; - -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import java.util.Collection; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; -import java.util.SortedMap; -import java.util.TreeMap; - -import org.apache.lucene.index.FieldInfo.IndexOptions; - -// nocommit: fix DWPT and change this to a more minimal FieldInfos.Builder that -// does *not* extend fieldinfos -final class MutableFieldInfos extends FieldInfos { - static final class FieldNumberBiMap { - - private final Map numberToName; - private final Map nameToNumber; - private int lowestUnassignedFieldNumber = -1; - - FieldNumberBiMap() { - this.nameToNumber = new HashMap(); - this.numberToName = new HashMap(); - } - - /** - * Returns the global field number for the given field name. If the name - * does not exist yet it tries to add it with the given preferred field - * number assigned if possible otherwise the first unassigned field number - * is used as the field number. - */ - synchronized int addOrGet(String fieldName, int preferredFieldNumber) { - Integer fieldNumber = nameToNumber.get(fieldName); - if (fieldNumber == null) { - final Integer preferredBoxed = Integer.valueOf(preferredFieldNumber); - - if (preferredFieldNumber != -1 && !numberToName.containsKey(preferredBoxed)) { - // cool - we can use this number globally - fieldNumber = preferredBoxed; - } else { - // find a new FieldNumber - while (numberToName.containsKey(++lowestUnassignedFieldNumber)) { - // might not be up to date - lets do the work once needed - } - fieldNumber = lowestUnassignedFieldNumber; - } - - numberToName.put(fieldNumber, fieldName); - nameToNumber.put(fieldName, fieldNumber); - } - - return fieldNumber.intValue(); - } - - /** - * Sets the given field number and name if not yet set. - */ - synchronized void setIfNotSet(int fieldNumber, String fieldName) { - final Integer boxedFieldNumber = Integer.valueOf(fieldNumber); - if (!numberToName.containsKey(boxedFieldNumber) - && !nameToNumber.containsKey(fieldName)) { - numberToName.put(boxedFieldNumber, fieldName); - nameToNumber.put(fieldName, boxedFieldNumber); - } else { - assert containsConsistent(boxedFieldNumber, fieldName); - } - } - - // used by assert - synchronized boolean containsConsistent(Integer number, String name) { - return name.equals(numberToName.get(number)) - && number.equals(nameToNumber.get(name)); - } - } - - private final SortedMap byNumber = new TreeMap(); - private final HashMap byName = new HashMap(); - private final FieldNumberBiMap globalFieldNumbers; - - private long version; // internal use to track changes - - public MutableFieldInfos() { - this(new FieldNumberBiMap()); - } - - public void add(FieldInfos other) { - for(FieldInfo fieldInfo : other){ - add(fieldInfo); - } - } - - /** - * Creates a new FieldInfos instance with the given {@link FieldNumberBiMap}. - */ - MutableFieldInfos(FieldNumberBiMap globalFieldNumbers) { - assert globalFieldNumbers != null; - this.globalFieldNumbers = globalFieldNumbers; - } - - /** - * adds the given field to this FieldInfos name / number mapping. The given FI - * must be present in the global field number mapping before this method it - * called - */ - private void putInternal(FieldInfo fi) { - assert !byNumber.containsKey(fi.number); - assert !byName.containsKey(fi.name); - assert globalFieldNumbers == null || globalFieldNumbers.containsConsistent(Integer.valueOf(fi.number), fi.name); - byNumber.put(fi.number, fi); - byName.put(fi.name, fi); - } - - private int nextFieldNumber(String name, int preferredFieldNumber) { - // get a global number for this field - final int fieldNumber = globalFieldNumbers.addOrGet(name, - preferredFieldNumber); - assert byNumber.get(fieldNumber) == null : "field number " + fieldNumber - + " already taken"; - return fieldNumber; - } - - /** - * Assumes the fields are not storing term vectors. - * - * @param names The names of the fields - * @param isIndexed Whether the fields are indexed or not - * - * @see #addOrUpdate(String, boolean) - */ - synchronized public void addOrUpdate(Collection names, boolean isIndexed) { - for (String name : names) { - addOrUpdate(name, isIndexed); - } - } - - /** - * Calls 5 parameter add with false for all TermVector parameters. - * - * @param name The name of the IndexableField - * @param isIndexed true if the field is indexed - * @see #addOrUpdate(String, boolean, boolean) - */ - synchronized public void addOrUpdate(String name, boolean isIndexed) { - addOrUpdate(name, isIndexed, false, false); - } - - /** If the field is not yet known, adds it. If it is known, checks to make - * sure that the isIndexed flag is the same as was given previously for this - * field. If not - marks it as being indexed. Same goes for the TermVector - * parameters. - * - * @param name The name of the field - * @param isIndexed true if the field is indexed - * @param storeTermVector true if the term vector should be stored - */ - synchronized public void addOrUpdate(String name, boolean isIndexed, boolean storeTermVector) { - addOrUpdate(name, isIndexed, storeTermVector, false); - } - - /** If the field is not yet known, adds it. If it is known, checks to make - * sure that the isIndexed flag is the same as was given previously for this - * field. If not - marks it as being indexed. Same goes for the TermVector - * parameters. - * - * @param name The name of the field - * @param isIndexed true if the field is indexed - * @param storeTermVector true if the term vector should be stored - * @param omitNorms true if the norms for the indexed field should be omitted - */ - synchronized public void addOrUpdate(String name, boolean isIndexed, boolean storeTermVector, - boolean omitNorms) { - addOrUpdate(name, isIndexed, storeTermVector, omitNorms, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS, null, null); - } - - /** If the field is not yet known, adds it. If it is known, checks to make - * sure that the isIndexed flag is the same as was given previously for this - * field. If not - marks it as being indexed. Same goes for the TermVector - * parameters. - * - * @param name The name of the field - * @param isIndexed true if the field is indexed - * @param storeTermVector true if the term vector should be stored - * @param omitNorms true if the norms for the indexed field should be omitted - * @param storePayloads true if payloads should be stored for this field - * @param indexOptions if term freqs should be omitted for this field - */ - synchronized public FieldInfo addOrUpdate(String name, boolean isIndexed, boolean storeTermVector, - boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValues, DocValues.Type normType) { - return addOrUpdateInternal(name, -1, isIndexed, storeTermVector, omitNorms, storePayloads, indexOptions, docValues, normType); - } - - // NOTE: this method does not carry over termVector - // booleans nor docValuesType; the indexer chain - // (TermVectorsConsumerPerField, DocFieldProcessor) must - // set these fields when they succeed in consuming - // the document: - public FieldInfo addOrUpdate(String name, IndexableFieldType fieldType) { - // TODO: really, indexer shouldn't even call this - // method (it's only called from DocFieldProcessor); - // rather, each component in the chain should update - // what it "owns". EG fieldType.indexOptions() should - // be updated by maybe FreqProxTermsWriterPerField: - return addOrUpdateInternal(name, -1, fieldType.indexed(), false, - fieldType.omitNorms(), false, - fieldType.indexOptions(), null, null); - } - - synchronized private FieldInfo addOrUpdateInternal(String name, int preferredFieldNumber, boolean isIndexed, - boolean storeTermVector, - boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValues, DocValues.Type normType) { - if (globalFieldNumbers == null) { - throw new IllegalStateException("FieldInfos are read-only, create a new instance with a global field map to make modifications to FieldInfos"); - } - FieldInfo fi = fieldInfo(name); - if (fi == null) { - final int fieldNumber = nextFieldNumber(name, preferredFieldNumber); - fi = addInternal(name, fieldNumber, isIndexed, storeTermVector, omitNorms, storePayloads, indexOptions, docValues, normType); - } else { - fi.update(isIndexed, storeTermVector, omitNorms, storePayloads, indexOptions); - if (docValues != null) { - fi.setDocValuesType(docValues); - } - if (normType != null) { - fi.setNormValueType(normType); - } - } - version++; - return fi; - } - - synchronized public FieldInfo add(FieldInfo fi) { - // IMPORTANT - reuse the field number if possible for consistent field numbers across segments - return addOrUpdateInternal(fi.name, fi.number, fi.isIndexed(), fi.hasVectors(), - fi.omitsNorms(), fi.hasPayloads(), - fi.getIndexOptions(), fi.getDocValuesType(), fi.getNormType()); - } - - /* - * NOTE: if you call this method from a public method make sure you check if we are modifiable and throw an exception otherwise - */ - private FieldInfo addInternal(String name, int fieldNumber, boolean isIndexed, - boolean storeTermVector, boolean omitNorms, boolean storePayloads, - IndexOptions indexOptions, DocValues.Type docValuesType, DocValues.Type normType) { - // don't check modifiable here since we use that to initially build up FIs - if (globalFieldNumbers != null) { - globalFieldNumbers.setIfNotSet(fieldNumber, name); - } - final FieldInfo fi = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, omitNorms, storePayloads, indexOptions, docValuesType, normType); - putInternal(fi); - return fi; - } - - public FieldInfo fieldInfo(String fieldName) { - return byName.get(fieldName); - } - - /** - * Return the fieldinfo object referenced by the fieldNumber. - * @param fieldNumber - * @return the FieldInfo object or null when the given fieldNumber - * doesn't exist. - */ - public FieldInfo fieldInfo(int fieldNumber) { - return (fieldNumber >= 0) ? byNumber.get(fieldNumber) : null; - } - - public Iterator iterator() { - return byNumber.values().iterator(); - } - - /** - * @return number of fields - */ - public int size() { - assert byNumber.size() == byName.size(); - return byNumber.size(); - } - - synchronized final long getVersion() { - return version; - } - - final ReadOnlyFieldInfos finish() { - FieldInfo infos[] = new FieldInfo[size()]; - int upto = 0; - for (FieldInfo info : byNumber.values()) { - infos[upto++] = info.clone(); - } - return new ReadOnlyFieldInfos(infos); - } - - /** - * Creates a new instance from the given instance. - */ - // nocommit - static MutableFieldInfos from(MutableFieldInfos other) { - return new MutableFieldInfos(other.globalFieldNumbers); - } -} diff --git a/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java b/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java index 57abcade003..8156baacb81 100644 --- a/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java +++ b/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java @@ -100,7 +100,7 @@ public final class ParallelAtomicReader extends AtomicReader { } // TODO: make this read-only in a cleaner way? - MutableFieldInfos builder = new MutableFieldInfos(); + FieldInfos.Builder builder = new FieldInfos.Builder(); // build FieldInfos and fieldToReader map: for (final AtomicReader reader : this.parallelReaders) { final FieldInfos readerFieldInfos = reader.getFieldInfos(); diff --git a/lucene/core/src/java/org/apache/lucene/index/ReadOnlyFieldInfos.java b/lucene/core/src/java/org/apache/lucene/index/ReadOnlyFieldInfos.java deleted file mode 100644 index 08a5f10d3ec..00000000000 --- a/lucene/core/src/java/org/apache/lucene/index/ReadOnlyFieldInfos.java +++ /dev/null @@ -1,115 +0,0 @@ -package org.apache.lucene.index; - -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.SortedMap; -import java.util.TreeMap; - -import org.apache.lucene.index.FieldInfo.IndexOptions; - -// nocommit: temporary -public final class ReadOnlyFieldInfos extends FieldInfos { - // nocommit - private final boolean hasFreq; - private final boolean hasProx; - private final boolean hasVectors; - private final boolean hasNorms; - private final boolean hasDocValues; - - private final SortedMap byNumber = new TreeMap(); - private final HashMap byName = new HashMap(); - private final Collection values; // for an unmodifiable iterator - - public ReadOnlyFieldInfos(FieldInfo[] infos) { - boolean hasVectors = false; - boolean hasProx = false; - boolean hasFreq = false; - boolean hasNorms = false; - boolean hasDocValues = false; - - for (FieldInfo info : infos) { - assert !byNumber.containsKey(info.number); - byNumber.put(info.number, info); - assert !byName.containsKey(info.name); - byName.put(info.name, info); - - hasVectors |= info.hasVectors(); - hasProx |= info.isIndexed() && info.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0; - hasFreq |= info.isIndexed() && info.getIndexOptions() != IndexOptions.DOCS_ONLY; - hasNorms |= info.hasNorms(); - hasDocValues |= info.hasDocValues(); - } - - this.hasVectors = hasVectors; - this.hasProx = hasProx; - this.hasFreq = hasFreq; - this.hasNorms = hasNorms; - this.hasDocValues = hasDocValues; - this.values = Collections.unmodifiableCollection(byNumber.values()); - } - - @Override - public boolean hasFreq() { - return hasFreq; - } - - @Override - public boolean hasProx() { - return hasProx; - } - - @Override - public boolean hasVectors() { - return hasVectors; - } - - @Override - public boolean hasNorms() { - return hasNorms; - } - - @Override - public boolean hasDocValues() { - return hasDocValues; - } - - @Override - public int size() { - assert byNumber.size() == byName.size(); - return byNumber.size(); - } - - @Override - public Iterator iterator() { - return values.iterator(); - } - - @Override - public FieldInfo fieldInfo(String fieldName) { - return byName.get(fieldName); - } - - @Override - public FieldInfo fieldInfo(int fieldNumber) { - return (fieldNumber >= 0) ? byNumber.get(fieldNumber) : null; - } -} diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java index be88e5a624a..963a55b59b0 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java +++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java @@ -34,7 +34,6 @@ import java.util.Set; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.SegmentInfosReader; import org.apache.lucene.codecs.SegmentInfosWriter; -import org.apache.lucene.index.MutableFieldInfos.FieldNumberBiMap; import org.apache.lucene.store.ChecksumIndexInput; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java b/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java index 2ed4c95bb15..ebd27d6eb46 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java +++ b/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java @@ -54,9 +54,9 @@ final class SegmentMerger { private final IOContext context; private final MergeState mergeState = new MergeState(); - private final MutableFieldInfos fieldInfosBuilder; + private final FieldInfos.Builder fieldInfosBuilder; - SegmentMerger(InfoStream infoStream, Directory dir, int termIndexInterval, String name, MergeState.CheckAbort checkAbort, PayloadProcessorProvider payloadProcessorProvider, MutableFieldInfos fieldInfosBuilder, Codec codec, IOContext context) { + SegmentMerger(InfoStream infoStream, Directory dir, int termIndexInterval, String name, MergeState.CheckAbort checkAbort, PayloadProcessorProvider payloadProcessorProvider, FieldInfos.Builder fieldInfosBuilder, Codec codec, IOContext context) { mergeState.infoStream = infoStream; mergeState.readers = new ArrayList(); mergeState.checkAbort = checkAbort; diff --git a/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java b/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java index da355fbae6a..24223e1aef0 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java @@ -88,7 +88,7 @@ public class TestCodecs extends LuceneTestCase { final boolean omitTF; final boolean storePayloads; - public FieldData(final String name, final MutableFieldInfos fieldInfos, final TermData[] terms, final boolean omitTF, final boolean storePayloads) { + public FieldData(final String name, final FieldInfos.Builder fieldInfos, final TermData[] terms, final boolean omitTF, final boolean storePayloads) { this.omitTF = omitTF; this.storePayloads = storePayloads; // TODO: change this test to use all three @@ -248,7 +248,7 @@ public class TestCodecs extends LuceneTestCase { terms[i] = new TermData(text, docs, null); } - final MutableFieldInfos builder = new MutableFieldInfos(new MutableFieldInfos.FieldNumberBiMap()); + final FieldInfos.Builder builder = new FieldInfos.Builder(); final FieldData field = new FieldData("field", builder, terms, true, false); final FieldData[] fields = new FieldData[] {field}; @@ -294,7 +294,7 @@ public class TestCodecs extends LuceneTestCase { } public void testRandomPostings() throws Throwable { - final MutableFieldInfos builder = new MutableFieldInfos(new MutableFieldInfos.FieldNumberBiMap()); + final FieldInfos.Builder builder = new FieldInfos.Builder(); final FieldData[] fields = new FieldData[NUM_FIELDS]; for(int i=0;i { diff --git a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java index b06099caeea..f30a30b52dc 100644 --- a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java +++ b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java @@ -25,8 +25,6 @@ import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; import java.util.Map; -import java.util.SortedMap; -import java.util.TreeMap; import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.TokenStream; @@ -203,7 +201,7 @@ public class MemoryIndex { private static final boolean DEBUG = false; - private final MemoryFieldInfos fieldInfos; + private HashMap fieldInfos = new HashMap(); /** * Sorts term entries into ascending order; also works for @@ -239,7 +237,6 @@ public class MemoryIndex { */ protected MemoryIndex(boolean storeOffsets) { this.stride = storeOffsets ? 3 : 1; - fieldInfos = new MemoryFieldInfos(); } /** @@ -356,7 +353,10 @@ public class MemoryIndex { int numOverlapTokens = 0; int pos = -1; - fieldInfos.addOrUpdate(fieldName); + if (!fieldInfos.containsKey(fieldName)) { + fieldInfos.put(fieldName, + new FieldInfo(fieldName, true, fieldInfos.size(), false, false, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS, null, null)); + } TermToBytesRefAttribute termAtt = stream.getAttribute(TermToBytesRefAttribute.class); PositionIncrementAttribute posIncrAttribute = stream.addAttribute(PositionIncrementAttribute.class); @@ -555,49 +555,6 @@ public class MemoryIndex { return result.toString(); } - - /////////////////////////////////////////////////////////////////////////////// - // Nested classes: - /////////////////////////////////////////////////////////////////////////////// - private static final class MemoryFieldInfos extends FieldInfos { - private final SortedMap byNumber = new TreeMap(); - private final HashMap byName = new HashMap(); - - @Override - public FieldInfos clone() { - return this; // I think this is ok here. - } - - @Override - public FieldInfo fieldInfo(String fieldName) { - return byName.get(fieldName); - } - - @Override - public FieldInfo fieldInfo(int fieldNumber) { - return byNumber.get(fieldNumber); - } - - @Override - public Iterator iterator() { - return byNumber.values().iterator(); - } - - @Override - public int size() { - return byNumber.size(); - } - - public void addOrUpdate(String name) { - if (!byName.containsKey(name)) { - int number = byNumber.isEmpty() ? 0 : byNumber.lastKey() + 1; - FieldInfo info = new FieldInfo(name, true, number, false, false, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS, null, null); - byNumber.put(number, info); - byName.put(name, info); - } - } - } - /** * Index data structure for a field; Contains the tokenized term texts and * their positions. @@ -757,7 +714,7 @@ public class MemoryIndex { @Override public FieldInfos getFieldInfos() { - return fieldInfos; + return new FieldInfos(fieldInfos.values().toArray(new FieldInfo[fieldInfos.size()])); } private class MemoryFields extends Fields { diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWFieldInfosReader.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWFieldInfosReader.java index fc9c9c310af..9ecdbf05144 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWFieldInfosReader.java +++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWFieldInfosReader.java @@ -26,7 +26,6 @@ import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexFormatTooNewException; import org.apache.lucene.index.IndexFormatTooOldException; -import org.apache.lucene.index.ReadOnlyFieldInfos; import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.FieldInfo.IndexOptions; import org.apache.lucene.store.Directory; @@ -99,7 +98,7 @@ class PreFlexRWFieldInfosReader extends FieldInfosReader { if (input.getFilePointer() != input.length()) { throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")"); } - return new ReadOnlyFieldInfos(infos); + return new FieldInfos(infos); } finally { input.close(); } diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java b/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java index def871b0027..737bc0fdbcd 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java +++ b/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java @@ -39,7 +39,7 @@ public final class FieldFilterAtomicReader extends FilterAtomicReader { filteredInfos.add(fi); } } - fieldInfos = new ReadOnlyFieldInfos(filteredInfos.toArray(new FieldInfo[filteredInfos.size()])); + fieldInfos = new FieldInfos(filteredInfos.toArray(new FieldInfo[filteredInfos.size()])); } boolean hasField(String field) { diff --git a/solr/core/src/test/org/apache/solr/search/TestDocSet.java b/solr/core/src/test/org/apache/solr/search/TestDocSet.java index 549cb147eec..222277501d3 100644 --- a/solr/core/src/test/org/apache/solr/search/TestDocSet.java +++ b/solr/core/src/test/org/apache/solr/search/TestDocSet.java @@ -31,7 +31,6 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.AtomicReaderContext; import org.apache.lucene.index.MultiReader; import org.apache.lucene.index.IndexReaderContext; -import org.apache.lucene.index.ReadOnlyFieldInfos; import org.apache.lucene.search.DocIdSet; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.Filter; @@ -367,7 +366,7 @@ public class TestDocSet extends LuceneTestCase { @Override public FieldInfos getFieldInfos() { - return new ReadOnlyFieldInfos(new FieldInfo[0]); + return new FieldInfos(new FieldInfo[0]); } @Override