LUCENE-5127: FixedGapTermsIndex should use monotonic compression

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1508147 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Muir 2013-07-29 17:34:52 +00:00
parent a16d24160f
commit 02a2f383b2
63 changed files with 717 additions and 999 deletions

View File

@ -27,6 +27,13 @@ Changes in backwards compatibility policy
no longer support multiple "dictionaries" as there is only one dictionary available.
(Dawid Weiss)
* LUCENE-5127: Reduce RAM usage of FixedGapTermsIndex. Remove
IndexWriterConfig.setTermIndexInterval, IndexWriterConfig.setReaderTermsIndexDivisor,
and termsIndexDivisor from StandardDirectoryReader. These options have been no-ops
with the default codec since Lucene 4.0. If you want to configure the interval for
this term index, pass it directly in your codec, where it can also be configured
per-field. (Robert Muir)
New Features
* LUCENE-4747: Move to Java 7 as minimum Java version.

View File

@ -313,11 +313,6 @@ public class BlockTermsReader extends FieldsProducer {
calls next() (which is not "typical"), then we'll do the real seek */
private boolean seekPending;
/* How many blocks we've read since last seek. Once this
is >= indexEnum.getDivisor() we set indexIsCurrent to false (since
the index can no long bracket seek-within-block). */
private int blocksSinceSeek;
private byte[] termSuffixes;
private ByteArrayDataInput termSuffixesReader = new ByteArrayDataInput();
@ -420,8 +415,7 @@ public class BlockTermsReader extends FieldsProducer {
assert result;
indexIsCurrent = true;
didIndexNext = false;
blocksSinceSeek = 0;
didIndexNext = false;
if (doOrd) {
state.ord = indexEnum.ord()-1;
@ -729,7 +723,6 @@ public class BlockTermsReader extends FieldsProducer {
indexIsCurrent = true;
didIndexNext = false;
blocksSinceSeek = 0;
seekPending = false;
state.ord = indexEnum.ord()-1;
@ -802,8 +795,7 @@ public class BlockTermsReader extends FieldsProducer {
postingsReader.readTermsBlock(in, fieldInfo, state);
blocksSinceSeek++;
indexIsCurrent = indexIsCurrent && (blocksSinceSeek < indexReader.getDivisor());
indexIsCurrent = false;
//System.out.println(" indexIsCurrent=" + indexIsCurrent);
return true;

View File

@ -27,7 +27,7 @@ import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.PagedBytes;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
import java.util.HashMap;
import java.util.Comparator;
@ -43,21 +43,15 @@ import org.apache.lucene.index.IndexFileNames;
*/
public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
// NOTE: long is overkill here, since this number is 128
// by default and only indexDivisor * 128 if you change
// the indexDivisor at search time. But, we use this in a
// NOTE: long is overkill here, but we use this in a
// number of places to multiply out the actual ord, and we
// will overflow int during those multiplies. So to avoid
// having to upgrade each multiple to long in multiple
// places (error prone), we use long here:
private long totalIndexInterval;
private int indexDivisor;
final private int indexInterval;
// Closed if indexLoaded is true:
private IndexInput in;
private volatile boolean indexLoaded;
private final long indexInterval;
private final int packedIntsVersion;
private final int blocksize;
private final Comparator<BytesRef> termComp;
@ -72,35 +66,24 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
// start of the field info data
private long dirOffset;
private final int version;
public FixedGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, Comparator<BytesRef> termComp, String segmentSuffix, IOContext context)
public FixedGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, Comparator<BytesRef> termComp, String segmentSuffix, IOContext context)
throws IOException {
this.termComp = termComp;
assert indexDivisor == -1 || indexDivisor > 0;
in = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION), context);
final IndexInput in = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION), context);
boolean success = false;
try {
version = readHeader(in);
indexInterval = in.readInt();
readHeader(in);
indexInterval = in.readVInt();
if (indexInterval < 1) {
throw new CorruptIndexException("invalid indexInterval: " + indexInterval + " (resource=" + in + ")");
}
this.indexDivisor = indexDivisor;
if (indexDivisor < 0) {
totalIndexInterval = indexInterval;
} else {
// In case terms index gets loaded, later, on demand
totalIndexInterval = indexInterval * indexDivisor;
}
assert totalIndexInterval > 0;
packedIntsVersion = in.readVInt();
blocksize = in.readVInt();
seekDir(in, dirOffset);
@ -112,7 +95,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
//System.out.println("FGR: init seg=" + segment + " div=" + indexDivisor + " nF=" + numFields);
for(int i=0;i<numFields;i++) {
final int field = in.readVInt();
final int numIndexTerms = in.readVInt();
final long numIndexTerms = in.readVInt(); // TODO: change this to a vLong if we fix writer to support > 2B index terms
if (numIndexTerms < 0) {
throw new CorruptIndexException("invalid numIndexTerms: " + numIndexTerms + " (resource=" + in + ")");
}
@ -124,47 +107,33 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
throw new CorruptIndexException("invalid packedIndexStart: " + packedIndexStart + " indexStart: " + indexStart + "numIndexTerms: " + numIndexTerms + " (resource=" + in + ")");
}
final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
FieldIndexData previous = fields.put(fieldInfo, new FieldIndexData(fieldInfo, numIndexTerms, indexStart, termsStart, packedIndexStart, packedOffsetsStart));
FieldIndexData previous = fields.put(fieldInfo, new FieldIndexData(in, indexStart, termsStart, packedIndexStart, packedOffsetsStart, numIndexTerms));
if (previous != null) {
throw new CorruptIndexException("duplicate field: " + fieldInfo.name + " (resource=" + in + ")");
}
}
success = true;
} finally {
if (!success) {
if (success) {
IOUtils.close(in);
} else {
IOUtils.closeWhileHandlingException(in);
}
if (indexDivisor > 0) {
in.close();
in = null;
if (success) {
indexLoaded = true;
}
termBytesReader = termBytes.freeze(true);
}
termBytesReader = termBytes.freeze(true);
}
}
@Override
public int getDivisor() {
return indexDivisor;
}
private int readHeader(IndexInput input) throws IOException {
int version = CodecUtil.checkHeader(input, FixedGapTermsIndexWriter.CODEC_NAME,
FixedGapTermsIndexWriter.VERSION_START, FixedGapTermsIndexWriter.VERSION_CURRENT);
if (version < FixedGapTermsIndexWriter.VERSION_APPEND_ONLY) {
dirOffset = input.readLong();
}
return version;
private void readHeader(IndexInput input) throws IOException {
CodecUtil.checkHeader(input, FixedGapTermsIndexWriter.CODEC_NAME,
FixedGapTermsIndexWriter.VERSION_CURRENT, FixedGapTermsIndexWriter.VERSION_CURRENT);
}
private class IndexEnum extends FieldIndexEnum {
private final FieldIndexData.CoreFieldIndex fieldIndex;
private final FieldIndexData fieldIndex;
private final BytesRef term = new BytesRef();
private long ord;
public IndexEnum(FieldIndexData.CoreFieldIndex fieldIndex) {
public IndexEnum(FieldIndexData fieldIndex) {
this.fieldIndex = fieldIndex;
}
@ -175,12 +144,11 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
@Override
public long seek(BytesRef target) {
int lo = 0; // binary search
int hi = fieldIndex.numIndexTerms - 1;
assert totalIndexInterval > 0 : "totalIndexInterval=" + totalIndexInterval;
long lo = 0; // binary search
long hi = fieldIndex.numIndexTerms - 1;
while (hi >= lo) {
int mid = (lo + hi) >>> 1;
long mid = (lo + hi) >>> 1;
final long offset = fieldIndex.termOffsets.get(mid);
final int length = (int) (fieldIndex.termOffsets.get(1+mid) - offset);
@ -193,7 +161,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
lo = mid + 1;
} else {
assert mid >= 0;
ord = mid*totalIndexInterval;
ord = mid*indexInterval;
return fieldIndex.termsStart + fieldIndex.termsDictOffsets.get(mid);
}
}
@ -207,17 +175,17 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
final int length = (int) (fieldIndex.termOffsets.get(1+hi) - offset);
termBytesReader.fillSlice(term, fieldIndex.termBytesStart + offset, length);
ord = hi*totalIndexInterval;
ord = hi*indexInterval;
return fieldIndex.termsStart + fieldIndex.termsDictOffsets.get(hi);
}
@Override
public long next() {
final int idx = 1 + (int) (ord / totalIndexInterval);
final long idx = 1 + (ord / indexInterval);
if (idx >= fieldIndex.numIndexTerms) {
return -1;
}
ord += totalIndexInterval;
ord += indexInterval;
final long offset = fieldIndex.termOffsets.get(idx);
final int length = (int) (fieldIndex.termOffsets.get(1+idx) - offset);
@ -232,13 +200,13 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
@Override
public long seek(long ord) {
int idx = (int) (ord / totalIndexInterval);
long idx = ord / indexInterval;
// caller must ensure ord is in bounds
assert idx < fieldIndex.numIndexTerms;
final long offset = fieldIndex.termOffsets.get(idx);
final int length = (int) (fieldIndex.termOffsets.get(1+idx) - offset);
termBytesReader.fillSlice(term, fieldIndex.termBytesStart + offset, length);
this.ord = idx * totalIndexInterval;
this.ord = idx * indexInterval;
return fieldIndex.termsStart + fieldIndex.termsDictOffsets.get(idx);
}
}
@ -249,176 +217,58 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
}
private final class FieldIndexData {
volatile CoreFieldIndex coreIndex;
private final long indexStart;
private final long termsStart;
private final long packedIndexStart;
private final long packedOffsetsStart;
private final int numIndexTerms;
public FieldIndexData(FieldInfo fieldInfo, int numIndexTerms, long indexStart, long termsStart, long packedIndexStart,
long packedOffsetsStart) throws IOException {
// where this field's terms begin in the packed byte[]
// data
final long termBytesStart;
// offset into index termBytes
final MonotonicBlockPackedReader termOffsets;
// index pointers into main terms dict
final MonotonicBlockPackedReader termsDictOffsets;
final long numIndexTerms;
final long termsStart;
public FieldIndexData(IndexInput in, long indexStart, long termsStart, long packedIndexStart, long packedOffsetsStart, long numIndexTerms) throws IOException {
this.termsStart = termsStart;
this.indexStart = indexStart;
this.packedIndexStart = packedIndexStart;
this.packedOffsetsStart = packedOffsetsStart;
termBytesStart = termBytes.getPointer();
IndexInput clone = in.clone();
clone.seek(indexStart);
this.numIndexTerms = numIndexTerms;
if (indexDivisor > 0) {
loadTermsIndex();
}
}
private void loadTermsIndex() throws IOException {
if (coreIndex == null) {
coreIndex = new CoreFieldIndex(indexStart, termsStart, packedIndexStart, packedOffsetsStart, numIndexTerms);
}
}
private final class CoreFieldIndex {
// where this field's terms begin in the packed byte[]
// data
final long termBytesStart;
// offset into index termBytes
final PackedInts.Reader termOffsets;
// index pointers into main terms dict
final PackedInts.Reader termsDictOffsets;
final int numIndexTerms;
final long termsStart;
public CoreFieldIndex(long indexStart, long termsStart, long packedIndexStart, long packedOffsetsStart, int numIndexTerms) throws IOException {
this.termsStart = termsStart;
termBytesStart = termBytes.getPointer();
IndexInput clone = in.clone();
clone.seek(indexStart);
// -1 is passed to mean "don't load term index", but
// if we are then later loaded it's overwritten with
// a real value
assert indexDivisor > 0;
this.numIndexTerms = 1+(numIndexTerms-1) / indexDivisor;
assert this.numIndexTerms > 0: "numIndexTerms=" + numIndexTerms + " indexDivisor=" + indexDivisor;
if (indexDivisor == 1) {
// Default (load all index terms) is fast -- slurp in the images from disk:
try {
final long numTermBytes = packedIndexStart - indexStart;
termBytes.copy(clone, numTermBytes);
// records offsets into main terms dict file
termsDictOffsets = PackedInts.getReader(clone);
assert termsDictOffsets.size() == numIndexTerms;
// records offsets into byte[] term data
termOffsets = PackedInts.getReader(clone);
assert termOffsets.size() == 1+numIndexTerms;
} finally {
clone.close();
}
} else {
// Get packed iterators
final IndexInput clone1 = in.clone();
final IndexInput clone2 = in.clone();
try {
// Subsample the index terms
clone1.seek(packedIndexStart);
final PackedInts.ReaderIterator termsDictOffsetsIter = PackedInts.getReaderIterator(clone1, PackedInts.DEFAULT_BUFFER_SIZE);
clone2.seek(packedOffsetsStart);
final PackedInts.ReaderIterator termOffsetsIter = PackedInts.getReaderIterator(clone2, PackedInts.DEFAULT_BUFFER_SIZE);
// TODO: often we can get by w/ fewer bits per
// value, below.. .but this'd be more complex:
// we'd have to try @ fewer bits and then grow
// if we overflowed it.
PackedInts.Mutable termsDictOffsetsM = PackedInts.getMutable(this.numIndexTerms, termsDictOffsetsIter.getBitsPerValue(), PackedInts.DEFAULT);
PackedInts.Mutable termOffsetsM = PackedInts.getMutable(this.numIndexTerms+1, termOffsetsIter.getBitsPerValue(), PackedInts.DEFAULT);
termsDictOffsets = termsDictOffsetsM;
termOffsets = termOffsetsM;
int upto = 0;
long termOffsetUpto = 0;
while(upto < this.numIndexTerms) {
// main file offset copies straight over
termsDictOffsetsM.set(upto, termsDictOffsetsIter.next());
termOffsetsM.set(upto, termOffsetUpto);
long termOffset = termOffsetsIter.next();
long nextTermOffset = termOffsetsIter.next();
final int numTermBytes = (int) (nextTermOffset - termOffset);
clone.seek(indexStart + termOffset);
assert indexStart + termOffset < clone.length() : "indexStart=" + indexStart + " termOffset=" + termOffset + " len=" + clone.length();
assert indexStart + termOffset + numTermBytes < clone.length();
termBytes.copy(clone, numTermBytes);
termOffsetUpto += numTermBytes;
upto++;
if (upto == this.numIndexTerms) {
break;
}
// skip terms:
termsDictOffsetsIter.next();
for(int i=0;i<indexDivisor-2;i++) {
termOffsetsIter.next();
termsDictOffsetsIter.next();
}
}
termOffsetsM.set(upto, termOffsetUpto);
} finally {
clone1.close();
clone2.close();
clone.close();
}
}
assert this.numIndexTerms > 0: "numIndexTerms=" + numIndexTerms;
// slurp in the images from disk:
try {
final long numTermBytes = packedIndexStart - indexStart;
termBytes.copy(clone, numTermBytes);
// records offsets into main terms dict file
termsDictOffsets = new MonotonicBlockPackedReader(clone, packedIntsVersion, blocksize, numIndexTerms, false);
// records offsets into byte[] term data
termOffsets = new MonotonicBlockPackedReader(clone, packedIntsVersion, blocksize, 1+numIndexTerms, false);
} finally {
clone.close();
}
}
}
@Override
public FieldIndexEnum getFieldEnum(FieldInfo fieldInfo) {
final FieldIndexData fieldData = fields.get(fieldInfo);
if (fieldData.coreIndex == null) {
return null;
} else {
return new IndexEnum(fieldData.coreIndex);
}
return new IndexEnum(fields.get(fieldInfo));
}
@Override
public void close() throws IOException {
if (in != null && !indexLoaded) {
in.close();
}
}
public void close() throws IOException {}
private void seekDir(IndexInput input, long dirOffset) throws IOException {
if (version >= FixedGapTermsIndexWriter.VERSION_APPEND_ONLY) {
input.seek(input.length() - 8);
dirOffset = input.readLong();
}
input.seek(input.length() - 8);
dirOffset = input.readLong();
input.seek(dirOffset);
}
}

View File

@ -18,15 +18,16 @@ package org.apache.lucene.codecs.blockterms;
*/
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.TermStats;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.packed.MonotonicAppendingLongBuffer;
import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
import org.apache.lucene.util.packed.PackedInts;
import java.util.List;
@ -50,23 +51,32 @@ public class FixedGapTermsIndexWriter extends TermsIndexWriterBase {
final static String CODEC_NAME = "SIMPLE_STANDARD_TERMS_INDEX";
final static int VERSION_START = 0;
final static int VERSION_APPEND_ONLY = 1;
final static int VERSION_CURRENT = VERSION_APPEND_ONLY;
final static int VERSION_MONOTONIC_ADDRESSING = 2;
final static int VERSION_CURRENT = VERSION_MONOTONIC_ADDRESSING;
final static int BLOCKSIZE = 4096;
final private int termIndexInterval;
public static final int DEFAULT_TERM_INDEX_INTERVAL = 32;
private final List<SimpleFieldWriter> fields = new ArrayList<SimpleFieldWriter>();
@SuppressWarnings("unused") private final FieldInfos fieldInfos; // unread
public FixedGapTermsIndexWriter(SegmentWriteState state) throws IOException {
this(state, DEFAULT_TERM_INDEX_INTERVAL);
}
public FixedGapTermsIndexWriter(SegmentWriteState state, int termIndexInterval) throws IOException {
if (termIndexInterval <= 0) {
throw new IllegalArgumentException("invalid termIndexInterval: " + termIndexInterval);
}
this.termIndexInterval = termIndexInterval;
final String indexFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_INDEX_EXTENSION);
termIndexInterval = state.termIndexInterval;
out = state.directory.createOutput(indexFileName, state.context);
boolean success = false;
try {
fieldInfos = state.fieldInfos;
writeHeader(out);
out.writeInt(termIndexInterval);
out.writeVInt(termIndexInterval);
out.writeVInt(PackedInts.VERSION_CURRENT);
out.writeVInt(BLOCKSIZE);
success = true;
} finally {
if (!success) {
@ -114,22 +124,25 @@ public class FixedGapTermsIndexWriter extends TermsIndexWriterBase {
long packedOffsetsStart;
private long numTerms;
// TODO: we could conceivably make a PackedInts wrapper
// that auto-grows... then we wouldn't force 6 bytes RAM
// per index term:
private short[] termLengths;
private int[] termsPointerDeltas;
private long lastTermsPointer;
private long totTermLength;
private RAMOutputStream offsetsBuffer = new RAMOutputStream();
private MonotonicBlockPackedWriter termOffsets = new MonotonicBlockPackedWriter(offsetsBuffer, BLOCKSIZE);
private long currentOffset;
private RAMOutputStream addressBuffer = new RAMOutputStream();
private MonotonicBlockPackedWriter termAddresses = new MonotonicBlockPackedWriter(addressBuffer, BLOCKSIZE);
private final BytesRef lastTerm = new BytesRef();
SimpleFieldWriter(FieldInfo fieldInfo, long termsFilePointer) {
this.fieldInfo = fieldInfo;
indexStart = out.getFilePointer();
termsStart = lastTermsPointer = termsFilePointer;
termLengths = new short[0];
termsPointerDeltas = new int[0];
termsStart = termsFilePointer;
// we write terms+1 offsets, term n's length is n+1 - n
try {
termOffsets.add(0L);
} catch (IOException bogus) {
throw new RuntimeException(bogus);
}
}
@Override
@ -157,21 +170,13 @@ public class FixedGapTermsIndexWriter extends TermsIndexWriterBase {
// against prior term
out.writeBytes(text.bytes, text.offset, indexedTermLength);
if (termLengths.length == numIndexTerms) {
termLengths = ArrayUtil.grow(termLengths);
}
if (termsPointerDeltas.length == numIndexTerms) {
termsPointerDeltas = ArrayUtil.grow(termsPointerDeltas);
}
// save delta terms pointer
termsPointerDeltas[numIndexTerms] = (int) (termsFilePointer - lastTermsPointer);
lastTermsPointer = termsFilePointer;
termAddresses.add(termsFilePointer - termsStart);
// save term length (in bytes)
assert indexedTermLength <= Short.MAX_VALUE;
termLengths[numIndexTerms] = (short) indexedTermLength;
totTermLength += indexedTermLength;
currentOffset += indexedTermLength;
termOffsets.add(currentOffset);
lastTerm.copyBytes(text);
numIndexTerms++;
@ -183,32 +188,20 @@ public class FixedGapTermsIndexWriter extends TermsIndexWriterBase {
// write primary terms dict offsets
packedIndexStart = out.getFilePointer();
PackedInts.Writer w = PackedInts.getWriter(out, numIndexTerms, PackedInts.bitsRequired(termsFilePointer), PackedInts.DEFAULT);
// relative to our indexStart
long upto = 0;
for(int i=0;i<numIndexTerms;i++) {
upto += termsPointerDeltas[i];
w.add(upto);
}
w.finish();
termAddresses.finish();
addressBuffer.writeTo(out);
packedOffsetsStart = out.getFilePointer();
// write offsets into the byte[] terms
w = PackedInts.getWriter(out, 1+numIndexTerms, PackedInts.bitsRequired(totTermLength), PackedInts.DEFAULT);
upto = 0;
for(int i=0;i<numIndexTerms;i++) {
w.add(upto);
upto += termLengths[i];
}
w.add(upto);
w.finish();
termOffsets.finish();
offsetsBuffer.writeTo(out);
// our referrer holds onto us, while other fields are
// being written, so don't tie up this RAM:
termLengths = null;
termsPointerDeltas = null;
termOffsets = termAddresses = null;
addressBuffer = offsetsBuffer = null;
}
}

View File

@ -47,8 +47,6 @@ public abstract class TermsIndexReaderBase implements Closeable {
public abstract boolean supportsOrd();
public abstract int getDivisor();
/**
* Similar to TermsEnum, except, the only "metadata" it
* reports for a given indexed term is the long fileOffset

View File

@ -32,8 +32,7 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.fst.Builder;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.fst.BytesRefFSTEnum;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.PositiveIntOutputs;
@ -45,11 +44,6 @@ import org.apache.lucene.util.fst.Util; // for toDot
public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
private final PositiveIntOutputs fstOutputs = PositiveIntOutputs.getSingleton();
private int indexDivisor;
// Closed if indexLoaded is true:
private IndexInput in;
private volatile boolean indexLoaded;
final HashMap<FieldInfo,FieldIndexData> fields = new HashMap<FieldInfo,FieldIndexData>();
@ -59,17 +53,15 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
private final int version;
final String segment;
public VariableGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, String segmentSuffix, IOContext context)
public VariableGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, String segmentSuffix, IOContext context)
throws IOException {
in = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION), new IOContext(context, true));
final IndexInput in = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION), new IOContext(context, true));
this.segment = segment;
boolean success = false;
assert indexDivisor == -1 || indexDivisor > 0;
try {
version = readHeader(in);
this.indexDivisor = indexDivisor;
seekDir(in, dirOffset);
@ -83,27 +75,20 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
final int field = in.readVInt();
final long indexStart = in.readVLong();
final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
FieldIndexData previous = fields.put(fieldInfo, new FieldIndexData(fieldInfo, indexStart));
FieldIndexData previous = fields.put(fieldInfo, new FieldIndexData(in, fieldInfo, indexStart));
if (previous != null) {
throw new CorruptIndexException("duplicate field: " + fieldInfo.name + " (resource=" + in + ")");
}
}
success = true;
} finally {
if (indexDivisor > 0) {
in.close();
in = null;
if (success) {
indexLoaded = true;
}
if (success) {
IOUtils.close(in);
} else {
IOUtils.closeWhileHandlingException(in);
}
}
}
@Override
public int getDivisor() {
return indexDivisor;
}
private int readHeader(IndexInput input) throws IOException {
int version = CodecUtil.checkHeader(input, VariableGapTermsIndexWriter.CODEC_NAME,
@ -168,52 +153,21 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
}
private final class FieldIndexData {
private final FST<Long> fst;
private final long indexStart;
// Set only if terms index is loaded:
private volatile FST<Long> fst;
public FieldIndexData(IndexInput in, FieldInfo fieldInfo, long indexStart) throws IOException {
IndexInput clone = in.clone();
clone.seek(indexStart);
fst = new FST<Long>(clone, fstOutputs);
clone.close();
public FieldIndexData(FieldInfo fieldInfo, long indexStart) throws IOException {
this.indexStart = indexStart;
if (indexDivisor > 0) {
loadTermsIndex();
}
}
private void loadTermsIndex() throws IOException {
if (fst == null) {
IndexInput clone = in.clone();
clone.seek(indexStart);
fst = new FST<Long>(clone, fstOutputs);
clone.close();
/*
final String dotFileName = segment + "_" + fieldInfo.name + ".dot";
Writer w = new OutputStreamWriter(new FileOutputStream(dotFileName));
Util.toDot(fst, w, false, false);
System.out.println("FST INDEX: SAVED to " + dotFileName);
w.close();
*/
if (indexDivisor > 1) {
// subsample
final IntsRef scratchIntsRef = new IntsRef();
final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, outputs);
final BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<Long>(fst);
BytesRefFSTEnum.InputOutput<Long> result;
int count = indexDivisor;
while((result = fstEnum.next()) != null) {
if (count == indexDivisor) {
builder.add(Util.toIntsRef(result.input, scratchIntsRef), result.output);
count = 0;
}
count++;
}
fst = builder.finish();
}
}
/*
final String dotFileName = segment + "_" + fieldInfo.name + ".dot";
Writer w = new OutputStreamWriter(new FileOutputStream(dotFileName));
Util.toDot(fst, w, false, false);
System.out.println("FST INDEX: SAVED to " + dotFileName);
w.close();
*/
}
}
@ -228,11 +182,7 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
}
@Override
public void close() throws IOException {
if (in != null && !indexLoaded) {
in.close();
}
}
public void close() throws IOException {}
private void seekDir(IndexInput input, long dirOffset) throws IOException {
if (version >= VariableGapTermsIndexWriter.VERSION_APPEND_ONLY) {

View File

@ -103,8 +103,7 @@ public abstract class PulsingPostingsFormat extends PostingsFormat {
state.directory, state.fieldInfos, state.segmentInfo,
pulsingReader,
state.context,
state.segmentSuffix,
state.termsIndexDivisor);
state.segmentSuffix);
success = true;
return ret;
} finally {

View File

@ -25,10 +25,8 @@ import org.apache.lucene.util._TestUtil;
/**
* Basic tests of a PF using FixedGap terms dictionary
*/
// TODO: we should add an instantiation for VarGap too to TestFramework, and a test in this package
// TODO: ensure both of these are also in rotation in RandomCodec
public class TestFixedGapPostingsFormat extends BasePostingsFormatTestCase {
private final Codec codec = _TestUtil.alwaysPostingsFormat(new Lucene41WithOrds());
private final Codec codec = _TestUtil.alwaysPostingsFormat(new Lucene41WithOrds(_TestUtil.nextInt(random(), 1, 1000)));
@Override
protected Codec getCodec() {

View File

@ -0,0 +1,35 @@
package org.apache.lucene.codecs.blockterms;
/*
* 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.codecs.Codec;
import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapFixedInterval;
import org.apache.lucene.index.BasePostingsFormatTestCase;
import org.apache.lucene.util._TestUtil;
/**
* Basic tests of a PF using VariableGap terms dictionary (fixed interval)
*/
public class TestVarGapDocFreqIntervalPostingsFormat extends BasePostingsFormatTestCase {
private final Codec codec = _TestUtil.alwaysPostingsFormat(new Lucene41VarGapFixedInterval(_TestUtil.nextInt(random(), 1, 1000)));
@Override
protected Codec getCodec() {
return codec;
}
}

View File

@ -0,0 +1,35 @@
package org.apache.lucene.codecs.blockterms;
/*
* 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.codecs.Codec;
import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapDocFreqInterval;
import org.apache.lucene.index.BasePostingsFormatTestCase;
import org.apache.lucene.util._TestUtil;
/**
* Basic tests of a PF using VariableGap terms dictionary (fixed interval, docFreq threshold)
*/
public class TestVarGapFixedIntervalPostingsFormat extends BasePostingsFormatTestCase {
private final Codec codec = _TestUtil.alwaysPostingsFormat(new Lucene41VarGapDocFreqInterval(_TestUtil.nextInt(random(), 1, 100), _TestUtil.nextInt(random(), 1, 1000)));
@Override
protected Codec getCodec() {
return codec;
}
}

View File

@ -67,9 +67,9 @@ import org.apache.lucene.util.fst.Util;
* does not support a pluggable terms index
* implementation).
*
* <p><b>NOTE</b>: this terms dictionary does not support
* index divisor when opening an IndexReader. Instead, you
* can change the min/maxItemsPerBlock during indexing.</p>
* <p><b>NOTE</b>: this terms dictionary supports
* min/maxItemsPerBlock during indexing to control how
* much memory the terms index uses.</p>
*
* <p>The data structure used by this implementation is very
* similar to a burst trie
@ -112,7 +112,7 @@ public class BlockTreeTermsReader extends FieldsProducer {
/** Sole constructor. */
public BlockTreeTermsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo info,
PostingsReaderBase postingsReader, IOContext ioContext,
String segmentSuffix, int indexDivisor)
String segmentSuffix)
throws IOException {
this.postingsReader = postingsReader;
@ -126,13 +126,11 @@ public class BlockTreeTermsReader extends FieldsProducer {
try {
version = readHeader(in);
if (indexDivisor != -1) {
indexIn = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION),
indexIn = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION),
ioContext);
int indexVersion = readIndexHeader(indexIn);
if (indexVersion != version) {
throw new CorruptIndexException("mixmatched version files: " + in + "=" + version + "," + indexIn + "=" + indexVersion);
}
int indexVersion = readIndexHeader(indexIn);
if (indexVersion != version) {
throw new CorruptIndexException("mixmatched version files: " + in + "=" + version + "," + indexIn + "=" + indexVersion);
}
// Have PostingsReader init itself
@ -140,9 +138,7 @@ public class BlockTreeTermsReader extends FieldsProducer {
// Read per-field details
seekDir(in, dirOffset);
if (indexDivisor != -1) {
seekDir(indexIn, indexDirOffset);
}
seekDir(indexIn, indexDirOffset);
final int numFields = in.readVInt();
if (numFields < 0) {
@ -171,15 +167,13 @@ public class BlockTreeTermsReader extends FieldsProducer {
if (sumTotalTermFreq != -1 && sumTotalTermFreq < sumDocFreq) { // #positions must be >= #postings
throw new CorruptIndexException("invalid sumTotalTermFreq: " + sumTotalTermFreq + " sumDocFreq: " + sumDocFreq + " (resource=" + in + ")");
}
final long indexStartFP = indexDivisor != -1 ? indexIn.readVLong() : 0;
final long indexStartFP = indexIn.readVLong();
FieldReader previous = fields.put(fieldInfo.name, new FieldReader(fieldInfo, numTerms, rootCode, sumTotalTermFreq, sumDocFreq, docCount, indexStartFP, indexIn));
if (previous != null) {
throw new CorruptIndexException("duplicate field: " + fieldInfo.name + " (resource=" + in + ")");
}
}
if (indexDivisor != -1) {
indexIn.close();
}
indexIn.close();
success = true;
} finally {

View File

@ -258,8 +258,7 @@ public class Lucene40PostingsFormat extends PostingsFormat {
state.segmentInfo,
postings,
state.context,
state.segmentSuffix,
state.termsIndexDivisor);
state.segmentSuffix);
success = true;
return ret;
} finally {

View File

@ -439,8 +439,7 @@ public final class Lucene41PostingsFormat extends PostingsFormat {
state.segmentInfo,
postingsReader,
state.context,
state.segmentSuffix,
state.termsIndexDivisor);
state.segmentSuffix);
success = true;
return ret;
} finally {

View File

@ -533,7 +533,7 @@ public class CheckIndex {
}
if (infoStream != null)
infoStream.print(" test: open reader.........");
reader = new SegmentReader(info, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, IOContext.DEFAULT);
reader = new SegmentReader(info, IOContext.DEFAULT);
segInfoStat.openReaderPassed = true;

View File

@ -52,9 +52,6 @@ import org.apache.lucene.store.NoSuchDirectoryException;
*/
public abstract class DirectoryReader extends BaseCompositeReader<AtomicReader> {
/** Default termInfosIndexDivisor. */
public static final int DEFAULT_TERMS_INDEX_DIVISOR = 1;
/** The index directory. */
protected final Directory directory;
@ -64,29 +61,7 @@ public abstract class DirectoryReader extends BaseCompositeReader<AtomicReader>
* @throws IOException if there is a low-level IO error
*/
public static DirectoryReader open(final Directory directory) throws IOException {
return StandardDirectoryReader.open(directory, null, DEFAULT_TERMS_INDEX_DIVISOR);
}
/** Expert: Returns a IndexReader reading the index in the given
* Directory with the given termInfosIndexDivisor.
* @param directory the index directory
* @param termInfosIndexDivisor Subsamples which indexed
* terms are loaded into RAM. This has the same effect as {@link
* IndexWriterConfig#setTermIndexInterval} except that setting
* must be done at indexing time while this setting can be
* set per reader. When set to N, then one in every
* N*termIndexInterval terms in the index is loaded into
* memory. By setting this to a value > 1 you can reduce
* memory usage, at the expense of higher latency when
* loading a TermInfo. The default value is 1. Set this
* to -1 to skip loading the terms index entirely.
* <b>NOTE:</b> divisor settings &gt; 1 do not apply to all PostingsFormat
* implementations, including the default one in this release. It only makes
* sense for terms indexes that can efficiently re-sample terms at load time.
* @throws IOException if there is a low-level IO error
*/
public static DirectoryReader open(final Directory directory, int termInfosIndexDivisor) throws IOException {
return StandardDirectoryReader.open(directory, null, termInfosIndexDivisor);
return StandardDirectoryReader.open(directory, null);
}
/**
@ -118,29 +93,7 @@ public abstract class DirectoryReader extends BaseCompositeReader<AtomicReader>
* @throws IOException if there is a low-level IO error
*/
public static DirectoryReader open(final IndexCommit commit) throws IOException {
return StandardDirectoryReader.open(commit.getDirectory(), commit, DEFAULT_TERMS_INDEX_DIVISOR);
}
/** Expert: returns an IndexReader reading the index in the given
* {@link IndexCommit} and termInfosIndexDivisor.
* @param commit the commit point to open
* @param termInfosIndexDivisor Subsamples which indexed
* terms are loaded into RAM. This has the same effect as {@link
* IndexWriterConfig#setTermIndexInterval} except that setting
* must be done at indexing time while this setting can be
* set per reader. When set to N, then one in every
* N*termIndexInterval terms in the index is loaded into
* memory. By setting this to a value > 1 you can reduce
* memory usage, at the expense of higher latency when
* loading a TermInfo. The default value is 1. Set this
* to -1 to skip loading the terms index entirely.
* <b>NOTE:</b> divisor settings &gt; 1 do not apply to all PostingsFormat
* implementations, including the default one in this release. It only makes
* sense for terms indexes that can efficiently re-sample terms at load time.
* @throws IOException if there is a low-level IO error
*/
public static DirectoryReader open(final IndexCommit commit, int termInfosIndexDivisor) throws IOException {
return StandardDirectoryReader.open(commit.getDirectory(), commit, termInfosIndexDivisor);
return StandardDirectoryReader.open(commit.getDirectory(), commit);
}
/**

View File

@ -468,7 +468,6 @@ class DocumentsWriterPerThread {
assert deleteSlice == null : "all deletes must be applied in prepareFlush";
segmentInfo.setDocCount(numDocsInRAM);
flushState = new SegmentWriteState(infoStream, directory, segmentInfo, fieldInfos.finish(),
writer.getConfig().getTermIndexInterval(),
pendingDeletes, new IOContext(new FlushInfo(numDocsInRAM, bytesUsed())));
final double startMBUsed = parent.flushControl.netBytes() / 1024. / 1024.;

View File

@ -2437,10 +2437,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
* to call this method multiple times, each time with a small set of readers.
* In principle, if you use a merge policy with a {@code mergeFactor} or
* {@code maxMergeAtOnce} parameter, you should pass that many readers in one
* call. Also, if the given readers are {@link DirectoryReader}s, they can be
* opened with {@code termIndexInterval=-1} to save RAM, since during merge
* the in-memory structure is not used. See
* {@link DirectoryReader#open(Directory, int)}.
* call.
*
* <p>
* <b>NOTE</b>: if you call {@link #close(boolean)} with <tt>false</tt>, which
@ -2488,7 +2485,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
SegmentInfo info = new SegmentInfo(directory, Constants.LUCENE_MAIN_VERSION, mergedName, -1,
false, codec, null, null);
SegmentMerger merger = new SegmentMerger(mergeReaders, info, infoStream, trackingDir, config.getTermIndexInterval(),
SegmentMerger merger = new SegmentMerger(mergeReaders, info, infoStream, trackingDir,
MergeState.CheckAbort.NONE, globalFieldNumberMap, context);
MergeState mergeState;
@ -3670,7 +3667,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// Hold onto the "live" reader; we will use this to
// commit merged deletes
final ReadersAndLiveDocs rld = readerPool.get(info, true);
SegmentReader reader = rld.getMergeReader(context);
SegmentReader reader = rld.getReader(context);
assert reader != null;
// Carefully pull the most recent live docs:
@ -3727,7 +3724,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
// we pass merge.getMergeReaders() instead of merge.readers to allow the
// OneMerge to return a view over the actual segments to merge
final SegmentMerger merger = new SegmentMerger(merge.getMergeReaders(),
merge.info.info, infoStream, dirWrapper, config.getTermIndexInterval(),
merge.info.info, infoStream, dirWrapper,
checkAbort, globalFieldNumberMap, context);
merge.checkAborted(directory);

View File

@ -72,9 +72,6 @@ public final class IndexWriterConfig extends LiveIndexWriterConfig implements Cl
CREATE_OR_APPEND
}
/** Default value is 32. Change using {@link #setTermIndexInterval(int)}. */
public static final int DEFAULT_TERM_INDEX_INTERVAL = 32; // TODO: this should be private to the codec, not settable here
/** Denotes a flush trigger is disabled. */
public final static int DISABLE_AUTO_FLUSH = -1;
@ -100,9 +97,6 @@ public final class IndexWriterConfig extends LiveIndexWriterConfig implements Cl
/** Default setting for {@link #setReaderPooling}. */
public final static boolean DEFAULT_READER_POOLING = false;
/** Default value is 1. Change using {@link #setReaderTermsIndexDivisor(int)}. */
public static final int DEFAULT_READER_TERMS_INDEX_DIVISOR = DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR;
/** Default value is 1945. Change using {@link #setRAMPerThreadHardLimitMB(int)} */
public static final int DEFAULT_RAM_PER_THREAD_HARD_LIMIT_MB = 1945;
@ -503,16 +497,6 @@ public final class IndexWriterConfig extends LiveIndexWriterConfig implements Cl
return super.getRAMBufferSizeMB();
}
@Override
public int getReaderTermsIndexDivisor() {
return super.getReaderTermsIndexDivisor();
}
@Override
public int getTermIndexInterval() {
return super.getTermIndexInterval();
}
/** If non-null, information about merges, deletes and a
* message when maxFieldLength is reached will be printed
* to this.
@ -554,16 +538,6 @@ public final class IndexWriterConfig extends LiveIndexWriterConfig implements Cl
return (IndexWriterConfig) super.setRAMBufferSizeMB(ramBufferSizeMB);
}
@Override
public IndexWriterConfig setReaderTermsIndexDivisor(int divisor) {
return (IndexWriterConfig) super.setReaderTermsIndexDivisor(divisor);
}
@Override
public IndexWriterConfig setTermIndexInterval(int interval) {
return (IndexWriterConfig) super.setTermIndexInterval(interval);
}
@Override
public IndexWriterConfig setUseCompoundFile(boolean useCompoundFile) {
return (IndexWriterConfig) super.setUseCompoundFile(useCompoundFile);

View File

@ -19,7 +19,6 @@ package org.apache.lucene.index;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; // javadocs
import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain;
import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
@ -41,9 +40,7 @@ public class LiveIndexWriterConfig {
private volatile int maxBufferedDocs;
private volatile double ramBufferSizeMB;
private volatile int maxBufferedDeleteTerms;
private volatile int readerTermsIndexDivisor;
private volatile IndexReaderWarmer mergedSegmentWarmer;
private volatile int termIndexInterval; // TODO: this should be private to the codec, not settable here
// modified by IndexWriterConfig
/** {@link IndexDeletionPolicy} controlling when commit
@ -108,9 +105,7 @@ public class LiveIndexWriterConfig {
ramBufferSizeMB = IndexWriterConfig.DEFAULT_RAM_BUFFER_SIZE_MB;
maxBufferedDocs = IndexWriterConfig.DEFAULT_MAX_BUFFERED_DOCS;
maxBufferedDeleteTerms = IndexWriterConfig.DEFAULT_MAX_BUFFERED_DELETE_TERMS;
readerTermsIndexDivisor = IndexWriterConfig.DEFAULT_READER_TERMS_INDEX_DIVISOR;
mergedSegmentWarmer = null;
termIndexInterval = IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL; // TODO: this should be private to the codec, not settable here
delPolicy = new KeepOnlyLastCommitDeletionPolicy();
commit = null;
useCompoundFile = IndexWriterConfig.DEFAULT_USE_COMPOUND_FILE_SYSTEM;
@ -140,8 +135,6 @@ public class LiveIndexWriterConfig {
maxBufferedDocs = config.getMaxBufferedDocs();
mergedSegmentWarmer = config.getMergedSegmentWarmer();
ramBufferSizeMB = config.getRAMBufferSizeMB();
readerTermsIndexDivisor = config.getReaderTermsIndexDivisor();
termIndexInterval = config.getTermIndexInterval();
matchVersion = config.matchVersion;
analyzer = config.getAnalyzer();
delPolicy = config.getIndexDeletionPolicy();
@ -165,69 +158,6 @@ public class LiveIndexWriterConfig {
public Analyzer getAnalyzer() {
return analyzer;
}
/**
* Expert: set the interval between indexed terms. Large values cause less
* memory to be used by IndexReader, but slow random-access to terms. Small
* values cause more memory to be used by an IndexReader, and speed
* random-access to terms.
* <p>
* This parameter determines the amount of computation required per query
* term, regardless of the number of documents that contain that term. In
* particular, it is the maximum number of other terms that must be scanned
* before a term is located and its frequency and position information may be
* processed. In a large index with user-entered query terms, query processing
* time is likely to be dominated not by term lookup but rather by the
* processing of frequency and positional data. In a small index or when many
* uncommon query terms are generated (e.g., by wildcard queries) term lookup
* may become a dominant cost.
* <p>
* In particular, <code>numUniqueTerms/interval</code> terms are read into
* memory by an IndexReader, and, on average, <code>interval/2</code> terms
* must be scanned for each random term access.
*
* <p>
* Takes effect immediately, but only applies to newly flushed/merged
* segments.
*
* <p>
* <b>NOTE:</b> This parameter does not apply to all PostingsFormat implementations,
* including the default one in this release. It only makes sense for term indexes
* that are implemented as a fixed gap between terms. For example,
* {@link Lucene41PostingsFormat} implements the term index instead based upon how
* terms share prefixes. To configure its parameters (the minimum and maximum size
* for a block), you would instead use {@link Lucene41PostingsFormat#Lucene41PostingsFormat(int, int)}.
* which can also be configured on a per-field basis:
* <pre class="prettyprint">
* //customize Lucene41PostingsFormat, passing minBlockSize=50, maxBlockSize=100
* final PostingsFormat tweakedPostings = new Lucene41PostingsFormat(50, 100);
* iwc.setCodec(new Lucene42Codec() {
* &#64;Override
* public PostingsFormat getPostingsFormatForField(String field) {
* if (field.equals("fieldWithTonsOfTerms"))
* return tweakedPostings;
* else
* return super.getPostingsFormatForField(field);
* }
* });
* </pre>
* Note that other implementations may have their own parameters, or no parameters at all.
*
* @see IndexWriterConfig#DEFAULT_TERM_INDEX_INTERVAL
*/
public LiveIndexWriterConfig setTermIndexInterval(int interval) { // TODO: this should be private to the codec, not settable here
this.termIndexInterval = interval;
return this;
}
/**
* Returns the interval between indexed terms.
*
* @see #setTermIndexInterval(int)
*/
public int getTermIndexInterval() { // TODO: this should be private to the codec, not settable here
return termIndexInterval;
}
/**
* Determines the minimal number of delete terms required before the buffered
@ -390,37 +320,6 @@ public class LiveIndexWriterConfig {
public IndexReaderWarmer getMergedSegmentWarmer() {
return mergedSegmentWarmer;
}
/**
* Sets the termsIndexDivisor passed to any readers that IndexWriter opens,
* for example when applying deletes or creating a near-real-time reader in
* {@link DirectoryReader#open(IndexWriter, boolean)}. If you pass -1, the
* terms index won't be loaded by the readers. This is only useful in advanced
* situations when you will only .next() through all terms; attempts to seek
* will hit an exception.
*
* <p>
* Takes effect immediately, but only applies to readers opened after this
* call
* <p>
* <b>NOTE:</b> divisor settings &gt; 1 do not apply to all PostingsFormat
* implementations, including the default one in this release. It only makes
* sense for terms indexes that can efficiently re-sample terms at load time.
*/
public LiveIndexWriterConfig setReaderTermsIndexDivisor(int divisor) {
if (divisor <= 0 && divisor != -1) {
throw new IllegalArgumentException("divisor must be >= 1, or -1 (got " + divisor + ")");
}
readerTermsIndexDivisor = divisor;
return this;
}
/** Returns the {@code termInfosIndexDivisor}.
*
* @see #setReaderTermsIndexDivisor(int) */
public int getReaderTermsIndexDivisor() {
return readerTermsIndexDivisor;
}
/** Returns the {@link OpenMode} set by {@link IndexWriterConfig#setOpenMode(OpenMode)}. */
public OpenMode getOpenMode() {
@ -583,8 +482,6 @@ public class LiveIndexWriterConfig {
sb.append("maxBufferedDocs=").append(getMaxBufferedDocs()).append("\n");
sb.append("maxBufferedDeleteTerms=").append(getMaxBufferedDeleteTerms()).append("\n");
sb.append("mergedSegmentWarmer=").append(getMergedSegmentWarmer()).append("\n");
sb.append("readerTermsIndexDivisor=").append(getReaderTermsIndexDivisor()).append("\n");
sb.append("termIndexInterval=").append(getTermIndexInterval()).append("\n"); // TODO: this should be private to the codec, not settable here
sb.append("delPolicy=").append(getIndexDeletionPolicy().getClass().getName()).append("\n");
IndexCommit commit = getIndexCommit();
sb.append("commit=").append(commit == null ? "null" : commit).append("\n");

View File

@ -43,16 +43,6 @@ class ReadersAndLiveDocs {
// Set once (null, and then maybe set, and never set again):
private SegmentReader reader;
// TODO: it's sometimes wasteful that we hold open two
// separate SRs (one for merging one for
// reading)... maybe just use a single SR? The gains of
// not loading the terms index (for merging in the
// non-NRT case) are far less now... and if the app has
// any deletes it'll open real readers anyway.
// Set once (null, and then maybe set, and never set again):
private SegmentReader mergeReader;
// Holds the current shared (readable and writable
// liveDocs). This is null when there are no deleted
// docs, and it's copy-on-write (cloned whenever we need
@ -118,7 +108,7 @@ class ReadersAndLiveDocs {
if (reader == null) {
// We steal returned ref:
reader = new SegmentReader(info, writer.getConfig().getReaderTermsIndexDivisor(), context);
reader = new SegmentReader(info, context);
if (liveDocs == null) {
liveDocs = reader.getLiveDocs();
}
@ -131,37 +121,6 @@ class ReadersAndLiveDocs {
return reader;
}
// Get reader for merging (does not load the terms
// index):
public synchronized SegmentReader getMergeReader(IOContext context) throws IOException {
//System.out.println(" livedocs=" + rld.liveDocs);
if (mergeReader == null) {
if (reader != null) {
// Just use the already opened non-merge reader
// for merging. In the NRT case this saves us
// pointless double-open:
//System.out.println("PROMOTE non-merge reader seg=" + rld.info);
// Ref for us:
reader.incRef();
mergeReader = reader;
//System.out.println(Thread.currentThread().getName() + ": getMergeReader share seg=" + info.name);
} else {
//System.out.println(Thread.currentThread().getName() + ": getMergeReader seg=" + info.name);
// We steal returned ref:
mergeReader = new SegmentReader(info, -1, context);
if (liveDocs == null) {
liveDocs = mergeReader.getLiveDocs();
}
}
}
// Ref for caller
mergeReader.incRef();
return mergeReader;
}
public synchronized void release(SegmentReader sr) throws IOException {
assert info == sr.getSegmentInfo();
sr.decRef();
@ -185,23 +144,12 @@ class ReadersAndLiveDocs {
public synchronized void dropReaders() throws IOException {
// TODO: can we somehow use IOUtils here...? problem is
// we are calling .decRef not .close)...
try {
if (reader != null) {
//System.out.println(" pool.drop info=" + info + " rc=" + reader.getRefCount());
try {
reader.decRef();
} finally {
reader = null;
}
}
} finally {
if (mergeReader != null) {
//System.out.println(" pool.drop info=" + info + " merge rc=" + mergeReader.getRefCount());
try {
mergeReader.decRef();
} finally {
mergeReader = null;
}
if (reader != null) {
//System.out.println(" pool.drop info=" + info + " rc=" + reader.getRefCount());
try {
reader.decRef();
} finally {
reader = null;
}
}

View File

@ -57,8 +57,6 @@ final class SegmentCoreReaders {
final DocValuesProducer dvProducer;
final DocValuesProducer normsProducer;
final int termsIndexDivisor;
private final SegmentReader owner;
final StoredFieldsReader fieldsReaderOrig;
@ -100,11 +98,7 @@ final class SegmentCoreReaders {
private final Set<CoreClosedListener> coreClosedListeners =
Collections.synchronizedSet(new LinkedHashSet<CoreClosedListener>());
SegmentCoreReaders(SegmentReader owner, Directory dir, SegmentInfoPerCommit si, IOContext context, int termsIndexDivisor) throws IOException {
if (termsIndexDivisor == 0) {
throw new IllegalArgumentException("indexDivisor must be < 0 (don't load terms index) or greater than 0 (got 0)");
}
SegmentCoreReaders(SegmentReader owner, Directory dir, SegmentInfoPerCommit si, IOContext context) throws IOException {
final Codec codec = si.info.getCodec();
final Directory cfsDir; // confusing name: if (cfs) its the cfsdir, otherwise its the segment's directory.
@ -120,9 +114,8 @@ final class SegmentCoreReaders {
}
fieldInfos = codec.fieldInfosFormat().getFieldInfosReader().read(cfsDir, si.info.name, IOContext.READONCE);
this.termsIndexDivisor = termsIndexDivisor;
final PostingsFormat format = codec.postingsFormat();
final SegmentReadState segmentReadState = new SegmentReadState(cfsDir, si.info, fieldInfos, context, termsIndexDivisor);
final SegmentReadState segmentReadState = new SegmentReadState(cfsDir, si.info, fieldInfos, context);
// Ask codec for its Fields
fields = format.fieldsProducer(segmentReadState);
assert fields != null;

View File

@ -42,7 +42,6 @@ import org.apache.lucene.util.InfoStream;
*/
final class SegmentMerger {
private final Directory directory;
private final int termIndexInterval;
private final Codec codec;
@ -52,11 +51,10 @@ final class SegmentMerger {
private final FieldInfos.Builder fieldInfosBuilder;
// note, just like in codec apis Directory 'dir' is NOT the same as segmentInfo.dir!!
SegmentMerger(List<AtomicReader> readers, SegmentInfo segmentInfo, InfoStream infoStream, Directory dir, int termIndexInterval,
SegmentMerger(List<AtomicReader> readers, SegmentInfo segmentInfo, InfoStream infoStream, Directory dir,
MergeState.CheckAbort checkAbort, FieldInfos.FieldNumbers fieldNumbers, IOContext context) {
mergeState = new MergeState(readers, segmentInfo, infoStream, checkAbort);
directory = dir;
this.termIndexInterval = termIndexInterval;
this.codec = segmentInfo.getCodec();
this.context = context;
this.fieldInfosBuilder = new FieldInfos.Builder(fieldNumbers);
@ -91,7 +89,7 @@ final class SegmentMerger {
assert numMerged == mergeState.segmentInfo.getDocCount();
final SegmentWriteState segmentWriteState = new SegmentWriteState(mergeState.infoStream, directory, mergeState.segmentInfo,
mergeState.fieldInfos, termIndexInterval, null, context);
mergeState.fieldInfos, null, context);
if (mergeState.infoStream.isEnabled("SM")) {
t0 = System.nanoTime();
}

View File

@ -41,17 +41,6 @@ public class SegmentReadState {
* Directory#openInput(String,IOContext)}. */
public final IOContext context;
/** The {@code termInfosIndexDivisor} to use, if
* appropriate (not all {@link PostingsFormat}s support
* it; in particular the current default does not).
*
* <p> NOTE: if this is &lt; 0, that means "defer terms index
* load until needed". But if the codec must load the
* terms index on init (preflex is the only once currently
* that must do so), then it should negate this value to
* get the app's terms divisor */
public int termsIndexDivisor;
/** Unique suffix for any postings files read for this
* segment. {@link PerFieldPostingsFormat} sets this for
* each of the postings formats it wraps. If you create
@ -62,8 +51,8 @@ public class SegmentReadState {
/** Create a {@code SegmentReadState}. */
public SegmentReadState(Directory dir, SegmentInfo info,
FieldInfos fieldInfos, IOContext context, int termsIndexDivisor) {
this(dir, info, fieldInfos, context, termsIndexDivisor, "");
FieldInfos fieldInfos, IOContext context) {
this(dir, info, fieldInfos, context, "");
}
/** Create a {@code SegmentReadState}. */
@ -71,13 +60,11 @@ public class SegmentReadState {
SegmentInfo info,
FieldInfos fieldInfos,
IOContext context,
int termsIndexDivisor,
String segmentSuffix) {
this.directory = dir;
this.segmentInfo = info;
this.fieldInfos = fieldInfos;
this.context = context;
this.termsIndexDivisor = termsIndexDivisor;
this.segmentSuffix = segmentSuffix;
}
@ -88,7 +75,6 @@ public class SegmentReadState {
this.segmentInfo = other.segmentInfo;
this.fieldInfos = other.fieldInfos;
this.context = other.context;
this.termsIndexDivisor = other.termsIndexDivisor;
this.segmentSuffix = newSegmentSuffix;
}
}

View File

@ -51,9 +51,9 @@ public final class SegmentReader extends AtomicReader {
* @throws IOException if there is a low-level IO error
*/
// TODO: why is this public?
public SegmentReader(SegmentInfoPerCommit si, int termInfosIndexDivisor, IOContext context) throws IOException {
public SegmentReader(SegmentInfoPerCommit si, IOContext context) throws IOException {
this.si = si;
core = new SegmentCoreReaders(this, si.info.dir, si, context, termInfosIndexDivisor);
core = new SegmentCoreReaders(this, si.info.dir, si, context);
boolean success = false;
try {
if (si.hasDeletions()) {
@ -217,12 +217,6 @@ public final class SegmentReader extends AtomicReader {
return this;
}
/** Returns term infos index divisor originally passed to
* {@link #SegmentReader(SegmentInfoPerCommit, int, IOContext)}. */
public int getTermInfosIndexDivisor() {
return core.termsIndexDivisor;
}
@Override
public NumericDocValues getNumericDocValues(String field) throws IOException {
ensureOpen();

View File

@ -66,13 +66,6 @@ public class SegmentWriteState {
* write/read must be derived using this suffix (use
* {@link IndexFileNames#segmentFileName(String,String,String)}). */
public final String segmentSuffix;
/** Expert: The fraction of terms in the "dictionary" which should be stored
* in RAM. Smaller values use more memory, but make searching slightly
* faster, while larger values use less memory and make searching slightly
* slower. Searching is typically not dominated by dictionary lookup, so
* tweaking this is rarely useful.*/
public int termIndexInterval; // TODO: this should be private to the codec, not settable here or in IWC
/** {@link IOContext} for all writes; you should pass this
* to {@link Directory#createOutput(String,IOContext)}. */
@ -80,13 +73,12 @@ public class SegmentWriteState {
/** Sole constructor. */
public SegmentWriteState(InfoStream infoStream, Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos,
int termIndexInterval, BufferedDeletes segDeletes, IOContext context) {
BufferedDeletes segDeletes, IOContext context) {
this.infoStream = infoStream;
this.segDeletes = segDeletes;
this.directory = directory;
this.segmentInfo = segmentInfo;
this.fieldInfos = fieldInfos;
this.termIndexInterval = termIndexInterval;
segmentSuffix = "";
this.context = context;
}
@ -99,7 +91,6 @@ public class SegmentWriteState {
directory = state.directory;
segmentInfo = state.segmentInfo;
fieldInfos = state.fieldInfos;
termIndexInterval = state.termIndexInterval;
context = state.context;
this.segmentSuffix = segmentSuffix;
segDeletes = state.segDeletes;

View File

@ -33,22 +33,19 @@ final class StandardDirectoryReader extends DirectoryReader {
private final IndexWriter writer;
private final SegmentInfos segmentInfos;
private final int termInfosIndexDivisor;
private final boolean applyAllDeletes;
/** called only from static open() methods */
StandardDirectoryReader(Directory directory, AtomicReader[] readers, IndexWriter writer,
SegmentInfos sis, int termInfosIndexDivisor, boolean applyAllDeletes) {
SegmentInfos sis, boolean applyAllDeletes) {
super(directory, readers);
this.writer = writer;
this.segmentInfos = sis;
this.termInfosIndexDivisor = termInfosIndexDivisor;
this.applyAllDeletes = applyAllDeletes;
}
/** called from DirectoryReader.open(...) methods */
static DirectoryReader open(final Directory directory, final IndexCommit commit,
final int termInfosIndexDivisor) throws IOException {
static DirectoryReader open(final Directory directory, final IndexCommit commit) throws IOException {
return (DirectoryReader) new SegmentInfos.FindSegmentsFile(directory) {
@Override
protected Object doBody(String segmentFileName) throws IOException {
@ -59,7 +56,7 @@ final class StandardDirectoryReader extends DirectoryReader {
IOException prior = null;
boolean success = false;
try {
readers[i] = new SegmentReader(sis.info(i), termInfosIndexDivisor, IOContext.READ);
readers[i] = new SegmentReader(sis.info(i), IOContext.READ);
success = true;
} catch(IOException ex) {
prior = ex;
@ -68,7 +65,7 @@ final class StandardDirectoryReader extends DirectoryReader {
IOUtils.closeWhileHandlingException(prior, readers);
}
}
return new StandardDirectoryReader(directory, readers, null, sis, termInfosIndexDivisor, false);
return new StandardDirectoryReader(directory, readers, null, sis, false);
}
}.run(commit);
}
@ -119,12 +116,11 @@ final class StandardDirectoryReader extends DirectoryReader {
}
}
return new StandardDirectoryReader(dir, readers.toArray(new SegmentReader[readers.size()]),
writer, segmentInfos, writer.getConfig().getReaderTermsIndexDivisor(), applyAllDeletes);
writer, segmentInfos, applyAllDeletes);
}
/** This constructor is only used for {@link #doOpenIfChanged(SegmentInfos)} */
private static DirectoryReader open(Directory directory, SegmentInfos infos, List<? extends AtomicReader> oldReaders,
int termInfosIndexDivisor) throws IOException {
private static DirectoryReader open(Directory directory, SegmentInfos infos, List<? extends AtomicReader> oldReaders) throws IOException {
// we put the old SegmentReaders in a map, that allows us
// to lookup a reader using its segment name
@ -162,7 +158,7 @@ final class StandardDirectoryReader extends DirectoryReader {
if (newReaders[i] == null || infos.info(i).info.getUseCompoundFile() != newReaders[i].getSegmentInfo().info.getUseCompoundFile()) {
// this is a new reader; in case we hit an exception we can close it safely
newReader = new SegmentReader(infos.info(i), termInfosIndexDivisor, IOContext.READ);
newReader = new SegmentReader(infos.info(i), IOContext.READ);
readerShared[i] = false;
newReaders[i] = newReader;
} else {
@ -212,7 +208,7 @@ final class StandardDirectoryReader extends DirectoryReader {
}
}
}
return new StandardDirectoryReader(directory, newReaders, null, infos, termInfosIndexDivisor, false);
return new StandardDirectoryReader(directory, newReaders, null, infos, false);
}
@Override
@ -313,7 +309,7 @@ final class StandardDirectoryReader extends DirectoryReader {
}
DirectoryReader doOpenIfChanged(SegmentInfos infos) throws IOException {
return StandardDirectoryReader.open(directory, infos, getSequentialSubReaders(), termInfosIndexDivisor);
return StandardDirectoryReader.open(directory, infos, getSequentialSubReaders());
}
@Override

View File

@ -24,7 +24,7 @@ import org.apache.lucene.index.BasePostingsFormatTestCase;
* Tests Lucene40PostingsFormat
*/
public class TestLucene40PostingsFormat extends BasePostingsFormatTestCase {
private final Codec codec = new Lucene40Codec();
private final Codec codec = new Lucene40RWCodec();
@Override
protected Codec getCodec() {

View File

@ -192,7 +192,7 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
if (VERBOSE) {
System.out.println("\nTEST: assertQuery " + t);
}
IndexReader reader = DirectoryReader.open(dir, 1);
IndexReader reader = DirectoryReader.open(dir);
IndexSearcher searcher = newSearcher(reader);
TopDocs search = searcher.search(new TermQuery(t), num + 10);
assertEquals(num, search.totalHits);

View File

@ -296,7 +296,7 @@ public class TestCodecs extends LuceneTestCase {
Codec codec = Codec.getDefault();
final SegmentInfo si = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, SEGMENT, 10000, false, codec, null, null);
final FieldsProducer reader = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random()), DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR));
final FieldsProducer reader = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random())));
final Iterator<String> fieldsEnum = reader.iterator();
String fieldName = fieldsEnum.next();
@ -357,7 +357,7 @@ public class TestCodecs extends LuceneTestCase {
if (VERBOSE) {
System.out.println("TEST: now read postings");
}
final FieldsProducer terms = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random()), DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR));
final FieldsProducer terms = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random())));
final Verify[] threads = new Verify[NUM_TEST_THREADS-1];
for(int i=0;i<NUM_TEST_THREADS-1;i++) {
@ -655,10 +655,9 @@ public class TestCodecs extends LuceneTestCase {
private void write(final FieldInfos fieldInfos, final Directory dir, final FieldData[] fields) throws Throwable {
final int termIndexInterval = _TestUtil.nextInt(random(), 13, 27);
final Codec codec = Codec.getDefault();
final SegmentInfo si = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, SEGMENT, 10000, false, codec, null, null);
final SegmentWriteState state = new SegmentWriteState(InfoStream.getDefault(), dir, si, fieldInfos, termIndexInterval, null, newIOContext(random()));
final SegmentWriteState state = new SegmentWriteState(InfoStream.getDefault(), dir, si, fieldInfos, null, newIOContext(random()));
final FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(state);
Arrays.sort(fields);

View File

@ -822,54 +822,6 @@ public void testFilesOpenClose() throws IOException {
dir.close();
}
// LUCENE-1609: don't load terms index
public void testNoTermsIndex() throws Throwable {
Directory dir = newDirectory();
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat())));
Document doc = new Document();
doc.add(newTextField("field", "a b c d e f g h i j k l m n o p q r s t u v w x y z", Field.Store.NO));
doc.add(newTextField("number", "0 1 2 3 4 5 6 7 8 9", Field.Store.NO));
writer.addDocument(doc);
writer.addDocument(doc);
writer.close();
DirectoryReader r = DirectoryReader.open(dir, -1);
try {
r.docFreq(new Term("field", "f"));
fail("did not hit expected exception");
} catch (IllegalStateException ise) {
// expected
}
assertEquals(-1, ((SegmentReader) r.leaves().get(0).reader()).getTermInfosIndexDivisor());
writer = new IndexWriter(
dir,
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).
setCodec(_TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat())).
setMergePolicy(newLogMergePolicy(10))
);
writer.addDocument(doc);
writer.close();
// LUCENE-1718: ensure re-open carries over no terms index:
DirectoryReader r2 = DirectoryReader.openIfChanged(r);
assertNotNull(r2);
assertNull(DirectoryReader.openIfChanged(r2));
r.close();
List<AtomicReaderContext> leaves = r2.leaves();
assertEquals(2, leaves.size());
for(AtomicReaderContext ctx : leaves) {
try {
ctx.reader().docFreq(new Term("field", "f"));
fail("did not hit expected exception");
} catch (IllegalStateException ise) {
// expected
}
}
r2.close();
dir.close();
}
// LUCENE-2046
public void testPrepareCommitIsCurrent() throws Throwable {
Directory dir = newDirectory();

View File

@ -212,15 +212,15 @@ public class TestDoc extends LuceneTestCase {
private SegmentInfoPerCommit merge(Directory dir, SegmentInfoPerCommit si1, SegmentInfoPerCommit si2, String merged, boolean useCompoundFile)
throws Exception {
IOContext context = newIOContext(random());
SegmentReader r1 = new SegmentReader(si1, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, context);
SegmentReader r2 = new SegmentReader(si2, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, context);
SegmentReader r1 = new SegmentReader(si1, context);
SegmentReader r2 = new SegmentReader(si2, context);
final Codec codec = Codec.getDefault();
TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(si1.info.dir);
final SegmentInfo si = new SegmentInfo(si1.info.dir, Constants.LUCENE_MAIN_VERSION, merged, -1, false, codec, null, null);
SegmentMerger merger = new SegmentMerger(Arrays.<AtomicReader>asList(r1, r2),
si, InfoStream.getDefault(), trackingDir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL,
si, InfoStream.getDefault(), trackingDir,
MergeState.CheckAbort.NONE, new FieldInfos.FieldNumbers(), context);
MergeState mergeState = merger.merge();
@ -245,7 +245,7 @@ public class TestDoc extends LuceneTestCase {
private void printSegment(PrintWriter out, SegmentInfoPerCommit si)
throws Exception {
SegmentReader reader = new SegmentReader(si, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random()));
SegmentReader reader = new SegmentReader(si, newIOContext(random()));
for (int i = 0; i < reader.numDocs(); i++)
out.println(reader.document(i));

View File

@ -65,7 +65,7 @@ public class TestDocumentWriter extends LuceneTestCase {
SegmentInfoPerCommit info = writer.newestSegment();
writer.close();
//After adding the document, we should be able to read it back in
SegmentReader reader = new SegmentReader(info, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random()));
SegmentReader reader = new SegmentReader(info, newIOContext(random()));
assertTrue(reader != null);
StoredDocument doc = reader.document(0);
assertTrue(doc != null);
@ -126,7 +126,7 @@ public class TestDocumentWriter extends LuceneTestCase {
writer.commit();
SegmentInfoPerCommit info = writer.newestSegment();
writer.close();
SegmentReader reader = new SegmentReader(info, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random()));
SegmentReader reader = new SegmentReader(info, newIOContext(random()));
DocsAndPositionsEnum termPositions = MultiFields.getTermPositionsEnum(reader, MultiFields.getLiveDocs(reader),
"repeated", new BytesRef("repeated"));
@ -198,7 +198,7 @@ public class TestDocumentWriter extends LuceneTestCase {
writer.commit();
SegmentInfoPerCommit info = writer.newestSegment();
writer.close();
SegmentReader reader = new SegmentReader(info, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random()));
SegmentReader reader = new SegmentReader(info, newIOContext(random()));
DocsAndPositionsEnum termPositions = MultiFields.getTermPositionsEnum(reader, reader.getLiveDocs(), "f1", new BytesRef("a"));
assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
@ -241,7 +241,7 @@ public class TestDocumentWriter extends LuceneTestCase {
writer.commit();
SegmentInfoPerCommit info = writer.newestSegment();
writer.close();
SegmentReader reader = new SegmentReader(info, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random()));
SegmentReader reader = new SegmentReader(info, newIOContext(random()));
DocsAndPositionsEnum termPositions = reader.termPositionsEnum(new Term("preanalyzed", "term1"));
assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);

View File

@ -1305,36 +1305,6 @@ public class TestIndexWriter extends LuceneTestCase {
dir.close();
}
public void testIndexDivisor() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig config = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
config.setTermIndexInterval(2);
IndexWriter w = new IndexWriter(dir, config);
StringBuilder s = new StringBuilder();
// must be > 256
for(int i=0;i<300;i++) {
s.append(' ').append(i);
}
Document d = new Document();
Field f = newTextField("field", s.toString(), Field.Store.NO);
d.add(f);
w.addDocument(d);
AtomicReader r = getOnlySegmentReader(w.getReader());
TermsEnum t = r.fields().terms("field").iterator(null);
int count = 0;
while(t.next() != null) {
final DocsEnum docs = _TestUtil.docs(random(), t, null, null, DocsEnum.FLAG_NONE);
assertEquals(0, docs.nextDoc());
assertEquals(DocIdSetIterator.NO_MORE_DOCS, docs.nextDoc());
count++;
}
assertEquals(300, count);
r.close();
w.close();
dir.close();
}
public void testDeleteUnusedFiles() throws Exception {
for(int iter=0;iter<2;iter++) {
Directory dir = newMockDirectory(); // relies on windows semantics
@ -1716,20 +1686,6 @@ public class TestIndexWriter extends LuceneTestCase {
dir.close();
}
// LUCENE-3183
public void testEmptyFieldNameTIIOne() throws IOException {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
iwc.setTermIndexInterval(1);
iwc.setReaderTermsIndexDivisor(1);
IndexWriter writer = new IndexWriter(dir, iwc);
Document doc = new Document();
doc.add(newTextField("", "a b c", Field.Store.NO));
writer.addDocument(doc);
writer.close();
dir.close();
}
public void testDeleteAllNRTLeftoverFiles() throws Exception {
Directory d = new MockDirectoryWrapper(random(), new RAMDirectory());

View File

@ -66,7 +66,6 @@ public class TestIndexWriterConfig extends LuceneTestCase {
assertEquals(OpenMode.CREATE_OR_APPEND, conf.getOpenMode());
// we don't need to assert this, it should be unspecified
assertTrue(IndexSearcher.getDefaultSimilarity() == conf.getSimilarity());
assertEquals(IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL, conf.getTermIndexInterval());
assertEquals(IndexWriterConfig.getDefaultWriteLockTimeout(), conf.getWriteLockTimeout());
assertEquals(IndexWriterConfig.WRITE_LOCK_TIMEOUT, IndexWriterConfig.getDefaultWriteLockTimeout());
assertEquals(IndexWriterConfig.DEFAULT_MAX_BUFFERED_DELETE_TERMS, conf.getMaxBufferedDeleteTerms());
@ -75,7 +74,6 @@ public class TestIndexWriterConfig extends LuceneTestCase {
assertEquals(IndexWriterConfig.DEFAULT_READER_POOLING, conf.getReaderPooling());
assertTrue(DocumentsWriterPerThread.defaultIndexingChain == conf.getIndexingChain());
assertNull(conf.getMergedSegmentWarmer());
assertEquals(IndexWriterConfig.DEFAULT_READER_TERMS_INDEX_DIVISOR, conf.getReaderTermsIndexDivisor());
assertEquals(TieredMergePolicy.class, conf.getMergePolicy().getClass());
assertEquals(ThreadAffinityDocumentsWriterThreadPool.class, conf.getIndexerThreadPool().getClass());
assertEquals(FlushByRamOrCountsPolicy.class, conf.getFlushPolicy().getClass());
@ -92,7 +90,6 @@ public class TestIndexWriterConfig extends LuceneTestCase {
getters.add("getMergeScheduler");
getters.add("getOpenMode");
getters.add("getSimilarity");
getters.add("getTermIndexInterval");
getters.add("getWriteLockTimeout");
getters.add("getDefaultWriteLockTimeout");
getters.add("getMaxBufferedDeleteTerms");
@ -104,7 +101,6 @@ public class TestIndexWriterConfig extends LuceneTestCase {
getters.add("getMaxThreadStates");
getters.add("getReaderPooling");
getters.add("getIndexerThreadPool");
getters.add("getReaderTermsIndexDivisor");
getters.add("getFlushPolicy");
getters.add("getRAMPerThreadHardLimitMB");
getters.add("getCodec");
@ -200,14 +196,12 @@ public class TestIndexWriterConfig extends LuceneTestCase {
public void testConstants() throws Exception {
// Tests that the values of the constants does not change
assertEquals(1000, IndexWriterConfig.WRITE_LOCK_TIMEOUT);
assertEquals(32, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL);
assertEquals(-1, IndexWriterConfig.DISABLE_AUTO_FLUSH);
assertEquals(IndexWriterConfig.DISABLE_AUTO_FLUSH, IndexWriterConfig.DEFAULT_MAX_BUFFERED_DELETE_TERMS);
assertEquals(IndexWriterConfig.DISABLE_AUTO_FLUSH, IndexWriterConfig.DEFAULT_MAX_BUFFERED_DOCS);
assertEquals(16.0, IndexWriterConfig.DEFAULT_RAM_BUFFER_SIZE_MB, 0.0);
assertEquals(false, IndexWriterConfig.DEFAULT_READER_POOLING);
assertEquals(true, IndexWriterConfig.DEFAULT_USE_COMPOUND_FILE_SYSTEM);
assertEquals(DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, IndexWriterConfig.DEFAULT_READER_TERMS_INDEX_DIVISOR);
}
@Test
@ -341,23 +335,6 @@ public class TestIndexWriterConfig extends LuceneTestCase {
} catch (IllegalArgumentException e) {
// this is expected
}
// Test setReaderTermsIndexDivisor
try {
conf.setReaderTermsIndexDivisor(0);
fail("should not have succeeded to set termsIndexDivisor to 0");
} catch (IllegalArgumentException e) {
// this is expected
}
// Setting to -1 is ok
conf.setReaderTermsIndexDivisor(-1);
try {
conf.setReaderTermsIndexDivisor(-2);
fail("should not have succeeded to set termsIndexDivisor to < -1");
} catch (IllegalArgumentException e) {
// this is expected
}
try {
conf.setRAMPerThreadHardLimitMB(2048);

View File

@ -130,7 +130,6 @@ public class TestIndexWriterForceMerge extends LuceneTestCase {
for(int j=0;j<500;j++) {
TestIndexWriter.addDocWithIndex(writer, j);
}
final int termIndexInterval = writer.getConfig().getTermIndexInterval();
// force one extra segment w/ different doc store so
// we see the doc stores get merged
writer.commit();
@ -152,10 +151,7 @@ public class TestIndexWriterForceMerge extends LuceneTestCase {
dir.resetMaxUsedSizeInBytes();
dir.setTrackDiskUsage(true);
// Import to use same term index interval else a
// smaller one here could increase the disk usage and
// cause a false failure:
writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random())).setOpenMode(OpenMode.APPEND).setTermIndexInterval(termIndexInterval).setMergePolicy(newLogMergePolicy()));
writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random())).setOpenMode(OpenMode.APPEND).setMergePolicy(newLogMergePolicy()));
writer.forceMerge(1);
writer.close();
long maxDiskUsage = dir.getMaxUsedSizeInBytes();

View File

@ -314,7 +314,7 @@ public class TestIndexWriterReader extends LuceneTestCase {
boolean doFullMerge = true;
Directory dir1 = newDirectory();
IndexWriter writer = new IndexWriter(dir1, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random())).setReaderTermsIndexDivisor(2));
IndexWriter writer = new IndexWriter(dir1, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random())));
// create the index
createIndexNoClose(!doFullMerge, "index1", writer);
writer.flush(false, true);
@ -1006,35 +1006,6 @@ public class TestIndexWriterReader extends LuceneTestCase {
assertTrue(didWarm.get());
}
public void testNoTermsIndex() throws Exception {
// Some Codecs don't honor the ReaderTermsIndexDivisor, so skip the test if
// they're picked.
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random())).setReaderTermsIndexDivisor(-1);
// Don't proceed if picked Codec is in the list of illegal ones.
final String format = _TestUtil.getPostingsFormat("f");
assumeFalse("Format: " + format + " does not support ReaderTermsIndexDivisor!",
(format.equals("SimpleText") || format.equals("Memory") || format.equals("Direct")));
Directory dir = newDirectory();
IndexWriter w = new IndexWriter(dir, conf);
Document doc = new Document();
doc.add(new TextField("f", "val", Field.Store.NO));
w.addDocument(doc);
SegmentReader r = getOnlySegmentReader(DirectoryReader.open(w, true));
try {
_TestUtil.docs(random(), r, "f", new BytesRef("val"), null, null, DocsEnum.FLAG_NONE);
fail("should have failed to seek since terms index was not loaded.");
} catch (IllegalStateException e) {
// expected - we didn't load the term index
} finally {
r.close();
w.close();
dir.close();
}
}
public void testReopenAfterNoRealChange() throws Exception {
Directory d = newDirectory();
IndexWriter w = new IndexWriter(

View File

@ -54,8 +54,8 @@ public class TestSegmentMerger extends LuceneTestCase {
SegmentInfoPerCommit info1 = DocHelper.writeDoc(random(), merge1Dir, doc1);
DocHelper.setupDoc(doc2);
SegmentInfoPerCommit info2 = DocHelper.writeDoc(random(), merge2Dir, doc2);
reader1 = new SegmentReader(info1, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random()));
reader2 = new SegmentReader(info2, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random()));
reader1 = new SegmentReader(info1, newIOContext(random()));
reader2 = new SegmentReader(info2, newIOContext(random()));
}
@Override
@ -81,7 +81,7 @@ public class TestSegmentMerger extends LuceneTestCase {
final SegmentInfo si = new SegmentInfo(mergedDir, Constants.LUCENE_MAIN_VERSION, mergedSegment, -1, false, codec, null, null);
SegmentMerger merger = new SegmentMerger(Arrays.<AtomicReader>asList(reader1, reader2),
si, InfoStream.getDefault(), mergedDir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL,
si, InfoStream.getDefault(), mergedDir,
MergeState.CheckAbort.NONE, new FieldInfos.FieldNumbers(), newIOContext(random()));
MergeState mergeState = merger.merge();
int docsMerged = mergeState.segmentInfo.getDocCount();
@ -91,7 +91,7 @@ public class TestSegmentMerger extends LuceneTestCase {
new SegmentInfo(mergedDir, Constants.LUCENE_MAIN_VERSION, mergedSegment, docsMerged,
false, codec, null, null),
0, -1L),
DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random()));
newIOContext(random()));
assertTrue(mergedReader != null);
assertTrue(mergedReader.numDocs() == 2);
StoredDocument newDoc1 = mergedReader.document(0);

View File

@ -43,7 +43,7 @@ public class TestSegmentReader extends LuceneTestCase {
dir = newDirectory();
DocHelper.setupDoc(testDoc);
SegmentInfoPerCommit info = DocHelper.writeDoc(random(), dir, testDoc);
reader = new SegmentReader(info, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, IOContext.READ);
reader = new SegmentReader(info, IOContext.READ);
}
@Override

View File

@ -50,16 +50,11 @@ public class TestSegmentTermDocs extends LuceneTestCase {
public void test() {
assertTrue(dir != null);
}
public void testTermDocs() throws IOException {
testTermDocs(1);
}
public void testTermDocs(int indexDivisor) throws IOException {
public void testTermDocs() throws IOException {
//After adding the document, we should be able to read it back in
SegmentReader reader = new SegmentReader(info, indexDivisor, newIOContext(random()));
SegmentReader reader = new SegmentReader(info, newIOContext(random()));
assertTrue(reader != null);
assertEquals(indexDivisor, reader.getTermInfosIndexDivisor());
TermsEnum terms = reader.fields().terms(DocHelper.TEXT_FIELD_2_KEY).iterator(null);
terms.seekCeil(new BytesRef("field"));
@ -74,13 +69,9 @@ public class TestSegmentTermDocs extends LuceneTestCase {
}
public void testBadSeek() throws IOException {
testBadSeek(1);
}
public void testBadSeek(int indexDivisor) throws IOException {
{
//After adding the document, we should be able to read it back in
SegmentReader reader = new SegmentReader(info, indexDivisor, newIOContext(random()));
SegmentReader reader = new SegmentReader(info, newIOContext(random()));
assertTrue(reader != null);
DocsEnum termDocs = _TestUtil.docs(random(), reader,
"textField2",
@ -94,7 +85,7 @@ public class TestSegmentTermDocs extends LuceneTestCase {
}
{
//After adding the document, we should be able to read it back in
SegmentReader reader = new SegmentReader(info, indexDivisor, newIOContext(random()));
SegmentReader reader = new SegmentReader(info, newIOContext(random()));
assertTrue(reader != null);
DocsEnum termDocs = _TestUtil.docs(random(), reader,
"junk",
@ -108,10 +99,6 @@ public class TestSegmentTermDocs extends LuceneTestCase {
}
public void testSkipTo() throws IOException {
testSkipTo(1);
}
public void testSkipTo(int indexDivisor) throws IOException {
Directory dir = newDirectory();
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy()));
@ -131,7 +118,7 @@ public class TestSegmentTermDocs extends LuceneTestCase {
writer.forceMerge(1);
writer.close();
IndexReader reader = DirectoryReader.open(dir, indexDivisor);
IndexReader reader = DirectoryReader.open(dir);
DocsEnum tdocs = _TestUtil.docs(random(), reader,
ta.field(),
@ -267,15 +254,7 @@ public class TestSegmentTermDocs extends LuceneTestCase {
reader.close();
dir.close();
}
public void testIndexDivisor() throws IOException {
testDoc = new Document();
DocHelper.setupDoc(testDoc);
DocHelper.writeDoc(random(), dir, testDoc);
testTermDocs(2);
testBadSeek(2);
testSkipTo(2);
}
private void addDoc(IndexWriter writer, String value) throws IOException
{

View File

@ -45,7 +45,7 @@ public class TaxonomyMergeUtils {
// merge the taxonomies
destTaxWriter.addTaxonomy(srcTaxDir, map);
int ordinalMap[] = map.getMap();
DirectoryReader reader = DirectoryReader.open(srcIndexDir, -1);
DirectoryReader reader = DirectoryReader.open(srcIndexDir);
List<AtomicReaderContext> leaves = reader.leaves();
int numReaders = leaves.size();
AtomicReader wrappedLeaves[] = new AtomicReader[numReaders];

View File

@ -45,9 +45,15 @@ import org.apache.lucene.util.BytesRef;
* {@link FixedGapTermsIndexWriter}.
*/
public final class Lucene41WithOrds extends PostingsFormat {
final int termIndexInterval;
public Lucene41WithOrds() {
this(FixedGapTermsIndexWriter.DEFAULT_TERM_INDEX_INTERVAL);
}
public Lucene41WithOrds(int termIndexInterval) {
super("Lucene41WithOrds");
this.termIndexInterval = termIndexInterval;
}
@Override
@ -61,7 +67,7 @@ public final class Lucene41WithOrds extends PostingsFormat {
TermsIndexWriterBase indexWriter;
boolean success = false;
try {
indexWriter = new FixedGapTermsIndexWriter(state);
indexWriter = new FixedGapTermsIndexWriter(state, termIndexInterval);
success = true;
} finally {
if (!success) {
@ -97,7 +103,6 @@ public final class Lucene41WithOrds extends PostingsFormat {
indexReader = new FixedGapTermsIndexReader(state.directory,
state.fieldInfos,
state.segmentInfo.name,
state.termsIndexDivisor,
BytesRef.getUTF8SortedAsUnicodeComparator(),
state.segmentSuffix, state.context);
success = true;

View File

@ -0,0 +1,144 @@
package org.apache.lucene.codecs.lucene41vargap;
/*
* 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 java.io.IOException;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.blockterms.BlockTermsReader;
import org.apache.lucene.codecs.blockterms.BlockTermsWriter;
import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexWriter;
import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase;
import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase;
import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexReader;
import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexWriter;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; // javadocs
import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
// TODO: we could make separate base class that can wrapp
// any PostingsBaseFormat and make it ord-able...
/**
* Customized version of {@link Lucene41PostingsFormat} that uses
* {@link VariableGapTermsIndexWriter} with a fixed interval, but
* forcing high docfreq terms to be indexed terms.
*/
public final class Lucene41VarGapDocFreqInterval extends PostingsFormat {
final int termIndexInterval;
final int docFreqThreshold;
public Lucene41VarGapDocFreqInterval() {
this(1000000, FixedGapTermsIndexWriter.DEFAULT_TERM_INDEX_INTERVAL);
}
public Lucene41VarGapDocFreqInterval(int docFreqThreshold, int termIndexInterval) {
super("Lucene41VarGapFixedInterval");
this.termIndexInterval = termIndexInterval;
this.docFreqThreshold = docFreqThreshold;
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase docs = new Lucene41PostingsWriter(state);
// TODO: should we make the terms index more easily
// pluggable? Ie so that this codec would record which
// index impl was used, and switch on loading?
// Or... you must make a new Codec for this?
TermsIndexWriterBase indexWriter;
boolean success = false;
try {
indexWriter = new VariableGapTermsIndexWriter(state, new VariableGapTermsIndexWriter.EveryNOrDocFreqTermSelector(docFreqThreshold, termIndexInterval));
success = true;
} finally {
if (!success) {
docs.close();
}
}
success = false;
try {
// Must use BlockTermsWriter (not BlockTree) because
// BlockTree doens't support ords (yet)...
FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, docs);
success = true;
return ret;
} finally {
if (!success) {
try {
docs.close();
} finally {
indexWriter.close();
}
}
}
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postings = new Lucene41PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
TermsIndexReaderBase indexReader;
boolean success = false;
try {
indexReader = new VariableGapTermsIndexReader(state.directory,
state.fieldInfos,
state.segmentInfo.name,
state.segmentSuffix, state.context);
success = true;
} finally {
if (!success) {
postings.close();
}
}
success = false;
try {
FieldsProducer ret = new BlockTermsReader(indexReader,
state.directory,
state.fieldInfos,
state.segmentInfo,
postings,
state.context,
state.segmentSuffix);
success = true;
return ret;
} finally {
if (!success) {
try {
postings.close();
} finally {
indexReader.close();
}
}
}
}
/** Extension of freq postings file */
static final String FREQ_EXTENSION = "frq";
/** Extension of prox postings file */
static final String PROX_EXTENSION = "prx";
}

View File

@ -0,0 +1,141 @@
package org.apache.lucene.codecs.lucene41vargap;
/*
* 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 java.io.IOException;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.blockterms.BlockTermsReader;
import org.apache.lucene.codecs.blockterms.BlockTermsWriter;
import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexWriter;
import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase;
import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase;
import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexReader;
import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexWriter;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; // javadocs
import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
// TODO: we could make separate base class that can wrapp
// any PostingsBaseFormat and make it ord-able...
/**
* Customized version of {@link Lucene41PostingsFormat} that uses
* {@link VariableGapTermsIndexWriter} with a fixed interval.
*/
public final class Lucene41VarGapFixedInterval extends PostingsFormat {
final int termIndexInterval;
public Lucene41VarGapFixedInterval() {
this(FixedGapTermsIndexWriter.DEFAULT_TERM_INDEX_INTERVAL);
}
public Lucene41VarGapFixedInterval(int termIndexInterval) {
super("Lucene41VarGapFixedInterval");
this.termIndexInterval = termIndexInterval;
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase docs = new Lucene41PostingsWriter(state);
// TODO: should we make the terms index more easily
// pluggable? Ie so that this codec would record which
// index impl was used, and switch on loading?
// Or... you must make a new Codec for this?
TermsIndexWriterBase indexWriter;
boolean success = false;
try {
indexWriter = new VariableGapTermsIndexWriter(state, new VariableGapTermsIndexWriter.EveryNTermSelector(termIndexInterval));
success = true;
} finally {
if (!success) {
docs.close();
}
}
success = false;
try {
// Must use BlockTermsWriter (not BlockTree) because
// BlockTree doens't support ords (yet)...
FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, docs);
success = true;
return ret;
} finally {
if (!success) {
try {
docs.close();
} finally {
indexWriter.close();
}
}
}
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postings = new Lucene41PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
TermsIndexReaderBase indexReader;
boolean success = false;
try {
indexReader = new VariableGapTermsIndexReader(state.directory,
state.fieldInfos,
state.segmentInfo.name,
state.segmentSuffix, state.context);
success = true;
} finally {
if (!success) {
postings.close();
}
}
success = false;
try {
FieldsProducer ret = new BlockTermsReader(indexReader,
state.directory,
state.fieldInfos,
state.segmentInfo,
postings,
state.context,
state.segmentSuffix);
success = true;
return ret;
} finally {
if (!success) {
try {
postings.close();
} finally {
indexReader.close();
}
}
}
}
/** Extension of freq postings file */
static final String FREQ_EXTENSION = "frq";
/** Extension of prox postings file */
static final String PROX_EXTENSION = "prx";
}

View File

@ -0,0 +1,25 @@
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
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.
-->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
Codecs for testing that support {@link org.apache.lucene.codecs.blockterms.VariableGapTermsIndexReader}
</body>
</html>

View File

@ -169,7 +169,6 @@ public final class MockFixedIntBlockPostingsFormat extends PostingsFormat {
indexReader = new FixedGapTermsIndexReader(state.directory,
state.fieldInfos,
state.segmentInfo.name,
state.termsIndexDivisor,
BytesRef.getUTF8SortedAsUnicodeComparator(), state.segmentSuffix,
IOContext.DEFAULT);
success = true;

View File

@ -194,7 +194,6 @@ public final class MockVariableIntBlockPostingsFormat extends PostingsFormat {
indexReader = new FixedGapTermsIndexReader(state.directory,
state.fieldInfos,
state.segmentInfo.name,
state.termsIndexDivisor,
BytesRef.getUTF8SortedAsUnicodeComparator(),
state.segmentSuffix, state.context);
success = true;

View File

@ -220,11 +220,11 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
final TermsIndexWriterBase indexWriter;
try {
if (random.nextBoolean()) {
state.termIndexInterval = _TestUtil.nextInt(random, 1, 100);
int termIndexInterval = _TestUtil.nextInt(random, 1, 100);
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: fixed-gap terms index (tii=" + state.termIndexInterval + ")");
System.out.println("MockRandomCodec: fixed-gap terms index (tii=" + termIndexInterval + ")");
}
indexWriter = new FixedGapTermsIndexWriter(state);
indexWriter = new FixedGapTermsIndexWriter(state, termIndexInterval);
} else {
final VariableGapTermsIndexWriter.IndexTermSelector selector;
final int n2 = random.nextInt(3);
@ -340,8 +340,7 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
state.segmentInfo,
postingsReader,
state.context,
state.segmentSuffix,
state.termsIndexDivisor);
state.segmentSuffix);
success = true;
} finally {
if (!success) {
@ -359,20 +358,14 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
final boolean doFixedGap = random.nextBoolean();
// randomness diverges from writer, here:
if (state.termsIndexDivisor != -1) {
state.termsIndexDivisor = _TestUtil.nextInt(random, 1, 10);
}
if (doFixedGap) {
// if termsIndexDivisor is set to -1, we should not touch it. It means a
// test explicitly instructed not to load the terms index.
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: fixed-gap terms index (divisor=" + state.termsIndexDivisor + ")");
System.out.println("MockRandomCodec: fixed-gap terms index");
}
indexReader = new FixedGapTermsIndexReader(state.directory,
state.fieldInfos,
state.segmentInfo.name,
state.termsIndexDivisor,
BytesRef.getUTF8SortedAsUnicodeComparator(),
state.segmentSuffix, state.context);
} else {
@ -383,12 +376,11 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
random.nextLong();
}
if (LuceneTestCase.VERBOSE) {
System.out.println("MockRandomCodec: variable-gap terms index (divisor=" + state.termsIndexDivisor + ")");
System.out.println("MockRandomCodec: variable-gap terms index");
}
indexReader = new VariableGapTermsIndexReader(state.directory,
state.fieldInfos,
state.segmentInfo.name,
state.termsIndexDivisor,
state.segmentSuffix, state.context);
}

View File

@ -92,7 +92,6 @@ public final class MockSepPostingsFormat extends PostingsFormat {
indexReader = new FixedGapTermsIndexReader(state.directory,
state.fieldInfos,
state.segmentInfo.name,
state.termsIndexDivisor,
BytesRef.getUTF8SortedAsUnicodeComparator(),
state.segmentSuffix, state.context);
success = true;

View File

@ -84,8 +84,7 @@ public final class NestedPulsingPostingsFormat extends PostingsFormat {
state.directory, state.fieldInfos, state.segmentInfo,
pulsingReader,
state.context,
state.segmentSuffix,
state.termsIndexDivisor);
state.segmentSuffix);
success = true;
return ret;
} finally {

View File

@ -1022,7 +1022,7 @@ public abstract class BaseDocValuesFormatTestCase extends LuceneTestCase {
writer.close(true);
DirectoryReader reader = DirectoryReader.open(dir, 1);
DirectoryReader reader = DirectoryReader.open(dir);
assertEquals(1, reader.leaves().size());
IndexSearcher searcher = new IndexSearcher(reader);

View File

@ -38,6 +38,8 @@ import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsConsumer;
import org.apache.lucene.codecs.TermStats;
import org.apache.lucene.codecs.TermsConsumer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.store.Directory;
@ -483,7 +485,7 @@ public abstract class BasePostingsFormatTestCase extends LuceneTestCase {
SegmentWriteState writeState = new SegmentWriteState(null, dir,
segmentInfo, newFieldInfos,
32, null, new IOContext(new FlushInfo(maxDoc, bytes)));
null, new IOContext(new FlushInfo(maxDoc, bytes)));
FieldsConsumer fieldsConsumer = codec.postingsFormat().fieldsConsumer(writeState);
for(Map.Entry<String,Map<BytesRef,Long>> fieldEnt : fields.entrySet()) {
@ -567,7 +569,7 @@ public abstract class BasePostingsFormatTestCase extends LuceneTestCase {
currentFieldInfos = newFieldInfos;
SegmentReadState readState = new SegmentReadState(dir, segmentInfo, newFieldInfos, IOContext.DEFAULT, 1);
SegmentReadState readState = new SegmentReadState(dir, segmentInfo, newFieldInfos, IOContext.DEFAULT);
return codec.postingsFormat().fieldsProducer(readState);
}
@ -1130,4 +1132,85 @@ public abstract class BasePostingsFormatTestCase extends LuceneTestCase {
_TestUtil.rmDir(path);
}
}
public void testEmptyField() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
iwc.setCodec(getCodec());
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
Document doc = new Document();
doc.add(newStringField("", "something", Field.Store.NO));
iw.addDocument(doc);
DirectoryReader ir = iw.getReader();
AtomicReader ar = getOnlySegmentReader(ir);
Fields fields = ar.fields();
assertTrue(fields.size() == 1);
Terms terms = ar.terms("");
assertNotNull(terms);
TermsEnum termsEnum = terms.iterator(null);
assertNotNull(termsEnum.next());
assertEquals(termsEnum.term(), new BytesRef("something"));
assertNull(termsEnum.next());
ir.close();
iw.close();
dir.close();
}
public void testEmptyFieldAndEmptyTerm() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
iwc.setCodec(getCodec());
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
Document doc = new Document();
doc.add(newStringField("", "", Field.Store.NO));
iw.addDocument(doc);
DirectoryReader ir = iw.getReader();
AtomicReader ar = getOnlySegmentReader(ir);
Fields fields = ar.fields();
assertTrue(fields.size() == 1);
Terms terms = ar.terms("");
assertNotNull(terms);
TermsEnum termsEnum = terms.iterator(null);
assertNotNull(termsEnum.next());
assertEquals(termsEnum.term(), new BytesRef(""));
assertNull(termsEnum.next());
ir.close();
iw.close();
dir.close();
}
// tests that ghost fields still work
// TODO: can this be improved?
public void testGhosts() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
iwc.setCodec(getCodec());
iwc.setMergePolicy(newLogMergePolicy());
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
Document doc = new Document();
iw.addDocument(doc);
doc.add(newStringField("ghostField", "something", Field.Store.NO));
iw.addDocument(doc);
iw.forceMerge(1);
iw.deleteDocuments(new Term("ghostField", "something")); // delete the only term for the field
iw.forceMerge(1);
DirectoryReader ir = iw.getReader();
AtomicReader ar = getOnlySegmentReader(ir);
Fields fields = ar.fields();
// Ghost busting terms dict impls will have
// fields.size() == 0; all others must be == 1:
assertTrue(fields.size() <= 1);
Terms terms = fields.terms("ghostField");
if (terms != null) {
TermsEnum termsEnum = terms.iterator(null);
BytesRef term = termsEnum.next();
if (term != null) {
DocsEnum docsEnum = termsEnum.docs(null, null);
assertTrue(docsEnum.nextDoc() == DocsEnum.NO_MORE_DOCS);
}
}
ir.close();
iw.close();
dir.close();
}
}

View File

@ -33,6 +33,8 @@ import org.apache.lucene.codecs.asserting.AssertingDocValuesFormat;
import org.apache.lucene.codecs.asserting.AssertingPostingsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapDocFreqInterval;
import org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapFixedInterval;
import org.apache.lucene.codecs.lucene42.Lucene42Codec;
import org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat;
import org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings;
@ -137,7 +139,9 @@ public class RandomCodec extends Lucene42Codec {
new MockVariableIntBlockPostingsFormat( _TestUtil.nextInt(random, 1, 127)),
new MockRandomPostingsFormat(random),
new NestedPulsingPostingsFormat(),
new Lucene41WithOrds(),
new Lucene41WithOrds(_TestUtil.nextInt(random, 1, 1000)),
new Lucene41VarGapFixedInterval(_TestUtil.nextInt(random, 1, 1000)),
new Lucene41VarGapDocFreqInterval(_TestUtil.nextInt(random, 1, 100), _TestUtil.nextInt(random, 1, 1000)),
new SimpleTextPostingsFormat(),
new AssertingPostingsFormat(),
new MemoryPostingsFormat(true, random.nextFloat()),

View File

@ -308,7 +308,7 @@ public class RandomIndexWriter implements Closeable {
}
w.commit();
if (r.nextBoolean()) {
return DirectoryReader.open(w.getDirectory(), _TestUtil.nextInt(r, 1, 10));
return DirectoryReader.open(w.getDirectory());
} else {
return w.getReader(applyDeletions);
}

View File

@ -753,15 +753,6 @@ public abstract class LuceneTestCase extends Assert {
c.setMaxBufferedDocs(_TestUtil.nextInt(r, 16, 1000));
}
}
if (r.nextBoolean()) {
if (rarely(r)) {
// crazy value
c.setTermIndexInterval(r.nextBoolean() ? _TestUtil.nextInt(r, 1, 31) : _TestUtil.nextInt(r, 129, 1000));
} else {
// reasonable value
c.setTermIndexInterval(_TestUtil.nextInt(r, 32, 128));
}
}
if (r.nextBoolean()) {
int maxNumThreadStates = rarely(r) ? _TestUtil.nextInt(r, 5, 20) // crazy value
: _TestUtil.nextInt(r, 1, 4); // reasonable value
@ -816,7 +807,6 @@ public abstract class LuceneTestCase extends Assert {
}
c.setUseCompoundFile(r.nextBoolean());
c.setReaderPooling(r.nextBoolean());
c.setReaderTermsIndexDivisor(_TestUtil.nextInt(r, 1, 4));
return c;
}

View File

@ -20,6 +20,8 @@ org.apache.lucene.codecs.mocksep.MockSepPostingsFormat
org.apache.lucene.codecs.nestedpulsing.NestedPulsingPostingsFormat
org.apache.lucene.codecs.ramonly.RAMOnlyPostingsFormat
org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds
org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapFixedInterval
org.apache.lucene.codecs.lucene41vargap.Lucene41VarGapDocFreqInterval
org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings
org.apache.lucene.codecs.asserting.AssertingPostingsFormat
org.apache.lucene.codecs.lucene40.Lucene40RWPostingsFormat

View File

@ -27,12 +27,8 @@ import org.apache.solr.util.plugin.NamedListInitializedPlugin;
* Factory used to build a new IndexReader instance.
*/
public abstract class IndexReaderFactory implements NamedListInitializedPlugin {
protected int termInfosIndexDivisor = 1;//IndexReader.DEFAULT_TERMS_INDEX_DIVISOR; Set this once Lucene makes this public.
/**
* Potentially initializes {@link #termInfosIndexDivisor}. Overriding classes should call super.init() in order
* to make sure termInfosIndexDivisor is set.
* <p>
* <code>init</code> will be called just once, immediately after creation.
* init will be called just once, immediately after creation.
* <p>
* The args are user-level initialization parameters that may be specified
* when declaring an indexReaderFactory in solrconfig.xml
@ -40,18 +36,10 @@ public abstract class IndexReaderFactory implements NamedListInitializedPlugin {
*/
@Override
public void init(NamedList args) {
Integer v = (Integer)args.get("setTermIndexDivisor");
if (v != null) {
termInfosIndexDivisor = v.intValue();
}
}
/**
*
* @return The setting of {@link #termInfosIndexDivisor}
*/
public int getTermInfosIndexDivisor() {
return termInfosIndexDivisor;
Object v = args.get("setTermIndexDivisor");
if (v != null) {
throw new IllegalArgumentException("Illegal parameter 'setTermIndexDivisor'");
}
}
/**

View File

@ -31,6 +31,6 @@ public class StandardIndexReaderFactory extends IndexReaderFactory {
@Override
public DirectoryReader newReader(Directory indexDir, SolrCore core) throws IOException {
return DirectoryReader.open(indexDir, termInfosIndexDivisor);
return DirectoryReader.open(indexDir);
}
}

View File

@ -69,7 +69,6 @@ public class SolrIndexConfig {
public final String lockType;
public final PluginInfo mergePolicyInfo;
public final PluginInfo mergeSchedulerInfo;
public final int termIndexInterval;
public final PluginInfo mergedSegmentWarmerInfo;
@ -95,7 +94,6 @@ public class SolrIndexConfig {
ramBufferSizeMB = 100;
writeLockTimeout = -1;
lockType = LOCK_TYPE_NATIVE;
termIndexInterval = IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL;
mergePolicyInfo = null;
mergeSchedulerInfo = null;
defaultMergePolicyClassName = TieredMergePolicy.class.getName();
@ -148,7 +146,10 @@ public class SolrIndexConfig {
mergeSchedulerInfo = getPluginInfo(prefix + "/mergeScheduler", solrConfig, def.mergeSchedulerInfo);
mergePolicyInfo = getPluginInfo(prefix + "/mergePolicy", solrConfig, def.mergePolicyInfo);
termIndexInterval = solrConfig.getInt(prefix + "/termIndexInterval", def.termIndexInterval);
String val = solrConfig.get(prefix + "/termIndexInterval", null);
if (val != null) {
throw new IllegalArgumentException("Illegal parameter 'termIndexInterval'");
}
boolean infoStreamEnabled = solrConfig.getBool(prefix + "/infoStream", false);
if(infoStreamEnabled) {
@ -198,9 +199,6 @@ public class SolrIndexConfig {
if (ramBufferSizeMB != -1)
iwc.setRAMBufferSizeMB(ramBufferSizeMB);
if (termIndexInterval != -1)
iwc.setTermIndexInterval(termIndexInterval);
if (writeLockTimeout != -1)
iwc.setWriteLockTimeout(writeLockTimeout);

View File

@ -32,7 +32,6 @@
<indexConfig>
<useCompoundFile>${useCompoundFile:false}</useCompoundFile>
<termIndexInterval>256</termIndexInterval>
<mergePolicy class="org.apache.lucene.index.TieredMergePolicy"/>
<mergeScheduler class="org.apache.lucene.index.ConcurrentMergeScheduler"/>
</indexConfig>
@ -40,9 +39,7 @@
<updateHandler class="solr.DirectUpdateHandler2">
</updateHandler>
<indexReaderFactory name="IndexReaderFactory" class="org.apache.solr.core.StandardIndexReaderFactory">
<int name="setTermIndexDivisor">12</int>
</indexReaderFactory >
<indexReaderFactory name="IndexReaderFactory" class="org.apache.solr.core.StandardIndexReaderFactory"/>
<requestHandler name="standard" class="solr.StandardRequestHandler"></requestHandler>

View File

@ -33,8 +33,5 @@ public class IndexReaderFactoryTest extends AbstractSolrTestCase {
IndexReaderFactory readerFactory = h.getCore().getIndexReaderFactory();
assertNotNull("Factory is null", readerFactory);
assertTrue("readerFactory is not an instanceof " + AlternateDirectoryTest.TestIndexReaderFactory.class, readerFactory instanceof StandardIndexReaderFactory);
assertTrue("termInfoIndexDivisor not set to 12", readerFactory.getTermInfosIndexDivisor() == 12);
}
}

View File

@ -114,27 +114,6 @@ public class TestConfig extends SolrTestCaseJ4 {
assertTrue(handler.getHiddenFiles().contains("PROTWORDS.TXT"));
}
@Test
public void testTermIndexInterval() throws Exception {
RefCounted<IndexWriter> iw = ((DirectUpdateHandler2) h.getCore()
.getUpdateHandler()).getSolrCoreState().getIndexWriter(h.getCore());
int interval = 0;
try {
IndexWriter writer = iw.get();
interval = writer.getConfig().getTermIndexInterval();
} finally {
iw.decref();
}
assertEquals(256, interval);
}
@Test
public void testTermIndexDivisor() throws Exception {
IndexReaderFactory irf = h.getCore().getIndexReaderFactory();
StandardIndexReaderFactory sirf = (StandardIndexReaderFactory) irf;
assertEquals(12, sirf.termInfosIndexDivisor);
}
// If defaults change, add test methods to cover each version
@Test
public void testDefaults() throws Exception {

View File

@ -257,11 +257,6 @@
<unlockOnStartup>false</unlockOnStartup>
-->
<!-- Expert: Controls how often Lucene loads terms into memory
Default is 128 and is likely good for most everyone.
-->
<!-- <termIndexInterval>128</termIndexInterval> -->
<!-- If true, IndexReaders will be reopened (often more efficient)
instead of closed and then opened. Default: true
-->
@ -440,15 +435,6 @@
<str name="someArg">Some Value</str>
</indexReaderFactory >
-->
<!-- By explicitly declaring the Factory, the termIndexDivisor can
be specified.
-->
<!--
<indexReaderFactory name="IndexReaderFactory"
class="solr.StandardIndexReaderFactory">
<int name="setTermIndexDivisor">12</int>
</indexReaderFactory >
-->
<!-- ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
Query section - these settings control query time things like caches

View File

@ -257,11 +257,6 @@
<!--
<unlockOnStartup>false</unlockOnStartup>
-->
<!-- Expert: Controls how often Lucene loads terms into memory
Default is 128 and is likely good for most everyone.
-->
<!-- <termIndexInterval>128</termIndexInterval> -->
<!-- If true, IndexReaders will be reopened (often more efficient)
instead of closed and then opened. Default: true
@ -441,15 +436,6 @@
<str name="someArg">Some Value</str>
</indexReaderFactory >
-->
<!-- By explicitly declaring the Factory, the termIndexDivisor can
be specified.
-->
<!--
<indexReaderFactory name="IndexReaderFactory"
class="solr.StandardIndexReaderFactory">
<int name="setTermIndexDivisor">12</int>
</indexReaderFactory >
-->
<!-- ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
Query section - these settings control query time things like caches