mirror of https://github.com/apache/lucene.git
LUCENE-2881: back out until corruption issues can be resolved
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1075413 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
6421e1aa77
commit
1337e9f1df
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 DocConsumer {
|
|||
for ( DocConsumerPerThread thread : threads) {
|
||||
DocFieldProcessorPerThread perThread = (DocFieldProcessorPerThread) thread;
|
||||
childThreadsAndFields.put(perThread.consumer, perThread.fields());
|
||||
perThread.trimFields(state);
|
||||
}
|
||||
fieldsWriter.flush(state);
|
||||
consumer.flush(childThreadsAndFields, state);
|
||||
|
@ -59,14 +63,7 @@ final class DocFieldProcessor extends DocConsumer {
|
|||
// 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
|
||||
|
|
|
@ -41,13 +41,14 @@ final class DocFieldProcessorPerThread extends DocConsumerPerThread {
|
|||
float docBoost;
|
||||
int fieldGen;
|
||||
final DocFieldProcessor docFieldProcessor;
|
||||
final FieldInfos fieldInfos;
|
||||
final DocFieldConsumerPerThread consumer;
|
||||
|
||||
// Holds all fields seen in current doc
|
||||
DocFieldProcessorPerField[] fields = new DocFieldProcessorPerField[1];
|
||||
int fieldCount;
|
||||
|
||||
// Hash table for all fields 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 extends DocConsumerPerThread {
|
|||
public DocFieldProcessorPerThread(DocumentsWriterThreadState threadState, DocFieldProcessor docFieldProcessor) throws IOException {
|
||||
this.docState = threadState.docState;
|
||||
this.docFieldProcessor = docFieldProcessor;
|
||||
this.fieldInfos = docFieldProcessor.fieldInfos;
|
||||
this.consumer = docFieldProcessor.consumer.addThread(this);
|
||||
fieldsWriter = docFieldProcessor.fieldsWriter.addThread(docState);
|
||||
}
|
||||
|
@ -73,7 +75,6 @@ final class DocFieldProcessorPerThread extends DocConsumerPerThread {
|
|||
field = next;
|
||||
}
|
||||
}
|
||||
doAfterFlush();
|
||||
fieldsWriter.abort();
|
||||
consumer.abort();
|
||||
}
|
||||
|
@ -91,15 +92,45 @@ final class DocFieldProcessorPerThread extends DocConsumerPerThread {
|
|||
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 extends DocConsumerPerThread {
|
|||
}
|
||||
|
||||
@Override
|
||||
public DocumentsWriter.DocWriter processDocument(FieldInfos fieldInfos) throws IOException {
|
||||
public DocumentsWriter.DocWriter processDocument() throws IOException {
|
||||
|
||||
consumer.startDocument();
|
||||
fieldsWriter.startDocument();
|
||||
|
|
|
@ -39,6 +39,13 @@ final class DocInverter extends DocFieldConsumer {
|
|||
this.endConsumer = endConsumer;
|
||||
}
|
||||
|
||||
@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 {
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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.IndexInput;
|
|||
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.StringHelper;
|
|||
* 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 Iterable<FieldInfo> {
|
|||
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 Iterable<FieldInfo> {
|
|||
* @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 Iterable<FieldInfo> {
|
|||
}
|
||||
}
|
||||
|
||||
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 Iterable<FieldInfo> {
|
|||
|
||||
/** 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 Iterable<FieldInfo> {
|
|||
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 Iterable<FieldInfo> {
|
|||
}
|
||||
|
||||
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 Iterable<FieldInfo> {
|
|||
}
|
||||
|
||||
public FieldInfo fieldInfo(String fieldName) {
|
||||
return localFieldInfos.get(fieldName);
|
||||
return byName.get(fieldName);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -383,37 +270,25 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
* 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 Iterable<FieldInfo> {
|
|||
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 Iterable<FieldInfo> {
|
|||
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 Iterable<FieldInfo> {
|
|||
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 Iterable<FieldInfo> {
|
|||
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()) {
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.lucene.document.Document;
|
|||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.index.PayloadProcessorProvider.DirPayloadProcessor;
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.store.BufferedIndexInput;
|
||||
|
@ -220,7 +221,6 @@ public class IndexWriter implements Closeable {
|
|||
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 Closeable {
|
|||
|
||||
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,13 +854,22 @@ public class IndexWriter implements Closeable {
|
|||
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();
|
||||
}
|
||||
|
@ -2266,7 +2272,7 @@ public class IndexWriter implements Closeable {
|
|||
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 Closeable {
|
|||
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 Closeable {
|
|||
// 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 Closeable {
|
|||
|
||||
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 Closeable {
|
|||
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 Closeable {
|
|||
// because codec must know if prox was written for
|
||||
// this segment:
|
||||
//System.out.println("merger set hasProx=" + merger.hasProx() + " seg=" + merge.info.name);
|
||||
merge.info.clearFilesCache();
|
||||
merge.info.setHasProx(merger.fieldInfos().hasProx());
|
||||
|
||||
boolean useCompoundFile;
|
||||
synchronized (this) { // Guard segmentInfos
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.lucene.store.IndexOutput;
|
|||
|
||||
final class NormsWriter extends InvertedDocEndConsumer {
|
||||
|
||||
private FieldInfos fieldInfos;
|
||||
@Override
|
||||
public InvertedDocEndConsumerPerThread addThread(DocInverterPerThread docInverterPerThread) {
|
||||
return new NormsWriterPerThread(docInverterPerThread, this);
|
||||
|
@ -47,6 +48,11 @@ final class NormsWriter extends InvertedDocEndConsumer {
|
|||
// 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 InvertedDocEndConsumer {
|
|||
|
||||
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 InvertedDocEndConsumer {
|
|||
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 InvertedDocEndConsumer {
|
|||
// 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++)
|
||||
|
|
|
@ -67,7 +67,7 @@ final class PerFieldCodecWrapper extends Codec {
|
|||
|
||||
@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 Codec {
|
|||
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));
|
||||
}
|
||||
|
|
|
@ -74,20 +74,22 @@ final class SegmentCodecs implements Cloneable {
|
|||
}
|
||||
|
||||
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));
|
||||
|
|
|
@ -17,22 +17,21 @@ package org.apache.lucene.index;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.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 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.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.HashSet;
|
||||
import java.util.HashMap;
|
||||
import java.util.ArrayList;
|
||||
|
||||
/**
|
||||
* 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
|
||||
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 hasVectors = CHECK_FIELDINFOS; // True if this segment wrote term vectors
|
||||
|
||||
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);
|
||||
|
|
|
@ -17,26 +17,26 @@ package org.apache.lucene.index;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Vector;
|
||||
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import org.apache.lucene.index.codecs.SegmentInfosReader;
|
||||
import org.apache.lucene.index.codecs.SegmentInfosWriter;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.store.NoSuchDirectoryException;
|
||||
import org.apache.lucene.index.codecs.CodecProvider;
|
||||
import org.apache.lucene.index.codecs.SegmentInfosReader;
|
||||
import org.apache.lucene.index.codecs.SegmentInfosWriter;
|
||||
import org.apache.lucene.util.ThreadInterruptedException;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.util.Vector;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* A collection of segmentInfo objects with methods for operating on
|
||||
* those segments in relation to the file system.
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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 IndexReader implements Cloneable {
|
|||
}
|
||||
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 IndexReader implements Cloneable {
|
|||
&& (!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 IndexReader implements Cloneable {
|
|||
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 IndexReader implements Cloneable {
|
|||
}
|
||||
|
||||
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 IndexReader implements Cloneable {
|
|||
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 IndexReader implements Cloneable {
|
|||
private void openNorms(Directory cfsDir, int readBufferSize) throws IOException {
|
||||
long nextNormSeek = SegmentMerger.NORMS_HEADER.length; //skip header (header unused for now)
|
||||
int maxDoc = maxDoc();
|
||||
for (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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -56,6 +56,12 @@ final class TermsHash extends InvertedDocConsumer {
|
|||
return new TermsHashPerThread(docInverterPerThread, this, nextTermsHash, primaryPerThread);
|
||||
}
|
||||
|
||||
@Override
|
||||
void setFieldInfos(FieldInfos fieldInfos) {
|
||||
this.fieldInfos = fieldInfos;
|
||||
consumer.setFieldInfos(fieldInfos);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
consumer.abort();
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,15 +19,14 @@ package org.apache.lucene.index.codecs.preflex;
|
|||
|
||||
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.SegmentInfo;
|
|||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.index.CompoundFileReader;
|
||||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
|
@ -94,11 +94,13 @@ public class PreFlexFields extends FieldsProducer {
|
|||
// so that if an index update removes them we'll still have them
|
||||
freqStream = dir.openInput(info.name + ".frq", readBufferSize);
|
||||
boolean anyProx = false;
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -528,9 +528,10 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
|
|||
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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 LuceneTestCase {
|
|||
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 LuceneTestCase {
|
|||
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 LuceneTestCase {
|
|||
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 LuceneTestCase {
|
|||
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;
|
||||
}
|
||||
|
|
|
@ -1,222 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
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.store.Directory;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestConsistentFieldNumbers extends LuceneTestCase {
|
||||
|
||||
@Test
|
||||
public void testSameFieldNumbersAcrossSegments() throws Exception {
|
||||
for (int i = 0; i < 2; i++) {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()).setMergePolicy(NoMergePolicy.COMPOUND_FILES));
|
||||
|
||||
Document d1 = new Document();
|
||||
d1.add(new Field("f1", "first field", Store.YES, Index.ANALYZED, TermVector.NO));
|
||||
d1.add(new Field("f2", "second field", Store.YES, Index.ANALYZED, TermVector.NO));
|
||||
writer.addDocument(d1);
|
||||
|
||||
if (i == 1) {
|
||||
writer.close();
|
||||
writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()).setMergePolicy(NoMergePolicy.COMPOUND_FILES));
|
||||
} else {
|
||||
writer.commit();
|
||||
}
|
||||
|
||||
Document d2 = new Document();
|
||||
d2.add(new Field("f2", "second field", Store.YES, Index.ANALYZED, TermVector.NO));
|
||||
d2.add(new Field("f1", "first field", Store.YES, Index.ANALYZED, TermVector.YES));
|
||||
d2.add(new Field("f3", "third field", Store.YES, Index.ANALYZED, TermVector.NO));
|
||||
d2.add(new Field("f4", "fourth field", Store.YES, Index.ANALYZED, TermVector.NO));
|
||||
writer.addDocument(d2);
|
||||
|
||||
writer.close();
|
||||
|
||||
SegmentInfos sis = new SegmentInfos();
|
||||
sis.read(dir);
|
||||
assertEquals(2, sis.size());
|
||||
|
||||
FieldInfos fis1 = sis.info(0).getFieldInfos();
|
||||
FieldInfos fis2 = sis.info(1).getFieldInfos();
|
||||
|
||||
assertEquals("f1", fis1.fieldInfo(0).name);
|
||||
assertEquals("f2", fis1.fieldInfo(1).name);
|
||||
assertEquals("f1", fis2.fieldInfo(0).name);
|
||||
assertEquals("f2", fis2.fieldInfo(1).name);
|
||||
assertEquals("f3", fis2.fieldInfo(2).name);
|
||||
assertEquals("f4", fis2.fieldInfo(3).name);
|
||||
|
||||
writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()));
|
||||
writer.optimize();
|
||||
writer.close();
|
||||
|
||||
sis = new SegmentInfos();
|
||||
sis.read(dir);
|
||||
assertEquals(1, sis.size());
|
||||
|
||||
FieldInfos fis3 = sis.info(0).getFieldInfos();
|
||||
|
||||
assertEquals("f1", fis3.fieldInfo(0).name);
|
||||
assertEquals("f2", fis3.fieldInfo(1).name);
|
||||
assertEquals("f3", fis3.fieldInfo(2).name);
|
||||
assertEquals("f4", fis3.fieldInfo(3).name);
|
||||
|
||||
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAddIndexes() throws Exception {
|
||||
Directory dir1 = newDirectory();
|
||||
Directory dir2 = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir1, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()).setMergePolicy(NoMergePolicy.COMPOUND_FILES));
|
||||
|
||||
Document d1 = new Document();
|
||||
d1.add(new Field("f1", "first field", Store.YES, Index.ANALYZED, TermVector.NO));
|
||||
d1.add(new Field("f2", "second field", Store.YES, Index.ANALYZED, TermVector.NO));
|
||||
writer.addDocument(d1);
|
||||
|
||||
writer.close();
|
||||
writer = new IndexWriter(dir2, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()).setMergePolicy(NoMergePolicy.COMPOUND_FILES));
|
||||
|
||||
Document d2 = new Document();
|
||||
d2.add(new Field("f2", "second field", Store.YES, Index.ANALYZED, TermVector.NO));
|
||||
d2.add(new Field("f1", "first field", Store.YES, Index.ANALYZED, TermVector.YES));
|
||||
d2.add(new Field("f3", "third field", Store.YES, Index.ANALYZED, TermVector.NO));
|
||||
d2.add(new Field("f4", "fourth field", Store.YES, Index.ANALYZED, TermVector.NO));
|
||||
writer.addDocument(d2);
|
||||
|
||||
writer.close();
|
||||
|
||||
writer = new IndexWriter(dir1, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()).setMergePolicy(NoMergePolicy.COMPOUND_FILES));
|
||||
writer.addIndexes(dir2);
|
||||
writer.close();
|
||||
|
||||
SegmentInfos sis = new SegmentInfos();
|
||||
sis.read(dir1);
|
||||
assertEquals(2, sis.size());
|
||||
|
||||
FieldInfos fis1 = sis.info(0).getFieldInfos();
|
||||
FieldInfos fis2 = sis.info(1).getFieldInfos();
|
||||
|
||||
assertEquals("f1", fis1.fieldInfo(0).name);
|
||||
assertEquals("f2", fis1.fieldInfo(1).name);
|
||||
// make sure the ordering of the "external" segment is preserved
|
||||
assertEquals("f2", fis2.fieldInfo(0).name);
|
||||
assertEquals("f1", fis2.fieldInfo(1).name);
|
||||
assertEquals("f3", fis2.fieldInfo(2).name);
|
||||
assertEquals("f4", fis2.fieldInfo(3).name);
|
||||
|
||||
writer = new IndexWriter(dir1, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()));
|
||||
writer.optimize();
|
||||
writer.close();
|
||||
|
||||
sis = new SegmentInfos();
|
||||
sis.read(dir1);
|
||||
assertEquals(1, sis.size());
|
||||
|
||||
FieldInfos fis3 = sis.info(0).getFieldInfos();
|
||||
|
||||
// after merging the ordering should be identical to the first segment
|
||||
assertEquals("f1", fis3.fieldInfo(0).name);
|
||||
assertEquals("f2", fis3.fieldInfo(1).name);
|
||||
assertEquals("f3", fis3.fieldInfo(2).name);
|
||||
assertEquals("f4", fis3.fieldInfo(3).name);
|
||||
|
||||
dir1.close();
|
||||
dir2.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testManyFields() throws Exception {
|
||||
final int NUM_DOCS = 2000;
|
||||
final int MAX_FIELDS = 50;
|
||||
|
||||
int[][] docs = new int[NUM_DOCS][4];
|
||||
for (int i = 0; i < docs.length; i++) {
|
||||
for (int j = 0; j < docs[i].length;j++) {
|
||||
docs[i][j] = random.nextInt(MAX_FIELDS);
|
||||
}
|
||||
}
|
||||
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()));
|
||||
|
||||
for (int i = 0; i < NUM_DOCS; i++) {
|
||||
Document d = new Document();
|
||||
for (int j = 0; j < docs[i].length; j++) {
|
||||
d.add(getField(docs[i][j]));
|
||||
}
|
||||
|
||||
writer.addDocument(d);
|
||||
}
|
||||
|
||||
writer.optimize();
|
||||
writer.close();
|
||||
|
||||
SegmentInfos sis = new SegmentInfos();
|
||||
sis.read(dir);
|
||||
for (SegmentInfo si : sis) {
|
||||
FieldInfos fis = si.getFieldInfos();
|
||||
|
||||
for (FieldInfo fi : fis) {
|
||||
Field expected = getField(Integer.parseInt(fi.name));
|
||||
assertEquals(expected.isIndexed(), fi.isIndexed);
|
||||
assertEquals(expected.isTermVectorStored(), fi.storeTermVector);
|
||||
assertEquals(expected.isStorePositionWithTermVector(), fi.storePositionWithTermVector);
|
||||
assertEquals(expected.isStoreOffsetWithTermVector(), fi.storeOffsetWithTermVector);
|
||||
}
|
||||
}
|
||||
|
||||
dir.close();
|
||||
}
|
||||
|
||||
private Field getField(int number) {
|
||||
int mode = number % 16;
|
||||
String fieldName = "" + number;
|
||||
switch (mode) {
|
||||
case 0: return new Field(fieldName, "some text", Store.YES, Index.ANALYZED, TermVector.NO);
|
||||
case 1: return new Field(fieldName, "some text", Store.NO, Index.ANALYZED, TermVector.NO);
|
||||
case 2: return new Field(fieldName, "some text", Store.YES, Index.NOT_ANALYZED, TermVector.NO);
|
||||
case 3: return new Field(fieldName, "some text", Store.NO, Index.NOT_ANALYZED, TermVector.NO);
|
||||
case 4: return new Field(fieldName, "some text", Store.YES, Index.ANALYZED, TermVector.WITH_OFFSETS);
|
||||
case 5: return new Field(fieldName, "some text", Store.NO, Index.ANALYZED, TermVector.WITH_OFFSETS);
|
||||
case 6: return new Field(fieldName, "some text", Store.YES, Index.NOT_ANALYZED, TermVector.WITH_OFFSETS);
|
||||
case 7: return new Field(fieldName, "some text", Store.NO, Index.NOT_ANALYZED, TermVector.WITH_OFFSETS);
|
||||
case 8: return new Field(fieldName, "some text", Store.YES, Index.ANALYZED, TermVector.WITH_POSITIONS);
|
||||
case 9: return new Field(fieldName, "some text", Store.NO, Index.ANALYZED, TermVector.WITH_POSITIONS);
|
||||
case 10: return new Field(fieldName, "some text", Store.YES, Index.NOT_ANALYZED, TermVector.WITH_POSITIONS);
|
||||
case 11: return new Field(fieldName, "some text", Store.NO, Index.NOT_ANALYZED, TermVector.WITH_POSITIONS);
|
||||
case 12: return new Field(fieldName, "some text", Store.YES, Index.ANALYZED, TermVector.WITH_POSITIONS_OFFSETS);
|
||||
case 13: return new Field(fieldName, "some text", Store.NO, Index.ANALYZED, TermVector.WITH_POSITIONS_OFFSETS);
|
||||
case 14: return new Field(fieldName, "some text", Store.YES, Index.NOT_ANALYZED, TermVector.WITH_POSITIONS_OFFSETS);
|
||||
case 15: return new Field(fieldName, "some text", Store.NO, Index.NOT_ANALYZED, TermVector.WITH_POSITIONS_OFFSETS);
|
||||
default: return null;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -204,8 +204,8 @@ public class TestDoc extends LuceneTestCase {
|
|||
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);
|
||||
|
|
|
@ -25,20 +25,20 @@ import org.apache.lucene.analysis.MockAnalyzer;
|
|||
import org.apache.lucene.analysis.MockTokenizer;
|
||||
import org.apache.lucene.analysis.TokenFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.Fieldable;
|
||||
import org.apache.lucene.document.Field.Index;
|
||||
import org.apache.lucene.document.Field.Store;
|
||||
import org.apache.lucene.document.Field.TermVector;
|
||||
import org.apache.lucene.document.Fieldable;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util._TestUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
public class TestDocumentWriter extends LuceneTestCase {
|
||||
private Directory dir;
|
||||
|
@ -98,7 +98,8 @@ public class TestDocumentWriter extends LuceneTestCase {
|
|||
|
||||
// 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));
|
||||
}
|
||||
|
|
|
@ -92,9 +92,10 @@ public class TestIndexFileDeleter extends LuceneTestCase {
|
|||
CompoundFileReader cfsReader = new CompoundFileReader(dir, "_2.cfs");
|
||||
FieldInfos fieldInfos = new FieldInfos(cfsReader, "_2.fnm");
|
||||
int contentFieldIndex = -1;
|
||||
for (FieldInfo fi : fieldInfos) {
|
||||
for(i=0;i<fieldInfos.size();i++) {
|
||||
FieldInfo fi = fieldInfos.fieldInfo(i);
|
||||
if (fi.name.equals("content")) {
|
||||
contentFieldIndex = fi.number;
|
||||
contentFieldIndex = i;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -216,7 +216,7 @@ public class TestPerFieldCodecSupport extends LuceneTestCase {
|
|||
IndexFileNames.FIELD_INFOS_EXTENSION));
|
||||
FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
|
||||
assertEquals("faild for segment index: " + i, codec[i],
|
||||
codecInfo.codecs[fieldInfo.getCodecId()]);
|
||||
codecInfo.codecs[fieldInfo.codecId]);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -79,8 +79,8 @@ public class TestSegmentMerger extends LuceneTestCase {
|
|||
int docsMerged = merger.merge();
|
||||
assertTrue(docsMerged == 2);
|
||||
//Should be able to open a new SegmentReader against the new directory
|
||||
SegmentReader mergedReader = SegmentReader.get(false, mergedDir, new SegmentInfo(mergedSegment, docsMerged, mergedDir, false,
|
||||
merger.getSegmentCodecs(), merger.fieldInfos()),
|
||||
SegmentReader mergedReader = SegmentReader.get(false, mergedDir, new SegmentInfo(mergedSegment, docsMerged, mergedDir, false, merger.fieldInfos().hasProx(),
|
||||
merger.getSegmentCodecs(), merger.fieldInfos().hasVectors()),
|
||||
BufferedIndexInput.BUFFER_SIZE, true, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR);
|
||||
|
||||
assertTrue(mergedReader != null);
|
||||
|
|
Loading…
Reference in New Issue