mirror of https://github.com/apache/lucene.git
LUCENE-3679: replace IR.getFieldNames with IR.getFieldInfos
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1229401 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
5dba63c719
commit
cdb2ee8a7b
|
@ -720,6 +720,11 @@ Changes in backwards compatibility policy
|
|||
break in the binary format so if you had built and saved any FSTs
|
||||
then you need to rebuild them. (Robert Muir, Mike McCandless)
|
||||
|
||||
* LUCENE-3679: The expert IndexReader.getFieldNames(FieldOption) API
|
||||
has been removed and replaced with the experimental getFieldInfos
|
||||
API. All IndexReader subclasses must implement getFieldInfos.
|
||||
(Mike McCandless)
|
||||
|
||||
Security fixes
|
||||
|
||||
* LUCENE-3588: Try harder to prevent SIGSEGV on cloned MMapIndexInputs:
|
||||
|
|
|
@ -128,8 +128,6 @@ public abstract class BaseFragmentsBuilder implements FragmentsBuilder {
|
|||
public void stringField(FieldInfo fieldInfo, String value) throws IOException {
|
||||
FieldType ft = new FieldType(TextField.TYPE_STORED);
|
||||
ft.setStoreTermVectors(fieldInfo.storeTermVector);
|
||||
ft.setStoreTermVectorOffsets(fieldInfo.storeOffsetWithTermVector);
|
||||
ft.setStoreTermVectorPositions(fieldInfo.storePositionWithTermVector);
|
||||
fields.add(new Field(fieldInfo.name, value, ft));
|
||||
}
|
||||
|
||||
|
|
|
@ -21,7 +21,6 @@ import java.io.IOException;
|
|||
import java.io.StringReader;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
|
@ -34,13 +33,14 @@ import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
|||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.FieldInvertState;
|
||||
import org.apache.lucene.index.Fields;
|
||||
import org.apache.lucene.index.FieldsEnum;
|
||||
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.OrdTermState;
|
||||
import org.apache.lucene.index.StoredFieldVisitor;
|
||||
|
@ -198,6 +198,8 @@ public class MemoryIndex {
|
|||
|
||||
private static final boolean DEBUG = false;
|
||||
|
||||
private final FieldInfos fieldInfos;
|
||||
|
||||
/**
|
||||
* Sorts term entries into ascending order; also works for
|
||||
* Arrays.binarySearch() and Arrays.sort()
|
||||
|
@ -232,6 +234,7 @@ public class MemoryIndex {
|
|||
*/
|
||||
private MemoryIndex(boolean storeOffsets) {
|
||||
this.stride = storeOffsets ? 3 : 1;
|
||||
fieldInfos = new FieldInfos();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -348,6 +351,8 @@ public class MemoryIndex {
|
|||
int numOverlapTokens = 0;
|
||||
int pos = -1;
|
||||
|
||||
fieldInfos.addOrUpdate(fieldName, true);
|
||||
|
||||
TermToBytesRefAttribute termAtt = stream.getAttribute(TermToBytesRefAttribute.class);
|
||||
PositionIncrementAttribute posIncrAttribute = stream.addAttribute(PositionIncrementAttribute.class);
|
||||
OffsetAttribute offsetAtt = stream.addAttribute(OffsetAttribute.class);
|
||||
|
@ -765,6 +770,11 @@ public class MemoryIndex {
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldInfos getFieldInfos() {
|
||||
return fieldInfos;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docFreq(String field, BytesRef term) {
|
||||
Info info = getInfo(field);
|
||||
|
@ -1113,22 +1123,6 @@ public class MemoryIndex {
|
|||
if (DEBUG) System.err.println("MemoryIndexReader.doClose");
|
||||
}
|
||||
|
||||
// lucene >= 1.9 (remove this method for lucene-1.4.3)
|
||||
@Override
|
||||
public Collection<String> getFieldNames(FieldOption fieldOption) {
|
||||
if (DEBUG) System.err.println("MemoryIndexReader.getFieldNamesOption");
|
||||
if (fieldOption == FieldOption.UNINDEXED)
|
||||
return Collections.<String>emptySet();
|
||||
if (fieldOption == FieldOption.INDEXED_NO_TERMVECTOR)
|
||||
return Collections.<String>emptySet();
|
||||
if (fieldOption == FieldOption.TERMVECTOR_WITH_OFFSET && stride == 1)
|
||||
return Collections.<String>emptySet();
|
||||
if (fieldOption == FieldOption.TERMVECTOR_WITH_POSITION_OFFSET && stride == 1)
|
||||
return Collections.<String>emptySet();
|
||||
|
||||
return Collections.unmodifiableSet(fields.keySet());
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValues docValues(String field) throws IOException {
|
||||
return null;
|
||||
|
|
|
@ -69,8 +69,6 @@ public class Lucene40FieldInfosReader extends FieldInfosReader {
|
|||
byte bits = input.readByte();
|
||||
boolean isIndexed = (bits & Lucene40FieldInfosWriter.IS_INDEXED) != 0;
|
||||
boolean storeTermVector = (bits & Lucene40FieldInfosWriter.STORE_TERMVECTOR) != 0;
|
||||
boolean storePositionsWithTermVector = (bits & Lucene40FieldInfosWriter.STORE_POSITIONS_WITH_TERMVECTOR) != 0;
|
||||
boolean storeOffsetWithTermVector = (bits & Lucene40FieldInfosWriter.STORE_OFFSET_WITH_TERMVECTOR) != 0;
|
||||
boolean omitNorms = (bits & Lucene40FieldInfosWriter.OMIT_NORMS) != 0;
|
||||
boolean storePayloads = (bits & Lucene40FieldInfosWriter.STORE_PAYLOADS) != 0;
|
||||
final IndexOptions indexOptions;
|
||||
|
@ -147,7 +145,6 @@ public class Lucene40FieldInfosReader extends FieldInfosReader {
|
|||
}
|
||||
}
|
||||
infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector,
|
||||
storePositionsWithTermVector, storeOffsetWithTermVector,
|
||||
omitNorms, storePayloads, indexOptions, docValuesType);
|
||||
}
|
||||
|
||||
|
|
|
@ -47,8 +47,6 @@ public class Lucene40FieldInfosWriter extends FieldInfosWriter {
|
|||
|
||||
static final byte IS_INDEXED = 0x1;
|
||||
static final byte STORE_TERMVECTOR = 0x2;
|
||||
static final byte STORE_POSITIONS_WITH_TERMVECTOR = 0x4;
|
||||
static final byte STORE_OFFSET_WITH_TERMVECTOR = 0x8;
|
||||
static final byte OMIT_NORMS = 0x10;
|
||||
static final byte STORE_PAYLOADS = 0x20;
|
||||
static final byte OMIT_TERM_FREQ_AND_POSITIONS = 0x40;
|
||||
|
@ -66,14 +64,13 @@ public class Lucene40FieldInfosWriter extends FieldInfosWriter {
|
|||
byte bits = 0x0;
|
||||
if (fi.isIndexed) bits |= IS_INDEXED;
|
||||
if (fi.storeTermVector) bits |= STORE_TERMVECTOR;
|
||||
if (fi.storePositionWithTermVector) bits |= STORE_POSITIONS_WITH_TERMVECTOR;
|
||||
if (fi.storeOffsetWithTermVector) bits |= STORE_OFFSET_WITH_TERMVECTOR;
|
||||
if (fi.omitNorms) bits |= OMIT_NORMS;
|
||||
if (fi.storePayloads) bits |= STORE_PAYLOADS;
|
||||
if (fi.indexOptions == IndexOptions.DOCS_ONLY)
|
||||
if (fi.indexOptions == IndexOptions.DOCS_ONLY) {
|
||||
bits |= OMIT_TERM_FREQ_AND_POSITIONS;
|
||||
else if (fi.indexOptions == IndexOptions.DOCS_AND_FREQS)
|
||||
} else if (fi.indexOptions == IndexOptions.DOCS_AND_FREQS) {
|
||||
bits |= OMIT_POSITIONS;
|
||||
}
|
||||
output.writeString(fi.name);
|
||||
output.writeInt(fi.number);
|
||||
output.writeByte(bits);
|
||||
|
|
|
@ -79,14 +79,6 @@ public class SimpleTextFieldInfosReader extends FieldInfosReader {
|
|||
assert StringHelper.startsWith(scratch, STORETV);
|
||||
boolean storeTermVector = Boolean.parseBoolean(readString(STORETV.length, scratch));
|
||||
|
||||
SimpleTextUtil.readLine(input, scratch);
|
||||
assert StringHelper.startsWith(scratch, STORETVPOS);
|
||||
boolean storePositionsWithTermVector = Boolean.parseBoolean(readString(STORETVPOS.length, scratch));
|
||||
|
||||
SimpleTextUtil.readLine(input, scratch);
|
||||
assert StringHelper.startsWith(scratch, STORETVOFF);
|
||||
boolean storeOffsetWithTermVector = Boolean.parseBoolean(readString(STORETVOFF.length, scratch));
|
||||
|
||||
SimpleTextUtil.readLine(input, scratch);
|
||||
assert StringHelper.startsWith(scratch, PAYLOADS);
|
||||
boolean storePayloads = Boolean.parseBoolean(readString(PAYLOADS.length, scratch));
|
||||
|
@ -115,7 +107,6 @@ public class SimpleTextFieldInfosReader extends FieldInfosReader {
|
|||
hasFreq |= isIndexed && indexOptions != IndexOptions.DOCS_ONLY;
|
||||
|
||||
infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector,
|
||||
storePositionsWithTermVector, storeOffsetWithTermVector,
|
||||
omitNorms, storePayloads, indexOptions, docValuesType);
|
||||
}
|
||||
|
||||
|
|
|
@ -80,14 +80,6 @@ public class SimpleTextFieldInfosWriter extends FieldInfosWriter {
|
|||
SimpleTextUtil.write(out, Boolean.toString(fi.storeTermVector), scratch);
|
||||
SimpleTextUtil.writeNewline(out);
|
||||
|
||||
SimpleTextUtil.write(out, STORETVPOS);
|
||||
SimpleTextUtil.write(out, Boolean.toString(fi.storePositionWithTermVector), scratch);
|
||||
SimpleTextUtil.writeNewline(out);
|
||||
|
||||
SimpleTextUtil.write(out, STORETVOFF);
|
||||
SimpleTextUtil.write(out, Boolean.toString(fi.storeOffsetWithTermVector), scratch);
|
||||
SimpleTextUtil.writeNewline(out);
|
||||
|
||||
SimpleTextUtil.write(out, PAYLOADS);
|
||||
SimpleTextUtil.write(out, Boolean.toString(fi.storePayloads), scratch);
|
||||
SimpleTextUtil.writeNewline(out);
|
||||
|
|
|
@ -64,8 +64,6 @@ public class DocumentStoredFieldVisitor extends StoredFieldVisitor {
|
|||
public void stringField(FieldInfo fieldInfo, String value) throws IOException {
|
||||
final FieldType ft = new FieldType(TextField.TYPE_STORED);
|
||||
ft.setStoreTermVectors(fieldInfo.storeTermVector);
|
||||
ft.setStoreTermVectorPositions(fieldInfo.storePositionWithTermVector);
|
||||
ft.setStoreTermVectorOffsets(fieldInfo.storeOffsetWithTermVector);
|
||||
ft.setStoreTermVectors(fieldInfo.storeTermVector);
|
||||
ft.setIndexed(fieldInfo.isIndexed);
|
||||
ft.setOmitNorms(fieldInfo.omitNorms);
|
||||
|
|
|
@ -18,9 +18,6 @@ package org.apache.lucene.index;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -55,6 +52,11 @@ abstract class BaseMultiReader<R extends IndexReader> extends IndexReader {
|
|||
topLevelContext = ReaderUtil.buildReaderContext(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldInfos getFieldInfos() {
|
||||
throw new UnsupportedOperationException("call getFieldInfos() on each sub reader, or use ReaderUtil.getMergedFieldInfos, instead");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Fields fields() throws IOException {
|
||||
throw new UnsupportedOperationException("please use MultiFields.getFields, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level Fields");
|
||||
|
@ -127,17 +129,6 @@ abstract class BaseMultiReader<R extends IndexReader> extends IndexReader {
|
|||
return total;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<String> getFieldNames (IndexReader.FieldOption fieldNames) {
|
||||
ensureOpen();
|
||||
// maintain a unique set of field names
|
||||
final Set<String> fieldSet = new HashSet<String>();
|
||||
for (IndexReader reader : subReaders) {
|
||||
fieldSet.addAll(reader.getFieldNames(fieldNames));
|
||||
}
|
||||
return fieldSet;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexReader[] getSequentialSubReaders() {
|
||||
return subReaders;
|
||||
|
|
|
@ -23,7 +23,6 @@ import java.io.PrintStream;
|
|||
import java.lang.reflect.Array;
|
||||
import java.text.NumberFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
|
@ -552,16 +551,16 @@ public class CheckIndex {
|
|||
if (reader.maxDoc() != info.docCount)
|
||||
throw new RuntimeException("SegmentReader.maxDoc() " + reader.maxDoc() + " != SegmentInfos.docCount " + info.docCount);
|
||||
|
||||
// Test getFieldNames()
|
||||
// Test getFieldInfos()
|
||||
if (infoStream != null) {
|
||||
infoStream.print(" test: fields..............");
|
||||
}
|
||||
Collection<String> fieldNames = reader.getFieldNames(IndexReader.FieldOption.ALL);
|
||||
msg("OK [" + fieldNames.size() + " fields]");
|
||||
segInfoStat.numFields = fieldNames.size();
|
||||
FieldInfos fieldInfos = reader.getFieldInfos();
|
||||
msg("OK [" + fieldInfos.size() + " fields]");
|
||||
segInfoStat.numFields = fieldInfos.size();
|
||||
|
||||
// Test Field Norms
|
||||
segInfoStat.fieldNormStatus = testFieldNorms(fieldNames, reader);
|
||||
segInfoStat.fieldNormStatus = testFieldNorms(fieldInfos, reader);
|
||||
|
||||
// Test the Term Index
|
||||
segInfoStat.termIndexStatus = testTermIndex(reader);
|
||||
|
@ -631,7 +630,7 @@ public class CheckIndex {
|
|||
/**
|
||||
* Test field norms.
|
||||
*/
|
||||
private Status.FieldNormStatus testFieldNorms(Collection<String> fieldNames, SegmentReader reader) {
|
||||
private Status.FieldNormStatus testFieldNorms(FieldInfos fieldInfos, SegmentReader reader) {
|
||||
final Status.FieldNormStatus status = new Status.FieldNormStatus();
|
||||
|
||||
try {
|
||||
|
@ -639,29 +638,27 @@ public class CheckIndex {
|
|||
if (infoStream != null) {
|
||||
infoStream.print(" test: field norms.........");
|
||||
}
|
||||
FieldInfos infos = reader.fieldInfos();
|
||||
DocValues dv;
|
||||
for (final String fieldName : fieldNames) {
|
||||
FieldInfo info = infos.fieldInfo(fieldName);
|
||||
if (reader.hasNorms(fieldName)) {
|
||||
dv = reader.normValues(fieldName);
|
||||
for (FieldInfo info : fieldInfos) {
|
||||
if (reader.hasNorms(info.name)) {
|
||||
dv = reader.normValues(info.name);
|
||||
assert dv != null;
|
||||
if (dv.getSource().hasArray()) {
|
||||
Object array = dv.getSource().getArray();
|
||||
if (Array.getLength(array) != reader.maxDoc()) {
|
||||
throw new RuntimeException("norms for field: " + fieldName + " are of the wrong size");
|
||||
throw new RuntimeException("norms for field: " + info.name + " are of the wrong size");
|
||||
}
|
||||
}
|
||||
if (!info.isIndexed || info.omitNorms) {
|
||||
throw new RuntimeException("field: " + fieldName + " should omit norms but has them!");
|
||||
throw new RuntimeException("field: " + info.name + " should omit norms but has them!");
|
||||
}
|
||||
++status.totFields;
|
||||
} else {
|
||||
if (reader.normValues(fieldName) != null) {
|
||||
throw new RuntimeException("field: " + fieldName + " should omit norms but has them!");
|
||||
if (reader.normValues(info.name) != null) {
|
||||
throw new RuntimeException("field: " + info.name + " should omit norms but has them!");
|
||||
}
|
||||
if (info.isIndexed && !info.omitNorms) {
|
||||
throw new RuntimeException("field: " + fieldName + " should have norms but omits them!");
|
||||
throw new RuntimeException("field: " + info.name + " should have norms but omits them!");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,6 +1,5 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
import org.apache.lucene.index.DocValues;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -27,15 +26,11 @@ public final class FieldInfo {
|
|||
public boolean isIndexed;
|
||||
private DocValues.Type docValues;
|
||||
|
||||
|
||||
// true if term vector for this field should be stored
|
||||
// True if any document indexed term vectors
|
||||
public boolean storeTermVector;
|
||||
public boolean storeOffsetWithTermVector;
|
||||
public boolean storePositionWithTermVector;
|
||||
|
||||
public boolean omitNorms; // omit norms associated with indexed fields
|
||||
public IndexOptions indexOptions;
|
||||
|
||||
public boolean storePayloads; // whether this field stores payloads together with term positions
|
||||
|
||||
/**
|
||||
|
@ -56,7 +51,6 @@ public final class FieldInfo {
|
|||
* @lucene.experimental
|
||||
*/
|
||||
public FieldInfo(String name, boolean isIndexed, int number, boolean storeTermVector,
|
||||
boolean storePositionWithTermVector, boolean storeOffsetWithTermVector,
|
||||
boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValues) {
|
||||
this.name = name;
|
||||
this.isIndexed = isIndexed;
|
||||
|
@ -64,15 +58,11 @@ public final class FieldInfo {
|
|||
this.docValues = docValues;
|
||||
if (isIndexed) {
|
||||
this.storeTermVector = storeTermVector;
|
||||
this.storeOffsetWithTermVector = storeOffsetWithTermVector;
|
||||
this.storePositionWithTermVector = storePositionWithTermVector;
|
||||
this.storePayloads = storePayloads;
|
||||
this.omitNorms = omitNorms;
|
||||
this.indexOptions = indexOptions;
|
||||
} else { // for non-indexed fields, leave defaults
|
||||
this.storeTermVector = false;
|
||||
this.storeOffsetWithTermVector = false;
|
||||
this.storePositionWithTermVector = false;
|
||||
this.storePayloads = false;
|
||||
this.omitNorms = false;
|
||||
this.indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
|
||||
|
@ -82,14 +72,12 @@ public final class FieldInfo {
|
|||
|
||||
@Override
|
||||
public Object clone() {
|
||||
FieldInfo clone = new FieldInfo(name, isIndexed, number, storeTermVector, storePositionWithTermVector,
|
||||
storeOffsetWithTermVector, omitNorms, storePayloads, indexOptions, docValues);
|
||||
return clone;
|
||||
return new FieldInfo(name, isIndexed, number, storeTermVector,
|
||||
omitNorms, storePayloads, indexOptions, docValues);
|
||||
}
|
||||
|
||||
// should only be called by FieldInfos#addOrUpdate
|
||||
void update(boolean isIndexed, boolean storeTermVector, boolean storePositionWithTermVector,
|
||||
boolean storeOffsetWithTermVector, boolean omitNorms, boolean storePayloads, IndexOptions indexOptions) {
|
||||
void update(boolean isIndexed, boolean storeTermVector, boolean omitNorms, boolean storePayloads, IndexOptions indexOptions) {
|
||||
|
||||
if (this.isIndexed != isIndexed) {
|
||||
this.isIndexed = true; // once indexed, always index
|
||||
|
@ -98,12 +86,6 @@ public final class FieldInfo {
|
|||
if (this.storeTermVector != storeTermVector) {
|
||||
this.storeTermVector = true; // once vector, always vector
|
||||
}
|
||||
if (this.storePositionWithTermVector != storePositionWithTermVector) {
|
||||
this.storePositionWithTermVector = true; // once vector, always vector
|
||||
}
|
||||
if (this.storeOffsetWithTermVector != storeOffsetWithTermVector) {
|
||||
this.storeOffsetWithTermVector = true; // once vector, always vector
|
||||
}
|
||||
if (this.storePayloads != storePayloads) {
|
||||
this.storePayloads = true;
|
||||
}
|
||||
|
@ -139,9 +121,7 @@ public final class FieldInfo {
|
|||
return docValues;
|
||||
}
|
||||
|
||||
public void setStoreTermVectors(boolean withPositions, boolean withOffsets) {
|
||||
public void setStoreTermVectors() {
|
||||
storeTermVector = true;
|
||||
storePositionWithTermVector |= withPositions;
|
||||
storeOffsetWithTermVector |= withOffsets;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,7 +25,6 @@ import java.util.SortedMap;
|
|||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
|
||||
/** Access to the Field Info file that describes document fields and whether or
|
||||
* not they are indexed. Each segment has a separate Field Info file. Objects
|
||||
|
@ -122,6 +121,16 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
}
|
||||
}
|
||||
|
||||
public FieldInfos() {
|
||||
this(new FieldNumberBiMap());
|
||||
}
|
||||
|
||||
public void add(FieldInfos other) {
|
||||
for(FieldInfo fieldInfo : other){
|
||||
add(fieldInfo);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new FieldInfos instance with the given {@link FieldNumberBiMap}.
|
||||
* If the {@link FieldNumberBiMap} is <code>null</code> this instance will be read-only.
|
||||
|
@ -202,13 +211,10 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
*
|
||||
* @param names The names of the fields
|
||||
* @param storeTermVectors Whether the fields store term vectors or not
|
||||
* @param storePositionWithTermVector true if positions should be stored.
|
||||
* @param storeOffsetWithTermVector true if offsets should be stored
|
||||
*/
|
||||
synchronized public void addOrUpdateIndexed(Collection<String> names, boolean storeTermVectors, boolean storePositionWithTermVector,
|
||||
boolean storeOffsetWithTermVector) {
|
||||
synchronized public void addOrUpdateIndexed(Collection<String> names, boolean storeTermVectors) {
|
||||
for (String name : names) {
|
||||
addOrUpdate(name, true, storeTermVectors, storePositionWithTermVector, storeOffsetWithTermVector);
|
||||
addOrUpdate(name, true, storeTermVectors);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -231,21 +237,10 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
*
|
||||
* @param name The name of the IndexableField
|
||||
* @param isIndexed true if the field is indexed
|
||||
* @see #addOrUpdate(String, boolean, boolean, boolean, boolean)
|
||||
* @see #addOrUpdate(String, boolean, boolean)
|
||||
*/
|
||||
synchronized public void addOrUpdate(String name, boolean isIndexed) {
|
||||
addOrUpdate(name, isIndexed, false, false, false, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Calls 5 parameter add with false for term vector positions and offsets.
|
||||
*
|
||||
* @param name The name of the field
|
||||
* @param isIndexed true if the field is indexed
|
||||
* @param storeTermVector true if the term vector should be stored
|
||||
*/
|
||||
synchronized public void addOrUpdate(String name, boolean isIndexed, boolean storeTermVector){
|
||||
addOrUpdate(name, isIndexed, storeTermVector, false, false, false);
|
||||
addOrUpdate(name, isIndexed, false, false);
|
||||
}
|
||||
|
||||
/** If the field is not yet known, adds it. If it is known, checks to make
|
||||
|
@ -256,13 +251,9 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
* @param name The name of the field
|
||||
* @param isIndexed true if the field is indexed
|
||||
* @param storeTermVector true if the term vector should be stored
|
||||
* @param storePositionWithTermVector true if the term vector with positions should be stored
|
||||
* @param storeOffsetWithTermVector true if the term vector with offsets should be stored
|
||||
*/
|
||||
synchronized public void addOrUpdate(String name, boolean isIndexed, boolean storeTermVector,
|
||||
boolean storePositionWithTermVector, boolean storeOffsetWithTermVector) {
|
||||
|
||||
addOrUpdate(name, isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, false);
|
||||
synchronized public void addOrUpdate(String name, boolean isIndexed, boolean storeTermVector) {
|
||||
addOrUpdate(name, isIndexed, storeTermVector, false);
|
||||
}
|
||||
|
||||
/** If the field is not yet known, adds it. If it is known, checks to make
|
||||
|
@ -273,14 +264,11 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
* @param name The name of the field
|
||||
* @param isIndexed true if the field is indexed
|
||||
* @param storeTermVector true if the term vector should be stored
|
||||
* @param storePositionWithTermVector true if the term vector with positions should be stored
|
||||
* @param storeOffsetWithTermVector true if the term vector with offsets should be stored
|
||||
* @param omitNorms true if the norms for the indexed field should be omitted
|
||||
*/
|
||||
synchronized public void addOrUpdate(String name, boolean isIndexed, boolean storeTermVector,
|
||||
boolean storePositionWithTermVector, boolean storeOffsetWithTermVector, boolean omitNorms) {
|
||||
addOrUpdate(name, isIndexed, storeTermVector, storePositionWithTermVector,
|
||||
storeOffsetWithTermVector, omitNorms, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS, null);
|
||||
boolean omitNorms) {
|
||||
addOrUpdate(name, isIndexed, storeTermVector, omitNorms, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS, null);
|
||||
}
|
||||
|
||||
/** If the field is not yet known, adds it. If it is known, checks to make
|
||||
|
@ -291,17 +279,13 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
* @param name The name of the field
|
||||
* @param isIndexed true if the field is indexed
|
||||
* @param storeTermVector true if the term vector should be stored
|
||||
* @param storePositionWithTermVector true if the term vector with positions should be stored
|
||||
* @param storeOffsetWithTermVector true if the term vector with offsets should be stored
|
||||
* @param omitNorms true if the norms for the indexed field should be omitted
|
||||
* @param storePayloads true if payloads should be stored for this field
|
||||
* @param indexOptions if term freqs should be omitted for this field
|
||||
*/
|
||||
synchronized public FieldInfo addOrUpdate(String name, boolean isIndexed, boolean storeTermVector,
|
||||
boolean storePositionWithTermVector, boolean storeOffsetWithTermVector,
|
||||
boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValues) {
|
||||
return addOrUpdateInternal(name, -1, isIndexed, storeTermVector, storePositionWithTermVector,
|
||||
storeOffsetWithTermVector, omitNorms, storePayloads, indexOptions, docValues);
|
||||
return addOrUpdateInternal(name, -1, isIndexed, storeTermVector, omitNorms, storePayloads, indexOptions, docValues);
|
||||
}
|
||||
|
||||
// NOTE: this method does not carry over termVector
|
||||
|
@ -315,13 +299,13 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
// rather, each component in the chain should update
|
||||
// what it "owns". EG fieldType.indexOptions() should
|
||||
// be updated by maybe FreqProxTermsWriterPerField:
|
||||
return addOrUpdateInternal(name, -1, fieldType.indexed(), false, false, false,
|
||||
return addOrUpdateInternal(name, -1, fieldType.indexed(), false,
|
||||
fieldType.omitNorms(), false,
|
||||
fieldType.indexOptions(), null);
|
||||
}
|
||||
|
||||
synchronized private FieldInfo addOrUpdateInternal(String name, int preferredFieldNumber, boolean isIndexed,
|
||||
boolean storeTermVector, boolean storePositionWithTermVector, boolean storeOffsetWithTermVector,
|
||||
boolean storeTermVector,
|
||||
boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValues) {
|
||||
if (globalFieldNumbers == null) {
|
||||
throw new IllegalStateException("FieldInfos are read-only, create a new instance with a global field map to make modifications to FieldInfos");
|
||||
|
@ -329,9 +313,9 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
FieldInfo fi = fieldInfo(name);
|
||||
if (fi == null) {
|
||||
final int fieldNumber = nextFieldNumber(name, preferredFieldNumber);
|
||||
fi = addInternal(name, fieldNumber, isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, indexOptions, docValues);
|
||||
fi = addInternal(name, fieldNumber, isIndexed, storeTermVector, omitNorms, storePayloads, indexOptions, docValues);
|
||||
} else {
|
||||
fi.update(isIndexed, storeTermVector, storePositionWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, indexOptions);
|
||||
fi.update(isIndexed, storeTermVector, omitNorms, storePayloads, indexOptions);
|
||||
fi.setDocValuesType(docValues);
|
||||
}
|
||||
version++;
|
||||
|
@ -341,7 +325,6 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
synchronized public FieldInfo add(FieldInfo fi) {
|
||||
// IMPORTANT - reuse the field number if possible for consistent field numbers across segments
|
||||
return addOrUpdateInternal(fi.name, fi.number, fi.isIndexed, fi.storeTermVector,
|
||||
fi.storePositionWithTermVector, fi.storeOffsetWithTermVector,
|
||||
fi.omitNorms, fi.storePayloads,
|
||||
fi.indexOptions, fi.getDocValuesType());
|
||||
}
|
||||
|
@ -350,14 +333,13 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
* NOTE: if you call this method from a public method make sure you check if we are modifiable and throw an exception otherwise
|
||||
*/
|
||||
private FieldInfo addInternal(String name, int fieldNumber, boolean isIndexed,
|
||||
boolean storeTermVector, boolean storePositionWithTermVector,
|
||||
boolean storeOffsetWithTermVector, boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValuesType) {
|
||||
boolean storeTermVector, boolean omitNorms, boolean storePayloads,
|
||||
IndexOptions indexOptions, DocValues.Type docValuesType) {
|
||||
// don't check modifiable here since we use that to initially build up FIs
|
||||
if (globalFieldNumbers != null) {
|
||||
globalFieldNumbers.setIfNotSet(fieldNumber, name);
|
||||
}
|
||||
final FieldInfo fi = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, storePositionWithTermVector,
|
||||
storeOffsetWithTermVector, omitNorms, storePayloads, indexOptions, docValuesType);
|
||||
final FieldInfo fi = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, omitNorms, storePayloads, indexOptions, docValuesType);
|
||||
putInternal(fi);
|
||||
return fi;
|
||||
}
|
||||
|
|
|
@ -22,7 +22,6 @@ import org.apache.lucene.util.Bits;
|
|||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Map;
|
||||
import java.util.Comparator;
|
||||
|
||||
|
@ -294,6 +293,11 @@ public class FilterIndexReader extends IndexReader {
|
|||
return in.getLiveDocs();
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldInfos getFieldInfos() {
|
||||
return in.getFieldInfos();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Fields getTermVectors(int docID)
|
||||
throws IOException {
|
||||
|
@ -342,12 +346,6 @@ public class FilterIndexReader extends IndexReader {
|
|||
in.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<String> getFieldNames(IndexReader.FieldOption fieldNames) {
|
||||
ensureOpen();
|
||||
return in.getFieldNames(fieldNames);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getVersion() {
|
||||
ensureOpen();
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.List;
|
||||
|
@ -29,7 +28,6 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.DocumentStoredFieldVisitor;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.search.SearcherManager; // javadocs
|
||||
import org.apache.lucene.store.*;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -108,39 +106,6 @@ public abstract class IndexReader implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Constants describing field properties, for example used for
|
||||
* {@link IndexReader#getFieldNames(FieldOption)}.
|
||||
*/
|
||||
public static enum FieldOption {
|
||||
/** All fields */
|
||||
ALL,
|
||||
/** All indexed fields */
|
||||
INDEXED,
|
||||
/** All fields that store payloads */
|
||||
STORES_PAYLOADS,
|
||||
/** All fields that omit tf */
|
||||
OMIT_TERM_FREQ_AND_POSITIONS,
|
||||
/** All fields that omit positions */
|
||||
OMIT_POSITIONS,
|
||||
/** All fields which are not indexed */
|
||||
UNINDEXED,
|
||||
/** All fields which are indexed with termvectors enabled */
|
||||
INDEXED_WITH_TERMVECTOR,
|
||||
/** All fields which are indexed but don't have termvectors enabled */
|
||||
INDEXED_NO_TERMVECTOR,
|
||||
/** All fields with termvectors enabled. Please note that only standard termvector fields are returned */
|
||||
TERMVECTOR,
|
||||
/** All fields with termvectors with position values enabled */
|
||||
TERMVECTOR_WITH_POSITION,
|
||||
/** All fields with termvectors with offset values enabled */
|
||||
TERMVECTOR_WITH_OFFSET,
|
||||
/** All fields with termvectors with offset values and position values enabled */
|
||||
TERMVECTOR_WITH_POSITION_OFFSET,
|
||||
/** All fields holding doc values */
|
||||
DOC_VALUES
|
||||
}
|
||||
|
||||
private volatile boolean closed;
|
||||
|
||||
private final AtomicInteger refCount = new AtomicInteger();
|
||||
|
@ -897,15 +862,14 @@ public abstract class IndexReader implements Closeable {
|
|||
/** Implements close. */
|
||||
protected abstract void doClose() throws IOException;
|
||||
|
||||
|
||||
/**
|
||||
* Get a list of unique field names that exist in this index and have the specified
|
||||
* field option information.
|
||||
* @param fldOption specifies which field option should be available for the returned fields
|
||||
* @return Collection of Strings indicating the names of the fields.
|
||||
* @see IndexReader.FieldOption
|
||||
* Get the {@link FieldInfos} describing all fields in
|
||||
* this reader. NOTE: do not make any changes to the
|
||||
* returned FieldInfos!
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract Collection<String> getFieldNames(FieldOption fldOption);
|
||||
public abstract FieldInfos getFieldInfos();
|
||||
|
||||
/** Returns the {@link Bits} representing live (not
|
||||
* deleted) docs. A set bit indicates the doc ID has not
|
||||
|
|
|
@ -17,12 +17,13 @@ package org.apache.lucene.index;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
|
||||
|
||||
/** An IndexReader which reads multiple, parallel indexes. Each index added
|
||||
* must have the same number of documents, but typically each contains
|
||||
|
@ -53,6 +54,7 @@ public class ParallelReader extends IndexReader {
|
|||
private int maxDoc;
|
||||
private int numDocs;
|
||||
private boolean hasDeletions;
|
||||
private final FieldInfos fieldInfos;
|
||||
|
||||
private final ParallelFields fields = new ParallelFields();
|
||||
|
||||
|
@ -68,6 +70,7 @@ public class ParallelReader extends IndexReader {
|
|||
public ParallelReader(boolean closeSubReaders) throws IOException {
|
||||
super();
|
||||
this.incRefReaders = !closeSubReaders;
|
||||
fieldInfos = new FieldInfos();
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
|
@ -120,12 +123,13 @@ public class ParallelReader extends IndexReader {
|
|||
throw new IllegalArgumentException
|
||||
("All readers must have same numDocs: "+numDocs+"!="+reader.numDocs());
|
||||
|
||||
Collection<String> fields = reader.getFieldNames(IndexReader.FieldOption.ALL);
|
||||
readerToFields.put(reader, fields);
|
||||
for (final String field : fields) { // update fieldToReader map
|
||||
if (fieldToReader.get(field) == null) {
|
||||
fieldToReader.put(field, reader);
|
||||
this.fields.addField(field, MultiFields.getFields(reader).terms(field));
|
||||
final FieldInfos readerFieldInfos = ReaderUtil.getMergedFieldInfos(reader);
|
||||
for(FieldInfo fieldInfo : readerFieldInfos) { // update fieldToReader map
|
||||
// NOTE: first reader having a given field "wins":
|
||||
if (fieldToReader.get(fieldInfo.name) == null) {
|
||||
fieldInfos.add(fieldInfo);
|
||||
fieldToReader.put(fieldInfo.name, reader);
|
||||
this.fields.addField(fieldInfo.name, MultiFields.getFields(reader).terms(fieldInfo.name));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -193,6 +197,11 @@ public class ParallelReader extends IndexReader {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldInfos getFieldInfos() {
|
||||
return fieldInfos;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getLiveDocs() {
|
||||
ensureOpen();
|
||||
|
@ -383,17 +392,6 @@ public class ParallelReader extends IndexReader {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<String> getFieldNames (IndexReader.FieldOption fieldNames) {
|
||||
ensureOpen();
|
||||
Set<String> fieldSet = new HashSet<String>();
|
||||
for (final IndexReader reader : readers) {
|
||||
Collection<String> names = reader.getFieldNames(fieldNames);
|
||||
fieldSet.addAll(names);
|
||||
}
|
||||
return fieldSet;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReaderContext getTopReaderContext() {
|
||||
ensureOpen();
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -30,9 +29,6 @@ import org.apache.lucene.codecs.FieldsConsumer;
|
|||
import org.apache.lucene.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.codecs.StoredFieldsWriter;
|
||||
import org.apache.lucene.codecs.TermVectorsWriter;
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.index.IndexReader.FieldOption;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -135,18 +131,6 @@ final class SegmentMerger {
|
|||
return mergeState;
|
||||
}
|
||||
|
||||
private static void addIndexed(IndexReader reader, FieldInfos fInfos,
|
||||
Collection<String> names, boolean storeTermVectors,
|
||||
boolean storePositionWithTermVector, boolean storeOffsetWithTermVector,
|
||||
boolean storePayloads, IndexOptions indexOptions)
|
||||
throws IOException {
|
||||
for (String field : names) {
|
||||
fInfos.addOrUpdate(field, true, storeTermVectors,
|
||||
storePositionWithTermVector, storeOffsetWithTermVector, !reader
|
||||
.hasNorms(field), storePayloads, indexOptions, null);
|
||||
}
|
||||
}
|
||||
|
||||
private void setMatchingSegmentReaders() {
|
||||
// If the i'th reader is a SegmentReader and has
|
||||
// identical fieldName -> number mapping, then this
|
||||
|
@ -160,10 +144,15 @@ final class SegmentMerger {
|
|||
// stored fields:
|
||||
for (int i = 0; i < numReaders; i++) {
|
||||
MergeState.IndexReaderAndLiveDocs reader = mergeState.readers.get(i);
|
||||
// TODO: we may be able to broaden this to
|
||||
// non-SegmentReaders, since FieldInfos is now
|
||||
// required? But... this'd also require exposing
|
||||
// bulk-copy (TVs and stored fields) API in foreign
|
||||
// readers..
|
||||
if (reader.reader instanceof SegmentReader) {
|
||||
SegmentReader segmentReader = (SegmentReader) reader.reader;
|
||||
boolean same = true;
|
||||
FieldInfos segmentFieldInfos = segmentReader.fieldInfos();
|
||||
FieldInfos segmentFieldInfos = segmentReader.getFieldInfos();
|
||||
for (FieldInfo fi : segmentFieldInfos) {
|
||||
if (!mergeState.fieldInfos.fieldName(fi.number).equals(fi.name)) {
|
||||
same = false;
|
||||
|
@ -208,9 +197,7 @@ final class SegmentMerger {
|
|||
|
||||
for (MergeState.IndexReaderAndLiveDocs readerAndLiveDocs : mergeState.readers) {
|
||||
final IndexReader reader = readerAndLiveDocs.reader;
|
||||
if (reader instanceof SegmentReader) {
|
||||
SegmentReader segmentReader = (SegmentReader) reader;
|
||||
FieldInfos readerFieldInfos = segmentReader.fieldInfos();
|
||||
FieldInfos readerFieldInfos = reader.getFieldInfos();
|
||||
for (FieldInfo fi : readerFieldInfos) {
|
||||
FieldInfo merged = mergeState.fieldInfos.add(fi);
|
||||
// update the type promotion mapping for this reader
|
||||
|
@ -219,26 +206,6 @@ final class SegmentMerger {
|
|||
docValuesTypes.put(merged, mergeDocValuesType(previous, reader.docValues(fi.name)));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
addIndexed(reader, mergeState.fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_POSITION_OFFSET), true, true, true, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
|
||||
addIndexed(reader, mergeState.fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_POSITION), true, true, false, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
|
||||
addIndexed(reader, mergeState.fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_OFFSET), true, false, true, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
|
||||
addIndexed(reader, mergeState.fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR), true, false, false, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
|
||||
addIndexed(reader, mergeState.fieldInfos, reader.getFieldNames(FieldOption.OMIT_POSITIONS), false, false, false, false, IndexOptions.DOCS_AND_FREQS);
|
||||
addIndexed(reader, mergeState.fieldInfos, reader.getFieldNames(FieldOption.OMIT_TERM_FREQ_AND_POSITIONS), false, false, false, false, IndexOptions.DOCS_ONLY);
|
||||
addIndexed(reader, mergeState.fieldInfos, reader.getFieldNames(FieldOption.STORES_PAYLOADS), false, false, false, true, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
|
||||
addIndexed(reader, mergeState.fieldInfos, reader.getFieldNames(FieldOption.INDEXED), false, false, false, false, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
|
||||
mergeState.fieldInfos.addOrUpdate(reader.getFieldNames(FieldOption.UNINDEXED), false);
|
||||
Collection<String> dvNames = reader.getFieldNames(FieldOption.DOC_VALUES);
|
||||
mergeState.fieldInfos.addOrUpdate(dvNames, false);
|
||||
for (String dvName : dvNames) {
|
||||
FieldInfo merged = mergeState.fieldInfos.fieldInfo(dvName);
|
||||
DocValues docValues = reader.docValues(dvName);
|
||||
merged.setDocValuesType(docValues.type());
|
||||
TypePromoter previous = docValuesTypes.get(merged);
|
||||
docValuesTypes.put(merged, mergeDocValuesType(previous, docValues));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// update any promoted doc values types:
|
||||
|
|
|
@ -18,16 +18,11 @@ package org.apache.lucene.index;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.codecs.PerDocProducer;
|
||||
import org.apache.lucene.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.codecs.TermVectorsReader;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.search.FieldCache; // javadocs
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.BitVector;
|
||||
|
@ -162,7 +157,8 @@ public final class SegmentReader extends IndexReader {
|
|||
return liveDocs != null;
|
||||
}
|
||||
|
||||
FieldInfos fieldInfos() {
|
||||
@Override
|
||||
public FieldInfos getFieldInfos() {
|
||||
return core.fieldInfos;
|
||||
}
|
||||
|
||||
|
@ -198,62 +194,6 @@ public final class SegmentReader extends IndexReader {
|
|||
return si.docCount;
|
||||
}
|
||||
|
||||
/**
|
||||
* @see IndexReader#getFieldNames(org.apache.lucene.index.IndexReader.FieldOption)
|
||||
*/
|
||||
@Override
|
||||
public Collection<String> getFieldNames(IndexReader.FieldOption fieldOption) {
|
||||
ensureOpen();
|
||||
|
||||
Set<String> fieldSet = new HashSet<String>();
|
||||
for (FieldInfo fi : core.fieldInfos) {
|
||||
if (fieldOption == IndexReader.FieldOption.ALL) {
|
||||
fieldSet.add(fi.name);
|
||||
}
|
||||
else if (!fi.isIndexed && fieldOption == IndexReader.FieldOption.UNINDEXED) {
|
||||
fieldSet.add(fi.name);
|
||||
}
|
||||
else if (fi.indexOptions == IndexOptions.DOCS_ONLY && fieldOption == IndexReader.FieldOption.OMIT_TERM_FREQ_AND_POSITIONS) {
|
||||
fieldSet.add(fi.name);
|
||||
}
|
||||
else if (fi.indexOptions == IndexOptions.DOCS_AND_FREQS && fieldOption == IndexReader.FieldOption.OMIT_POSITIONS) {
|
||||
fieldSet.add(fi.name);
|
||||
}
|
||||
else if (fi.storePayloads && fieldOption == IndexReader.FieldOption.STORES_PAYLOADS) {
|
||||
fieldSet.add(fi.name);
|
||||
}
|
||||
else if (fi.isIndexed && fieldOption == IndexReader.FieldOption.INDEXED) {
|
||||
fieldSet.add(fi.name);
|
||||
}
|
||||
else if (fi.isIndexed && fi.storeTermVector == false && fieldOption == IndexReader.FieldOption.INDEXED_NO_TERMVECTOR) {
|
||||
fieldSet.add(fi.name);
|
||||
}
|
||||
else if (fi.storeTermVector == true &&
|
||||
fi.storePositionWithTermVector == false &&
|
||||
fi.storeOffsetWithTermVector == false &&
|
||||
fieldOption == IndexReader.FieldOption.TERMVECTOR) {
|
||||
fieldSet.add(fi.name);
|
||||
}
|
||||
else if (fi.isIndexed && fi.storeTermVector && fieldOption == IndexReader.FieldOption.INDEXED_WITH_TERMVECTOR) {
|
||||
fieldSet.add(fi.name);
|
||||
}
|
||||
else if (fi.storePositionWithTermVector && fi.storeOffsetWithTermVector == false && fieldOption == IndexReader.FieldOption.TERMVECTOR_WITH_POSITION) {
|
||||
fieldSet.add(fi.name);
|
||||
}
|
||||
else if (fi.storeOffsetWithTermVector && fi.storePositionWithTermVector == false && fieldOption == IndexReader.FieldOption.TERMVECTOR_WITH_OFFSET) {
|
||||
fieldSet.add(fi.name);
|
||||
}
|
||||
else if ((fi.storeOffsetWithTermVector && fi.storePositionWithTermVector) &&
|
||||
fieldOption == IndexReader.FieldOption.TERMVECTOR_WITH_POSITION_OFFSET) {
|
||||
fieldSet.add(fi.name);
|
||||
}
|
||||
else if (fi.hasDocValues() && fieldOption == IndexReader.FieldOption.DOC_VALUES) {
|
||||
fieldSet.add(fi.name);
|
||||
}
|
||||
}
|
||||
return fieldSet;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNorms(String field) {
|
||||
ensureOpen();
|
||||
|
|
|
@ -103,4 +103,9 @@ public final class SlowMultiReaderWrapper extends FilterIndexReader {
|
|||
ensureOpen();
|
||||
return readerContext;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldInfos getFieldInfos() {
|
||||
return ReaderUtil.getMergedFieldInfos(in);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -151,7 +151,7 @@ final class TermVectorsConsumerPerField extends TermsHashConsumerPerField {
|
|||
termsHashPerField.reset();
|
||||
|
||||
// commit the termVectors once successful success - FI will otherwise reset them
|
||||
fieldInfo.setStoreTermVectors(doVectorPositions, doVectorOffsets);
|
||||
fieldInfo.setStoreTermVectors();
|
||||
}
|
||||
|
||||
void shrinkHash() {
|
||||
|
|
|
@ -18,9 +18,13 @@ package org.apache.lucene.util;
|
|||
*/
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
|
||||
import org.apache.lucene.index.IndexReader.CompositeReaderContext;
|
||||
|
@ -292,4 +296,26 @@ public final class ReaderUtil {
|
|||
}
|
||||
return hi;
|
||||
}
|
||||
|
||||
public static Collection<String> getIndexedFields(IndexReader reader) {
|
||||
final Collection<String> fields = new HashSet<String>();
|
||||
for(FieldInfo fieldInfo : getMergedFieldInfos(reader)) {
|
||||
if (fieldInfo.isIndexed) {
|
||||
fields.add(fieldInfo.name);
|
||||
}
|
||||
}
|
||||
return fields;
|
||||
}
|
||||
|
||||
/** Call this to get the (merged) FieldInfos for a
|
||||
* composite reader */
|
||||
public static FieldInfos getMergedFieldInfos(IndexReader reader) {
|
||||
final List<IndexReader> subReaders = new ArrayList<IndexReader>();
|
||||
ReaderUtil.gatherSubReaders(subReaders, reader);
|
||||
final FieldInfos fieldInfos = new FieldInfos();
|
||||
for(IndexReader subReader : subReaders) {
|
||||
fieldInfos.add(subReader.getFieldInfos());
|
||||
}
|
||||
return fieldInfos;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1740,10 +1740,6 @@ document.write("Last Published: " + document.lastModified);
|
|||
without term vectors.
|
||||
</li>
|
||||
|
||||
<li>If the third lowest-order bit is set (0x04), term positions are stored with the term vectors.</li>
|
||||
|
||||
<li>If the fourth lowest-order bit is set (0x08), term offsets are stored with the term vectors.</li>
|
||||
|
||||
<li>If the fifth lowest-order bit is set (0x10), norms are omitted for the indexed field.</li>
|
||||
|
||||
<li>If the sixth lowest-order bit is set (0x20), payloads are stored for the indexed field.</li>
|
||||
|
@ -1908,7 +1904,7 @@ document.write("Last Published: " + document.lastModified);
|
|||
</li>
|
||||
|
||||
</ol>
|
||||
<a name="N106F0"></a><a name="Term Dictionary"></a>
|
||||
<a name="N106EA"></a><a name="Term Dictionary"></a>
|
||||
<h3 class="boxed">Term Dictionary</h3>
|
||||
<p>
|
||||
The term dictionary is represented as two files:
|
||||
|
@ -2100,7 +2096,7 @@ document.write("Last Published: " + document.lastModified);
|
|||
</li>
|
||||
|
||||
</ol>
|
||||
<a name="N10774"></a><a name="Frequencies"></a>
|
||||
<a name="N1076E"></a><a name="Frequencies"></a>
|
||||
<h3 class="boxed">Frequencies</h3>
|
||||
<p>
|
||||
The .frq file contains the lists of documents
|
||||
|
@ -2228,7 +2224,7 @@ document.write("Last Published: " + document.lastModified);
|
|||
entry in level-1. In the example has entry 15 on level 1 a pointer to entry 15 on level 0 and entry 31 on level 1 a pointer
|
||||
to entry 31 on level 0.
|
||||
</p>
|
||||
<a name="N107FC"></a><a name="Positions"></a>
|
||||
<a name="N107F6"></a><a name="Positions"></a>
|
||||
<h3 class="boxed">Positions</h3>
|
||||
<p>
|
||||
The .prx file contains the lists of positions that
|
||||
|
@ -2298,7 +2294,7 @@ document.write("Last Published: " + document.lastModified);
|
|||
Payload. If PayloadLength is not stored, then this Payload has the same
|
||||
length as the Payload at the previous position.
|
||||
</p>
|
||||
<a name="N10838"></a><a name="Normalization Factors"></a>
|
||||
<a name="N10832"></a><a name="Normalization Factors"></a>
|
||||
<h3 class="boxed">Normalization Factors</h3>
|
||||
<p>There's a single .nrm file containing all norms:
|
||||
</p>
|
||||
|
@ -2378,7 +2374,7 @@ document.write("Last Published: " + document.lastModified);
|
|||
</p>
|
||||
<p>Separate norm files are created (when adequate) for both compound and non compound segments.
|
||||
</p>
|
||||
<a name="N10889"></a><a name="Term Vectors"></a>
|
||||
<a name="N10883"></a><a name="Term Vectors"></a>
|
||||
<h3 class="boxed">Term Vectors</h3>
|
||||
<p>
|
||||
Term Vector support is an optional on a field by
|
||||
|
@ -2514,7 +2510,7 @@ document.write("Last Published: " + document.lastModified);
|
|||
</li>
|
||||
|
||||
</ol>
|
||||
<a name="N10925"></a><a name="Deleted Documents"></a>
|
||||
<a name="N1091F"></a><a name="Deleted Documents"></a>
|
||||
<h3 class="boxed">Deleted Documents</h3>
|
||||
<p>The .del file is
|
||||
optional, and only exists when a segment contains deletions.
|
||||
|
@ -2578,7 +2574,7 @@ document.write("Last Published: " + document.lastModified);
|
|||
</div>
|
||||
|
||||
|
||||
<a name="N1095F"></a><a name="Limitations"></a>
|
||||
<a name="N10959"></a><a name="Limitations"></a>
|
||||
<h2 class="boxed">Limitations</h2>
|
||||
<div class="section">
|
||||
<p>
|
||||
|
|
|
@ -1216,8 +1216,6 @@
|
|||
bit is one for fields that have term vectors stored, and zero for fields
|
||||
without term vectors.
|
||||
</li>
|
||||
<li>If the third lowest-order bit is set (0x04), term positions are stored with the term vectors.</li>
|
||||
<li>If the fourth lowest-order bit is set (0x08), term offsets are stored with the term vectors.</li>
|
||||
<li>If the fifth lowest-order bit is set (0x10), norms are omitted for the indexed field.</li>
|
||||
<li>If the sixth lowest-order bit is set (0x20), payloads are stored for the indexed field.</li>
|
||||
<li>If the seventh lowest-order bit is set (0x40), term frequencies and positions omitted for the indexed field.</li>
|
||||
|
|
|
@ -296,8 +296,6 @@ public class TestConsistentFieldNumbers extends LuceneTestCase {
|
|||
Field expected = getField(Integer.parseInt(fi.name));
|
||||
assertEquals(expected.fieldType().indexed(), fi.isIndexed);
|
||||
assertEquals(expected.fieldType().storeTermVectors(), fi.storeTermVector);
|
||||
assertEquals(expected.fieldType().storeTermVectorPositions(), fi.storePositionWithTermVector);
|
||||
assertEquals(expected.fieldType().storeTermVectorOffsets(), fi.storeOffsetWithTermVector);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -94,7 +94,7 @@ public class TestDocumentWriter extends LuceneTestCase {
|
|||
|
||||
// test that the norms are not present in the segment if
|
||||
// omitNorms is true
|
||||
for (FieldInfo fi : reader.fieldInfos()) {
|
||||
for (FieldInfo fi : reader.getFieldInfos()) {
|
||||
if (fi.isIndexed) {
|
||||
assertTrue(fi.omitNorms == !reader.hasNorms(fi.name));
|
||||
}
|
||||
|
@ -327,7 +327,7 @@ public class TestDocumentWriter extends LuceneTestCase {
|
|||
_TestUtil.checkIndex(dir);
|
||||
|
||||
SegmentReader reader = getOnlySegmentReader(IndexReader.open(dir));
|
||||
FieldInfos fi = reader.fieldInfos();
|
||||
FieldInfos fi = reader.getFieldInfos();
|
||||
// f1
|
||||
assertFalse("f1 should have no norms", reader.hasNorms("f1"));
|
||||
assertEquals("omitTermFreqAndPositions field bit should not be set for f1", IndexOptions.DOCS_AND_FREQS_AND_POSITIONS, fi.fieldInfo("f1").indexOptions);
|
||||
|
|
|
@ -27,7 +27,6 @@ import org.apache.lucene.analysis.Analyzer;
|
|||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.IndexReader.FieldOption;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -35,6 +34,7 @@ import org.apache.lucene.util.BytesRef;
|
|||
import org.apache.lucene.util.FixedBitSet;
|
||||
import org.apache.lucene.util.LineFileDocs;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
import org.apache.lucene.util.automaton.AutomatonTestUtil;
|
||||
import org.apache.lucene.util.automaton.CompiledAutomaton;
|
||||
import org.apache.lucene.util.automaton.RegExp;
|
||||
|
@ -517,12 +517,23 @@ public class TestDuelingCodecs extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
private static Set<String> getDVFields(IndexReader reader) {
|
||||
Set<String> fields = new HashSet<String>();
|
||||
for(FieldInfo fi : ReaderUtil.getMergedFieldInfos(reader)) {
|
||||
if (fi.hasDocValues()) {
|
||||
fields.add(fi.name);
|
||||
}
|
||||
}
|
||||
|
||||
return fields;
|
||||
}
|
||||
|
||||
/**
|
||||
* checks that docvalues across all fields are equivalent
|
||||
*/
|
||||
public void assertDocValues(IndexReader leftReader, IndexReader rightReader) throws Exception {
|
||||
Set<String> leftValues = new HashSet<String>(leftReader.getFieldNames(FieldOption.DOC_VALUES));
|
||||
Set<String> rightValues = new HashSet<String>(rightReader.getFieldNames(FieldOption.DOC_VALUES));
|
||||
Set<String> leftValues = getDVFields(leftReader);
|
||||
Set<String> rightValues = getDVFields(rightReader);
|
||||
assertEquals(info, leftValues, rightValues);
|
||||
|
||||
for (String field : leftValues) {
|
||||
|
|
|
@ -133,21 +133,14 @@ public class TestFieldInfos extends LuceneTestCase {
|
|||
}
|
||||
try {
|
||||
readOnly.addOrUpdate("bogus", random.nextBoolean(), random.nextBoolean(),
|
||||
random.nextBoolean(), random.nextBoolean());
|
||||
random.nextBoolean());
|
||||
fail("instance should be read only");
|
||||
} catch (IllegalStateException e) {
|
||||
// expected
|
||||
}
|
||||
try {
|
||||
readOnly.addOrUpdate("bogus", random.nextBoolean(), random.nextBoolean(),
|
||||
random.nextBoolean(), random.nextBoolean(), random.nextBoolean());
|
||||
fail("instance should be read only");
|
||||
} catch (IllegalStateException e) {
|
||||
// expected
|
||||
}
|
||||
try {
|
||||
readOnly.addOrUpdate("bogus", random.nextBoolean(), random.nextBoolean(),
|
||||
random.nextBoolean(), random.nextBoolean(), random.nextBoolean(),
|
||||
random.nextBoolean(),
|
||||
random.nextBoolean(), random.nextBoolean() ? IndexOptions.DOCS_ONLY : IndexOptions.DOCS_AND_FREQS_AND_POSITIONS, null);
|
||||
fail("instance should be read only");
|
||||
} catch (IllegalStateException e) {
|
||||
|
|
|
@ -80,24 +80,18 @@ public class TestFieldsReader extends LuceneTestCase {
|
|||
assertTrue(field != null);
|
||||
assertTrue(field.fieldType().storeTermVectors());
|
||||
|
||||
assertTrue(field.fieldType().storeTermVectorOffsets());
|
||||
assertTrue(field.fieldType().storeTermVectorPositions());
|
||||
assertFalse(field.fieldType().omitNorms());
|
||||
assertTrue(field.fieldType().indexOptions() == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
|
||||
|
||||
field = (Field) doc.getField(DocHelper.TEXT_FIELD_3_KEY);
|
||||
assertTrue(field != null);
|
||||
assertFalse(field.fieldType().storeTermVectors());
|
||||
assertFalse(field.fieldType().storeTermVectorOffsets());
|
||||
assertFalse(field.fieldType().storeTermVectorPositions());
|
||||
assertTrue(field.fieldType().omitNorms());
|
||||
assertTrue(field.fieldType().indexOptions() == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
|
||||
|
||||
field = (Field) doc.getField(DocHelper.NO_TF_KEY);
|
||||
assertTrue(field != null);
|
||||
assertFalse(field.fieldType().storeTermVectors());
|
||||
assertFalse(field.fieldType().storeTermVectorOffsets());
|
||||
assertFalse(field.fieldType().storeTermVectorPositions());
|
||||
assertFalse(field.fieldType().omitNorms());
|
||||
assertTrue(field.fieldType().indexOptions() == IndexOptions.DOCS_ONLY);
|
||||
|
||||
|
|
|
@ -18,20 +18,20 @@ package org.apache.lucene.index;
|
|||
*/
|
||||
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Method;
|
||||
import java.lang.reflect.Modifier;
|
||||
import java.util.HashSet;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
|
||||
public class TestFilterIndexReader extends LuceneTestCase {
|
||||
|
||||
private static class TestReader extends FilterIndexReader {
|
||||
|
@ -121,6 +121,11 @@ public class TestFilterIndexReader extends LuceneTestCase {
|
|||
public Fields fields() throws IOException {
|
||||
return new TestFields(super.fields());
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldInfos getFieldInfos() {
|
||||
return ReaderUtil.getMergedFieldInfos(in);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -36,7 +36,6 @@ import org.apache.lucene.document.Field;
|
|||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.index.IndexReader.FieldOption;
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.FieldCache;
|
||||
|
@ -46,6 +45,7 @@ import org.apache.lucene.store.NoSuchDirectoryException;
|
|||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
import org.apache.lucene.util._TestUtil;
|
||||
import org.junit.Assume;
|
||||
|
||||
|
@ -102,11 +102,11 @@ public class TestIndexReader extends LuceneTestCase {
|
|||
writer.close();
|
||||
// set up reader
|
||||
IndexReader reader = IndexReader.open(d);
|
||||
Collection<String> fieldNames = reader.getFieldNames(IndexReader.FieldOption.ALL);
|
||||
assertTrue(fieldNames.contains("keyword"));
|
||||
assertTrue(fieldNames.contains("text"));
|
||||
assertTrue(fieldNames.contains("unindexed"));
|
||||
assertTrue(fieldNames.contains("unstored"));
|
||||
FieldInfos fieldInfos = ReaderUtil.getMergedFieldInfos(reader);
|
||||
assertNotNull(fieldInfos.fieldInfo("keyword"));
|
||||
assertNotNull(fieldInfos.fieldInfo("text"));
|
||||
assertNotNull(fieldInfos.fieldInfo("unindexed"));
|
||||
assertNotNull(fieldInfos.fieldInfo("unstored"));
|
||||
reader.close();
|
||||
// add more documents
|
||||
writer = new IndexWriter(
|
||||
|
@ -160,61 +160,66 @@ public class TestIndexReader extends LuceneTestCase {
|
|||
}
|
||||
|
||||
writer.close();
|
||||
|
||||
// verify fields again
|
||||
reader = IndexReader.open(d);
|
||||
fieldNames = reader.getFieldNames(IndexReader.FieldOption.ALL);
|
||||
assertEquals(13, fieldNames.size()); // the following fields
|
||||
assertTrue(fieldNames.contains("keyword"));
|
||||
assertTrue(fieldNames.contains("text"));
|
||||
assertTrue(fieldNames.contains("unindexed"));
|
||||
assertTrue(fieldNames.contains("unstored"));
|
||||
assertTrue(fieldNames.contains("keyword2"));
|
||||
assertTrue(fieldNames.contains("text2"));
|
||||
assertTrue(fieldNames.contains("unindexed2"));
|
||||
assertTrue(fieldNames.contains("unstored2"));
|
||||
assertTrue(fieldNames.contains("tvnot"));
|
||||
assertTrue(fieldNames.contains("termvector"));
|
||||
assertTrue(fieldNames.contains("tvposition"));
|
||||
assertTrue(fieldNames.contains("tvoffset"));
|
||||
assertTrue(fieldNames.contains("tvpositionoffset"));
|
||||
fieldInfos = ReaderUtil.getMergedFieldInfos(reader);
|
||||
|
||||
Collection<String> allFieldNames = new HashSet<String>();
|
||||
Collection<String> indexedFieldNames = new HashSet<String>();
|
||||
Collection<String> notIndexedFieldNames = new HashSet<String>();
|
||||
Collection<String> tvFieldNames = new HashSet<String>();
|
||||
|
||||
for(FieldInfo fieldInfo : fieldInfos) {
|
||||
final String name = fieldInfo.name;
|
||||
allFieldNames.add(name);
|
||||
if (fieldInfo.isIndexed) {
|
||||
indexedFieldNames.add(name);
|
||||
} else {
|
||||
notIndexedFieldNames.add(name);
|
||||
}
|
||||
if (fieldInfo.storeTermVector) {
|
||||
tvFieldNames.add(name);
|
||||
}
|
||||
}
|
||||
|
||||
assertTrue(allFieldNames.contains("keyword"));
|
||||
assertTrue(allFieldNames.contains("text"));
|
||||
assertTrue(allFieldNames.contains("unindexed"));
|
||||
assertTrue(allFieldNames.contains("unstored"));
|
||||
assertTrue(allFieldNames.contains("keyword2"));
|
||||
assertTrue(allFieldNames.contains("text2"));
|
||||
assertTrue(allFieldNames.contains("unindexed2"));
|
||||
assertTrue(allFieldNames.contains("unstored2"));
|
||||
assertTrue(allFieldNames.contains("tvnot"));
|
||||
assertTrue(allFieldNames.contains("termvector"));
|
||||
assertTrue(allFieldNames.contains("tvposition"));
|
||||
assertTrue(allFieldNames.contains("tvoffset"));
|
||||
assertTrue(allFieldNames.contains("tvpositionoffset"));
|
||||
|
||||
// verify that only indexed fields were returned
|
||||
fieldNames = reader.getFieldNames(IndexReader.FieldOption.INDEXED);
|
||||
assertEquals(11, fieldNames.size()); // 6 original + the 5 termvector fields
|
||||
assertTrue(fieldNames.contains("keyword"));
|
||||
assertTrue(fieldNames.contains("text"));
|
||||
assertTrue(fieldNames.contains("unstored"));
|
||||
assertTrue(fieldNames.contains("keyword2"));
|
||||
assertTrue(fieldNames.contains("text2"));
|
||||
assertTrue(fieldNames.contains("unstored2"));
|
||||
assertTrue(fieldNames.contains("tvnot"));
|
||||
assertTrue(fieldNames.contains("termvector"));
|
||||
assertTrue(fieldNames.contains("tvposition"));
|
||||
assertTrue(fieldNames.contains("tvoffset"));
|
||||
assertTrue(fieldNames.contains("tvpositionoffset"));
|
||||
assertEquals(11, indexedFieldNames.size()); // 6 original + the 5 termvector fields
|
||||
assertTrue(indexedFieldNames.contains("keyword"));
|
||||
assertTrue(indexedFieldNames.contains("text"));
|
||||
assertTrue(indexedFieldNames.contains("unstored"));
|
||||
assertTrue(indexedFieldNames.contains("keyword2"));
|
||||
assertTrue(indexedFieldNames.contains("text2"));
|
||||
assertTrue(indexedFieldNames.contains("unstored2"));
|
||||
assertTrue(indexedFieldNames.contains("tvnot"));
|
||||
assertTrue(indexedFieldNames.contains("termvector"));
|
||||
assertTrue(indexedFieldNames.contains("tvposition"));
|
||||
assertTrue(indexedFieldNames.contains("tvoffset"));
|
||||
assertTrue(indexedFieldNames.contains("tvpositionoffset"));
|
||||
|
||||
// verify that only unindexed fields were returned
|
||||
fieldNames = reader.getFieldNames(IndexReader.FieldOption.UNINDEXED);
|
||||
assertEquals(2, fieldNames.size()); // the following fields
|
||||
assertTrue(fieldNames.contains("unindexed"));
|
||||
assertTrue(fieldNames.contains("unindexed2"));
|
||||
assertEquals(2, notIndexedFieldNames.size()); // the following fields
|
||||
assertTrue(notIndexedFieldNames.contains("unindexed"));
|
||||
assertTrue(notIndexedFieldNames.contains("unindexed2"));
|
||||
|
||||
// verify index term vector fields
|
||||
fieldNames = reader.getFieldNames(IndexReader.FieldOption.TERMVECTOR);
|
||||
assertEquals(1, fieldNames.size()); // 1 field has term vector only
|
||||
assertTrue(fieldNames.contains("termvector"));
|
||||
assertEquals(tvFieldNames.toString(), 4, tvFieldNames.size()); // 4 field has term vector only
|
||||
assertTrue(tvFieldNames.contains("termvector"));
|
||||
|
||||
fieldNames = reader.getFieldNames(IndexReader.FieldOption.TERMVECTOR_WITH_POSITION);
|
||||
assertEquals(1, fieldNames.size()); // 4 fields are indexed with term vectors
|
||||
assertTrue(fieldNames.contains("tvposition"));
|
||||
|
||||
fieldNames = reader.getFieldNames(IndexReader.FieldOption.TERMVECTOR_WITH_OFFSET);
|
||||
assertEquals(1, fieldNames.size()); // 4 fields are indexed with term vectors
|
||||
assertTrue(fieldNames.contains("tvoffset"));
|
||||
|
||||
fieldNames = reader.getFieldNames(IndexReader.FieldOption.TERMVECTOR_WITH_POSITION_OFFSET);
|
||||
assertEquals(1, fieldNames.size()); // 4 fields are indexed with term vectors
|
||||
assertTrue(fieldNames.contains("tvpositionoffset"));
|
||||
reader.close();
|
||||
d.close();
|
||||
}
|
||||
|
@ -519,19 +524,19 @@ public class TestIndexReader extends LuceneTestCase {
|
|||
}
|
||||
|
||||
// check field names
|
||||
Collection<String> fields1 = index1.getFieldNames(FieldOption.ALL);
|
||||
Collection<String> fields2 = index1.getFieldNames(FieldOption.ALL);
|
||||
assertEquals("IndexReaders have different numbers of fields.", fields1.size(), fields2.size());
|
||||
Iterator<String> it1 = fields1.iterator();
|
||||
Iterator<String> it2 = fields1.iterator();
|
||||
while (it1.hasNext()) {
|
||||
assertEquals("Different field names.", it1.next(), it2.next());
|
||||
FieldInfos fieldInfos1 = ReaderUtil.getMergedFieldInfos(index1);
|
||||
FieldInfos fieldInfos2 = ReaderUtil.getMergedFieldInfos(index2);
|
||||
assertEquals("IndexReaders have different numbers of fields.", fieldInfos1.size(), fieldInfos2.size());
|
||||
final int numFields = fieldInfos1.size();
|
||||
for(int fieldID=0;fieldID<numFields;fieldID++) {
|
||||
final FieldInfo fieldInfo1 = fieldInfos1.fieldInfo(fieldID);
|
||||
final FieldInfo fieldInfo2 = fieldInfos2.fieldInfo(fieldID);
|
||||
assertEquals("Different field names.", fieldInfo1.name, fieldInfo2.name);
|
||||
}
|
||||
|
||||
// check norms
|
||||
it1 = fields1.iterator();
|
||||
while (it1.hasNext()) {
|
||||
String curField = it1.next();
|
||||
for(FieldInfo fieldInfo : fieldInfos1) {
|
||||
String curField = fieldInfo.name;
|
||||
DocValues norms1 = MultiDocValues.getNormDocValues(index1, curField);
|
||||
DocValues norms2 = MultiDocValues.getNormDocValues(index2, curField);
|
||||
if (norms1 != null && norms2 != null)
|
||||
|
|
|
@ -67,7 +67,7 @@ public class TestOmitNorms extends LuceneTestCase {
|
|||
writer.close();
|
||||
|
||||
SegmentReader reader = getOnlySegmentReader(IndexReader.open(ram));
|
||||
FieldInfos fi = reader.fieldInfos();
|
||||
FieldInfos fi = reader.getFieldInfos();
|
||||
assertTrue("OmitNorms field bit should be set.", fi.fieldInfo("f1").omitNorms);
|
||||
assertTrue("OmitNorms field bit should be set.", fi.fieldInfo("f2").omitNorms);
|
||||
|
||||
|
@ -121,7 +121,7 @@ public class TestOmitNorms extends LuceneTestCase {
|
|||
writer.close();
|
||||
|
||||
SegmentReader reader = getOnlySegmentReader(IndexReader.open(ram));
|
||||
FieldInfos fi = reader.fieldInfos();
|
||||
FieldInfos fi = reader.getFieldInfos();
|
||||
assertTrue("OmitNorms field bit should be set.", fi.fieldInfo("f1").omitNorms);
|
||||
assertTrue("OmitNorms field bit should be set.", fi.fieldInfo("f2").omitNorms);
|
||||
|
||||
|
@ -169,7 +169,7 @@ public class TestOmitNorms extends LuceneTestCase {
|
|||
writer.close();
|
||||
|
||||
SegmentReader reader = getOnlySegmentReader(IndexReader.open(ram));
|
||||
FieldInfos fi = reader.fieldInfos();
|
||||
FieldInfos fi = reader.getFieldInfos();
|
||||
assertTrue("OmitNorms field bit should not be set.", !fi.fieldInfo("f1").omitNorms);
|
||||
assertTrue("OmitNorms field bit should be set.", fi.fieldInfo("f2").omitNorms);
|
||||
|
||||
|
|
|
@ -154,7 +154,7 @@ public class TestOmitPositions extends LuceneTestCase {
|
|||
writer.close();
|
||||
|
||||
SegmentReader reader = getOnlySegmentReader(IndexReader.open(ram));
|
||||
FieldInfos fi = reader.fieldInfos();
|
||||
FieldInfos fi = reader.getFieldInfos();
|
||||
// docs + docs = docs
|
||||
assertEquals(IndexOptions.DOCS_ONLY, fi.fieldInfo("f1").indexOptions);
|
||||
// docs + docs/freqs = docs
|
||||
|
|
|
@ -101,7 +101,7 @@ public class TestOmitTf extends LuceneTestCase {
|
|||
writer.close();
|
||||
|
||||
SegmentReader reader = getOnlySegmentReader(IndexReader.open(ram));
|
||||
FieldInfos fi = reader.fieldInfos();
|
||||
FieldInfos fi = reader.getFieldInfos();
|
||||
assertEquals("OmitTermFreqAndPositions field bit should be set.", IndexOptions.DOCS_ONLY, fi.fieldInfo("f1").indexOptions);
|
||||
assertEquals("OmitTermFreqAndPositions field bit should be set.", IndexOptions.DOCS_ONLY, fi.fieldInfo("f2").indexOptions);
|
||||
|
||||
|
@ -153,7 +153,7 @@ public class TestOmitTf extends LuceneTestCase {
|
|||
writer.close();
|
||||
|
||||
SegmentReader reader = getOnlySegmentReader(IndexReader.open(ram));
|
||||
FieldInfos fi = reader.fieldInfos();
|
||||
FieldInfos fi = reader.getFieldInfos();
|
||||
assertEquals("OmitTermFreqAndPositions field bit should be set.", IndexOptions.DOCS_ONLY, fi.fieldInfo("f1").indexOptions);
|
||||
assertEquals("OmitTermFreqAndPositions field bit should be set.", IndexOptions.DOCS_ONLY, fi.fieldInfo("f2").indexOptions);
|
||||
|
||||
|
@ -196,7 +196,7 @@ public class TestOmitTf extends LuceneTestCase {
|
|||
writer.close();
|
||||
|
||||
SegmentReader reader = getOnlySegmentReader(IndexReader.open(ram));
|
||||
FieldInfos fi = reader.fieldInfos();
|
||||
FieldInfos fi = reader.getFieldInfos();
|
||||
assertEquals("OmitTermFreqAndPositions field bit should not be set.", IndexOptions.DOCS_AND_FREQS_AND_POSITIONS, fi.fieldInfo("f1").indexOptions);
|
||||
assertEquals("OmitTermFreqAndPositions field bit should be set.", IndexOptions.DOCS_ONLY, fi.fieldInfo("f2").indexOptions);
|
||||
|
||||
|
|
|
@ -18,14 +18,12 @@ package org.apache.lucene.index;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.search.BooleanClause.Occur;
|
||||
import org.apache.lucene.search.similarities.DefaultSimilarity;
|
||||
import org.apache.lucene.search.*;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
@ -75,12 +73,12 @@ public class TestParallelReader extends LuceneTestCase {
|
|||
ParallelReader pr = new ParallelReader();
|
||||
pr.add(IndexReader.open(dir1));
|
||||
pr.add(IndexReader.open(dir2));
|
||||
Collection<String> fieldNames = pr.getFieldNames(IndexReader.FieldOption.ALL);
|
||||
assertEquals(4, fieldNames.size());
|
||||
assertTrue(fieldNames.contains("f1"));
|
||||
assertTrue(fieldNames.contains("f2"));
|
||||
assertTrue(fieldNames.contains("f3"));
|
||||
assertTrue(fieldNames.contains("f4"));
|
||||
FieldInfos fieldInfos = pr.getFieldInfos();
|
||||
assertEquals(4, fieldInfos.size());
|
||||
assertNotNull(fieldInfos.fieldInfo("f1"));
|
||||
assertNotNull(fieldInfos.fieldInfo("f2"));
|
||||
assertNotNull(fieldInfos.fieldInfo("f3"));
|
||||
assertNotNull(fieldInfos.fieldInfo("f4"));
|
||||
pr.close();
|
||||
dir1.close();
|
||||
dir2.close();
|
||||
|
|
|
@ -112,7 +112,7 @@ public class TestPayloads extends LuceneTestCase {
|
|||
writer.close();
|
||||
|
||||
SegmentReader reader = getOnlySegmentReader(IndexReader.open(ram));
|
||||
FieldInfos fi = reader.fieldInfos();
|
||||
FieldInfos fi = reader.getFieldInfos();
|
||||
assertFalse("Payload field bit should not be set.", fi.fieldInfo("f1").storePayloads);
|
||||
assertTrue("Payload field bit should be set.", fi.fieldInfo("f2").storePayloads);
|
||||
assertFalse("Payload field bit should not be set.", fi.fieldInfo("f3").storePayloads);
|
||||
|
@ -139,7 +139,7 @@ public class TestPayloads extends LuceneTestCase {
|
|||
writer.close();
|
||||
|
||||
reader = getOnlySegmentReader(IndexReader.open(ram));
|
||||
fi = reader.fieldInfos();
|
||||
fi = reader.getFieldInfos();
|
||||
assertFalse("Payload field bit should not be set.", fi.fieldInfo("f1").storePayloads);
|
||||
assertTrue("Payload field bit should be set.", fi.fieldInfo("f2").storePayloads);
|
||||
assertTrue("Payload field bit should be set.", fi.fieldInfo("f3").storePayloads);
|
||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.lucene.index;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
|
@ -30,6 +29,7 @@ import org.apache.lucene.util.InfoStream;
|
|||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util._TestUtil;
|
||||
|
||||
|
||||
public class TestSegmentMerger extends LuceneTestCase {
|
||||
//The variables for the new merged segment
|
||||
private Directory mergedDir;
|
||||
|
@ -107,10 +107,15 @@ public class TestSegmentMerger extends LuceneTestCase {
|
|||
assertTrue(termDocs != null);
|
||||
assertTrue(termDocs.nextDoc() != DocsEnum.NO_MORE_DOCS);
|
||||
|
||||
Collection<String> stored = mergedReader.getFieldNames(IndexReader.FieldOption.INDEXED_WITH_TERMVECTOR);
|
||||
assertTrue(stored != null);
|
||||
int tvCount = 0;
|
||||
for(FieldInfo fieldInfo : mergedReader.getFieldInfos()) {
|
||||
if (fieldInfo.storeTermVector) {
|
||||
tvCount++;
|
||||
}
|
||||
}
|
||||
|
||||
//System.out.println("stored size: " + stored.size());
|
||||
assertTrue("We do not have 3 fields that were indexed with term vector",stored.size() == 3);
|
||||
assertEquals("We do not have 3 fields that were indexed with term vector", 3, tvCount);
|
||||
|
||||
Terms vector = mergedReader.getTermVectors(0).terms(DocHelper.TEXT_FIELD_2_KEY);
|
||||
assertNotNull(vector);
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.document.Document;
|
||||
|
@ -74,33 +74,42 @@ public class TestSegmentReader extends LuceneTestCase {
|
|||
}
|
||||
|
||||
public void testGetFieldNameVariations() {
|
||||
Collection<String> result = reader.getFieldNames(IndexReader.FieldOption.ALL);
|
||||
assertTrue(result != null);
|
||||
assertTrue(result.size() == DocHelper.all.size());
|
||||
for (Iterator<String> iter = result.iterator(); iter.hasNext();) {
|
||||
String s = iter.next();
|
||||
//System.out.println("Name: " + s);
|
||||
Collection<String> allFieldNames = new HashSet<String>();
|
||||
Collection<String> indexedFieldNames = new HashSet<String>();
|
||||
Collection<String> notIndexedFieldNames = new HashSet<String>();
|
||||
Collection<String> tvFieldNames = new HashSet<String>();
|
||||
Collection<String> noTVFieldNames = new HashSet<String>();
|
||||
|
||||
for(FieldInfo fieldInfo : reader.getFieldInfos()) {
|
||||
final String name = fieldInfo.name;
|
||||
allFieldNames.add(name);
|
||||
if (fieldInfo.isIndexed) {
|
||||
indexedFieldNames.add(name);
|
||||
} else {
|
||||
notIndexedFieldNames.add(name);
|
||||
}
|
||||
if (fieldInfo.storeTermVector) {
|
||||
tvFieldNames.add(name);
|
||||
} else if (fieldInfo.isIndexed) {
|
||||
noTVFieldNames.add(name);
|
||||
}
|
||||
}
|
||||
|
||||
assertTrue(allFieldNames.size() == DocHelper.all.size());
|
||||
for (String s : allFieldNames) {
|
||||
assertTrue(DocHelper.nameValues.containsKey(s) == true || s.equals(""));
|
||||
}
|
||||
result = reader.getFieldNames(IndexReader.FieldOption.INDEXED);
|
||||
assertTrue(result != null);
|
||||
assertTrue(result.size() == DocHelper.indexed.size());
|
||||
for (Iterator<String> iter = result.iterator(); iter.hasNext();) {
|
||||
String s = iter.next();
|
||||
|
||||
assertTrue(indexedFieldNames.size() == DocHelper.indexed.size());
|
||||
for (String s : indexedFieldNames) {
|
||||
assertTrue(DocHelper.indexed.containsKey(s) == true || s.equals(""));
|
||||
}
|
||||
|
||||
result = reader.getFieldNames(IndexReader.FieldOption.UNINDEXED);
|
||||
assertTrue(result != null);
|
||||
assertTrue(result.size() == DocHelper.unindexed.size());
|
||||
assertTrue(notIndexedFieldNames.size() == DocHelper.unindexed.size());
|
||||
//Get all indexed fields that are storing term vectors
|
||||
result = reader.getFieldNames(IndexReader.FieldOption.INDEXED_WITH_TERMVECTOR);
|
||||
assertTrue(result != null);
|
||||
assertTrue(result.size() == DocHelper.termvector.size());
|
||||
assertTrue(tvFieldNames.size() == DocHelper.termvector.size());
|
||||
|
||||
result = reader.getFieldNames(IndexReader.FieldOption.INDEXED_NO_TERMVECTOR);
|
||||
assertTrue(result != null);
|
||||
assertTrue(result.size() == DocHelper.notermvector.size());
|
||||
assertTrue(noTVFieldNames.size() == DocHelper.notermvector.size());
|
||||
}
|
||||
|
||||
public void testTerms() throws IOException {
|
||||
|
|
|
@ -16,22 +16,23 @@ package org.apache.lucene.analysis.query;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.analysis.AnalyzerWrapper;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.MultiFields;
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.core.StopFilter;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
import org.apache.lucene.util.Version;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.AnalyzerWrapper;
|
||||
import org.apache.lucene.analysis.core.StopFilter;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.MultiFields;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
import org.apache.lucene.util.Version;
|
||||
|
||||
/**
|
||||
* An {@link Analyzer} used primarily at query time to wrap another analyzer and provide a layer of protection
|
||||
* which prevents very common words from being passed into queries.
|
||||
|
@ -84,7 +85,7 @@ public final class QueryAutoStopWordAnalyzer extends AnalyzerWrapper {
|
|||
Analyzer delegate,
|
||||
IndexReader indexReader,
|
||||
int maxDocFreq) throws IOException {
|
||||
this(matchVersion, delegate, indexReader, indexReader.getFieldNames(IndexReader.FieldOption.INDEXED), maxDocFreq);
|
||||
this(matchVersion, delegate, indexReader, ReaderUtil.getIndexedFields(indexReader), maxDocFreq);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -104,7 +105,7 @@ public final class QueryAutoStopWordAnalyzer extends AnalyzerWrapper {
|
|||
Analyzer delegate,
|
||||
IndexReader indexReader,
|
||||
float maxPercentDocs) throws IOException {
|
||||
this(matchVersion, delegate, indexReader, indexReader.getFieldNames(IndexReader.FieldOption.INDEXED), maxPercentDocs);
|
||||
this(matchVersion, delegate, indexReader, ReaderUtil.getIndexedFields(indexReader), maxPercentDocs);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.lucene.search.similarities.TFIDFSimilarity;
|
|||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.lucene.util.PriorityQueue;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
|
||||
|
||||
|
@ -569,7 +570,7 @@ public final class MoreLikeThis {
|
|||
public Query like(int docNum) throws IOException {
|
||||
if (fieldNames == null) {
|
||||
// gather list of valid fields from lucene
|
||||
Collection<String> fields = ir.getFieldNames(IndexReader.FieldOption.INDEXED);
|
||||
Collection<String> fields = ReaderUtil.getIndexedFields(ir);
|
||||
fieldNames = fields.toArray(new String[fields.size()]);
|
||||
}
|
||||
|
||||
|
|
|
@ -27,13 +27,11 @@ import org.apache.lucene.document.Document;
|
|||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.index.*;
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import static org.apache.lucene.index.FieldInfo.IndexOptions.DOCS_ONLY;
|
||||
import static org.apache.lucene.index.FieldInfo.IndexOptions.DOCS_AND_FREQS;
|
||||
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
import org.apache.lucene.util.PriorityQueue;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
import org.apache.solr.analysis.CharFilterFactory;
|
||||
import org.apache.solr.analysis.TokenFilterFactory;
|
||||
|
@ -55,7 +53,9 @@ import org.apache.solr.schema.SchemaField;
|
|||
import org.apache.solr.search.SolrIndexSearcher;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
|
||||
import static org.apache.lucene.index.FieldInfo.IndexOptions.DOCS_AND_FREQS;
|
||||
import static org.apache.lucene.index.FieldInfo.IndexOptions.DOCS_ONLY;
|
||||
|
||||
/**
|
||||
* This handler exposes the internal lucene index. It is inspired by and
|
||||
|
@ -289,11 +289,15 @@ public class LukeRequestHandler extends RequestHandlerBase
|
|||
IndexReader reader = searcher.getIndexReader();
|
||||
IndexSchema schema = searcher.getSchema();
|
||||
|
||||
Set<String> fieldNames = new TreeSet<String>();
|
||||
for(FieldInfo fieldInfo : ReaderUtil.getMergedFieldInfos(reader)) {
|
||||
fieldNames.add(fieldInfo.name);
|
||||
}
|
||||
|
||||
// Walk the term enum and keep a priority queue for each map in our set
|
||||
SimpleOrderedMap<Object> finfo = new SimpleOrderedMap<Object>();
|
||||
Fields theFields = MultiFields.getFields(reader);
|
||||
|
||||
Set<String> fieldNames = new TreeSet<String>(reader.getFieldNames(IndexReader.FieldOption.ALL));
|
||||
for (String fieldName : fieldNames) {
|
||||
if (fields != null && ! fields.contains(fieldName)) {
|
||||
continue; // we're not interested in this term
|
||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.lucene.store.FSDirectory;
|
|||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.OpenBitSet;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
import org.apache.solr.common.params.ModifiableSolrParams;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.apache.solr.common.util.SimpleOrderedMap;
|
||||
|
@ -180,7 +181,10 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
|
|||
}
|
||||
optimizer = solrConfig.filtOptEnabled ? new LuceneQueryOptimizer(solrConfig.filtOptCacheSize,solrConfig.filtOptThreshold) : null;
|
||||
|
||||
fieldNames = r.getFieldNames(IndexReader.FieldOption.ALL);
|
||||
fieldNames = new HashSet<String>();
|
||||
for(FieldInfo fieldInfo : ReaderUtil.getMergedFieldInfos(r)) {
|
||||
fieldNames.add(fieldInfo.name);
|
||||
}
|
||||
|
||||
// do this at the end since an exception in the constructor means we won't close
|
||||
numOpens.incrementAndGet();
|
||||
|
@ -421,8 +425,6 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
|
|||
public void stringField(FieldInfo fieldInfo, String value) throws IOException {
|
||||
final FieldType ft = new FieldType(TextField.TYPE_STORED);
|
||||
ft.setStoreTermVectors(fieldInfo.storeTermVector);
|
||||
ft.setStoreTermVectorPositions(fieldInfo.storePositionWithTermVector);
|
||||
ft.setStoreTermVectorOffsets(fieldInfo.storeOffsetWithTermVector);
|
||||
ft.setStoreTermVectors(fieldInfo.storeTermVector);
|
||||
ft.setIndexed(fieldInfo.isIndexed);
|
||||
ft.setOmitNorms(fieldInfo.omitNorms);
|
||||
|
|
|
@ -17,22 +17,23 @@
|
|||
|
||||
package org.apache.solr.search;
|
||||
|
||||
import java.util.Random;
|
||||
import java.util.Arrays;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.OpenBitSet;
|
||||
import org.apache.lucene.util.OpenBitSetIterator;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.FilterIndexReader;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.FilterIndexReader;
|
||||
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
|
||||
import org.apache.lucene.index.IndexReader.ReaderContext;
|
||||
import org.apache.lucene.index.MultiReader;
|
||||
import org.apache.lucene.search.Filter;
|
||||
import org.apache.lucene.search.DocIdSet;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.Filter;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.OpenBitSet;
|
||||
import org.apache.lucene.util.OpenBitSetIterator;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -358,6 +359,11 @@ public class TestDocSet extends LuceneTestCase {
|
|||
public IndexReader[] getSequentialSubReaders() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldInfos getFieldInfos() {
|
||||
return new FieldInfos();
|
||||
}
|
||||
};
|
||||
return r;
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue