From 26f104d35c2528672875771d2177dff5a33f90d5 Mon Sep 17 00:00:00 2001 From: Michael Busch Date: Mon, 21 Feb 2011 18:50:39 +0000 Subject: [PATCH] LUCENE-2881: Track FieldInfos per segment git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1073110 13f79535-47bb-0310-9956-ffa450edef68 --- lucene/CHANGES.txt | 5 + .../lucene/index/DocConsumerPerThread.java | 3 +- .../apache/lucene/index/DocFieldConsumer.java | 7 - .../lucene/index/DocFieldProcessor.java | 8 +- .../index/DocFieldProcessorPerThread.java | 51 +--- .../org/apache/lucene/index/DocInverter.java | 7 - .../apache/lucene/index/DocumentsWriter.java | 20 +- .../org/apache/lucene/index/FieldInfo.java | 15 +- .../org/apache/lucene/index/FieldInfos.java | 185 ++++++++++++--- .../org/apache/lucene/index/FieldsWriter.java | 9 +- .../org/apache/lucene/index/IndexWriter.java | 35 ++- .../lucene/index/InvertedDocConsumer.java | 6 - .../lucene/index/InvertedDocEndConsumer.java | 1 - .../org/apache/lucene/index/NormsWriter.java | 19 +- .../lucene/index/PerFieldCodecWrapper.java | 10 +- .../apache/lucene/index/SegmentCodecs.java | 10 +- .../org/apache/lucene/index/SegmentInfo.java | 175 ++++++++------ .../org/apache/lucene/index/SegmentInfos.java | 28 +-- .../apache/lucene/index/SegmentMerger.java | 29 +-- .../apache/lucene/index/SegmentReader.java | 34 ++- .../lucene/index/StoredFieldsWriter.java | 6 +- .../index/StoredFieldsWriterPerThread.java | 2 +- .../org/apache/lucene/index/TermsHash.java | 6 - .../lucene/index/TermsHashConsumer.java | 6 - .../index/codecs/preflex/PreFlexFields.java | 24 +- .../index/TestBackwardsCompatibility.java | 5 +- .../org/apache/lucene/index/TestCodecs.java | 16 +- .../index/TestConsistentFieldNumbers.java | 222 ++++++++++++++++++ .../test/org/apache/lucene/index/TestDoc.java | 4 +- .../lucene/index/TestDocumentWriter.java | 9 +- .../lucene/index/TestIndexFileDeleter.java | 5 +- .../index/TestPerFieldCodecSupport.java | 2 +- .../lucene/index/TestSegmentMerger.java | 4 +- 33 files changed, 633 insertions(+), 335 deletions(-) create mode 100644 lucene/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 0f41674c542..66083212f3b 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -160,6 +160,11 @@ Changes in Runtime Behavior LogMergePolicy impls, and call setRequireContiguousMerge(true). (Mike McCandless) +* LUCENE-2881: FieldInfos is now tracked per segment. Before it was tracked + per IndexWriter session, which resulted in FieldInfos that had the FieldInfo + properties from all previous segments combined. The corresponding file format + changes are backwards-compatible. (Michael Busch) + API Changes * LUCENE-2302, LUCENE-1458, LUCENE-2111, LUCENE-2514: Terms are no longer diff --git a/lucene/src/java/org/apache/lucene/index/DocConsumerPerThread.java b/lucene/src/java/org/apache/lucene/index/DocConsumerPerThread.java index 23a0305c8b2..098e6889894 100644 --- a/lucene/src/java/org/apache/lucene/index/DocConsumerPerThread.java +++ b/lucene/src/java/org/apache/lucene/index/DocConsumerPerThread.java @@ -27,7 +27,8 @@ abstract class DocConsumerPerThread { * DocumentsWriter.DocWriter and return it. * DocumentsWriter then calls finish() on this object * when it's its turn. */ - abstract DocumentsWriter.DocWriter processDocument() throws IOException; + abstract DocumentsWriter.DocWriter processDocument(FieldInfos fieldInfos) throws IOException; + abstract void doAfterFlush(); abstract void abort(); } diff --git a/lucene/src/java/org/apache/lucene/index/DocFieldConsumer.java b/lucene/src/java/org/apache/lucene/index/DocFieldConsumer.java index d1b17dcacc2..2abc0bb5531 100644 --- a/lucene/src/java/org/apache/lucene/index/DocFieldConsumer.java +++ b/lucene/src/java/org/apache/lucene/index/DocFieldConsumer.java @@ -22,9 +22,6 @@ import java.util.Collection; import java.util.Map; abstract class DocFieldConsumer { - - FieldInfos fieldInfos; - /** Called when DocumentsWriter decides to create a new * segment */ abstract void flush(Map> threadsAndFields, SegmentWriteState state) throws IOException; @@ -39,8 +36,4 @@ abstract class DocFieldConsumer { * The consumer should free RAM, if possible, returning * true if any RAM was in fact freed. */ abstract boolean freeRAM(); - - void setFieldInfos(FieldInfos fieldInfos) { - this.fieldInfos = fieldInfos; } -} diff --git a/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java b/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java index 54ed4133c60..6416f518983 100644 --- a/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java +++ b/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java @@ -34,16 +34,13 @@ import java.util.HashMap; final class DocFieldProcessor extends DocConsumer { final DocumentsWriter docWriter; - final FieldInfos fieldInfos; final DocFieldConsumer consumer; final StoredFieldsWriter fieldsWriter; public DocFieldProcessor(DocumentsWriter docWriter, DocFieldConsumer consumer) { this.docWriter = docWriter; this.consumer = consumer; - fieldInfos = docWriter.getFieldInfos(); - consumer.setFieldInfos(fieldInfos); - fieldsWriter = new StoredFieldsWriter(docWriter, fieldInfos); + fieldsWriter = new StoredFieldsWriter(docWriter); } @Override @@ -53,7 +50,6 @@ final class DocFieldProcessor extends DocConsumer { for ( DocConsumerPerThread thread : threads) { DocFieldProcessorPerThread perThread = (DocFieldProcessorPerThread) thread; childThreadsAndFields.put(perThread.consumer, perThread.fields()); - perThread.trimFields(state); } fieldsWriter.flush(state); consumer.flush(childThreadsAndFields, state); @@ -63,7 +59,7 @@ final class DocFieldProcessor extends DocConsumer { // FreqProxTermsWriter does this with // FieldInfo.storePayload. final String fileName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.FIELD_INFOS_EXTENSION); - fieldInfos.write(state.directory, fileName); + state.fieldInfos.write(state.directory, fileName); } @Override diff --git a/lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java b/lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java index 298b3bd2b75..96ed560463c 100644 --- a/lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java +++ b/lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java @@ -41,14 +41,13 @@ final class DocFieldProcessorPerThread extends DocConsumerPerThread { float docBoost; int fieldGen; final DocFieldProcessor docFieldProcessor; - final FieldInfos fieldInfos; final DocFieldConsumerPerThread consumer; // Holds all fields seen in current doc DocFieldProcessorPerField[] fields = new DocFieldProcessorPerField[1]; int fieldCount; - // Hash table for all fields ever seen + // Hash table for all fields seen in current segment DocFieldProcessorPerField[] fieldHash = new DocFieldProcessorPerField[2]; int hashMask = 1; int totalFieldCount; @@ -60,7 +59,6 @@ final class DocFieldProcessorPerThread extends DocConsumerPerThread { public DocFieldProcessorPerThread(DocumentsWriterThreadState threadState, DocFieldProcessor docFieldProcessor) throws IOException { this.docState = threadState.docState; this.docFieldProcessor = docFieldProcessor; - this.fieldInfos = docFieldProcessor.fieldInfos; this.consumer = docFieldProcessor.consumer.addThread(this); fieldsWriter = docFieldProcessor.fieldsWriter.addThread(docState); } @@ -75,6 +73,7 @@ final class DocFieldProcessorPerThread extends DocConsumerPerThread { field = next; } } + doAfterFlush(); fieldsWriter.abort(); consumer.abort(); } @@ -92,45 +91,15 @@ final class DocFieldProcessorPerThread extends DocConsumerPerThread { return fields; } - /** If there are fields we've seen but did not see again - * in the last run, then free them up. */ - - void trimFields(SegmentWriteState state) { - - for(int i=0;i { + private static final class FieldNumberBiMap { + private final Map numberToName; + private final Map nameToNumber; + + private FieldNumberBiMap() { + this.nameToNumber = new HashMap(); + this.numberToName = new HashMap(); + } + + synchronized int addOrGet(String fieldName, FieldInfoBiMap fieldInfoMap, int preferredFieldNumber) { + Integer fieldNumber = nameToNumber.get(fieldName); + if (fieldNumber == null) { + if (!numberToName.containsKey(preferredFieldNumber)) { + // cool - we can use this number globally + fieldNumber = preferredFieldNumber; + } else { + fieldNumber = findNextAvailableFieldNumber(preferredFieldNumber + 1, numberToName.keySet()); + } + + numberToName.put(fieldNumber, fieldName); + nameToNumber.put(fieldName, fieldNumber); + } + + return fieldNumber; + } + + synchronized void setIfNotSet(int fieldNumber, String fieldName) { + if (!numberToName.containsKey(fieldNumber) && !nameToNumber.containsKey(fieldName)) { + numberToName.put(fieldNumber, fieldName); + nameToNumber.put(fieldName, fieldNumber); + } + } + } + + private static final class FieldInfoBiMap implements Iterable { + private final SortedMap byNumber = new TreeMap(); + private final HashMap byName = new HashMap(); + private int nextAvailableNumber = 0; + + public void put(FieldInfo fi) { + assert !byNumber.containsKey(fi.number); + assert !byName.containsKey(fi.name); + + byNumber.put(fi.number, fi); + byName.put(fi.name, fi); + } + + public FieldInfo get(String fieldName) { + return byName.get(fieldName); + } + + public FieldInfo get(int fieldNumber) { + return byNumber.get(fieldNumber); + } + + public int size() { + assert byNumber.size() == byName.size(); + return byNumber.size(); + } + + @Override + public Iterator iterator() { + return byNumber.values().iterator(); + } + } // First used in 2.9; prior to 2.9 there was no format header public static final int FORMAT_START = -2; @@ -53,11 +125,18 @@ public final class FieldInfos { static final byte STORE_PAYLOADS = 0x20; static final byte OMIT_TERM_FREQ_AND_POSITIONS = 0x40; - private final ArrayList byNumber = new ArrayList(); - private final HashMap byName = new HashMap(); + private final FieldNumberBiMap globalFieldNumbers; + private final FieldInfoBiMap localFieldInfos; + private int format; public FieldInfos() { + this(new FieldNumberBiMap()); + } + + private FieldInfos(FieldNumberBiMap globalFieldNumbers) { + this.globalFieldNumbers = globalFieldNumbers; + this.localFieldInfos = new FieldInfoBiMap(); } /** @@ -68,6 +147,7 @@ public final class FieldInfos { * @throws IOException */ public FieldInfos(Directory d, String name) throws IOException { + this(new FieldNumberBiMap()); IndexInput input = d.openInput(name); try { read(input, name); @@ -76,17 +156,27 @@ public final class FieldInfos { } } + private static final int findNextAvailableFieldNumber(int nextPreferredNumber, Set unavailableNumbers) { + while (unavailableNumbers.contains(nextPreferredNumber)) { + nextPreferredNumber++; + } + + return nextPreferredNumber; + } + + public FieldInfos newFieldInfosWithGlobalFieldNumberMap() { + return new FieldInfos(this.globalFieldNumbers); + } + /** * Returns a deep clone of this FieldInfos instance. */ @Override synchronized public Object clone() { - FieldInfos fis = new FieldInfos(); - final int numField = byNumber.size(); - for(int i=0;i= 0) ? byNumber.get(fieldNumber) : null; + return (fieldNumber >= 0) ? localFieldInfos.get(fieldNumber) : null; + } + + public Iterator iterator() { + return localFieldInfos.iterator(); } public int size() { - return byNumber.size(); + return localFieldInfos.size(); } public boolean hasVectors() { - for (int i = 0; i < size(); i++) { - if (fieldInfo(i).storeTermVector) { + for (FieldInfo fi : this) { + if (fi.storeTermVector) { return true; } } @@ -287,8 +405,8 @@ public final class FieldInfos { } public boolean hasNorms() { - for (int i = 0; i < size(); i++) { - if (!fieldInfo(i).omitNorms) { + for (FieldInfo fi : this) { + if (!fi.omitNorms) { return true; } } @@ -307,8 +425,7 @@ public final class FieldInfos { public void write(IndexOutput output) throws IOException { output.writeVInt(FORMAT_CURRENT); output.writeVInt(size()); - for (int i = 0; i < size(); i++) { - FieldInfo fi = fieldInfo(i); + for (FieldInfo fi : this) { byte bits = 0x0; if (fi.isIndexed) bits |= IS_INDEXED; if (fi.storeTermVector) bits |= STORE_TERMVECTOR; @@ -318,7 +435,8 @@ public final class FieldInfos { if (fi.storePayloads) bits |= STORE_PAYLOADS; if (fi.omitTermFreqAndPositions) bits |= OMIT_TERM_FREQ_AND_POSITIONS; output.writeString(fi.name); - output.writeInt(fi.codecId); + output.writeInt(fi.number); + output.writeInt(fi.getCodecId()); output.writeByte(bits); } } @@ -338,6 +456,7 @@ public final class FieldInfos { for (int i = 0; i < size; i++) { String name = StringHelper.intern(input.readString()); // if this is a previous format codec 0 will be preflex! + final int fieldNumber = format <= FORMAT_PER_FIELD_CODEC? input.readInt():i; final int codecId = format <= FORMAT_PER_FIELD_CODEC? input.readInt():0; byte bits = input.readByte(); boolean isIndexed = (bits & IS_INDEXED) != 0; @@ -347,8 +466,8 @@ public final class FieldInfos { boolean omitNorms = (bits & OMIT_NORMS) != 0; boolean storePayloads = (bits & STORE_PAYLOADS) != 0; boolean omitTermFreqAndPositions = (bits & OMIT_TERM_FREQ_AND_POSITIONS) != 0; - final FieldInfo addInternal = addInternal(name, isIndexed, storeTermVector, storePositionsWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions); - addInternal.codecId = codecId; + final FieldInfo addInternal = addInternal(name, fieldNumber, isIndexed, storeTermVector, storePositionsWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions); + addInternal.setCodecId(codecId); } if (input.getFilePointer() != input.length()) { diff --git a/lucene/src/java/org/apache/lucene/index/FieldsWriter.java b/lucene/src/java/org/apache/lucene/index/FieldsWriter.java index 91ab30e7349..f694bb4342c 100644 --- a/lucene/src/java/org/apache/lucene/index/FieldsWriter.java +++ b/lucene/src/java/org/apache/lucene/index/FieldsWriter.java @@ -45,14 +45,12 @@ final class FieldsWriter { // If null - we were supplied with streams, if notnull - we manage them ourselves private Directory directory; private String segment; - private FieldInfos fieldInfos; private IndexOutput fieldsStream; private IndexOutput indexStream; - FieldsWriter(Directory directory, String segment, FieldInfos fn) throws IOException { + FieldsWriter(Directory directory, String segment) throws IOException { this.directory = directory; this.segment = segment; - fieldInfos = fn; boolean success = false; try { @@ -70,10 +68,9 @@ final class FieldsWriter { } } - FieldsWriter(IndexOutput fdx, IndexOutput fdt, FieldInfos fn) { + FieldsWriter(IndexOutput fdx, IndexOutput fdt) { directory = null; segment = null; - fieldInfos = fn; fieldsStream = fdt; indexStream = fdx; } @@ -166,7 +163,7 @@ final class FieldsWriter { assert fieldsStream.getFilePointer() == position; } - final void addDocument(Document doc) throws IOException { + final void addDocument(Document doc, FieldInfos fieldInfos) throws IOException { indexStream.writeLong(fieldsStream.getFilePointer()); int storedCount = 0; diff --git a/lucene/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/src/java/org/apache/lucene/index/IndexWriter.java index c5e3e5776e7..b548ef7a8cd 100644 --- a/lucene/src/java/org/apache/lucene/index/IndexWriter.java +++ b/lucene/src/java/org/apache/lucene/index/IndexWriter.java @@ -38,7 +38,6 @@ import org.apache.lucene.document.Document; import org.apache.lucene.index.IndexWriterConfig.OpenMode; import org.apache.lucene.index.PayloadProcessorProvider.DirPayloadProcessor; import org.apache.lucene.index.codecs.CodecProvider; -import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter; import org.apache.lucene.search.Query; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.BufferedIndexInput; @@ -221,6 +220,7 @@ public class IndexWriter implements Closeable { volatile long pendingCommitChangeCount; final SegmentInfos segmentInfos; // the segments + final FieldInfos fieldInfos; private DocumentsWriter docWriter; private IndexFileDeleter deleter; @@ -791,7 +791,10 @@ public class IndexWriter implements Closeable { setRollbackSegmentInfos(segmentInfos); - docWriter = new DocumentsWriter(directory, this, conf.getIndexingChain(), conf.getMaxThreadStates(), getCurrentFieldInfos(), bufferedDeletesStream); + // start with previous field numbers, but new FieldInfos + fieldInfos = getCurrentFieldInfos(); + docWriter = new DocumentsWriter(directory, this, conf.getIndexingChain(), conf.getMaxThreadStates(), + fieldInfos.newFieldInfosWithGlobalFieldNumberMap(), bufferedDeletesStream); docWriter.setInfoStream(infoStream); // Default deleter (for backwards compatibility) is @@ -854,23 +857,14 @@ public class IndexWriter implements Closeable { private FieldInfos getCurrentFieldInfos() throws IOException { final FieldInfos fieldInfos; if (segmentInfos.size() > 0) { - if (segmentInfos.getFormat() > DefaultSegmentInfosWriter.FORMAT_HAS_VECTORS) { - // Pre-4.0 index. In this case we sweep all - // segments, merging their FieldInfos: fieldInfos = new FieldInfos(); for(SegmentInfo info : segmentInfos) { final FieldInfos segFieldInfos = getFieldInfos(info); - final int fieldCount = segFieldInfos.size(); - for(int fieldNumber=0;fieldNumber BD final BufferedDeletesStream.ApplyDeletesResult result = bufferedDeletesStream.applyDeletes(readerPool, merge.segments); @@ -3165,7 +3159,7 @@ public class IndexWriter implements Closeable { SegmentMerger merger = new SegmentMerger(directory, termIndexInterval, mergedName, merge, codecs, payloadProcessorProvider, - ((FieldInfos) docWriter.getFieldInfos().clone())); + merge.info.getFieldInfos()); if (infoStream != null) { message("merging " + merge.segString(directory) + " mergeVectors=" + merger.fieldInfos().hasVectors()); @@ -3174,7 +3168,8 @@ public class IndexWriter implements Closeable { merge.readers = new ArrayList(); merge.readerClones = new ArrayList(); - merge.info.setHasVectors(merger.fieldInfos().hasVectors()); + merge.info.clearFilesCache(); + // This is try/finally to make sure merger's readers are // closed: @@ -3230,7 +3225,7 @@ public class IndexWriter implements Closeable { // because codec must know if prox was written for // this segment: //System.out.println("merger set hasProx=" + merger.hasProx() + " seg=" + merge.info.name); - merge.info.setHasProx(merger.fieldInfos().hasProx()); + merge.info.clearFilesCache(); boolean useCompoundFile; synchronized (this) { // Guard segmentInfos diff --git a/lucene/src/java/org/apache/lucene/index/InvertedDocConsumer.java b/lucene/src/java/org/apache/lucene/index/InvertedDocConsumer.java index 063a920b7b6..76ca1d7fddf 100644 --- a/lucene/src/java/org/apache/lucene/index/InvertedDocConsumer.java +++ b/lucene/src/java/org/apache/lucene/index/InvertedDocConsumer.java @@ -35,10 +35,4 @@ abstract class InvertedDocConsumer { /** Attempt to free RAM, returning true if any RAM was * freed */ abstract boolean freeRAM(); - - FieldInfos fieldInfos; - - void setFieldInfos(FieldInfos fieldInfos) { - this.fieldInfos = fieldInfos; } -} diff --git a/lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java b/lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java index 8c285661692..351529f381b 100644 --- a/lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java +++ b/lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java @@ -25,5 +25,4 @@ abstract class InvertedDocEndConsumer { abstract InvertedDocEndConsumerPerThread addThread(DocInverterPerThread docInverterPerThread); abstract void flush(Map> threadsAndFields, SegmentWriteState state) throws IOException; abstract void abort(); - abstract void setFieldInfos(FieldInfos fieldInfos); } diff --git a/lucene/src/java/org/apache/lucene/index/NormsWriter.java b/lucene/src/java/org/apache/lucene/index/NormsWriter.java index 82b4a730e5b..9148b1f220b 100644 --- a/lucene/src/java/org/apache/lucene/index/NormsWriter.java +++ b/lucene/src/java/org/apache/lucene/index/NormsWriter.java @@ -36,7 +36,6 @@ import org.apache.lucene.store.IndexOutput; final class NormsWriter extends InvertedDocEndConsumer { - private FieldInfos fieldInfos; @Override public InvertedDocEndConsumerPerThread addThread(DocInverterPerThread docInverterPerThread) { return new NormsWriterPerThread(docInverterPerThread, this); @@ -48,11 +47,6 @@ final class NormsWriter extends InvertedDocEndConsumer { // We only write the _X.nrm file at flush void files(Collection files) {} - @Override - void setFieldInfos(FieldInfos fieldInfos) { - this.fieldInfos = fieldInfos; - } - /** Produce _X.nrm if any document had a field with norms * not disabled */ @Override @@ -60,7 +54,7 @@ final class NormsWriter extends InvertedDocEndConsumer { final Map> byField = new HashMap>(); - if (!fieldInfos.hasNorms()) { + if (!state.fieldInfos.hasNorms()) { return; } @@ -96,15 +90,10 @@ final class NormsWriter extends InvertedDocEndConsumer { try { normsOut.writeBytes(SegmentMerger.NORMS_HEADER, 0, SegmentMerger.NORMS_HEADER.length); - final int numField = fieldInfos.size(); - int normCount = 0; - for(int fieldNumber=0;fieldNumber toMerge = byField.get(fieldInfo); + for (FieldInfo fi : state.fieldInfos) { + List toMerge = byField.get(fi); int upto = 0; if (toMerge != null) { @@ -158,7 +147,7 @@ final class NormsWriter extends InvertedDocEndConsumer { // Fill final hole with defaultNorm for(;upto producers = new HashMap(); boolean success = false; try { - for (int i = 0; i < fieldCount; i++) { - FieldInfo fi = fieldInfos.fieldInfo(i); + for (FieldInfo fi : fieldInfos) { if (fi.isIndexed) { // TODO this does not work for non-indexed fields fields.add(fi.name); - Codec codec = segmentCodecs.codecs[fi.codecId]; + Codec codec = segmentCodecs.codecs[fi.getCodecId()]; if (!producers.containsKey(codec)) { producers.put(codec, codec.fieldsProducer(new SegmentReadState(dir, - si, fieldInfos, readBufferSize, indexDivisor, ""+fi.codecId))); + si, fieldInfos, readBufferSize, indexDivisor, ""+fi.getCodecId()))); } codecs.put(fi.name, producers.get(codec)); } diff --git a/lucene/src/java/org/apache/lucene/index/SegmentCodecs.java b/lucene/src/java/org/apache/lucene/index/SegmentCodecs.java index 9be168028e7..0df82707f62 100644 --- a/lucene/src/java/org/apache/lucene/index/SegmentCodecs.java +++ b/lucene/src/java/org/apache/lucene/index/SegmentCodecs.java @@ -74,22 +74,20 @@ final class SegmentCodecs implements Cloneable { } static SegmentCodecs build(FieldInfos infos, CodecProvider provider) { - final int size = infos.size(); final Map codecRegistry = new IdentityHashMap(); final ArrayList codecs = new ArrayList(); - for (int i = 0; i < size; i++) { - final FieldInfo info = infos.fieldInfo(i); - if (info.isIndexed) { + for (FieldInfo fi : infos) { + if (fi.isIndexed) { final Codec fieldCodec = provider.lookup(provider - .getFieldCodec(info.name)); + .getFieldCodec(fi.name)); Integer ord = codecRegistry.get(fieldCodec); if (ord == null) { ord = Integer.valueOf(codecs.size()); codecRegistry.put(fieldCodec, ord); codecs.add(fieldCodec); } - info.codecId = ord.intValue(); + fi.setCodecId(ord.intValue()); } } return new SegmentCodecs(provider, codecs.toArray(Codec.EMPTY)); diff --git a/lucene/src/java/org/apache/lucene/index/SegmentInfo.java b/lucene/src/java/org/apache/lucene/index/SegmentInfo.java index 47d0b54795d..767e62d2b5c 100644 --- a/lucene/src/java/org/apache/lucene/index/SegmentInfo.java +++ b/lucene/src/java/org/apache/lucene/index/SegmentInfo.java @@ -17,21 +17,22 @@ package org.apache.lucene.index; * limitations under the License. */ -import org.apache.lucene.store.Directory; -import org.apache.lucene.store.IndexOutput; -import org.apache.lucene.store.IndexInput; -import org.apache.lucene.util.Constants; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + import org.apache.lucene.index.codecs.Codec; import org.apache.lucene.index.codecs.CodecProvider; import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter; -import java.io.IOException; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.HashSet; -import java.util.HashMap; -import java.util.ArrayList; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.Constants; /** * Information about a segment such as it's name, directory, and files related @@ -41,6 +42,9 @@ import java.util.ArrayList; */ public final class SegmentInfo { + @Deprecated + // remove with hasVector and hasProx + static final int CHECK_FIELDINFOS = -2; // hasVector and hasProx use this for bw compatibility static final int NO = -1; // e.g. no norms; no deletes; static final int YES = 1; // e.g. have norms; have deletes; static final int WITHOUT_GEN = 0; // a file name that has no GEN in it. @@ -62,7 +66,7 @@ public final class SegmentInfo { * - NO says this field has no separate norms * >= YES says this field has separate norms with the specified generation */ - private long[] normGen; + private Map normGen; private boolean isCompoundFile; @@ -80,9 +84,15 @@ public final class SegmentInfo { private int delCount; // How many deleted docs in this segment - private boolean hasProx; // True if this segment has any fields with omitTermFreqAndPositions==false + @Deprecated + // remove when we don't have to support old indexes anymore that had this field + private int hasProx = CHECK_FIELDINFOS; // True if this segment has any fields with omitTermFreqAndPositions==false - private boolean hasVectors; // True if this segment wrote term vectors + @Deprecated + // remove when we don't have to support old indexes anymore that had this field + private int hasVectors = CHECK_FIELDINFOS; // True if this segment wrote term vectors + + private FieldInfos fieldInfos; private SegmentCodecs segmentCodecs; @@ -100,7 +110,7 @@ public final class SegmentInfo { private long bufferedDeletesGen; public SegmentInfo(String name, int docCount, Directory dir, boolean isCompoundFile, - boolean hasProx, SegmentCodecs segmentCodecs, boolean hasVectors) { + SegmentCodecs segmentCodecs, FieldInfos fieldInfos) { this.name = name; this.docCount = docCount; this.dir = dir; @@ -108,18 +118,17 @@ public final class SegmentInfo { this.isCompoundFile = isCompoundFile; this.docStoreOffset = -1; this.docStoreSegment = name; - this.hasProx = hasProx; this.segmentCodecs = segmentCodecs; - this.hasVectors = hasVectors; delCount = 0; version = Constants.LUCENE_MAIN_VERSION; + this.fieldInfos = fieldInfos; } /** * Copy everything from src SegmentInfo into our instance. */ void reset(SegmentInfo src) { - clearFiles(); + clearFilesCache(); version = src.version; name = src.name; docCount = src.docCount; @@ -130,11 +139,14 @@ public final class SegmentInfo { docStoreIsCompoundFile = src.docStoreIsCompoundFile; hasVectors = src.hasVectors; hasProx = src.hasProx; + fieldInfos = src.fieldInfos == null ? null : (FieldInfos) src.fieldInfos.clone(); if (src.normGen == null) { normGen = null; } else { - normGen = new long[src.normGen.length]; - System.arraycopy(src.normGen, 0, normGen, 0, src.normGen.length); + normGen = new HashMap(src.normGen.size()); + for (Entry entry : src.normGen.entrySet()) { + normGen.put(entry.getKey(), entry.getValue()); + } } isCompoundFile = src.isCompoundFile; delCount = src.delCount; @@ -184,17 +196,35 @@ public final class SegmentInfo { if (numNormGen == NO) { normGen = null; } else { - normGen = new long[numNormGen]; + normGen = new HashMap(); for(int j=0;j(diagnostics); if (normGen != null) { - si.normGen = normGen.clone(); + si.normGen = new HashMap(); + for (Entry entry : normGen.entrySet()) { + si.normGen.put(entry.getKey(), entry.getValue()); } + } + si.hasProx = hasProx; si.hasVectors = hasVectors; si.version = version; return si; @@ -339,7 +378,12 @@ public final class SegmentInfo { * @param fieldNumber the field index to check */ public boolean hasSeparateNorms(int fieldNumber) { - return normGen != null && normGen[fieldNumber] != NO; + if (normGen == null) { + return false; + } + + Long gen = normGen.get(fieldNumber); + return gen != null && gen.longValue() != NO; } /** @@ -349,7 +393,7 @@ public final class SegmentInfo { if (normGen == null) { return false; } else { - for (long fieldNormGen : normGen) { + for (long fieldNormGen : normGen.values()) { if (fieldNormGen >= YES) { return true; } @@ -359,10 +403,9 @@ public final class SegmentInfo { return false; } - void initNormGen(int numFields) { + void initNormGen() { if (normGen == null) { // normGen is null if this segments file hasn't had any norms set against it yet - normGen = new long[numFields]; - Arrays.fill(normGen, NO); + normGen = new HashMap(); } } @@ -373,12 +416,13 @@ public final class SegmentInfo { * @param fieldIndex field whose norm file will be rewritten */ void advanceNormGen(int fieldIndex) { - if (normGen[fieldIndex] == NO) { - normGen[fieldIndex] = YES; + Long gen = normGen.get(fieldIndex); + if (gen == null || gen.longValue() == NO) { + normGen.put(fieldIndex, new Long(YES)); } else { - normGen[fieldIndex]++; + normGen.put(fieldIndex, gen+1); } - clearFiles(); + clearFilesCache(); } /** @@ -388,7 +432,7 @@ public final class SegmentInfo { */ public String getNormFileName(int number) { if (hasSeparateNorms(number)) { - return IndexFileNames.fileNameFromGeneration(name, "s" + number, normGen[number]); + return IndexFileNames.fileNameFromGeneration(name, "s" + number, normGen.get(number)); } else { // single file for all norms return IndexFileNames.fileNameFromGeneration(name, IndexFileNames.NORMS_EXTENSION, WITHOUT_GEN); @@ -403,7 +447,7 @@ public final class SegmentInfo { */ void setUseCompoundFile(boolean isCompoundFile) { this.isCompoundFile = isCompoundFile; - clearFiles(); + clearFilesCache(); } /** @@ -433,7 +477,7 @@ public final class SegmentInfo { void setDocStoreIsCompoundFile(boolean v) { docStoreIsCompoundFile = v; - clearFiles(); + clearFilesCache(); } public String getDocStoreSegment() { @@ -446,14 +490,14 @@ public final class SegmentInfo { void setDocStoreOffset(int offset) { docStoreOffset = offset; - clearFiles(); + clearFilesCache(); } void setDocStore(int offset, String segment, boolean isCompoundFile) { docStoreOffset = offset; docStoreSegment = segment; docStoreIsCompoundFile = isCompoundFile; - clearFiles(); + clearFilesCache(); } /** Save this segment's info. */ @@ -474,27 +518,24 @@ public final class SegmentInfo { if (normGen == null) { output.writeInt(NO); } else { - output.writeInt(normGen.length); - for (long fieldNormGen : normGen) { - output.writeLong(fieldNormGen); + output.writeInt(normGen.size()); + for (Entry entry : normGen.entrySet()) { + output.writeInt(entry.getKey()); + output.writeLong(entry.getValue()); } } output.writeByte((byte) (isCompoundFile ? YES : NO)); output.writeInt(delCount); - output.writeByte((byte) (hasProx ? 1:0)); + output.writeByte((byte) hasProx); segmentCodecs.write(output); output.writeStringStringMap(diagnostics); - output.writeByte((byte) (hasVectors ? 1 : 0)); - } - - void setHasProx(boolean hasProx) { - this.hasProx = hasProx; - clearFiles(); + output.writeByte((byte) hasVectors); } public boolean getHasProx() { - return hasProx; + return hasProx == CHECK_FIELDINFOS ? + (fieldInfos == null ? true : fieldInfos.hasProx()) : hasProx == YES; } /** Can only be called once. */ @@ -550,7 +591,7 @@ public final class SegmentInfo { } else { fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.FIELDS_INDEX_EXTENSION)); fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.FIELDS_EXTENSION)); - if (hasVectors) { + if (getHasVectors()) { fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_INDEX_EXTENSION)); fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION)); fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION)); @@ -559,7 +600,7 @@ public final class SegmentInfo { } else if (!useCompoundFile) { fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.FIELDS_INDEX_EXTENSION)); fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.FIELDS_EXTENSION)); - if (hasVectors) { + if (getHasVectors()) { fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.VECTORS_INDEX_EXTENSION)); fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION)); fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.VECTORS_FIELDS_EXTENSION)); @@ -572,11 +613,11 @@ public final class SegmentInfo { } if (normGen != null) { - for (int i = 0; i < normGen.length; i++) { - long gen = normGen[i]; + for (Entry entry : normGen.entrySet()) { + long gen = entry.getValue(); if (gen >= YES) { // Definitely a separate norm file, with generation: - fileSet.add(IndexFileNames.fileNameFromGeneration(name, IndexFileNames.SEPARATE_NORMS_EXTENSION + i, gen)); + fileSet.add(IndexFileNames.fileNameFromGeneration(name, IndexFileNames.SEPARATE_NORMS_EXTENSION + entry.getKey(), gen)); } } } @@ -588,7 +629,7 @@ public final class SegmentInfo { /* Called whenever any change is made that affects which * files this segment has. */ - private void clearFiles() { + void clearFilesCache() { files = null; sizeInBytesNoStore = -1; sizeInBytesWithStore = -1; @@ -623,7 +664,7 @@ public final class SegmentInfo { if (this.dir != dir) { s.append('x'); } - if (hasVectors) { + if (getHasVectors()) { s.append('v'); } s.append(docCount); diff --git a/lucene/src/java/org/apache/lucene/index/SegmentInfos.java b/lucene/src/java/org/apache/lucene/index/SegmentInfos.java index 493279ee17b..8016dde811c 100644 --- a/lucene/src/java/org/apache/lucene/index/SegmentInfos.java +++ b/lucene/src/java/org/apache/lucene/index/SegmentInfos.java @@ -17,26 +17,26 @@ package org.apache.lucene.index; * limitations under the License. */ +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.PrintStream; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Vector; + +import org.apache.lucene.index.codecs.CodecProvider; +import org.apache.lucene.index.codecs.SegmentInfosReader; +import org.apache.lucene.index.codecs.SegmentInfosWriter; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; import org.apache.lucene.store.NoSuchDirectoryException; -import org.apache.lucene.index.codecs.CodecProvider; -import org.apache.lucene.index.codecs.SegmentInfosReader; -import org.apache.lucene.index.codecs.SegmentInfosWriter; import org.apache.lucene.util.ThreadInterruptedException; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.PrintStream; -import java.util.Vector; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.HashMap; -import java.util.Map; - /** * A collection of segmentInfo objects with methods for operating on * those segments in relation to the file system. diff --git a/lucene/src/java/org/apache/lucene/index/SegmentMerger.java b/lucene/src/java/org/apache/lucene/index/SegmentMerger.java index a708c93ffcf..d8ca010127c 100644 --- a/lucene/src/java/org/apache/lucene/index/SegmentMerger.java +++ b/lucene/src/java/org/apache/lucene/index/SegmentMerger.java @@ -26,16 +26,16 @@ import java.util.List; import org.apache.lucene.document.Document; import org.apache.lucene.index.IndexReader.FieldOption; import org.apache.lucene.index.MergePolicy.MergeAbortedException; -import org.apache.lucene.index.codecs.CodecProvider; import org.apache.lucene.index.codecs.Codec; -import org.apache.lucene.index.codecs.MergeState; +import org.apache.lucene.index.codecs.CodecProvider; import org.apache.lucene.index.codecs.FieldsConsumer; +import org.apache.lucene.index.codecs.MergeState; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; import org.apache.lucene.util.Bits; -import org.apache.lucene.util.ReaderUtil; import org.apache.lucene.util.MultiBits; +import org.apache.lucene.util.ReaderUtil; /** * The SegmentMerger class combines two or more Segments, represented by an IndexReader ({@link #add}, @@ -75,8 +75,8 @@ final class SegmentMerger { this.payloadProcessorProvider = payloadProcessorProvider; directory = dir; this.codecs = codecs; - this.fieldInfos = fieldInfos; segment = name; + this.fieldInfos = fieldInfos; if (merge != null) { checkAbort = new MergeState.CheckAbort(merge, directory); } else { @@ -180,9 +180,8 @@ final class SegmentMerger { SegmentReader segmentReader = (SegmentReader) reader; boolean same = true; FieldInfos segmentFieldInfos = segmentReader.fieldInfos(); - int numFieldInfos = segmentFieldInfos.size(); - for (int j = 0; same && j < numFieldInfos; j++) { - same = fieldInfos.fieldName(j).equals(segmentFieldInfos.fieldName(j)); + for (FieldInfo fi : segmentFieldInfos) { + same = fieldInfos.fieldName(fi.number).equals(fi.name); } if (same) { matchingSegmentReaders[i] = segmentReader; @@ -208,9 +207,8 @@ final class SegmentMerger { if (reader instanceof SegmentReader) { SegmentReader segmentReader = (SegmentReader) reader; FieldInfos readerFieldInfos = segmentReader.fieldInfos(); - int numReaderFieldInfos = readerFieldInfos.size(); - for (int j = 0; j < numReaderFieldInfos; j++) { - fieldInfos.add(readerFieldInfos.fieldInfo(j)); + for (FieldInfo fi : readerFieldInfos) { + fieldInfos.add(fi); } } else { addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_POSITION_OFFSET), true, true, true, false, false); @@ -224,13 +222,13 @@ final class SegmentMerger { } } final SegmentCodecs codecInfo = SegmentCodecs.build(fieldInfos, this.codecs); - fieldInfos.write(directory, segment + ".fnm"); + fieldInfos.write(directory, segment + "." + IndexFileNames.FIELD_INFOS_EXTENSION); int docCount = 0; setMatchingSegmentReaders(); - final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment, fieldInfos); + final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment); try { int idx = 0; @@ -312,7 +310,7 @@ final class SegmentMerger { // NOTE: it's very important to first assign to doc then pass it to // termVectorsWriter.addAllDocVectors; see LUCENE-1282 Document doc = reader.document(j); - fieldsWriter.addDocument(doc); + fieldsWriter.addDocument(doc, fieldInfos); docCount++; checkAbort.work(300); } @@ -339,7 +337,7 @@ final class SegmentMerger { // NOTE: it's very important to first assign to doc then pass it to // termVectorsWriter.addAllDocVectors; see LUCENE-1282 Document doc = reader.document(docCount); - fieldsWriter.addDocument(doc); + fieldsWriter.addDocument(doc, fieldInfos); checkAbort.work(300); } } @@ -574,8 +572,7 @@ final class SegmentMerger { private void mergeNorms() throws IOException { IndexOutput output = null; try { - for (int i = 0, numFieldInfos = fieldInfos.size(); i < numFieldInfos; i++) { - final FieldInfo fi = fieldInfos.fieldInfo(i); + for (FieldInfo fi : fieldInfos) { if (fi.isIndexed && !fi.omitNorms) { if (output == null) { output = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.NORMS_EXTENSION)); diff --git a/lucene/src/java/org/apache/lucene/index/SegmentReader.java b/lucene/src/java/org/apache/lucene/index/SegmentReader.java index ac36827bfc6..1cb378101f3 100644 --- a/lucene/src/java/org/apache/lucene/index/SegmentReader.java +++ b/lucene/src/java/org/apache/lucene/index/SegmentReader.java @@ -22,23 +22,22 @@ import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; - import java.util.List; import java.util.Map; import java.util.Set; - import java.util.concurrent.atomic.AtomicInteger; + import org.apache.lucene.document.Document; import org.apache.lucene.document.FieldSelector; +import org.apache.lucene.index.codecs.FieldsProducer; import org.apache.lucene.store.BufferedIndexInput; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; import org.apache.lucene.util.BitVector; import org.apache.lucene.util.Bits; -import org.apache.lucene.util.CloseableThreadLocal; -import org.apache.lucene.index.codecs.FieldsProducer; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.CloseableThreadLocal; /** * @lucene.experimental @@ -120,7 +119,7 @@ public class SegmentReader extends IndexReader implements Cloneable { } cfsDir = dir0; - fieldInfos = new FieldInfos(cfsDir, IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELD_INFOS_EXTENSION)); + fieldInfos = si.getFieldInfos(); this.termsIndexDivisor = termsIndexDivisor; @@ -598,12 +597,12 @@ public class SegmentReader extends IndexReader implements Cloneable { && (!si.hasDeletions() || this.si.getDelFileName().equals(si.getDelFileName())); boolean normsUpToDate = true; - boolean[] fieldNormsChanged = new boolean[core.fieldInfos.size()]; - final int fieldCount = core.fieldInfos.size(); - for (int i = 0; i < fieldCount; i++) { - if (!this.si.getNormFileName(i).equals(si.getNormFileName(i))) { + Set fieldNormsChanged = new HashSet(); + for (FieldInfo fi : core.fieldInfos) { + int fieldNumber = fi.number; + if (!this.si.getNormFileName(fieldNumber).equals(si.getNormFileName(fieldNumber))) { normsUpToDate = false; - fieldNormsChanged[i] = true; + fieldNormsChanged.add(fieldNumber); } } @@ -659,11 +658,10 @@ public class SegmentReader extends IndexReader implements Cloneable { clone.norms = new HashMap(); // Clone norms - for (int i = 0; i < fieldNormsChanged.length; i++) { - + for (FieldInfo fi : core.fieldInfos) { // Clone unchanged norms to the cloned reader - if (doClone || !fieldNormsChanged[i]) { - final String curField = core.fieldInfos.fieldInfo(i).name; + if (doClone || !fieldNormsChanged.contains(fi.number)) { + final String curField = fi.name; Norm norm = this.norms.get(curField); if (norm != null) clone.norms.put(curField, (Norm) norm.clone()); @@ -735,7 +733,7 @@ public class SegmentReader extends IndexReader implements Cloneable { } if (normsDirty) { // re-write norms - si.initNormGen(core.fieldInfos.size()); + si.initNormGen(); for (final Norm norm : norms.values()) { if (norm.dirty) { norm.reWrite(si); @@ -880,8 +878,7 @@ public class SegmentReader extends IndexReader implements Cloneable { ensureOpen(); Set fieldSet = new HashSet(); - for (int i = 0; i < core.fieldInfos.size(); i++) { - FieldInfo fi = core.fieldInfos.fieldInfo(i); + for (FieldInfo fi : core.fieldInfos) { if (fieldOption == IndexReader.FieldOption.ALL) { fieldSet.add(fi.name); } @@ -959,8 +956,7 @@ public class SegmentReader extends IndexReader implements Cloneable { private void openNorms(Directory cfsDir, int readBufferSize) throws IOException { long nextNormSeek = SegmentMerger.NORMS_HEADER.length; //skip header (header unused for now) int maxDoc = maxDoc(); - for (int i = 0; i < core.fieldInfos.size(); i++) { - FieldInfo fi = core.fieldInfos.fieldInfo(i); + for (FieldInfo fi : core.fieldInfos) { if (norms.containsKey(fi.name)) { // in case this SegmentReader is being re-opened, we might be able to // reuse some norm instances and skip loading them here diff --git a/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java b/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java index 05416abd6f6..9f04dcb9786 100644 --- a/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java +++ b/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java @@ -27,15 +27,13 @@ final class StoredFieldsWriter { FieldsWriter fieldsWriter; final DocumentsWriter docWriter; - final FieldInfos fieldInfos; int lastDocID; PerDoc[] docFreeList = new PerDoc[1]; int freeCount; - public StoredFieldsWriter(DocumentsWriter docWriter, FieldInfos fieldInfos) { + public StoredFieldsWriter(DocumentsWriter docWriter) { this.docWriter = docWriter; - this.fieldInfos = fieldInfos; } public StoredFieldsWriterPerThread addThread(DocumentsWriter.DocState docState) throws IOException { @@ -62,7 +60,7 @@ final class StoredFieldsWriter { private synchronized void initFieldsWriter() throws IOException { if (fieldsWriter == null) { - fieldsWriter = new FieldsWriter(docWriter.directory, docWriter.getSegment(), fieldInfos); + fieldsWriter = new FieldsWriter(docWriter.directory, docWriter.getSegment()); lastDocID = 0; } } diff --git a/lucene/src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java b/lucene/src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java index a0e77ae887b..85c6b57583b 100644 --- a/lucene/src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java +++ b/lucene/src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java @@ -32,7 +32,7 @@ final class StoredFieldsWriterPerThread { public StoredFieldsWriterPerThread(DocumentsWriter.DocState docState, StoredFieldsWriter storedFieldsWriter) throws IOException { this.storedFieldsWriter = storedFieldsWriter; this.docState = docState; - localFieldsWriter = new FieldsWriter((IndexOutput) null, (IndexOutput) null, storedFieldsWriter.fieldInfos); + localFieldsWriter = new FieldsWriter((IndexOutput) null, (IndexOutput) null); } public void startDocument() { diff --git a/lucene/src/java/org/apache/lucene/index/TermsHash.java b/lucene/src/java/org/apache/lucene/index/TermsHash.java index 2ad2a827b5f..2c3bc8124ce 100644 --- a/lucene/src/java/org/apache/lucene/index/TermsHash.java +++ b/lucene/src/java/org/apache/lucene/index/TermsHash.java @@ -56,12 +56,6 @@ final class TermsHash extends InvertedDocConsumer { return new TermsHashPerThread(docInverterPerThread, this, nextTermsHash, primaryPerThread); } - @Override - void setFieldInfos(FieldInfos fieldInfos) { - this.fieldInfos = fieldInfos; - consumer.setFieldInfos(fieldInfos); - } - @Override public void abort() { consumer.abort(); diff --git a/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java b/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java index 5f23675ff1d..6488d332ce8 100644 --- a/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java +++ b/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java @@ -25,10 +25,4 @@ abstract class TermsHashConsumer { abstract TermsHashConsumerPerThread addThread(TermsHashPerThread perThread); abstract void flush(Map> threadsAndFields, final SegmentWriteState state) throws IOException; abstract void abort(); - - FieldInfos fieldInfos; - - void setFieldInfos(FieldInfos fieldInfos) { - this.fieldInfos = fieldInfos; } -} diff --git a/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java b/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java index 65b7460f782..28b19a52090 100644 --- a/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java +++ b/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java @@ -19,14 +19,15 @@ package org.apache.lucene.index.codecs.preflex; import java.io.IOException; import java.util.Collection; -import java.util.Iterator; -import java.util.TreeMap; -import java.util.HashMap; -import java.util.Map; import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.TreeMap; -import org.apache.lucene.index.DocsEnum; +import org.apache.lucene.index.CompoundFileReader; import org.apache.lucene.index.DocsAndPositionsEnum; +import org.apache.lucene.index.DocsEnum; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.FieldsEnum; @@ -35,7 +36,6 @@ import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.Term; import org.apache.lucene.index.Terms; import org.apache.lucene.index.TermsEnum; -import org.apache.lucene.index.CompoundFileReader; import org.apache.lucene.index.codecs.FieldsProducer; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IndexInput; @@ -94,13 +94,11 @@ public class PreFlexFields extends FieldsProducer { // so that if an index update removes them we'll still have them freqStream = dir.openInput(info.name + ".frq", readBufferSize); boolean anyProx = false; - final int numFields = fieldInfos.size(); - for(int i=0;i filesToDelete = merger.createCompoundFile(merged + ".cfs", info); diff --git a/lucene/src/test/org/apache/lucene/index/TestDocumentWriter.java b/lucene/src/test/org/apache/lucene/index/TestDocumentWriter.java index fa513194ac3..e3c84067848 100644 --- a/lucene/src/test/org/apache/lucene/index/TestDocumentWriter.java +++ b/lucene/src/test/org/apache/lucene/index/TestDocumentWriter.java @@ -25,20 +25,20 @@ import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.analysis.MockTokenizer; import org.apache.lucene.analysis.TokenFilter; import org.apache.lucene.analysis.TokenStream; +import org.apache.lucene.analysis.tokenattributes.CharTermAttribute; import org.apache.lucene.analysis.tokenattributes.PayloadAttribute; import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute; -import org.apache.lucene.analysis.tokenattributes.CharTermAttribute; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; -import org.apache.lucene.document.Fieldable; import org.apache.lucene.document.Field.Index; import org.apache.lucene.document.Field.Store; import org.apache.lucene.document.Field.TermVector; +import org.apache.lucene.document.Fieldable; import org.apache.lucene.store.Directory; import org.apache.lucene.util.AttributeSource; +import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util._TestUtil; -import org.apache.lucene.util.BytesRef; public class TestDocumentWriter extends LuceneTestCase { private Directory dir; @@ -98,8 +98,7 @@ public class TestDocumentWriter extends LuceneTestCase { // test that the norms are not present in the segment if // omitNorms is true - for (int i = 0; i < reader.core.fieldInfos.size(); i++) { - FieldInfo fi = reader.core.fieldInfos.fieldInfo(i); + for (FieldInfo fi : reader.core.fieldInfos) { if (fi.isIndexed) { assertTrue(fi.omitNorms == !reader.hasNorms(fi.name)); } diff --git a/lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java b/lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java index ea514a861ea..3035af8a957 100644 --- a/lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java +++ b/lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java @@ -92,10 +92,9 @@ public class TestIndexFileDeleter extends LuceneTestCase { CompoundFileReader cfsReader = new CompoundFileReader(dir, "_2.cfs"); FieldInfos fieldInfos = new FieldInfos(cfsReader, "_2.fnm"); int contentFieldIndex = -1; - for(i=0;i