You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2011/02/28 17:33:18 UTC
svn commit: r1075413 [1/2] - in /lucene/dev/trunk: lucene/
lucene/src/java/org/apache/lucene/index/
lucene/src/java/org/apache/lucene/index/codecs/preflex/
lucene/src/test/org/apache/lucene/index/ solr/example/
Author: rmuir
Date: Mon Feb 28 16:33:16 2011
New Revision: 1075413
URL: http://svn.apache.org/viewvc?rev=1075413&view=rev
Log:
LUCENE-2881: back out until corruption issues can be resolved
Removed:
lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java
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
lucene/dev/trunk/solr/example/ (props changed)
Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1075413&r1=1075412&r2=1075413&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Mon Feb 28 16:33:16 2011
@@ -163,11 +163,6 @@ 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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 2011
@@ -27,8 +27,7 @@ abstract class DocConsumerPerThread {
* DocumentsWriter.DocWriter and return it.
* DocumentsWriter then calls finish() on this object
* when it's its turn. */
- abstract DocumentsWriter.DocWriter processDocument(FieldInfos fieldInfos) throws IOException;
+ abstract DocumentsWriter.DocWriter processDocument() 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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 2011
@@ -22,6 +22,9 @@ 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;
@@ -36,4 +39,8 @@ 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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 2011
@@ -34,13 +34,16 @@ 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;
- fieldsWriter = new StoredFieldsWriter(docWriter);
+ fieldInfos = docWriter.getFieldInfos();
+ consumer.setFieldInfos(fieldInfos);
+ fieldsWriter = new StoredFieldsWriter(docWriter, fieldInfos);
}
@Override
@@ -50,6 +53,7 @@ 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);
@@ -59,14 +63,7 @@ final class DocFieldProcessor extends Do
// FreqProxTermsWriter does this with
// FieldInfo.storePayload.
final String fileName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.FIELD_INFOS_EXTENSION);
-
- // If this segment only has docs that hit non-aborting exceptions,
- // then no term vectors files will have been written; therefore we
- // need to update the fieldInfos and clear the term vectors bits
- if (!state.hasVectors) {
- state.fieldInfos.clearVectors();
- }
- state.fieldInfos.write(state.directory, fileName);
+ 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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 2011
@@ -41,13 +41,14 @@ 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 seen in current segment
+ // Hash table for all fields ever seen
DocFieldProcessorPerField[] fieldHash = new DocFieldProcessorPerField[2];
int hashMask = 1;
int totalFieldCount;
@@ -59,6 +60,7 @@ 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);
}
@@ -73,7 +75,6 @@ final class DocFieldProcessorPerThread e
field = next;
}
}
- doAfterFlush();
fieldsWriter.abort();
consumer.abort();
}
@@ -91,15 +92,45 @@ final class DocFieldProcessorPerThread e
return fields;
}
- /** 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;
+ /** 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);
}
+ totalFieldCount--;
+
+ } else {
+ // Reset
+ perField.lastGen = -1;
+ lastPerField = perField;
+ }
+
+ perField = perField.next;
+ }
+ }
+ }
+
private void rehash() {
final int newHashSize = (fieldHash.length*2);
assert newHashSize > fieldHash.length;
@@ -124,7 +155,7 @@ final class DocFieldProcessorPerThread e
}
@Override
- public DocumentsWriter.DocWriter processDocument(FieldInfos fieldInfos) throws IOException {
+ public DocumentsWriter.DocWriter processDocument() 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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 2011
@@ -40,6 +40,13 @@ 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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 2011
@@ -279,13 +279,12 @@ final class DocumentsWriter {
private int maxBufferedDocs = IndexWriterConfig.DEFAULT_MAX_BUFFERED_DOCS;
private boolean closed;
- private FieldInfos fieldInfos;
+ private final 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();
@@ -351,6 +350,10 @@ 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) {
@@ -479,14 +482,9 @@ 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;
@@ -604,7 +602,7 @@ final class DocumentsWriter {
pendingDeletes.docIDs.clear();
}
- newSegment = new SegmentInfo(segment, numDocs, directory, false, flushState.segmentCodecs, fieldInfos);
+ newSegment = new SegmentInfo(segment, numDocs, directory, false, fieldInfos.hasProx(), flushState.segmentCodecs, false);
Collection<DocConsumerPerThread> threads = new HashSet<DocConsumerPerThread>();
for (DocumentsWriterThreadState threadState : threadStates) {
@@ -615,7 +613,7 @@ final class DocumentsWriter {
consumer.flush(threads, flushState);
- newSegment.clearFilesCache();
+ newSegment.setHasVectors(flushState.hasVectors);
if (infoStream != null) {
message("new segment has " + (flushState.hasVectors ? "vectors" : "no vectors"));
@@ -798,7 +796,7 @@ final class DocumentsWriter {
// work
final DocWriter perDoc;
try {
- perDoc = state.consumer.processDocument(fieldInfos);
+ perDoc = state.consumer.processDocument();
} 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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 2011
@@ -32,7 +32,7 @@ public final class FieldInfo {
public boolean omitTermFreqAndPositions;
public boolean storePayloads; // whether this field stores payloads together with term positions
- private int codecId = -1; // set inside SegmentCodecs#build() during segment flush - this is used to identify the codec used to write this field
+ int codecId = 0; // 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,21 +57,10 @@ 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() {
- FieldInfo clone = new FieldInfo(name, isIndexed, number, storeTermVector, storePositionWithTermVector,
+ return 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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 2011
@@ -17,16 +17,6 @@ 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;
@@ -34,6 +24,9 @@ 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
@@ -41,71 +34,7 @@ import org.apache.lucene.util.StringHelp
* accessing this object.
* @lucene.experimental
*/
-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();
- }
-
- public Iterator<FieldInfo> iterator() {
- return byNumber.values().iterator();
- }
- }
+public final class FieldInfos {
// First used in 2.9; prior to 2.9 there was no format header
public static final int FORMAT_START = -2;
@@ -124,18 +53,11 @@ public final class FieldInfos implements
static final byte STORE_PAYLOADS = 0x20;
static final byte OMIT_TERM_FREQ_AND_POSITIONS = 0x40;
- private final FieldNumberBiMap globalFieldNumbers;
- private final FieldInfoBiMap localFieldInfos;
-
+ private final ArrayList<FieldInfo> byNumber = new ArrayList<FieldInfo>();
+ private final HashMap<String,FieldInfo> byName = new HashMap<String,FieldInfo>();
private int format;
public FieldInfos() {
- this(new FieldNumberBiMap());
- }
-
- private FieldInfos(FieldNumberBiMap globalFieldNumbers) {
- this.globalFieldNumbers = globalFieldNumbers;
- this.localFieldInfos = new FieldInfoBiMap();
}
/**
@@ -146,7 +68,6 @@ public final class FieldInfos implements
* @throws IOException
*/
public FieldInfos(Directory d, String name) throws IOException {
- this(new FieldNumberBiMap());
IndexInput input = d.openInput(name);
try {
read(input, name);
@@ -155,27 +76,17 @@ public final class FieldInfos implements
}
}
- 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(globalFieldNumbers);
- for (FieldInfo fi : this) {
- FieldInfo clone = (FieldInfo) (fi).clone();
- fis.localFieldInfos.put(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);
}
return fis;
}
@@ -191,7 +102,9 @@ public final class FieldInfos implements
/** Returns true if any fields do not omitTermFreqAndPositions */
public boolean hasProx() {
- for (FieldInfo fi : this) {
+ final int numFields = byNumber.size();
+ for(int i=0;i<numFields;i++) {
+ final FieldInfo fi = fieldInfo(i);
if (fi.isIndexed && !fi.omitTermFreqAndPositions) {
return true;
}
@@ -302,28 +215,9 @@ public final class FieldInfos implements
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) {
- 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);
+ return addInternal(name, isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions);
} else {
fi.update(isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions);
}
@@ -331,27 +225,20 @@ public final class FieldInfos implements
}
synchronized public FieldInfo add(FieldInfo fi) {
- 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,
+ return add(fi.name, fi.isIndexed, fi.storeTermVector,
fi.storePositionWithTermVector, fi.storeOffsetWithTermVector,
fi.omitNorms, fi.storePayloads,
fi.omitTermFreqAndPositions);
}
- private FieldInfo addInternal(String name, int fieldNumber, boolean isIndexed,
+ private FieldInfo addInternal(String name, boolean isIndexed,
boolean storeTermVector, boolean storePositionWithTermVector,
boolean storeOffsetWithTermVector, boolean omitNorms, boolean storePayloads, boolean omitTermFreqAndPositions) {
name = StringHelper.intern(name);
- globalFieldNumbers.setIfNotSet(fieldNumber, name);
- FieldInfo fi = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, storePositionWithTermVector,
+ FieldInfo fi = new FieldInfo(name, isIndexed, byNumber.size(), storeTermVector, storePositionWithTermVector,
storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions);
-
- assert localFieldInfos.get(fi.number) == null;
- localFieldInfos.put(fi);
+ byNumber.add(fi);
+ byName.put(name, fi);
return fi;
}
@@ -361,7 +248,7 @@ public final class FieldInfos implements
}
public FieldInfo fieldInfo(String fieldName) {
- return localFieldInfos.get(fieldName);
+ return byName.get(fieldName);
}
/**
@@ -383,37 +270,25 @@ public final class FieldInfos implements
* doesn't exist.
*/
public FieldInfo fieldInfo(int fieldNumber) {
- return (fieldNumber >= 0) ? localFieldInfos.get(fieldNumber) : null;
- }
-
- public Iterator<FieldInfo> iterator() {
- return localFieldInfos.iterator();
+ return (fieldNumber >= 0) ? byNumber.get(fieldNumber) : null;
}
public int size() {
- return localFieldInfos.size();
+ return byNumber.size();
}
public boolean hasVectors() {
- for (FieldInfo fi : this) {
- if (fi.storeTermVector) {
+ for (int i = 0; i < size(); i++) {
+ if (fieldInfo(i).storeTermVector) {
return true;
}
}
return false;
}
- void clearVectors() {
- for (FieldInfo fi : this) {
- fi.storeTermVector = false;
- fi.storeOffsetWithTermVector = false;
- fi.storePositionWithTermVector = false;
- }
- }
-
public boolean hasNorms() {
- for (FieldInfo fi : this) {
- if (!fi.omitNorms) {
+ for (int i = 0; i < size(); i++) {
+ if (!fieldInfo(i).omitNorms) {
return true;
}
}
@@ -432,7 +307,8 @@ public final class FieldInfos implements
public void write(IndexOutput output) throws IOException {
output.writeVInt(FORMAT_CURRENT);
output.writeVInt(size());
- for (FieldInfo fi : this) {
+ for (int i = 0; i < size(); i++) {
+ FieldInfo fi = fieldInfo(i);
byte bits = 0x0;
if (fi.isIndexed) bits |= IS_INDEXED;
if (fi.storeTermVector) bits |= STORE_TERMVECTOR;
@@ -442,8 +318,7 @@ public final class FieldInfos implements
if (fi.storePayloads) bits |= STORE_PAYLOADS;
if (fi.omitTermFreqAndPositions) bits |= OMIT_TERM_FREQ_AND_POSITIONS;
output.writeString(fi.name);
- output.writeInt(fi.number);
- output.writeInt(fi.getCodecId());
+ output.writeInt(fi.codecId);
output.writeByte(bits);
}
}
@@ -463,7 +338,6 @@ public final class FieldInfos implements
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;
@@ -473,8 +347,8 @@ public final class FieldInfos implements
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, fieldNumber, isIndexed, storeTermVector, storePositionsWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions);
- addInternal.setCodecId(codecId);
+ final FieldInfo addInternal = addInternal(name, isIndexed, storeTermVector, storePositionsWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, omitTermFreqAndPositions);
+ addInternal.codecId = 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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 2011
@@ -45,12 +45,14 @@ 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) throws IOException {
+ FieldsWriter(Directory directory, String segment, FieldInfos fn) throws IOException {
this.directory = directory;
this.segment = segment;
+ fieldInfos = fn;
boolean success = false;
try {
@@ -68,9 +70,10 @@ final class FieldsWriter {
}
}
- FieldsWriter(IndexOutput fdx, IndexOutput fdt) {
+ FieldsWriter(IndexOutput fdx, IndexOutput fdt, FieldInfos fn) {
directory = null;
segment = null;
+ fieldInfos = fn;
fieldsStream = fdt;
indexStream = fdx;
}
@@ -163,7 +166,7 @@ final class FieldsWriter {
assert fieldsStream.getFilePointer() == position;
}
- final void addDocument(Document doc, FieldInfos fieldInfos) throws IOException {
+ final void addDocument(Document doc) 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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 2011
@@ -38,6 +38,7 @@ 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;
@@ -220,7 +221,6 @@ public class IndexWriter implements Clos
volatile long pendingCommitChangeCount;
final SegmentInfos segmentInfos; // the segments
- final FieldInfos fieldInfos;
private DocumentsWriter docWriter;
private IndexFileDeleter deleter;
@@ -791,10 +791,7 @@ public class IndexWriter implements Clos
setRollbackSegmentInfos(segmentInfos);
- // start with previous field numbers, but new FieldInfos
- fieldInfos = getCurrentFieldInfos();
- docWriter = new DocumentsWriter(directory, this, conf.getIndexingChain(), conf.getMaxThreadStates(),
- fieldInfos.newFieldInfosWithGlobalFieldNumberMap(), bufferedDeletesStream);
+ docWriter = new DocumentsWriter(directory, this, conf.getIndexingChain(), conf.getMaxThreadStates(), getCurrentFieldInfos(), bufferedDeletesStream);
docWriter.setInfoStream(infoStream);
// Default deleter (for backwards compatibility) is
@@ -857,14 +854,23 @@ public class IndexWriter implements Clos
private FieldInfos getCurrentFieldInfos() throws IOException {
final FieldInfos fieldInfos;
if (segmentInfos.size() > 0) {
- fieldInfos = new FieldInfos();
- for(SegmentInfo info : segmentInfos) {
- final FieldInfos segFieldInfos = getFieldInfos(info);
- for (FieldInfo fi : segFieldInfos) {
- fieldInfos.add(fi);
+ 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));
}
}
} 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;
@@ -2266,7 +2272,7 @@ public class IndexWriter implements Clos
String mergedName = newSegmentName();
SegmentMerger merger = new SegmentMerger(directory, termIndexInterval,
mergedName, null, codecs, payloadProcessorProvider,
- fieldInfos.newFieldInfosWithGlobalFieldNumberMap());
+ ((FieldInfos) docWriter.getFieldInfos().clone()));
for (IndexReader reader : readers) // add new indexes
merger.add(reader);
@@ -2274,8 +2280,8 @@ public class IndexWriter implements Clos
int docCount = merger.merge(); // merge 'em
SegmentInfo info = new SegmentInfo(mergedName, docCount, directory,
- false, merger.getSegmentCodecs(),
- merger.fieldInfos());
+ false, merger.fieldInfos().hasProx(), merger.getSegmentCodecs(),
+ merger.fieldInfos().hasVectors());
setDiagnostics(info, "addIndexes(IndexReader...)");
boolean useCompoundFile;
@@ -3008,7 +3014,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, null, fieldInfos.newFieldInfosWithGlobalFieldNumberMap());
+ merge.info = new SegmentInfo(newSegmentName(), 0, directory, false, false, null, false);
// Lock order: IW -> BD
final BufferedDeletesStream.ApplyDeletesResult result = bufferedDeletesStream.applyDeletes(readerPool, merge.segments);
@@ -3159,7 +3165,7 @@ public class IndexWriter implements Clos
SegmentMerger merger = new SegmentMerger(directory, termIndexInterval, mergedName, merge,
codecs, payloadProcessorProvider,
- merge.info.getFieldInfos());
+ ((FieldInfos) docWriter.getFieldInfos().clone()));
if (infoStream != null) {
message("merging " + merge.segString(directory) + " mergeVectors=" + merger.fieldInfos().hasVectors());
@@ -3168,8 +3174,7 @@ public class IndexWriter implements Clos
merge.readers = new ArrayList<SegmentReader>();
merge.readerClones = new ArrayList<SegmentReader>();
- merge.info.clearFilesCache();
-
+ merge.info.setHasVectors(merger.fieldInfos().hasVectors());
// This is try/finally to make sure merger's readers are
// closed:
@@ -3225,7 +3230,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.clearFilesCache();
+ merge.info.setHasProx(merger.fieldInfos().hasProx());
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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 2011
@@ -35,4 +35,10 @@ 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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 2011
@@ -25,4 +25,5 @@ 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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 2011
@@ -36,6 +36,7 @@ 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);
@@ -47,6 +48,11 @@ 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
@@ -54,7 +60,7 @@ final class NormsWriter extends Inverted
final Map<FieldInfo,List<NormsWriterPerField>> byField = new HashMap<FieldInfo,List<NormsWriterPerField>>();
- if (!state.fieldInfos.hasNorms()) {
+ if (!fieldInfos.hasNorms()) {
return;
}
@@ -90,10 +96,15 @@ 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 (FieldInfo fi : state.fieldInfos) {
- List<NormsWriterPerField> toMerge = byField.get(fi);
+ for(int fieldNumber=0;fieldNumber<numField;fieldNumber++) {
+
+ final FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldNumber);
+
+ List<NormsWriterPerField> toMerge = byField.get(fieldInfo);
int upto = 0;
if (toMerge != null) {
@@ -147,7 +158,7 @@ final class NormsWriter extends Inverted
// Fill final hole with defaultNorm
for(;upto<state.numDocs;upto++)
normsOut.writeByte((byte) 0);
- } else if (fi.isIndexed && !fi.omitNorms) {
+ } else if (fieldInfo.isIndexed && !fieldInfo.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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 2011
@@ -67,7 +67,7 @@ final class PerFieldCodecWrapper extends
@Override
public TermsConsumer addField(FieldInfo field) throws IOException {
- final FieldsConsumer fields = consumers.get(field.getCodecId());
+ final FieldsConsumer fields = consumers.get(field.codecId);
return fields.addField(field);
}
@@ -100,16 +100,18 @@ 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 (FieldInfo fi : fieldInfos) {
+ for (int i = 0; i < fieldCount; i++) {
+ FieldInfo fi = fieldInfos.fieldInfo(i);
if (fi.isIndexed) { // TODO this does not work for non-indexed fields
fields.add(fi.name);
- Codec codec = segmentCodecs.codecs[fi.getCodecId()];
+ Codec codec = segmentCodecs.codecs[fi.codecId];
if (!producers.containsKey(codec)) {
producers.put(codec, codec.fieldsProducer(new SegmentReadState(dir,
- si, fieldInfos, readBufferSize, indexDivisor, ""+fi.getCodecId())));
+ si, fieldInfos, readBufferSize, indexDivisor, ""+fi.codecId)));
}
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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 2011
@@ -74,20 +74,22 @@ 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 (FieldInfo fi : infos) {
- if (fi.isIndexed) {
+ for (int i = 0; i < size; i++) {
+ final FieldInfo info = infos.fieldInfo(i);
+ if (info.isIndexed) {
final Codec fieldCodec = provider.lookup(provider
- .getFieldCodec(fi.name));
+ .getFieldCodec(info.name));
Integer ord = codecRegistry.get(fieldCodec);
if (ord == null) {
ord = Integer.valueOf(codecs.size());
codecRegistry.put(fieldCodec, ord);
codecs.add(fieldCodec);
}
- fi.setCodecId(ord.intValue());
+ info.codecId = 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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 2011
@@ -17,22 +17,21 @@ 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.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
+import java.util.Arrays;
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 org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.util.Constants;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.ArrayList;
/**
* Information about a segment such as it's name, directory, and files related
@@ -42,9 +41,6 @@ import org.apache.lucene.util.Constants;
*/
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.
@@ -66,7 +62,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 Map<Integer,Long> normGen;
+ private long[] normGen;
private boolean isCompoundFile;
@@ -84,15 +80,9 @@ public final class SegmentInfo {
private int delCount; // How many deleted docs in this segment
- @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 hasProx; // True if this segment has any fields with omitTermFreqAndPositions==false
- private FieldInfos fieldInfos;
+ private boolean hasVectors; // True if this segment wrote term vectors
private SegmentCodecs segmentCodecs;
@@ -110,7 +100,7 @@ public final class SegmentInfo {
private long bufferedDeletesGen;
public SegmentInfo(String name, int docCount, Directory dir, boolean isCompoundFile,
- SegmentCodecs segmentCodecs, FieldInfos fieldInfos) {
+ boolean hasProx, SegmentCodecs segmentCodecs, boolean hasVectors) {
this.name = name;
this.docCount = docCount;
this.dir = dir;
@@ -118,17 +108,18 @@ 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) {
- clearFilesCache();
+ clearFiles();
version = src.version;
name = src.name;
docCount = src.docCount;
@@ -139,14 +130,11 @@ 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 HashMap<Integer, Long>(src.normGen.size());
- for (Entry<Integer,Long> entry : src.normGen.entrySet()) {
- normGen.put(entry.getKey(), entry.getValue());
- }
+ normGen = new long[src.normGen.length];
+ System.arraycopy(src.normGen, 0, normGen, 0, src.normGen.length);
}
isCompoundFile = src.isCompoundFile;
delCount = src.delCount;
@@ -196,35 +184,17 @@ public final class SegmentInfo {
if (numNormGen == NO) {
normGen = null;
} else {
- normGen = new HashMap<Integer, Long>();
+ normGen = new long[numNormGen];
for(int j=0;j<numNormGen;j++) {
- int fieldNumber = j;
- if (format <= DefaultSegmentInfosWriter.FORMAT_4_0) {
- fieldNumber = input.readInt();
+ normGen[j] = input.readLong();
}
-
- 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();
+ hasProx = input.readByte() == YES;
// System.out.println(Thread.currentThread().getName() + ": si.read hasProx=" + hasProx + " seg=" + name);
segmentCodecs = new SegmentCodecs(codecs);
@@ -238,7 +208,7 @@ public final class SegmentInfo {
diagnostics = input.readStringStringMap();
if (format <= DefaultSegmentInfosWriter.FORMAT_HAS_VECTORS) {
- hasVectors = input.readByte();
+ hasVectors = input.readByte() == 1;
} else {
final String storesSegment;
final String ext;
@@ -259,11 +229,7 @@ public final class SegmentInfo {
dirToTest = dir;
}
try {
- if (dirToTest.fileExists(IndexFileNames.segmentFileName(storesSegment, "", IndexFileNames.VECTORS_INDEX_EXTENSION))) {
- hasVectors = YES;
- } else {
- hasVectors = NO;
- }
+ hasVectors = dirToTest.fileExists(IndexFileNames.segmentFileName(storesSegment, "", IndexFileNames.VECTORS_INDEX_EXTENSION));
} finally {
if (isCompoundFile) {
dirToTest.close();
@@ -308,13 +274,13 @@ public final class SegmentInfo {
}
}
- public boolean getHasVectors() {
- return hasVectors == CHECK_FIELDINFOS ?
- (fieldInfos == null ? true : fieldInfos.hasVectors()) : hasVectors == YES;
+ public boolean getHasVectors() throws IOException {
+ return hasVectors;
}
- public FieldInfos getFieldInfos() {
- return fieldInfos;
+ public void setHasVectors(boolean v) {
+ hasVectors = v;
+ clearFiles();
}
public boolean hasDeletions() {
@@ -332,18 +298,17 @@ public final class SegmentInfo {
} else {
delGen++;
}
- clearFilesCache();
+ clearFiles();
}
void clearDelGen() {
delGen = NO;
- clearFilesCache();
+ clearFiles();
}
@Override
public Object clone() {
- SegmentInfo si = new SegmentInfo(name, docCount, dir, isCompoundFile, segmentCodecs,
- fieldInfos == null ? null : (FieldInfos) fieldInfos.clone());
+ SegmentInfo si = new SegmentInfo(name, docCount, dir, isCompoundFile, hasProx, segmentCodecs, false);
si.docStoreOffset = docStoreOffset;
si.docStoreSegment = docStoreSegment;
si.docStoreIsCompoundFile = docStoreIsCompoundFile;
@@ -351,12 +316,8 @@ public final class SegmentInfo {
si.delCount = delCount;
si.diagnostics = new HashMap<String, String>(diagnostics);
if (normGen != null) {
- si.normGen = new HashMap<Integer, Long>();
- for (Entry<Integer,Long> entry : normGen.entrySet()) {
- si.normGen.put(entry.getKey(), entry.getValue());
- }
+ si.normGen = normGen.clone();
}
- si.hasProx = hasProx;
si.hasVectors = hasVectors;
si.version = version;
return si;
@@ -378,12 +339,7 @@ public final class SegmentInfo {
* @param fieldNumber the field index to check
*/
public boolean hasSeparateNorms(int fieldNumber) {
- if (normGen == null) {
- return false;
- }
-
- Long gen = normGen.get(fieldNumber);
- return gen != null && gen.longValue() != NO;
+ return normGen != null && normGen[fieldNumber] != NO;
}
/**
@@ -393,7 +349,7 @@ public final class SegmentInfo {
if (normGen == null) {
return false;
} else {
- for (long fieldNormGen : normGen.values()) {
+ for (long fieldNormGen : normGen) {
if (fieldNormGen >= YES) {
return true;
}
@@ -403,9 +359,10 @@ public final class SegmentInfo {
return false;
}
- void initNormGen() {
+ void initNormGen(int numFields) {
if (normGen == null) { // normGen is null if this segments file hasn't had any norms set against it yet
- normGen = new HashMap<Integer, Long>();
+ normGen = new long[numFields];
+ Arrays.fill(normGen, NO);
}
}
@@ -416,13 +373,12 @@ public final class SegmentInfo {
* @param fieldIndex field whose norm file will be rewritten
*/
void advanceNormGen(int fieldIndex) {
- Long gen = normGen.get(fieldIndex);
- if (gen == null || gen.longValue() == NO) {
- normGen.put(fieldIndex, new Long(YES));
+ if (normGen[fieldIndex] == NO) {
+ normGen[fieldIndex] = YES;
} else {
- normGen.put(fieldIndex, gen+1);
+ normGen[fieldIndex]++;
}
- clearFilesCache();
+ clearFiles();
}
/**
@@ -432,7 +388,7 @@ public final class SegmentInfo {
*/
public String getNormFileName(int number) {
if (hasSeparateNorms(number)) {
- return IndexFileNames.fileNameFromGeneration(name, "s" + number, normGen.get(number));
+ return IndexFileNames.fileNameFromGeneration(name, "s" + number, normGen[number]);
} else {
// single file for all norms
return IndexFileNames.fileNameFromGeneration(name, IndexFileNames.NORMS_EXTENSION, WITHOUT_GEN);
@@ -447,7 +403,7 @@ public final class SegmentInfo {
*/
void setUseCompoundFile(boolean isCompoundFile) {
this.isCompoundFile = isCompoundFile;
- clearFilesCache();
+ clearFiles();
}
/**
@@ -477,7 +433,7 @@ public final class SegmentInfo {
void setDocStoreIsCompoundFile(boolean v) {
docStoreIsCompoundFile = v;
- clearFilesCache();
+ clearFiles();
}
public String getDocStoreSegment() {
@@ -490,14 +446,14 @@ public final class SegmentInfo {
void setDocStoreOffset(int offset) {
docStoreOffset = offset;
- clearFilesCache();
+ clearFiles();
}
void setDocStore(int offset, String segment, boolean isCompoundFile) {
docStoreOffset = offset;
docStoreSegment = segment;
docStoreIsCompoundFile = isCompoundFile;
- clearFilesCache();
+ clearFiles();
}
/** Save this segment's info. */
@@ -518,24 +474,27 @@ public final class SegmentInfo {
if (normGen == null) {
output.writeInt(NO);
} else {
- output.writeInt(normGen.size());
- for (Entry<Integer,Long> entry : normGen.entrySet()) {
- output.writeInt(entry.getKey());
- output.writeLong(entry.getValue());
+ output.writeInt(normGen.length);
+ for (long fieldNormGen : normGen) {
+ output.writeLong(fieldNormGen);
}
}
output.writeByte((byte) (isCompoundFile ? YES : NO));
output.writeInt(delCount);
- output.writeByte((byte) hasProx);
+ output.writeByte((byte) (hasProx ? 1:0));
segmentCodecs.write(output);
output.writeStringStringMap(diagnostics);
- output.writeByte((byte) hasVectors);
+ output.writeByte((byte) (hasVectors ? 1 : 0));
+ }
+
+ void setHasProx(boolean hasProx) {
+ this.hasProx = hasProx;
+ clearFiles();
}
public boolean getHasProx() {
- return hasProx == CHECK_FIELDINFOS ?
- (fieldInfos == null ? true : fieldInfos.hasProx()) : hasProx == YES;
+ return hasProx;
}
/** Can only be called once. */
@@ -591,7 +550,7 @@ public final class SegmentInfo {
} else {
fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.FIELDS_INDEX_EXTENSION));
fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.FIELDS_EXTENSION));
- if (getHasVectors()) {
+ if (hasVectors) {
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));
@@ -600,7 +559,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 (getHasVectors()) {
+ if (hasVectors) {
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));
@@ -613,11 +572,11 @@ public final class SegmentInfo {
}
if (normGen != null) {
- for (Entry<Integer,Long> entry : normGen.entrySet()) {
- long gen = entry.getValue();
+ for (int i = 0; i < normGen.length; i++) {
+ long gen = normGen[i];
if (gen >= YES) {
// Definitely a separate norm file, with generation:
- fileSet.add(IndexFileNames.fileNameFromGeneration(name, IndexFileNames.SEPARATE_NORMS_EXTENSION + entry.getKey(), gen));
+ fileSet.add(IndexFileNames.fileNameFromGeneration(name, IndexFileNames.SEPARATE_NORMS_EXTENSION + i, gen));
}
}
}
@@ -629,7 +588,7 @@ public final class SegmentInfo {
/* Called whenever any change is made that affects which
* files this segment has. */
- void clearFilesCache() {
+ private void clearFiles() {
files = null;
sizeInBytesNoStore = -1;
sizeInBytesWithStore = -1;
@@ -664,7 +623,7 @@ public final class SegmentInfo {
if (this.dir != dir) {
s.append('x');
}
- if (getHasVectors()) {
+ if (hasVectors) {
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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 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.HashMap;
import java.util.HashSet;
+import java.util.HashMap;
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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 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.Codec;
import org.apache.lucene.index.codecs.CodecProvider;
-import org.apache.lucene.index.codecs.FieldsConsumer;
+import org.apache.lucene.index.codecs.Codec;
import org.apache.lucene.index.codecs.MergeState;
+import org.apache.lucene.index.codecs.FieldsConsumer;
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.MultiBits;
import org.apache.lucene.util.ReaderUtil;
+import org.apache.lucene.util.MultiBits;
/**
* 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;
- segment = name;
this.fieldInfos = fieldInfos;
+ segment = name;
if (merge != null) {
checkAbort = new MergeState.CheckAbort(merge, directory);
} else {
@@ -180,8 +180,9 @@ final class SegmentMerger {
SegmentReader segmentReader = (SegmentReader) reader;
boolean same = true;
FieldInfos segmentFieldInfos = segmentReader.fieldInfos();
- for (FieldInfo fi : segmentFieldInfos) {
- same = fieldInfos.fieldName(fi.number).equals(fi.name);
+ int numFieldInfos = segmentFieldInfos.size();
+ for (int j = 0; same && j < numFieldInfos; j++) {
+ same = fieldInfos.fieldName(j).equals(segmentFieldInfos.fieldName(j));
}
if (same) {
matchingSegmentReaders[i] = segmentReader;
@@ -207,8 +208,9 @@ final class SegmentMerger {
if (reader instanceof SegmentReader) {
SegmentReader segmentReader = (SegmentReader) reader;
FieldInfos readerFieldInfos = segmentReader.fieldInfos();
- for (FieldInfo fi : readerFieldInfos) {
- fieldInfos.add(fi);
+ int numReaderFieldInfos = readerFieldInfos.size();
+ for (int j = 0; j < numReaderFieldInfos; j++) {
+ fieldInfos.add(readerFieldInfos.fieldInfo(j));
}
} else {
addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_POSITION_OFFSET), true, true, true, false, false);
@@ -222,13 +224,13 @@ final class SegmentMerger {
}
}
final SegmentCodecs codecInfo = SegmentCodecs.build(fieldInfos, this.codecs);
- fieldInfos.write(directory, segment + "." + IndexFileNames.FIELD_INFOS_EXTENSION);
+ fieldInfos.write(directory, segment + ".fnm");
int docCount = 0;
setMatchingSegmentReaders();
- final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment);
+ final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment, fieldInfos);
try {
int idx = 0;
@@ -310,7 +312,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, fieldInfos);
+ fieldsWriter.addDocument(doc);
docCount++;
checkAbort.work(300);
}
@@ -337,7 +339,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, fieldInfos);
+ fieldsWriter.addDocument(doc);
checkAbort.work(300);
}
}
@@ -572,7 +574,8 @@ final class SegmentMerger {
private void mergeNorms() throws IOException {
IndexOutput output = null;
try {
- for (FieldInfo fi : fieldInfos) {
+ for (int i = 0, numFieldInfos = fieldInfos.size(); i < numFieldInfos; i++) {
+ final FieldInfo fi = fieldInfos.fieldInfo(i);
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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 2011
@@ -22,22 +22,23 @@ 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 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.BytesRef;
import org.apache.lucene.util.CloseableThreadLocal;
+import org.apache.lucene.index.codecs.FieldsProducer;
+import org.apache.lucene.util.BytesRef;
/**
* @lucene.experimental
@@ -119,7 +120,7 @@ public class SegmentReader extends Index
}
cfsDir = dir0;
- fieldInfos = si.getFieldInfos();
+ fieldInfos = new FieldInfos(cfsDir, IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELD_INFOS_EXTENSION));
this.termsIndexDivisor = termsIndexDivisor;
@@ -597,12 +598,12 @@ public class SegmentReader extends Index
&& (!si.hasDeletions() || this.si.getDelFileName().equals(si.getDelFileName()));
boolean normsUpToDate = true;
- Set<Integer> fieldNormsChanged = new HashSet<Integer>();
- for (FieldInfo fi : core.fieldInfos) {
- int fieldNumber = fi.number;
- if (!this.si.getNormFileName(fieldNumber).equals(si.getNormFileName(fieldNumber))) {
+ 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))) {
normsUpToDate = false;
- fieldNormsChanged.add(fieldNumber);
+ fieldNormsChanged[i] = true;
}
}
@@ -658,10 +659,11 @@ public class SegmentReader extends Index
clone.norms = new HashMap<String,Norm>();
// Clone norms
- for (FieldInfo fi : core.fieldInfos) {
+ for (int i = 0; i < fieldNormsChanged.length; i++) {
+
// Clone unchanged norms to the cloned reader
- if (doClone || !fieldNormsChanged.contains(fi.number)) {
- final String curField = fi.name;
+ if (doClone || !fieldNormsChanged[i]) {
+ final String curField = core.fieldInfos.fieldInfo(i).name;
Norm norm = this.norms.get(curField);
if (norm != null)
clone.norms.put(curField, (Norm) norm.clone());
@@ -733,7 +735,7 @@ public class SegmentReader extends Index
}
if (normsDirty) { // re-write norms
- si.initNormGen();
+ si.initNormGen(core.fieldInfos.size());
for (final Norm norm : norms.values()) {
if (norm.dirty) {
norm.reWrite(si);
@@ -878,7 +880,8 @@ public class SegmentReader extends Index
ensureOpen();
Set<String> fieldSet = new HashSet<String>();
- for (FieldInfo fi : core.fieldInfos) {
+ for (int i = 0; i < core.fieldInfos.size(); i++) {
+ FieldInfo fi = core.fieldInfos.fieldInfo(i);
if (fieldOption == IndexReader.FieldOption.ALL) {
fieldSet.add(fi.name);
}
@@ -956,7 +959,8 @@ 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 (FieldInfo fi : core.fieldInfos) {
+ for (int i = 0; i < core.fieldInfos.size(); i++) {
+ FieldInfo fi = core.fieldInfos.fieldInfo(i);
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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 2011
@@ -27,13 +27,15 @@ final class StoredFieldsWriter {
FieldsWriter fieldsWriter;
final DocumentsWriter docWriter;
+ final FieldInfos fieldInfos;
int lastDocID;
PerDoc[] docFreeList = new PerDoc[1];
int freeCount;
- public StoredFieldsWriter(DocumentsWriter docWriter) {
+ public StoredFieldsWriter(DocumentsWriter docWriter, FieldInfos fieldInfos) {
this.docWriter = docWriter;
+ this.fieldInfos = fieldInfos;
}
public StoredFieldsWriterPerThread addThread(DocumentsWriter.DocState docState) throws IOException {
@@ -60,7 +62,7 @@ final class StoredFieldsWriter {
private synchronized void initFieldsWriter() throws IOException {
if (fieldsWriter == null) {
- fieldsWriter = new FieldsWriter(docWriter.directory, docWriter.getSegment());
+ fieldsWriter = new FieldsWriter(docWriter.directory, docWriter.getSegment(), fieldInfos);
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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 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);
+ localFieldsWriter = new FieldsWriter((IndexOutput) null, (IndexOutput) null, storedFieldsWriter.fieldInfos);
}
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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 2011
@@ -57,6 +57,12 @@ 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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 2011
@@ -25,4 +25,10 @@ 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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 2011
@@ -19,15 +19,14 @@ package org.apache.lucene.index.codecs.p
import java.io.IOException;
import java.util.Collection;
-import java.util.Comparator;
-import java.util.HashMap;
import java.util.Iterator;
-import java.util.Map;
import java.util.TreeMap;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Comparator;
-import org.apache.lucene.index.CompoundFileReader;
-import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.FieldsEnum;
@@ -36,6 +35,7 @@ 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,11 +94,13 @@ 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;
- for (FieldInfo fi : fieldInfos) {
- if (fi.isIndexed) {
- fields.put(fi.name, fi);
- preTerms.put(fi.name, new PreTerms(fi));
- if (!fi.omitTermFreqAndPositions) {
+ 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) {
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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 2011
@@ -528,9 +528,10 @@ public class TestBackwardsCompatibility
CompoundFileReader cfsReader = new CompoundFileReader(dir, "_0.cfs");
FieldInfos fieldInfos = new FieldInfos(cfsReader, "_0.fnm");
int contentFieldIndex = -1;
- for (FieldInfo fi : fieldInfos) {
+ for(int i=0;i<fieldInfos.size();i++) {
+ FieldInfo fi = fieldInfos.fieldInfo(i);
if (fi.name.equals("content")) {
- contentFieldIndex = fi.number;
+ contentFieldIndex = i;
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=1075413&r1=1075412&r2=1075413&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 28 16:33:16 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;
import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.Field;
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.TermStats;
import org.apache.lucene.index.codecs.TermsConsumer;
+import org.apache.lucene.index.codecs.TermStats;
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, SegmentCodecs.build(clonedFieldInfos, CodecProvider.getDefault()), clonedFieldInfos);
+ final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, true, SegmentCodecs.build(fieldInfos, CodecProvider.getDefault()), fieldInfos.hasVectors());
+ si.setHasProx(false);
final FieldsProducer reader = si.getSegmentCodecs().codec().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, 64, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR));
@@ -289,10 +289,8 @@ 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, SegmentCodecs.build(clonedFieldInfos, CodecProvider.getDefault()), clonedFieldInfos);
+ final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, true, SegmentCodecs.build(fieldInfos, CodecProvider.getDefault()), fieldInfos.hasVectors());
if (VERBOSE) {
System.out.println("TEST: now read postings");
@@ -442,7 +440,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.getCodecId()] instanceof PreFlexCodec) {
+ if (si.getSegmentCodecs().codecs[field.fieldInfo.codecId] instanceof PreFlexCodec) {
// code below expects unicode sort order
continue;
}
@@ -596,7 +594,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.getCodecId()] instanceof PreFlexCodec) {
+ if (!allowPreFlex && codecInfo.codecs[field.fieldInfo.codecId] instanceof PreFlexCodec) {
// code below expects unicode sort order
continue;
}
Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDoc.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDoc.java?rev=1075413&r1=1075412&r2=1075413&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDoc.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDoc.java Mon Feb 28 16:33:16 2011
@@ -204,8 +204,8 @@ public class TestDoc extends LuceneTestC
r2.close();
final SegmentInfo info = new SegmentInfo(merged, si1.docCount + si2.docCount, si1.dir,
- false, merger.getSegmentCodecs(),
- merger.fieldInfos());
+ false, merger.fieldInfos().hasProx(), merger.getSegmentCodecs(),
+ merger.fieldInfos().hasVectors());
if (useCompoundFile) {
Collection<String> filesToDelete = merger.createCompoundFile(merged + ".cfs", info);
Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDocumentWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDocumentWriter.java?rev=1075413&r1=1075412&r2=1075413&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDocumentWriter.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestDocumentWriter.java Mon Feb 28 16:33:16 2011
@@ -25,20 +25,20 @@ import org.apache.lucene.analysis.MockAn
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,7 +98,8 @@ public class TestDocumentWriter extends
// test that the norms are not present in the segment if
// omitNorms is true
- for (FieldInfo fi : reader.core.fieldInfos) {
+ for (int i = 0; i < reader.core.fieldInfos.size(); i++) {
+ FieldInfo fi = reader.core.fieldInfos.fieldInfo(i);
if (fi.isIndexed) {
assertTrue(fi.omitNorms == !reader.hasNorms(fi.name));
}