You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by bu...@apache.org on 2011/02/21 19:50:40 UTC
svn commit: r1073110 [1/2] - in /lucene/dev/trunk/lucene: ./
src/java/org/apache/lucene/index/
src/java/org/apache/lucene/index/codecs/preflex/
src/test/org/apache/lucene/index/
Author: buschmi
Date: Mon Feb 21 18:50:39 2011
New Revision: 1073110
URL: http://svn.apache.org/viewvc?rev=1073110&view=rev
Log:
LUCENE-2881: Track FieldInfos per segment
Added:
lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java (with props)
Modified:
lucene/dev/trunk/lucene/CHANGES.txt
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocConsumerPerThread.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldConsumer.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocInverter.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfo.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfos.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldsWriter.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/InvertedDocConsumer.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/NormsWriter.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentCodecs.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHash.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java
lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCodecs.java
lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDoc.java
lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDocumentWriter.java
lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestPerFieldCodecSupport.java
lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestSegmentMerger.java
Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Mon Feb 21 18:50:39 2011
@@ -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
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocConsumerPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocConsumerPerThread.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocConsumerPerThread.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocConsumerPerThread.java Mon Feb 21 18:50:39 2011
@@ -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();
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldConsumer.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldConsumer.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldConsumer.java Mon Feb 21 18:50:39 2011
@@ -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<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>> 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;
}
-}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java Mon Feb 21 18:50:39 2011
@@ -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 Do
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 Do
// 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
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java Mon Feb 21 18:50:39 2011
@@ -41,14 +41,13 @@ final class DocFieldProcessorPerThread e
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 e
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 e
field = next;
}
}
+ doAfterFlush();
fieldsWriter.abort();
consumer.abort();
}
@@ -92,45 +91,15 @@ final class DocFieldProcessorPerThread e
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<fieldHash.length;i++) {
- DocFieldProcessorPerField perField = fieldHash[i];
- DocFieldProcessorPerField lastPerField = null;
-
- while (perField != null) {
-
- if (perField.lastGen == -1) {
-
- // This field was not seen since the previous
- // flush, so, free up its resources now
-
- // Unhash
- if (lastPerField == null)
- fieldHash[i] = perField.next;
- else
- lastPerField.next = perField.next;
-
- if (state.infoStream != null) {
- state.infoStream.println(" purge field=" + perField.fieldInfo.name);
+ /** In flush we reset the fieldHash to not maintain per-field state
+ * across segments */
+ @Override
+ void doAfterFlush() {
+ fieldHash = new DocFieldProcessorPerField[2];
+ hashMask = 1;
+ totalFieldCount = 0;
}
- totalFieldCount--;
-
- } else {
- // Reset
- perField.lastGen = -1;
- lastPerField = perField;
- }
-
- perField = perField.next;
- }
- }
- }
-
private void rehash() {
final int newHashSize = (fieldHash.length*2);
assert newHashSize > fieldHash.length;
@@ -155,7 +124,7 @@ final class DocFieldProcessorPerThread e
}
@Override
- public DocumentsWriter.DocWriter processDocument() throws IOException {
+ public DocumentsWriter.DocWriter processDocument(FieldInfos fieldInfos) throws IOException {
consumer.startDocument();
fieldsWriter.startDocument();
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocInverter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocInverter.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocInverter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocInverter.java Mon Feb 21 18:50:39 2011
@@ -40,13 +40,6 @@ final class DocInverter extends DocField
}
@Override
- void setFieldInfos(FieldInfos fieldInfos) {
- super.setFieldInfos(fieldInfos);
- consumer.setFieldInfos(fieldInfos);
- endConsumer.setFieldInfos(fieldInfos);
- }
-
- @Override
void flush(Map<DocFieldConsumerPerThread, Collection<DocFieldConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException {
Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> childThreadsAndFields = new HashMap<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>>();
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java Mon Feb 21 18:50:39 2011
@@ -279,12 +279,13 @@ final class DocumentsWriter {
private int maxBufferedDocs = IndexWriterConfig.DEFAULT_MAX_BUFFERED_DOCS;
private boolean closed;
- private final FieldInfos fieldInfos;
+ private FieldInfos fieldInfos;
private final BufferedDeletesStream bufferedDeletesStream;
private final IndexWriter.FlushControl flushControl;
- DocumentsWriter(Directory directory, IndexWriter writer, IndexingChain indexingChain, int maxThreadStates, FieldInfos fieldInfos, BufferedDeletesStream bufferedDeletesStream) throws IOException {
+ DocumentsWriter(Directory directory, IndexWriter writer, IndexingChain indexingChain, int maxThreadStates, FieldInfos fieldInfos,
+ BufferedDeletesStream bufferedDeletesStream) throws IOException {
this.directory = directory;
this.writer = writer;
this.similarityProvider = writer.getConfig().getSimilarityProvider();
@@ -350,10 +351,6 @@ final class DocumentsWriter {
return doFlush;
}
- public FieldInfos getFieldInfos() {
- return fieldInfos;
- }
-
/** If non-null, various details of indexing are printed
* here. */
synchronized void setInfoStream(PrintStream infoStream) {
@@ -482,9 +479,14 @@ final class DocumentsWriter {
private void doAfterFlush() throws IOException {
// All ThreadStates should be idle when we are called
assert allThreadsIdle();
+ for (DocumentsWriterThreadState threadState : threadStates) {
+ threadState.consumer.doAfterFlush();
+ }
+
threadBindings.clear();
waitQueue.reset();
segment = null;
+ fieldInfos = fieldInfos.newFieldInfosWithGlobalFieldNumberMap();
numDocs = 0;
nextDocID = 0;
bufferIsFull = false;
@@ -602,7 +604,7 @@ final class DocumentsWriter {
pendingDeletes.docIDs.clear();
}
- newSegment = new SegmentInfo(segment, numDocs, directory, false, fieldInfos.hasProx(), flushState.segmentCodecs, false);
+ newSegment = new SegmentInfo(segment, numDocs, directory, false, flushState.segmentCodecs, fieldInfos);
Collection<DocConsumerPerThread> threads = new HashSet<DocConsumerPerThread>();
for (DocumentsWriterThreadState threadState : threadStates) {
@@ -613,7 +615,7 @@ final class DocumentsWriter {
consumer.flush(threads, flushState);
- newSegment.setHasVectors(flushState.hasVectors);
+ newSegment.clearFilesCache();
if (infoStream != null) {
message("new segment has " + (flushState.hasVectors ? "vectors" : "no vectors"));
@@ -796,7 +798,7 @@ final class DocumentsWriter {
// work
final DocWriter perDoc;
try {
- perDoc = state.consumer.processDocument();
+ perDoc = state.consumer.processDocument(fieldInfos);
} finally {
docState.clear();
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfo.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfo.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfo.java Mon Feb 21 18:50:39 2011
@@ -32,7 +32,7 @@ public final class FieldInfo {
public boolean omitTermFreqAndPositions;
public boolean storePayloads; // whether this field stores payloads together with term positions
- int codecId = 0; // set inside SegmentCodecs#build() during segment flush - this is used to identify the codec used to write this field
+ private int codecId = -1; // set inside SegmentCodecs#build() during segment flush - this is used to identify the codec used to write this field
FieldInfo(String na, boolean tk, int nu, boolean storeTermVector,
boolean storePositionWithTermVector, boolean storeOffsetWithTermVector,
@@ -57,10 +57,21 @@ public final class FieldInfo {
}
}
+ public void setCodecId(int codecId) {
+ assert this.codecId == -1 : "CodecId can only be set once.";
+ this.codecId = codecId;
+ }
+
+ public int getCodecId() {
+ return codecId;
+ }
+
@Override
public Object clone() {
- return new FieldInfo(name, isIndexed, number, storeTermVector, storePositionWithTermVector,
+ FieldInfo clone = new FieldInfo(name, isIndexed, number, storeTermVector, storePositionWithTermVector,
storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions);
+ clone.codecId = this.codecId;
+ return clone;
}
void update(boolean isIndexed, boolean storeTermVector, boolean storePositionWithTermVector,
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfos.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfos.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldInfos.java Mon Feb 21 18:50:39 2011
@@ -17,6 +17,16 @@ package org.apache.lucene.index;
* limitations under the License.
*/
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Fieldable;
import org.apache.lucene.store.Directory;
@@ -24,9 +34,6 @@ import org.apache.lucene.store.IndexInpu
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.StringHelper;
-import java.io.IOException;
-import java.util.*;
-
/** Access to the Fieldable Info file that describes document fields and whether or
* not they are indexed. Each segment has a separate Fieldable Info file. Objects
* of this class are thread-safe for multiple readers, but only one thread can
@@ -34,7 +41,72 @@ import java.util.*;
* accessing this object.
* @lucene.experimental
*/
-public final class FieldInfos {
+public final class FieldInfos implements Iterable<FieldInfo> {
+ private static final class FieldNumberBiMap {
+ private final Map<Integer,String> numberToName;
+ private final Map<String,Integer> nameToNumber;
+
+ private FieldNumberBiMap() {
+ this.nameToNumber = new HashMap<String, Integer>();
+ this.numberToName = new HashMap<Integer, String>();
+ }
+
+ 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<FieldInfo> {
+ private final SortedMap<Integer,FieldInfo> byNumber = new TreeMap<Integer,FieldInfo>();
+ private final HashMap<String,FieldInfo> byName = new HashMap<String,FieldInfo>();
+ 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<FieldInfo> 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<FieldInfo> byNumber = new ArrayList<FieldInfo>();
- private final HashMap<String,FieldInfo> byName = new HashMap<String,FieldInfo>();
+ 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<Integer> 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<numField;i++) {
- FieldInfo fi = (FieldInfo) ( byNumber.get(i)).clone();
- fis.byNumber.add(fi);
- fis.byName.put(fi.name, fi);
+ FieldInfos fis = new FieldInfos(globalFieldNumbers);
+ for (FieldInfo fi : this) {
+ FieldInfo clone = (FieldInfo) (fi).clone();
+ fis.localFieldInfos.put(clone);
}
return fis;
}
@@ -102,9 +192,7 @@ public final class FieldInfos {
/** Returns true if any fields do not omitTermFreqAndPositions */
public boolean hasProx() {
- final int numFields = byNumber.size();
- for(int i=0;i<numFields;i++) {
- final FieldInfo fi = fieldInfo(i);
+ for (FieldInfo fi : this) {
if (fi.isIndexed && !fi.omitTermFreqAndPositions) {
return true;
}
@@ -215,9 +303,28 @@ public final class FieldInfos {
synchronized public FieldInfo add(String name, boolean isIndexed, boolean storeTermVector,
boolean storePositionWithTermVector, boolean storeOffsetWithTermVector,
boolean omitNorms, boolean storePayloads, boolean omitTermFreqAndPositions) {
+ return addOrUpdateInternal(name, -1, isIndexed, storeTermVector, storePositionWithTermVector,
+ storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions);
+ }
+
+ synchronized private FieldInfo addOrUpdateInternal(String name, int preferredFieldNumber, boolean isIndexed,
+ boolean storeTermVector, boolean storePositionWithTermVector, boolean storeOffsetWithTermVector,
+ boolean omitNorms, boolean storePayloads, boolean omitTermFreqAndPositions) {
+
FieldInfo fi = fieldInfo(name);
if (fi == null) {
- return addInternal(name, isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions);
+ if (preferredFieldNumber == -1) {
+ preferredFieldNumber = findNextAvailableFieldNumber(localFieldInfos.nextAvailableNumber, localFieldInfos.byNumber.keySet());
+ localFieldInfos.nextAvailableNumber = preferredFieldNumber;
+ }
+
+ // get a global number for this field
+ int fieldNumber = globalFieldNumbers.addOrGet(name, localFieldInfos, preferredFieldNumber);
+ if (localFieldInfos.get(fieldNumber) != null) {
+ // fall back if the global number is already taken
+ fieldNumber = preferredFieldNumber;
+ }
+ return addInternal(name, fieldNumber, isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions);
} else {
fi.update(isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions);
}
@@ -225,20 +332,27 @@ public final class FieldInfos {
}
synchronized public FieldInfo add(FieldInfo fi) {
- return add(fi.name, fi.isIndexed, fi.storeTermVector,
+ int preferredFieldNumber = fi.number;
+ FieldInfo other = localFieldInfos.get(preferredFieldNumber);
+ if (other == null || !other.name.equals(fi.name)) {
+ preferredFieldNumber = -1;
+ }
+ return addOrUpdateInternal(fi.name, preferredFieldNumber, fi.isIndexed, fi.storeTermVector,
fi.storePositionWithTermVector, fi.storeOffsetWithTermVector,
fi.omitNorms, fi.storePayloads,
fi.omitTermFreqAndPositions);
}
- private FieldInfo addInternal(String name, boolean isIndexed,
+ private FieldInfo addInternal(String name, int fieldNumber, boolean isIndexed,
boolean storeTermVector, boolean storePositionWithTermVector,
boolean storeOffsetWithTermVector, boolean omitNorms, boolean storePayloads, boolean omitTermFreqAndPositions) {
name = StringHelper.intern(name);
- FieldInfo fi = new FieldInfo(name, isIndexed, byNumber.size(), storeTermVector, storePositionWithTermVector,
+ globalFieldNumbers.setIfNotSet(fieldNumber, name);
+ FieldInfo fi = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, storePositionWithTermVector,
storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions);
- byNumber.add(fi);
- byName.put(name, fi);
+
+ assert localFieldInfos.get(fi.number) == null;
+ localFieldInfos.put(fi);
return fi;
}
@@ -248,7 +362,7 @@ public final class FieldInfos {
}
public FieldInfo fieldInfo(String fieldName) {
- return byName.get(fieldName);
+ return localFieldInfos.get(fieldName);
}
/**
@@ -270,16 +384,20 @@ public final class FieldInfos {
* doesn't exist.
*/
public FieldInfo fieldInfo(int fieldNumber) {
- return (fieldNumber >= 0) ? byNumber.get(fieldNumber) : null;
+ return (fieldNumber >= 0) ? localFieldInfos.get(fieldNumber) : null;
+ }
+
+ public Iterator<FieldInfo> 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()) {
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldsWriter.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FieldsWriter.java Mon Feb 21 18:50:39 2011
@@ -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;
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java Mon Feb 21 18:50:39 2011
@@ -38,7 +38,6 @@ import org.apache.lucene.document.Docume
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 Clos
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 Clos
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 Clos
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<fieldCount;fieldNumber++) {
- fieldInfos.add(segFieldInfos.fieldInfo(fieldNumber));
+ for (FieldInfo fi : segFieldInfos) {
+ fieldInfos.add(fi);
}
}
} else {
- // Already a 4.0 index; just seed the FieldInfos
- // from the last segment
- fieldInfos = getFieldInfos(segmentInfos.info(segmentInfos.size()-1));
- }
- } else {
fieldInfos = new FieldInfos();
}
return fieldInfos;
@@ -2272,7 +2266,7 @@ public class IndexWriter implements Clos
String mergedName = newSegmentName();
SegmentMerger merger = new SegmentMerger(directory, termIndexInterval,
mergedName, null, codecs, payloadProcessorProvider,
- ((FieldInfos) docWriter.getFieldInfos().clone()));
+ fieldInfos.newFieldInfosWithGlobalFieldNumberMap());
for (IndexReader reader : readers) // add new indexes
merger.add(reader);
@@ -2280,8 +2274,8 @@ public class IndexWriter implements Clos
int docCount = merger.merge(); // merge 'em
SegmentInfo info = new SegmentInfo(mergedName, docCount, directory,
- false, merger.fieldInfos().hasProx(), merger.getSegmentCodecs(),
- merger.fieldInfos().hasVectors());
+ false, merger.getSegmentCodecs(),
+ merger.fieldInfos());
setDiagnostics(info, "addIndexes(IndexReader...)");
boolean useCompoundFile;
@@ -3014,7 +3008,7 @@ public class IndexWriter implements Clos
// Bind a new segment name here so even with
// ConcurrentMergePolicy we keep deterministic segment
// names.
- merge.info = new SegmentInfo(newSegmentName(), 0, directory, false, false, null, false);
+ merge.info = new SegmentInfo(newSegmentName(), 0, directory, false, null, fieldInfos.newFieldInfosWithGlobalFieldNumberMap());
// Lock order: IW -> BD
final BufferedDeletesStream.ApplyDeletesResult result = bufferedDeletesStream.applyDeletes(readerPool, merge.segments);
@@ -3165,7 +3159,7 @@ public class IndexWriter implements Clos
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 Clos
merge.readers = new ArrayList<SegmentReader>();
merge.readerClones = new ArrayList<SegmentReader>();
- 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 Clos
// 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
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/InvertedDocConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/InvertedDocConsumer.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/InvertedDocConsumer.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/InvertedDocConsumer.java Mon Feb 21 18:50:39 2011
@@ -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;
}
-}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java Mon Feb 21 18:50:39 2011
@@ -25,5 +25,4 @@ abstract class InvertedDocEndConsumer {
abstract InvertedDocEndConsumerPerThread addThread(DocInverterPerThread docInverterPerThread);
abstract void flush(Map<InvertedDocEndConsumerPerThread,Collection<InvertedDocEndConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException;
abstract void abort();
- abstract void setFieldInfos(FieldInfos fieldInfos);
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/NormsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/NormsWriter.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/NormsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/NormsWriter.java Mon Feb 21 18:50:39 2011
@@ -36,7 +36,6 @@ import org.apache.lucene.store.IndexOutp
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 Inverted
// We only write the _X.nrm file at flush
void files(Collection<String> 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 Inverted
final Map<FieldInfo,List<NormsWriterPerField>> byField = new HashMap<FieldInfo,List<NormsWriterPerField>>();
- if (!fieldInfos.hasNorms()) {
+ if (!state.fieldInfos.hasNorms()) {
return;
}
@@ -96,15 +90,10 @@ final class NormsWriter extends Inverted
try {
normsOut.writeBytes(SegmentMerger.NORMS_HEADER, 0, SegmentMerger.NORMS_HEADER.length);
- final int numField = fieldInfos.size();
-
int normCount = 0;
- for(int fieldNumber=0;fieldNumber<numField;fieldNumber++) {
-
- final FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldNumber);
-
- List<NormsWriterPerField> toMerge = byField.get(fieldInfo);
+ for (FieldInfo fi : state.fieldInfos) {
+ List<NormsWriterPerField> toMerge = byField.get(fi);
int upto = 0;
if (toMerge != null) {
@@ -158,7 +147,7 @@ final class NormsWriter extends Inverted
// Fill final hole with defaultNorm
for(;upto<state.numDocs;upto++)
normsOut.writeByte((byte) 0);
- } else if (fieldInfo.isIndexed && !fieldInfo.omitNorms) {
+ } else if (fi.isIndexed && !fi.omitNorms) {
normCount++;
// Fill entire field with default norm:
for(;upto<state.numDocs;upto++)
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java Mon Feb 21 18:50:39 2011
@@ -67,7 +67,7 @@ final class PerFieldCodecWrapper extends
@Override
public TermsConsumer addField(FieldInfo field) throws IOException {
- final FieldsConsumer fields = consumers.get(field.codecId);
+ final FieldsConsumer fields = consumers.get(field.getCodecId());
return fields.addField(field);
}
@@ -100,18 +100,16 @@ final class PerFieldCodecWrapper extends
public FieldsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo si,
int readBufferSize, int indexDivisor) throws IOException {
- final int fieldCount = fieldInfos.size();
final Map<Codec, FieldsProducer> producers = new HashMap<Codec, FieldsProducer>();
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));
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentCodecs.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentCodecs.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentCodecs.java Mon Feb 21 18:50:39 2011
@@ -74,22 +74,20 @@ final class SegmentCodecs implements Clo
}
static SegmentCodecs build(FieldInfos infos, CodecProvider provider) {
- final int size = infos.size();
final Map<Codec, Integer> codecRegistry = new IdentityHashMap<Codec, Integer>();
final ArrayList<Codec> codecs = new ArrayList<Codec>();
- 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));
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java Mon Feb 21 18:50:39 2011
@@ -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 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.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 java.util.HashSet;
-import java.util.HashMap;
-import java.util.ArrayList;
+
+import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
+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<Integer,Long> 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
+
+ @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 boolean hasVectors; // 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<Integer, Long>(src.normGen.size());
+ for (Entry<Integer,Long> 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<Integer, Long>();
for(int j=0;j<numNormGen;j++) {
- normGen[j] = input.readLong();
+ int fieldNumber = j;
+ if (format <= DefaultSegmentInfosWriter.FORMAT_4_0) {
+ fieldNumber = input.readInt();
}
+
+ normGen.put(fieldNumber, input.readLong());
+ }
}
isCompoundFile = input.readByte() == YES;
+ Directory dir0 = dir;
+ if (isCompoundFile) {
+ dir0 = new CompoundFileReader(dir, IndexFileNames.segmentFileName(name, "", IndexFileNames.COMPOUND_FILE_EXTENSION));
+ }
+
+ try {
+ fieldInfos = new FieldInfos(dir0, IndexFileNames.segmentFileName(name, "", IndexFileNames.FIELD_INFOS_EXTENSION));
+ } finally {
+ if (dir != dir0) {
+ dir0.close();
+ }
+ }
+
delCount = input.readInt();
assert delCount <= docCount;
- hasProx = input.readByte() == YES;
+ hasProx = input.readByte();
// System.out.println(Thread.currentThread().getName() + ": si.read hasProx=" + hasProx + " seg=" + name);
segmentCodecs = new SegmentCodecs(codecs);
@@ -208,7 +238,7 @@ public final class SegmentInfo {
diagnostics = input.readStringStringMap();
if (format <= DefaultSegmentInfosWriter.FORMAT_HAS_VECTORS) {
- hasVectors = input.readByte() == 1;
+ hasVectors = input.readByte();
} else {
final String storesSegment;
final String ext;
@@ -229,7 +259,11 @@ public final class SegmentInfo {
dirToTest = dir;
}
try {
- hasVectors = dirToTest.fileExists(IndexFileNames.segmentFileName(storesSegment, "", IndexFileNames.VECTORS_INDEX_EXTENSION));
+ if (dirToTest.fileExists(IndexFileNames.segmentFileName(storesSegment, "", IndexFileNames.VECTORS_INDEX_EXTENSION))) {
+ hasVectors = YES;
+ } else {
+ hasVectors = NO;
+ }
} finally {
if (isCompoundFile) {
dirToTest.close();
@@ -274,13 +308,13 @@ public final class SegmentInfo {
}
}
- public boolean getHasVectors() throws IOException {
- return hasVectors;
+ public boolean getHasVectors() {
+ return hasVectors == CHECK_FIELDINFOS ?
+ (fieldInfos == null ? true : fieldInfos.hasVectors()) : hasVectors == YES;
}
- public void setHasVectors(boolean v) {
- hasVectors = v;
- clearFiles();
+ public FieldInfos getFieldInfos() {
+ return fieldInfos;
}
public boolean hasDeletions() {
@@ -298,17 +332,18 @@ public final class SegmentInfo {
} else {
delGen++;
}
- clearFiles();
+ clearFilesCache();
}
void clearDelGen() {
delGen = NO;
- clearFiles();
+ clearFilesCache();
}
@Override
public Object clone() {
- SegmentInfo si = new SegmentInfo(name, docCount, dir, isCompoundFile, hasProx, segmentCodecs, false);
+ SegmentInfo si = new SegmentInfo(name, docCount, dir, isCompoundFile, segmentCodecs,
+ fieldInfos == null ? null : (FieldInfos) fieldInfos.clone());
si.docStoreOffset = docStoreOffset;
si.docStoreSegment = docStoreSegment;
si.docStoreIsCompoundFile = docStoreIsCompoundFile;
@@ -316,8 +351,12 @@ public final class SegmentInfo {
si.delCount = delCount;
si.diagnostics = new HashMap<String, String>(diagnostics);
if (normGen != null) {
- si.normGen = normGen.clone();
+ si.normGen = new HashMap<Integer, Long>();
+ for (Entry<Integer,Long> 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<Integer, Long>();
}
}
@@ -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<Integer,Long> 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<Integer,Long> 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);
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java Mon Feb 21 18:50:39 2011
@@ -17,25 +17,25 @@ package org.apache.lucene.index;
* limitations under the License.
*/
-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.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.util.ThreadInterruptedException;
/**
* A collection of segmentInfo objects with methods for operating on
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java Mon Feb 21 18:50:39 2011
@@ -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));
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java Mon Feb 21 18:50:39 2011
@@ -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 Index
}
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 Index
&& (!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<Integer> fieldNormsChanged = new HashSet<Integer>();
+ 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 Index
clone.norms = new HashMap<String,Norm>();
// 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 Index
}
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 Index
ensureOpen();
Set<String> fieldSet = new HashSet<String>();
- 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 Index
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
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java Mon Feb 21 18:50:39 2011
@@ -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;
}
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java Mon Feb 21 18:50:39 2011
@@ -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() {
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHash.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHash.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHash.java Mon Feb 21 18:50:39 2011
@@ -57,12 +57,6 @@ final class TermsHash extends InvertedDo
}
@Override
- void setFieldInfos(FieldInfos fieldInfos) {
- this.fieldInfos = fieldInfos;
- consumer.setFieldInfos(fieldInfos);
- }
-
- @Override
public void abort() {
consumer.abort();
if (nextTermsHash != null)
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java Mon Feb 21 18:50:39 2011
@@ -25,10 +25,4 @@ abstract class TermsHashConsumer {
abstract TermsHashConsumerPerThread addThread(TermsHashPerThread perThread);
abstract void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException;
abstract void abort();
-
- FieldInfos fieldInfos;
-
- void setFieldInfos(FieldInfos fieldInfos) {
- this.fieldInfos = fieldInfos;
}
-}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java Mon Feb 21 18:50:39 2011
@@ -19,14 +19,15 @@ package org.apache.lucene.index.codecs.p
import java.io.IOException;
import java.util.Collection;
-import java.util.Iterator;
-import java.util.TreeMap;
+import java.util.Comparator;
import java.util.HashMap;
+import java.util.Iterator;
import java.util.Map;
-import java.util.Comparator;
+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.SegmentIn
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 Field
// 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<numFields;i++) {
- final FieldInfo fieldInfo = fieldInfos.fieldInfo(i);
- if (fieldInfo.isIndexed) {
- fields.put(fieldInfo.name, fieldInfo);
- preTerms.put(fieldInfo.name, new PreTerms(fieldInfo));
- if (!fieldInfo.omitTermFreqAndPositions) {
+ for (FieldInfo fi : fieldInfos) {
+ if (fi.isIndexed) {
+ fields.put(fi.name, fi);
+ preTerms.put(fi.name, new PreTerms(fi));
+ if (!fi.omitTermFreqAndPositions) {
anyProx = true;
}
}
Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Mon Feb 21 18:50:39 2011
@@ -528,10 +528,9 @@ public class TestBackwardsCompatibility
CompoundFileReader cfsReader = new CompoundFileReader(dir, "_0.cfs");
FieldInfos fieldInfos = new FieldInfos(cfsReader, "_0.fnm");
int contentFieldIndex = -1;
- for(int i=0;i<fieldInfos.size();i++) {
- FieldInfo fi = fieldInfos.fieldInfo(i);
+ for (FieldInfo fi : fieldInfos) {
if (fi.name.equals("content")) {
- contentFieldIndex = i;
+ contentFieldIndex = fi.number;
break;
}
}
Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCodecs.java?rev=1073110&r1=1073109&r2=1073110&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCodecs.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestCodecs.java Mon Feb 21 18:50:39 2011
@@ -23,14 +23,14 @@ import java.util.HashSet;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field.Store;
import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Field.Store;
import org.apache.lucene.index.codecs.CodecProvider;
import org.apache.lucene.index.codecs.FieldsConsumer;
import org.apache.lucene.index.codecs.FieldsProducer;
import org.apache.lucene.index.codecs.PostingsConsumer;
-import org.apache.lucene.index.codecs.TermsConsumer;
import org.apache.lucene.index.codecs.TermStats;
+import org.apache.lucene.index.codecs.TermsConsumer;
import org.apache.lucene.index.codecs.mocksep.MockSepCodec;
import org.apache.lucene.index.codecs.preflex.PreFlexCodec;
import org.apache.lucene.search.DocIdSetIterator;
@@ -238,9 +238,9 @@ public class TestCodecs extends LuceneTe
final FieldData[] fields = new FieldData[] {field};
final Directory dir = newDirectory();
+ FieldInfos clonedFieldInfos = (FieldInfos) fieldInfos.clone();
this.write(fieldInfos, dir, fields, true);
- final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, true, SegmentCodecs.build(fieldInfos, CodecProvider.getDefault()), fieldInfos.hasVectors());
- si.setHasProx(false);
+ final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, SegmentCodecs.build(clonedFieldInfos, CodecProvider.getDefault()), clonedFieldInfos);
final FieldsProducer reader = si.getSegmentCodecs().codec().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, 64, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR));
@@ -289,8 +289,10 @@ public class TestCodecs extends LuceneTe
if (VERBOSE) {
System.out.println("TEST: now write postings");
}
+
+ FieldInfos clonedFieldInfos = (FieldInfos) fieldInfos.clone();
this.write(fieldInfos, dir, fields, false);
- final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, true, SegmentCodecs.build(fieldInfos, CodecProvider.getDefault()), fieldInfos.hasVectors());
+ final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, SegmentCodecs.build(clonedFieldInfos, CodecProvider.getDefault()), clonedFieldInfos);
if (VERBOSE) {
System.out.println("TEST: now read postings");
@@ -440,7 +442,7 @@ public class TestCodecs extends LuceneTe
final FieldData field = fields[TestCodecs.random.nextInt(fields.length)];
final TermsEnum termsEnum = termsDict.terms(field.fieldInfo.name).iterator();
- if (si.getSegmentCodecs().codecs[field.fieldInfo.codecId] instanceof PreFlexCodec) {
+ if (si.getSegmentCodecs().codecs[field.fieldInfo.getCodecId()] instanceof PreFlexCodec) {
// code below expects unicode sort order
continue;
}
@@ -594,7 +596,7 @@ public class TestCodecs extends LuceneTe
final FieldsConsumer consumer = state.segmentCodecs.codec().fieldsConsumer(state);
Arrays.sort(fields);
for (final FieldData field : fields) {
- if (!allowPreFlex && codecInfo.codecs[field.fieldInfo.codecId] instanceof PreFlexCodec) {
+ if (!allowPreFlex && codecInfo.codecs[field.fieldInfo.getCodecId()] instanceof PreFlexCodec) {
// code below expects unicode sort order
continue;
}