mirror of https://github.com/apache/lucene.git
LUCENE-4446: wrap up cutover to blockpostingsformat
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene4446@1397858 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
b1a58d9a9c
commit
2ea2adcf6b
|
@ -27,7 +27,7 @@ import java.util.TreeMap;
|
|||
import org.apache.lucene.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat; // javadocs
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; // javadocs
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
|
@ -52,7 +52,7 @@ import org.apache.lucene.util.automaton.Transition;
|
|||
// - build depth-N prefix hash?
|
||||
// - or: longer dense skip lists than just next byte?
|
||||
|
||||
/** Wraps {@link Lucene40PostingsFormat} format for on-disk
|
||||
/** Wraps {@link Lucene41PostingsFormat} format for on-disk
|
||||
* storage, but then at read time loads and stores all
|
||||
* terms & postings directly in RAM as byte[], int[].
|
||||
*
|
||||
|
@ -100,12 +100,12 @@ public final class DirectPostingsFormat extends PostingsFormat {
|
|||
|
||||
@Override
|
||||
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
return PostingsFormat.forName("Lucene40").fieldsConsumer(state);
|
||||
return PostingsFormat.forName("Lucene41").fieldsConsumer(state);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
|
||||
FieldsProducer postings = PostingsFormat.forName("Lucene40").fieldsProducer(state);
|
||||
FieldsProducer postings = PostingsFormat.forName("Lucene41").fieldsProducer(state);
|
||||
if (state.context.context != IOContext.Context.MERGE) {
|
||||
FieldsProducer loadedPostings;
|
||||
try {
|
||||
|
|
|
@ -28,17 +28,17 @@ import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; // javadocs
|
|||
*/
|
||||
public class Pulsing41PostingsFormat extends PulsingPostingsFormat {
|
||||
|
||||
/** Inlines docFreq=1 terms, otherwise uses the normal "Lucene40" format. */
|
||||
/** Inlines docFreq=1 terms, otherwise uses the normal "Lucene41" format. */
|
||||
public Pulsing41PostingsFormat() {
|
||||
this(1);
|
||||
}
|
||||
|
||||
/** Inlines docFreq=<code>freqCutoff</code> terms, otherwise uses the normal "Lucene40" format. */
|
||||
/** Inlines docFreq=<code>freqCutoff</code> terms, otherwise uses the normal "Lucene41" format. */
|
||||
public Pulsing41PostingsFormat(int freqCutoff) {
|
||||
this(freqCutoff, BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
|
||||
}
|
||||
|
||||
/** Inlines docFreq=<code>freqCutoff</code> terms, otherwise uses the normal "Lucene40" format. */
|
||||
/** Inlines docFreq=<code>freqCutoff</code> terms, otherwise uses the normal "Lucene41" format. */
|
||||
public Pulsing41PostingsFormat(int freqCutoff, int minBlockSize, int maxBlockSize) {
|
||||
super("Pulsing41", new Lucene41PostingsBaseFormat(), freqCutoff, minBlockSize, maxBlockSize);
|
||||
}
|
||||
|
|
|
@ -119,7 +119,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
|
|||
loader.reload(classloader);
|
||||
}
|
||||
|
||||
private static Codec defaultCodec = Codec.forName("Lucene40");
|
||||
private static Codec defaultCodec = Codec.forName("Lucene41");
|
||||
|
||||
/** expert: returns the default codec used for newly created
|
||||
* {@link IndexWriterConfig}s.
|
||||
|
|
|
@ -21,7 +21,7 @@ package org.apache.lucene.codecs;
|
|||
* A codec that forwards all its method calls to another codec.
|
||||
* <p>
|
||||
* Extend this class when you need to reuse the functionality of an existing
|
||||
* codec. For example, if you want to build a codec that redefines Lucene40's
|
||||
* codec. For example, if you want to build a codec that redefines Lucene41's
|
||||
* {@link LiveDocsFormat}:
|
||||
* <pre class="prettyprint">
|
||||
* public final class CustomCodec extends FilterCodec {
|
||||
|
|
|
@ -36,12 +36,12 @@ import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
|||
* {@link FilterCodec}.
|
||||
*
|
||||
* @see org.apache.lucene.codecs.lucene40 package documentation for file format details.
|
||||
* @lucene.experimental
|
||||
* @deprecated Only for reading old 4.0 segments
|
||||
*/
|
||||
// NOTE: if we make largish changes in a minor release, easier to just make Lucene42Codec or whatever
|
||||
// if they are backwards compatible or smallish we can probably do the backwards in the postingsreader
|
||||
// (it writes a minor version, etc).
|
||||
// nocommit: make readonly and add impersonator
|
||||
@Deprecated
|
||||
public final class Lucene40Codec extends Codec {
|
||||
private final StoredFieldsFormat fieldsFormat = new Lucene40StoredFieldsFormat();
|
||||
private final TermVectorsFormat vectorsFormat = new Lucene40TermVectorsFormat();
|
||||
|
|
|
@ -29,9 +29,10 @@ import org.apache.lucene.index.SegmentWriteState;
|
|||
* Provides a {@link PostingsReaderBase} and {@link
|
||||
* PostingsWriterBase}.
|
||||
*
|
||||
* @lucene.experimental */
|
||||
* @deprecated Only for reading old 4.0 segments */
|
||||
|
||||
// TODO: should these also be named / looked up via SPI?
|
||||
@Deprecated
|
||||
public final class Lucene40PostingsBaseFormat extends PostingsBaseFormat {
|
||||
|
||||
/** Sole constructor. */
|
||||
|
@ -46,6 +47,6 @@ public final class Lucene40PostingsBaseFormat extends PostingsBaseFormat {
|
|||
|
||||
@Override
|
||||
public PostingsWriterBase postingsWriterBase(SegmentWriteState state) throws IOException {
|
||||
return new Lucene40PostingsWriter(state);
|
||||
throw new UnsupportedOperationException("this codec can only be used for reading");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -211,15 +211,18 @@ import org.apache.lucene.util.fst.FST; // javadocs
|
|||
* previous occurrence and an OffsetLength follows. Offset data is only written for
|
||||
* {@link IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS}.</p>
|
||||
*
|
||||
* @lucene.experimental */
|
||||
* @deprecated Only for reading old 4.0 segments */
|
||||
|
||||
// TODO: this class could be created by wrapping
|
||||
// BlockTreeTermsDict around Lucene40PostingsBaseFormat; ie
|
||||
// we should not duplicate the code from that class here:
|
||||
public final class Lucene40PostingsFormat extends PostingsFormat {
|
||||
@Deprecated
|
||||
public class Lucene40PostingsFormat extends PostingsFormat {
|
||||
|
||||
private final int minBlockSize;
|
||||
private final int maxBlockSize;
|
||||
/** minimum items (terms or sub-blocks) per block for BlockTree */
|
||||
protected final int minBlockSize;
|
||||
/** maximum items (terms or sub-blocks) per block for BlockTree */
|
||||
protected final int maxBlockSize;
|
||||
|
||||
/** Creates {@code Lucene40PostingsFormat} with default
|
||||
* settings. */
|
||||
|
@ -231,7 +234,7 @@ public final class Lucene40PostingsFormat extends PostingsFormat {
|
|||
* values for {@code minBlockSize} and {@code
|
||||
* maxBlockSize} passed to block terms dictionary.
|
||||
* @see BlockTreeTermsWriter#BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int) */
|
||||
public Lucene40PostingsFormat(int minBlockSize, int maxBlockSize) {
|
||||
private Lucene40PostingsFormat(int minBlockSize, int maxBlockSize) {
|
||||
super("Lucene40");
|
||||
this.minBlockSize = minBlockSize;
|
||||
assert minBlockSize > 1;
|
||||
|
@ -240,22 +243,7 @@ public final class Lucene40PostingsFormat extends PostingsFormat {
|
|||
|
||||
@Override
|
||||
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
PostingsWriterBase docs = new Lucene40PostingsWriter(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?
|
||||
boolean success = false;
|
||||
try {
|
||||
FieldsConsumer ret = new BlockTreeTermsWriter(state, docs, minBlockSize, maxBlockSize);
|
||||
success = true;
|
||||
return ret;
|
||||
} finally {
|
||||
if (!success) {
|
||||
docs.close();
|
||||
}
|
||||
}
|
||||
throw new UnsupportedOperationException("this codec can only be used for reading");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -45,10 +45,21 @@ import org.apache.lucene.util.IOUtils;
|
|||
* postings format.
|
||||
*
|
||||
* @see Lucene40PostingsFormat
|
||||
* @lucene.experimental */
|
||||
|
||||
* @deprecated Only for reading old 4.0 segments */
|
||||
@Deprecated
|
||||
public class Lucene40PostingsReader extends PostingsReaderBase {
|
||||
|
||||
final static String TERMS_CODEC = "Lucene40PostingsWriterTerms";
|
||||
final static String FRQ_CODEC = "Lucene40PostingsWriterFrq";
|
||||
final static String PRX_CODEC = "Lucene40PostingsWriterPrx";
|
||||
|
||||
//private static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
|
||||
|
||||
// Increment version to change it:
|
||||
final static int VERSION_START = 0;
|
||||
final static int VERSION_LONG_SKIP = 1;
|
||||
final static int VERSION_CURRENT = VERSION_LONG_SKIP;
|
||||
|
||||
private final IndexInput freqIn;
|
||||
private final IndexInput proxIn;
|
||||
// public static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
|
||||
|
@ -67,7 +78,7 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
|
|||
try {
|
||||
freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.FREQ_EXTENSION),
|
||||
ioContext);
|
||||
CodecUtil.checkHeader(freqIn, Lucene40PostingsWriter.FRQ_CODEC, Lucene40PostingsWriter.VERSION_START,Lucene40PostingsWriter.VERSION_CURRENT);
|
||||
CodecUtil.checkHeader(freqIn, FRQ_CODEC, VERSION_START, VERSION_CURRENT);
|
||||
// TODO: hasProx should (somehow!) become codec private,
|
||||
// but it's tricky because 1) FIS.hasProx is global (it
|
||||
// could be all fields that have prox are written by a
|
||||
|
@ -79,7 +90,7 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
|
|||
if (fieldInfos.hasProx()) {
|
||||
proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.PROX_EXTENSION),
|
||||
ioContext);
|
||||
CodecUtil.checkHeader(proxIn, Lucene40PostingsWriter.PRX_CODEC, Lucene40PostingsWriter.VERSION_START,Lucene40PostingsWriter.VERSION_CURRENT);
|
||||
CodecUtil.checkHeader(proxIn, PRX_CODEC, VERSION_START, VERSION_CURRENT);
|
||||
} else {
|
||||
proxIn = null;
|
||||
}
|
||||
|
@ -97,8 +108,7 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
|
|||
public void init(IndexInput termsIn) throws IOException {
|
||||
|
||||
// Make sure we are talking to the matching past writer
|
||||
CodecUtil.checkHeader(termsIn, Lucene40PostingsWriter.TERMS_CODEC,
|
||||
Lucene40PostingsWriter.VERSION_START, Lucene40PostingsWriter.VERSION_CURRENT);
|
||||
CodecUtil.checkHeader(termsIn, TERMS_CODEC, VERSION_START, VERSION_CURRENT);
|
||||
|
||||
skipInterval = termsIn.readInt();
|
||||
maxSkipLevels = termsIn.readInt();
|
||||
|
|
|
@ -28,8 +28,9 @@ import org.apache.lucene.store.IndexInput;
|
|||
* that stores positions and payloads.
|
||||
*
|
||||
* @see Lucene40PostingsFormat
|
||||
* @lucene.experimental
|
||||
* @deprecated Only for reading old 4.0 segments
|
||||
*/
|
||||
@Deprecated
|
||||
public class Lucene40SkipListReader extends MultiLevelSkipListReader {
|
||||
private boolean currentFieldStoresPayloads;
|
||||
private boolean currentFieldStoresOffsets;
|
||||
|
|
|
@ -380,10 +380,16 @@ public final class Lucene41PostingsFormat extends PostingsFormat {
|
|||
// NOTE: must be multiple of 64 because of PackedInts long-aligned encoding/decoding
|
||||
public final static int BLOCK_SIZE = 128;
|
||||
|
||||
/** Creates {@code Lucene41PostingsFormat} with default
|
||||
* settings. */
|
||||
public Lucene41PostingsFormat() {
|
||||
this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
|
||||
}
|
||||
|
||||
/** Creates {@code Lucene41PostingsFormat} with custom
|
||||
* values for {@code minBlockSize} and {@code
|
||||
* maxBlockSize} passed to block terms dictionary.
|
||||
* @see BlockTreeTermsWriter#BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int) */
|
||||
public Lucene41PostingsFormat(int minTermBlockSize, int maxTermBlockSize) {
|
||||
super("Lucene41");
|
||||
this.minTermBlockSize = minTermBlockSize;
|
||||
|
|
|
@ -63,10 +63,9 @@ public final class Lucene41PostingsWriter extends PostingsWriterBase {
|
|||
final static String POS_CODEC = "Lucene41PostingsWriterPos";
|
||||
final static String PAY_CODEC = "Lucene41PostingsWriterPay";
|
||||
|
||||
// Increment version to change it: nocommit: we can start at 0
|
||||
// Increment version to change it
|
||||
final static int VERSION_START = 0;
|
||||
final static int VERSION_NO_OFFSETS_IN_SKIPDATA = 1; // LUCENE-4443
|
||||
final static int VERSION_CURRENT = VERSION_NO_OFFSETS_IN_SKIPDATA;
|
||||
final static int VERSION_CURRENT = VERSION_START;
|
||||
|
||||
final IndexOutput docOut;
|
||||
final IndexOutput posOut;
|
||||
|
|
|
@ -153,7 +153,7 @@ its title, url, or an identifier to access a database. The set of stored fields
|
|||
returned for each hit when searching. This is keyed by document number.
|
||||
</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.codecs.lucene41Lucene41PostingsFormat Term dictionary}.
|
||||
{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term dictionary}.
|
||||
A dictionary containing all of the terms used in all of the
|
||||
indexed fields of all of the documents. The dictionary also contains the number
|
||||
of documents which contain the term, and pointers to the term's frequency and
|
||||
|
|
|
@ -61,8 +61,8 @@ name of your codec.
|
|||
If you just want to customise the {@link org.apache.lucene.codecs.PostingsFormat}, or use different postings
|
||||
formats for different fields, then you can register your custom postings format in the same way (in
|
||||
META-INF/services/org.apache.lucene.codecs.PostingsFormat), and then extend the default
|
||||
{@link org.apache.lucene.codecs.lucene40.Lucene40Codec} and override
|
||||
{@link org.apache.lucene.codecs.lucene40.Lucene40Codec#getPostingsFormatForField(String)} to return your custom
|
||||
{@link org.apache.lucene.codecs.lucene41.Lucene41Codec} and override
|
||||
{@link org.apache.lucene.codecs.lucene41.Lucene41Codec#getPostingsFormatForField(String)} to return your custom
|
||||
postings format.
|
||||
</p>
|
||||
</body>
|
||||
|
|
|
@ -191,7 +191,7 @@ public class LiveIndexWriterConfig {
|
|||
* 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 Lucene40PostingsFormat, passing minBlockSize=50, maxBlockSize=100
|
||||
* //customize Lucene41PostingsFormat, passing minBlockSize=50, maxBlockSize=100
|
||||
* final PostingsFormat tweakedPostings = new Lucene41PostingsFormat(50, 100);
|
||||
* iwc.setCodec(new Lucene41Codec() {
|
||||
* @Override
|
||||
|
|
|
@ -38,12 +38,12 @@ import org.apache.lucene.util.LineFileDocs;
|
|||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util._TestUtil;
|
||||
|
||||
// nocommit: really this should be in BaseTestPF or somewhere else? useful test!
|
||||
// TODO: really this should be in BaseTestPF or somewhere else? useful test!
|
||||
public class TestReuseDocsEnum extends LuceneTestCase {
|
||||
|
||||
public void testReuseDocsEnumNoReuse() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
Codec cp = _TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat());
|
||||
Codec cp = _TestUtil.alwaysPostingsFormat(new Lucene40RWPostingsFormat());
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir,
|
||||
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setCodec(cp));
|
||||
int numdocs = atLeast(20);
|
||||
|
@ -70,7 +70,7 @@ public class TestReuseDocsEnum extends LuceneTestCase {
|
|||
// tests for reuse only if bits are the same either null or the same instance
|
||||
public void testReuseDocsEnumSameBitsOrNull() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
Codec cp = _TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat());
|
||||
Codec cp = _TestUtil.alwaysPostingsFormat(new Lucene40RWPostingsFormat());
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir,
|
||||
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setCodec(cp));
|
||||
int numdocs = atLeast(20);
|
||||
|
@ -114,7 +114,7 @@ public class TestReuseDocsEnum extends LuceneTestCase {
|
|||
// make sure we never reuse from another reader even if it is the same field & codec etc
|
||||
public void testReuseDocsEnumDifferentReader() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
Codec cp = _TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat());
|
||||
Codec cp = _TestUtil.alwaysPostingsFormat(new Lucene40RWPostingsFormat());
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir,
|
||||
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setCodec(cp));
|
||||
int numdocs = atLeast(20);
|
||||
|
|
|
@ -45,16 +45,6 @@ import org.apache.lucene.util.IOUtils;
|
|||
* @lucene.experimental
|
||||
*/
|
||||
public final class Lucene40PostingsWriter extends PostingsWriterBase {
|
||||
final static String TERMS_CODEC = "Lucene40PostingsWriterTerms";
|
||||
final static String FRQ_CODEC = "Lucene40PostingsWriterFrq";
|
||||
final static String PRX_CODEC = "Lucene40PostingsWriterPrx";
|
||||
|
||||
//private static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
|
||||
|
||||
// Increment version to change it:
|
||||
final static int VERSION_START = 0;
|
||||
final static int VERSION_LONG_SKIP = 1;
|
||||
final static int VERSION_CURRENT = VERSION_LONG_SKIP;
|
||||
|
||||
final IndexOutput freqOut;
|
||||
final IndexOutput proxOut;
|
||||
|
@ -111,7 +101,7 @@ public final class Lucene40PostingsWriter extends PostingsWriterBase {
|
|||
boolean success = false;
|
||||
IndexOutput proxOut = null;
|
||||
try {
|
||||
CodecUtil.writeHeader(freqOut, FRQ_CODEC, VERSION_CURRENT);
|
||||
CodecUtil.writeHeader(freqOut, Lucene40PostingsReader.FRQ_CODEC, Lucene40PostingsReader.VERSION_CURRENT);
|
||||
// TODO: this is a best effort, if one of these fields has no postings
|
||||
// then we make an empty prx file, same as if we are wrapped in
|
||||
// per-field postingsformat. maybe... we shouldn't
|
||||
|
@ -121,7 +111,7 @@ public final class Lucene40PostingsWriter extends PostingsWriterBase {
|
|||
// prox file
|
||||
fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene40PostingsFormat.PROX_EXTENSION);
|
||||
proxOut = state.directory.createOutput(fileName, state.context);
|
||||
CodecUtil.writeHeader(proxOut, PRX_CODEC, VERSION_CURRENT);
|
||||
CodecUtil.writeHeader(proxOut, Lucene40PostingsReader.PRX_CODEC, Lucene40PostingsReader.VERSION_CURRENT);
|
||||
} else {
|
||||
// Every field omits TF so we will write no prox file
|
||||
proxOut = null;
|
||||
|
@ -146,7 +136,7 @@ public final class Lucene40PostingsWriter extends PostingsWriterBase {
|
|||
@Override
|
||||
public void start(IndexOutput termsOut) throws IOException {
|
||||
this.termsOut = termsOut;
|
||||
CodecUtil.writeHeader(termsOut, TERMS_CODEC, VERSION_CURRENT);
|
||||
CodecUtil.writeHeader(termsOut, Lucene40PostingsReader.TERMS_CODEC, Lucene40PostingsReader.VERSION_CURRENT);
|
||||
termsOut.writeInt(skipInterval); // write skipInterval
|
||||
termsOut.writeInt(maxSkipLevels); // write maxSkipLevels
|
||||
termsOut.writeInt(skipMinimum); // write skipMinimum
|
|
@ -0,0 +1,50 @@
|
|||
package org.apache.lucene.codecs.lucene40;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.BlockTreeTermsWriter;
|
||||
import org.apache.lucene.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.codecs.PostingsWriterBase;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Read-write version of {@link Lucene40PostingsFormat} for testing.
|
||||
*/
|
||||
public class Lucene40RWPostingsFormat extends Lucene40PostingsFormat {
|
||||
@Override
|
||||
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
PostingsWriterBase docs = new Lucene40PostingsWriter(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?
|
||||
boolean success = false;
|
||||
try {
|
||||
FieldsConsumer ret = new BlockTreeTermsWriter(state, docs, minBlockSize, maxBlockSize);
|
||||
success = true;
|
||||
return ret;
|
||||
} finally {
|
||||
if (!success) {
|
||||
docs.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -29,8 +29,9 @@ import org.apache.lucene.codecs.MultiLevelSkipListWriter;
|
|||
* that stores positions and payloads.
|
||||
*
|
||||
* @see Lucene40PostingsFormat
|
||||
* @lucene.experimental
|
||||
* @deprecated Only for reading old 4.0 segments
|
||||
*/
|
||||
@Deprecated
|
||||
public class Lucene40SkipListWriter extends MultiLevelSkipListWriter {
|
||||
private int[] lastSkipDoc;
|
||||
private int[] lastSkipPayloadLength;
|
|
@ -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>
|
||||
Support for testing {@link org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat}.
|
||||
</body>
|
||||
</html>
|
|
@ -38,8 +38,7 @@ 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.lucene40.Lucene40PostingsReader;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40PostingsWriter;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
|
||||
import org.apache.lucene.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
|
||||
import org.apache.lucene.codecs.mockintblock.MockVariableIntBlockPostingsFormat;
|
||||
|
@ -175,8 +174,8 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
|
|||
if (LuceneTestCase.VERBOSE) {
|
||||
System.out.println("MockRandomCodec: writing Standard postings");
|
||||
}
|
||||
// nocommit: way to randomize skipInterval and acceptibleOverHead?!
|
||||
postingsWriter = new Lucene40PostingsWriter(state, skipInterval);
|
||||
// TODO: randomize variables like acceptibleOverHead?!
|
||||
postingsWriter = new Lucene41PostingsWriter(state, skipInterval);
|
||||
}
|
||||
|
||||
if (random.nextBoolean()) {
|
||||
|
@ -315,8 +314,7 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
|
|||
if (LuceneTestCase.VERBOSE) {
|
||||
System.out.println("MockRandomCodec: reading Standard postings");
|
||||
}
|
||||
// nocommit
|
||||
postingsReader = new Lucene40PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
|
||||
postingsReader = new Lucene41PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
|
||||
}
|
||||
|
||||
if (random.nextBoolean()) {
|
||||
|
|
|
@ -99,7 +99,7 @@ public class RandomCodec extends Lucene41Codec {
|
|||
new Pulsing41PostingsFormat(1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock),
|
||||
// add pulsing again with (usually) different parameters
|
||||
new Pulsing41PostingsFormat(1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock),
|
||||
//TODO as a PostingsFormat which wraps others, we should allow TestBloomFilteredLucene40Postings to be constructed
|
||||
//TODO as a PostingsFormat which wraps others, we should allow TestBloomFilteredLucene41Postings to be constructed
|
||||
//with a choice of concrete PostingsFormats. Maybe useful to have a generic means of marking and dealing
|
||||
//with such "wrapper" classes?
|
||||
new TestBloomFilteredLucene41Postings(),
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.lucene.codecs.PostingsFormat;
|
|||
import org.apache.lucene.codecs.asserting.AssertingCodec;
|
||||
import org.apache.lucene.codecs.compressing.CompressingCodec;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40RWPostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41Codec;
|
||||
import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
|
||||
import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
|
||||
|
@ -133,13 +134,12 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule {
|
|||
savedCodec = Codec.getDefault();
|
||||
int randomVal = random.nextInt(10);
|
||||
|
||||
|
||||
if ("Lucene40".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) &&
|
||||
"random".equals(TEST_POSTINGSFORMAT) &&
|
||||
randomVal < 2 &&
|
||||
!shouldAvoidCodec("Lucene40"))) {
|
||||
codec = Codec.forName("Lucene40");
|
||||
// nocommit: assert (codec instanceof PreFlexRWCodec) : "fix your classpath to have tests-framework.jar before lucene-core.jar";
|
||||
assert (PostingsFormat.forName("Lucene40") instanceof Lucene40RWPostingsFormat) : "fix your classpath to have tests-framework.jar before lucene-core.jar";
|
||||
} else if (!"random".equals(TEST_POSTINGSFORMAT)) {
|
||||
final PostingsFormat format;
|
||||
if ("MockRandom".equals(TEST_POSTINGSFORMAT)) {
|
||||
|
|
|
@ -22,4 +22,4 @@ org.apache.lucene.codecs.ramonly.RAMOnlyPostingsFormat
|
|||
org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds
|
||||
org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings
|
||||
org.apache.lucene.codecs.asserting.AssertingPostingsFormat
|
||||
|
||||
org.apache.lucene.codecs.lucene40.Lucene40RWPostingsFormat
|
||||
|
|
|
@ -17,9 +17,9 @@
|
|||
-->
|
||||
<schema name="codec" version="1.2">
|
||||
<types>
|
||||
<fieldType name="string_pulsing" class="solr.StrField" postingsFormat="Pulsing40"/>
|
||||
<fieldType name="string_pulsing" class="solr.StrField" postingsFormat="Pulsing41"/>
|
||||
<fieldType name="string_simpletext" class="solr.StrField" postingsFormat="SimpleText"/>
|
||||
<fieldType name="string_standard" class="solr.StrField" postingsFormat="Lucene40"/>
|
||||
<fieldType name="string_standard" class="solr.StrField" postingsFormat="Lucene41"/>
|
||||
<fieldType name="string" class="solr.StrField" />
|
||||
|
||||
</types>
|
||||
|
|
|
@ -37,14 +37,14 @@ public class TestCodecSupport extends SolrTestCaseJ4 {
|
|||
Map<String, SchemaField> fields = h.getCore().getSchema().getFields();
|
||||
SchemaField schemaField = fields.get("string_pulsing_f");
|
||||
PerFieldPostingsFormat format = (PerFieldPostingsFormat) codec.postingsFormat();
|
||||
assertEquals("Pulsing40", format.getPostingsFormatForField(schemaField.getName()).getName());
|
||||
assertEquals("Pulsing41", format.getPostingsFormatForField(schemaField.getName()).getName());
|
||||
schemaField = fields.get("string_simpletext_f");
|
||||
assertEquals("SimpleText",
|
||||
format.getPostingsFormatForField(schemaField.getName()).getName());
|
||||
schemaField = fields.get("string_standard_f");
|
||||
assertEquals("Lucene40", format.getPostingsFormatForField(schemaField.getName()).getName());
|
||||
assertEquals("Lucene41", format.getPostingsFormatForField(schemaField.getName()).getName());
|
||||
schemaField = fields.get("string_f");
|
||||
assertEquals("Lucene40", format.getPostingsFormatForField(schemaField.getName()).getName());
|
||||
assertEquals("Lucene41", format.getPostingsFormatForField(schemaField.getName()).getName());
|
||||
}
|
||||
|
||||
public void testDynamicFields() {
|
||||
|
@ -53,10 +53,10 @@ public class TestCodecSupport extends SolrTestCaseJ4 {
|
|||
|
||||
assertEquals("SimpleText", format.getPostingsFormatForField("foo_simple").getName());
|
||||
assertEquals("SimpleText", format.getPostingsFormatForField("bar_simple").getName());
|
||||
assertEquals("Pulsing40", format.getPostingsFormatForField("foo_pulsing").getName());
|
||||
assertEquals("Pulsing40", format.getPostingsFormatForField("bar_pulsing").getName());
|
||||
assertEquals("Lucene40", format.getPostingsFormatForField("foo_standard").getName());
|
||||
assertEquals("Lucene40", format.getPostingsFormatForField("bar_standard").getName());
|
||||
assertEquals("Pulsing41", format.getPostingsFormatForField("foo_pulsing").getName());
|
||||
assertEquals("Pulsing41", format.getPostingsFormatForField("bar_pulsing").getName());
|
||||
assertEquals("Lucene41", format.getPostingsFormatForField("foo_standard").getName());
|
||||
assertEquals("Lucene41", format.getPostingsFormatForField("bar_standard").getName());
|
||||
}
|
||||
|
||||
public void testUnknownField() {
|
||||
|
|
Loading…
Reference in New Issue