LUCENE-4446: quick stab at a start... I think core tests pass but all else is TODO/untested

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene4446@1397416 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Muir 2012-10-12 02:00:19 +00:00
parent b6f7024afb
commit 54ff47eff0
40 changed files with 771 additions and 240 deletions

View File

@ -1,25 +0,0 @@
<!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>
BlockPostingsFormat file format.
</body>
</html>

View File

@ -18,28 +18,28 @@ package org.apache.lucene.codecs.pulsing;
*/
import org.apache.lucene.codecs.BlockTreeTermsWriter;
import org.apache.lucene.codecs.lucene40.Lucene40PostingsBaseFormat;
import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat; // javadocs
import org.apache.lucene.codecs.lucene41.Lucene41PostingsBaseFormat;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; // javadocs
/**
* Concrete pulsing implementation over {@link Lucene40PostingsFormat}.
* Concrete pulsing implementation over {@link Lucene41PostingsFormat}.
*
* @lucene.experimental
*/
public class Pulsing40PostingsFormat extends PulsingPostingsFormat {
public class Pulsing41PostingsFormat extends PulsingPostingsFormat {
/** Inlines docFreq=1 terms, otherwise uses the normal "Lucene40" format. */
public Pulsing40PostingsFormat() {
public Pulsing41PostingsFormat() {
this(1);
}
/** Inlines docFreq=<code>freqCutoff</code> terms, otherwise uses the normal "Lucene40" format. */
public Pulsing40PostingsFormat(int freqCutoff) {
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. */
public Pulsing40PostingsFormat(int freqCutoff, int minBlockSize, int maxBlockSize) {
super("Pulsing40", new Lucene40PostingsBaseFormat(), freqCutoff, minBlockSize, maxBlockSize);
public Pulsing41PostingsFormat(int freqCutoff, int minBlockSize, int maxBlockSize) {
super("Pulsing41", new Lucene41PostingsBaseFormat(), freqCutoff, minBlockSize, maxBlockSize);
}
}

View File

@ -13,9 +13,8 @@
# See the License for the specific language governing permissions and
# limitations under the License.
org.apache.lucene.codecs.pulsing.Pulsing40PostingsFormat
org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat
org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat
org.apache.lucene.codecs.memory.MemoryPostingsFormat
org.apache.lucene.codecs.bloom.BloomFilteringPostingsFormat
org.apache.lucene.codecs.memory.DirectPostingsFormat
org.apache.lucene.codecs.block.BlockPostingsFormat

View File

@ -20,7 +20,7 @@ package org.apache.lucene.codecs.blockterms;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
import org.apache.lucene.codecs.lucene40ords.Lucene40WithOrds;
import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
import org.apache.lucene.index.BasePostingsFormatTestCase;
/**
@ -29,7 +29,7 @@ import org.apache.lucene.index.BasePostingsFormatTestCase;
// 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 PostingsFormat postings = new Lucene40WithOrds();
private final PostingsFormat postings = new Lucene41WithOrds();
private final Codec codec = new Lucene40Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {

View File

@ -26,7 +26,7 @@ import org.apache.lucene.index.BasePostingsFormatTestCase;
* Basic tests for BloomPostingsFormat
*/
public class TestBloomPostingsFormat extends BasePostingsFormatTestCase {
private final PostingsFormat postings = new TestBloomFilteredLucene40Postings();
private final PostingsFormat postings = new TestBloomFilteredLucene41Postings();
private final Codec codec = new Lucene40Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {

View File

@ -25,7 +25,6 @@ import java.util.Locale;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.pulsing.Pulsing40PostingsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
@ -52,7 +51,7 @@ import org.apache.lucene.util._TestUtil;
public class Test10KPulsings extends LuceneTestCase {
public void test10kPulsed() throws Exception {
// we always run this test with pulsing codec.
Codec cp = _TestUtil.alwaysPostingsFormat(new Pulsing40PostingsFormat(1));
Codec cp = _TestUtil.alwaysPostingsFormat(new Pulsing41PostingsFormat(1));
File f = _TestUtil.getTempDir("10kpulsed");
BaseDirectoryWrapper dir = newFSDirectory(f);
@ -103,7 +102,7 @@ public class Test10KPulsings extends LuceneTestCase {
public void test10kNotPulsed() throws Exception {
// we always run this test with pulsing codec.
int freqCutoff = _TestUtil.nextInt(random(), 1, 10);
Codec cp = _TestUtil.alwaysPostingsFormat(new Pulsing40PostingsFormat(freqCutoff));
Codec cp = _TestUtil.alwaysPostingsFormat(new Pulsing41PostingsFormat(freqCutoff));
File f = _TestUtil.getTempDir("10knotpulsed");
BaseDirectoryWrapper dir = newFSDirectory(f);

View File

@ -19,7 +19,7 @@ package org.apache.lucene.codecs.pulsing;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
import org.apache.lucene.codecs.lucene41.Lucene41Codec;
import org.apache.lucene.index.BasePostingsFormatTestCase;
/**
@ -27,8 +27,8 @@ import org.apache.lucene.index.BasePostingsFormatTestCase;
*/
public class TestPulsingPostingsFormat extends BasePostingsFormatTestCase {
// TODO: randomize cutoff
private final PostingsFormat postings = new Pulsing40PostingsFormat();
private final Codec codec = new Lucene40Codec() {
private final PostingsFormat postings = new Pulsing41PostingsFormat();
private final Codec codec = new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postings;

View File

@ -45,7 +45,7 @@ public class TestPulsingReuse extends LuceneTestCase {
// TODO: this is a basic test. this thing is complicated, add more
public void testSophisticatedReuse() throws Exception {
// we always run this test with pulsing codec.
Codec cp = _TestUtil.alwaysPostingsFormat(new Pulsing40PostingsFormat(1));
Codec cp = _TestUtil.alwaysPostingsFormat(new Pulsing41PostingsFormat(1));
Directory dir = newDirectory();
RandomIndexWriter iw = new RandomIndexWriter(random(), dir,
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setCodec(cp));

View File

@ -1,4 +1,4 @@
package org.apache.lucene.codecs.block;
package org.apache.lucene.codecs.lucene41;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@ -28,7 +28,7 @@ import org.apache.lucene.util.packed.PackedInts.Decoder;
import org.apache.lucene.util.packed.PackedInts.FormatAndBits;
import org.apache.lucene.util.packed.PackedInts;
import static org.apache.lucene.codecs.block.BlockPostingsFormat.BLOCK_SIZE;
import static org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat.BLOCK_SIZE;
/**
* Encode all values in normal area with fixed bit width,

View File

@ -0,0 +1,122 @@
package org.apache.lucene.codecs.lucene41;
/*
* 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.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat;
import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat;
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40NormsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat;
import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
/**
* Implements the Lucene 4.1 index format, with configurable per-field postings formats.
* <p>
* If you want to reuse functionality of this codec in another codec, extend
* {@link FilterCodec}.
*
* @see org.apache.lucene.codecs.lucene41 package documentation for file format details.
* @lucene.experimental
*/
// 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).
public class Lucene41Codec extends Codec {
private final StoredFieldsFormat fieldsFormat = new Lucene40StoredFieldsFormat();
private final TermVectorsFormat vectorsFormat = new Lucene40TermVectorsFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene40FieldInfosFormat();
private final DocValuesFormat docValuesFormat = new Lucene40DocValuesFormat();
private final SegmentInfoFormat infosFormat = new Lucene40SegmentInfoFormat();
private final NormsFormat normsFormat = new Lucene40NormsFormat();
private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return Lucene41Codec.this.getPostingsFormatForField(field);
}
};
/** Sole constructor. */
public Lucene41Codec() {
super("Lucene41");
}
@Override
public final StoredFieldsFormat storedFieldsFormat() {
return fieldsFormat;
}
@Override
public final TermVectorsFormat termVectorsFormat() {
return vectorsFormat;
}
@Override
public final DocValuesFormat docValuesFormat() {
return docValuesFormat;
}
@Override
public final PostingsFormat postingsFormat() {
return postingsFormat;
}
@Override
public final FieldInfosFormat fieldInfosFormat() {
return fieldInfosFormat;
}
@Override
public final SegmentInfoFormat segmentInfoFormat() {
return infosFormat;
}
@Override
public final NormsFormat normsFormat() {
return normsFormat;
}
@Override
public final LiveDocsFormat liveDocsFormat() {
return liveDocsFormat;
}
/** Returns the postings format that should be used for writing
* new segments of <code>field</code>.
*
* The default implementation always returns "Lucene41"
*/
public PostingsFormat getPostingsFormatForField(String field) {
return defaultFormat;
}
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
}

View File

@ -0,0 +1,51 @@
package org.apache.lucene.codecs.lucene41;
/*
* 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.PostingsBaseFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
/**
* Provides a {@link PostingsReaderBase} and {@link
* PostingsWriterBase}.
*
* @lucene.experimental */
// TODO: should these also be named / looked up via SPI?
public final class Lucene41PostingsBaseFormat extends PostingsBaseFormat {
/** Sole constructor. */
public Lucene41PostingsBaseFormat() {
super("Lucene41");
}
@Override
public PostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException {
return new Lucene41PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
}
@Override
public PostingsWriterBase postingsWriterBase(SegmentWriteState state) throws IOException {
return new Lucene41PostingsWriter(state);
}
}

View File

@ -1,4 +1,4 @@
package org.apache.lucene.codecs.block;
package org.apache.lucene.codecs.lucene41;
/*
@ -38,7 +38,7 @@ import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.packed.PackedInts;
/**
* Block postings format, which encodes postings in packed integer blocks
* Lucene 4.1 postings format, which encodes postings in packed integer blocks
* for fast decode.
*
* <p><b>NOTE</b>: this format is still experimental and
@ -58,7 +58,7 @@ import org.apache.lucene.util.packed.PackedInts;
*
* <li>
* <b>Block structure</b>:
* <p>When the postings are long enough, BlockPostingsFormat will try to encode most integer data
* <p>When the postings are long enough, Lucene41PostingsFormat will try to encode most integer data
* as a packed block.</p>
* <p>Take a term with 259 documents as an example, the first 256 document ids are encoded as two packed
* blocks, while the remaining 3 are encoded as one VInt block. </p>
@ -161,7 +161,7 @@ import org.apache.lucene.util.packed.PackedInts;
* <li>SkipFPDelta determines the position of this term's SkipData within the .doc
* file. In particular, it is the length of the TermFreq data.
* SkipDelta is only stored if DocFreq is not smaller than SkipMinimum
* (i.e. 8 in BlockPostingsFormat).</li>
* (i.e. 8 in Lucene41PostingsFormat).</li>
* </ul>
* </dd>
* </dl>
@ -238,10 +238,10 @@ import org.apache.lucene.util.packed.PackedInts;
* We use this trick since the definition of skip entry is a little different from base interface.
* In {@link MultiLevelSkipListWriter}, skip data is assumed to be saved for
* skipInterval<sup>th</sup>, 2*skipInterval<sup>th</sup> ... posting in the list. However,
* in BlockPostingsFormat, the skip data is saved for skipInterval+1<sup>th</sup>,
* in Lucene41PostingsFormat, the skip data is saved for skipInterval+1<sup>th</sup>,
* 2*skipInterval+1<sup>th</sup> ... posting (skipInterval==PackedBlockSize in this case).
* When DocFreq is multiple of PackedBlockSize, MultiLevelSkipListWriter will expect one
* more skip data than BlockSkipWriter. </li>
* more skip data than Lucene41SkipWriter. </li>
* <li>SkipDatum is the metadata of one skip entry.
* For the first block (no matter packed or VInt), it is omitted.</li>
* <li>DocSkip records the document number of every PackedBlockSize<sup>th</sup> document number in
@ -351,7 +351,7 @@ import org.apache.lucene.util.packed.PackedInts;
* @lucene.experimental
*/
public final class BlockPostingsFormat extends PostingsFormat {
public final class Lucene41PostingsFormat extends PostingsFormat {
/**
* Filename extension for document number, frequencies, and skip data.
* See chapter: <a href="#Frequencies">Frequencies and Skip Data</a>
@ -380,12 +380,12 @@ public final class BlockPostingsFormat extends PostingsFormat {
// NOTE: must be multiple of 64 because of PackedInts long-aligned encoding/decoding
public final static int BLOCK_SIZE = 128;
public BlockPostingsFormat() {
public Lucene41PostingsFormat() {
this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
}
public BlockPostingsFormat(int minTermBlockSize, int maxTermBlockSize) {
super("Block");
public Lucene41PostingsFormat(int minTermBlockSize, int maxTermBlockSize) {
super("Lucene41");
this.minTermBlockSize = minTermBlockSize;
assert minTermBlockSize > 1;
this.maxTermBlockSize = maxTermBlockSize;
@ -399,7 +399,7 @@ public final class BlockPostingsFormat extends PostingsFormat {
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new BlockPostingsWriter(state);
PostingsWriterBase postingsWriter = new Lucene41PostingsWriter(state);
boolean success = false;
try {
@ -418,7 +418,7 @@ public final class BlockPostingsFormat extends PostingsFormat {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new BlockPostingsReader(state.dir,
PostingsReaderBase postingsReader = new Lucene41PostingsReader(state.dir,
state.fieldInfos,
state.segmentInfo,
state.context,

View File

@ -1,4 +1,4 @@
package org.apache.lucene.codecs.block;
package org.apache.lucene.codecs.lucene41;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -17,9 +17,9 @@ package org.apache.lucene.codecs.block;
* limitations under the License.
*/
import static org.apache.lucene.codecs.block.BlockPostingsFormat.BLOCK_SIZE;
import static org.apache.lucene.codecs.block.ForUtil.MAX_DATA_SIZE;
import static org.apache.lucene.codecs.block.ForUtil.MAX_ENCODED_SIZE;
import static org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat.BLOCK_SIZE;
import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_DATA_SIZE;
import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_ENCODED_SIZE;
import java.io.IOException;
import java.util.Arrays;
@ -49,10 +49,10 @@ import org.apache.lucene.util.IOUtils;
* Concrete class that reads docId(maybe frq,pos,offset,payloads) list
* with postings format.
*
* @see BlockSkipReader for details
*
* @see Lucene41SkipReader for details
* @lucene.experimental
*/
final class BlockPostingsReader extends PostingsReaderBase {
public final class Lucene41PostingsReader extends PostingsReaderBase {
private final IndexInput docIn;
private final IndexInput posIn;
@ -62,35 +62,35 @@ final class BlockPostingsReader extends PostingsReaderBase {
// public static boolean DEBUG = false;
public BlockPostingsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo segmentInfo, IOContext ioContext, String segmentSuffix) throws IOException {
public Lucene41PostingsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo segmentInfo, IOContext ioContext, String segmentSuffix) throws IOException {
boolean success = false;
IndexInput docIn = null;
IndexInput posIn = null;
IndexInput payIn = null;
try {
docIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, BlockPostingsFormat.DOC_EXTENSION),
docIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene41PostingsFormat.DOC_EXTENSION),
ioContext);
CodecUtil.checkHeader(docIn,
BlockPostingsWriter.DOC_CODEC,
BlockPostingsWriter.VERSION_CURRENT,
BlockPostingsWriter.VERSION_CURRENT);
Lucene41PostingsWriter.DOC_CODEC,
Lucene41PostingsWriter.VERSION_CURRENT,
Lucene41PostingsWriter.VERSION_CURRENT);
forUtil = new ForUtil(docIn);
if (fieldInfos.hasProx()) {
posIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, BlockPostingsFormat.POS_EXTENSION),
posIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene41PostingsFormat.POS_EXTENSION),
ioContext);
CodecUtil.checkHeader(posIn,
BlockPostingsWriter.POS_CODEC,
BlockPostingsWriter.VERSION_CURRENT,
BlockPostingsWriter.VERSION_CURRENT);
Lucene41PostingsWriter.POS_CODEC,
Lucene41PostingsWriter.VERSION_CURRENT,
Lucene41PostingsWriter.VERSION_CURRENT);
if (fieldInfos.hasPayloads() || fieldInfos.hasOffsets()) {
payIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, BlockPostingsFormat.PAY_EXTENSION),
payIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene41PostingsFormat.PAY_EXTENSION),
ioContext);
CodecUtil.checkHeader(payIn,
BlockPostingsWriter.PAY_CODEC,
BlockPostingsWriter.VERSION_CURRENT,
BlockPostingsWriter.VERSION_CURRENT);
Lucene41PostingsWriter.PAY_CODEC,
Lucene41PostingsWriter.VERSION_CURRENT,
Lucene41PostingsWriter.VERSION_CURRENT);
}
}
@ -109,9 +109,9 @@ final class BlockPostingsReader extends PostingsReaderBase {
public void init(IndexInput termsIn) throws IOException {
// Make sure we are talking to the matching postings writer
CodecUtil.checkHeader(termsIn,
BlockPostingsWriter.TERMS_CODEC,
BlockPostingsWriter.VERSION_CURRENT,
BlockPostingsWriter.VERSION_CURRENT);
Lucene41PostingsWriter.TERMS_CODEC,
Lucene41PostingsWriter.VERSION_CURRENT,
Lucene41PostingsWriter.VERSION_CURRENT);
final int indexBlockSize = termsIn.readVInt();
if (indexBlockSize != BLOCK_SIZE) {
throw new IllegalStateException("index-time BLOCK_SIZE (" + indexBlockSize + ") != read-time BLOCK_SIZE (" + BLOCK_SIZE + ")");
@ -321,7 +321,7 @@ final class BlockPostingsReader extends PostingsReaderBase {
private int docBufferUpto;
private BlockSkipReader skipper;
private Lucene41SkipReader skipper;
private boolean skipped;
final IndexInput startDocIn;
@ -353,7 +353,7 @@ final class BlockPostingsReader extends PostingsReaderBase {
private Bits liveDocs;
public BlockDocsEnum(FieldInfo fieldInfo) throws IOException {
this.startDocIn = BlockPostingsReader.this.docIn;
this.startDocIn = Lucene41PostingsReader.this.docIn;
this.docIn = startDocIn.clone();
indexHasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
indexHasPos = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
@ -486,8 +486,8 @@ final class BlockPostingsReader extends PostingsReaderBase {
if (skipper == null) {
// Lazy init: first time this enum has ever been used for skipping
skipper = new BlockSkipReader(docIn.clone(),
BlockPostingsWriter.maxSkipLevels,
skipper = new Lucene41SkipReader(docIn.clone(),
Lucene41PostingsWriter.maxSkipLevels,
BLOCK_SIZE,
indexHasPos,
indexHasOffsets,
@ -502,7 +502,7 @@ final class BlockPostingsReader extends PostingsReaderBase {
skipped = true;
}
// always plus one to fix the result, since skip position in BlockSkipReader
// always plus one to fix the result, since skip position in Lucene41SkipReader
// is a little different from MultiLevelSkipListReader
final int newDocUpto = skipper.skipTo(target) + 1;
@ -577,7 +577,7 @@ final class BlockPostingsReader extends PostingsReaderBase {
private int docBufferUpto;
private int posBufferUpto;
private BlockSkipReader skipper;
private Lucene41SkipReader skipper;
private boolean skipped;
final IndexInput startDocIn;
@ -628,9 +628,9 @@ final class BlockPostingsReader extends PostingsReaderBase {
private Bits liveDocs;
public BlockDocsAndPositionsEnum(FieldInfo fieldInfo) throws IOException {
this.startDocIn = BlockPostingsReader.this.docIn;
this.startDocIn = Lucene41PostingsReader.this.docIn;
this.docIn = startDocIn.clone();
this.posIn = BlockPostingsReader.this.posIn.clone();
this.posIn = Lucene41PostingsReader.this.posIn.clone();
encoded = new byte[MAX_ENCODED_SIZE];
indexHasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
indexHasPayloads = fieldInfo.hasPayloads();
@ -797,8 +797,8 @@ final class BlockPostingsReader extends PostingsReaderBase {
// if (DEBUG) {
// System.out.println(" create skipper");
// }
skipper = new BlockSkipReader(docIn.clone(),
BlockPostingsWriter.maxSkipLevels,
skipper = new Lucene41SkipReader(docIn.clone(),
Lucene41PostingsWriter.maxSkipLevels,
BLOCK_SIZE,
true,
indexHasOffsets,
@ -987,7 +987,7 @@ final class BlockPostingsReader extends PostingsReaderBase {
private int docBufferUpto;
private int posBufferUpto;
private BlockSkipReader skipper;
private Lucene41SkipReader skipper;
private boolean skipped;
final IndexInput startDocIn;
@ -1044,10 +1044,10 @@ final class BlockPostingsReader extends PostingsReaderBase {
private Bits liveDocs;
public EverythingEnum(FieldInfo fieldInfo) throws IOException {
this.startDocIn = BlockPostingsReader.this.docIn;
this.startDocIn = Lucene41PostingsReader.this.docIn;
this.docIn = startDocIn.clone();
this.posIn = BlockPostingsReader.this.posIn.clone();
this.payIn = BlockPostingsReader.this.payIn.clone();
this.posIn = Lucene41PostingsReader.this.posIn.clone();
this.payIn = Lucene41PostingsReader.this.payIn.clone();
encoded = new byte[MAX_ENCODED_SIZE];
indexHasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
if (indexHasOffsets) {
@ -1282,8 +1282,8 @@ final class BlockPostingsReader extends PostingsReaderBase {
// if (DEBUG) {
// System.out.println(" create skipper");
// }
skipper = new BlockSkipReader(docIn.clone(),
BlockPostingsWriter.maxSkipLevels,
skipper = new Lucene41SkipReader(docIn.clone(),
Lucene41PostingsWriter.maxSkipLevels,
BLOCK_SIZE,
true,
indexHasOffsets,

View File

@ -1,4 +1,4 @@
package org.apache.lucene.codecs.block;
package org.apache.lucene.codecs.lucene41;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -17,9 +17,9 @@ package org.apache.lucene.codecs.block;
* limitations under the License.
*/
import static org.apache.lucene.codecs.block.BlockPostingsFormat.BLOCK_SIZE;
import static org.apache.lucene.codecs.block.ForUtil.MAX_DATA_SIZE;
import static org.apache.lucene.codecs.block.ForUtil.MAX_ENCODED_SIZE;
import static org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat.BLOCK_SIZE;
import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_DATA_SIZE;
import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_ENCODED_SIZE;
import java.io.IOException;
import java.util.ArrayList;
@ -47,10 +47,10 @@ import org.apache.lucene.util.packed.PackedInts;
*
* Postings list for each term will be stored separately.
*
* @see BlockSkipWriter for details about skipping setting and postings layout.
*
* @see Lucene41SkipWriter for details about skipping setting and postings layout.
* @lucene.experimental
*/
final class BlockPostingsWriter extends PostingsWriterBase {
public final class Lucene41PostingsWriter extends PostingsWriterBase {
/**
* Expert: The maximum number of skip levels. Smaller values result in
@ -58,12 +58,12 @@ final class BlockPostingsWriter extends PostingsWriterBase {
*/
static final int maxSkipLevels = 10;
final static String TERMS_CODEC = "BlockPostingsWriterTerms";
final static String DOC_CODEC = "BlockPostingsWriterDoc";
final static String POS_CODEC = "BlockPostingsWriterPos";
final static String PAY_CODEC = "BlockPostingsWriterPay";
final static String TERMS_CODEC = "Lucene41PostingsWriterTerms";
final static String DOC_CODEC = "Lucene41PostingsWriterDoc";
final static String POS_CODEC = "Lucene41PostingsWriterPos";
final static String PAY_CODEC = "Lucene41PostingsWriterPay";
// Increment version to change it:
// Increment version to change it: nocommit: we can start at 0
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;
@ -112,12 +112,12 @@ final class BlockPostingsWriter extends PostingsWriterBase {
final byte[] encoded;
private final ForUtil forUtil;
private final BlockSkipWriter skipWriter;
private final Lucene41SkipWriter skipWriter;
public BlockPostingsWriter(SegmentWriteState state, float acceptableOverheadRatio) throws IOException {
public Lucene41PostingsWriter(SegmentWriteState state, float acceptableOverheadRatio) throws IOException {
super();
docOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, BlockPostingsFormat.DOC_EXTENSION),
docOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene41PostingsFormat.DOC_EXTENSION),
state.context);
IndexOutput posOut = null;
IndexOutput payOut = null;
@ -127,7 +127,7 @@ final class BlockPostingsWriter extends PostingsWriterBase {
forUtil = new ForUtil(acceptableOverheadRatio, docOut);
if (state.fieldInfos.hasProx()) {
posDeltaBuffer = new int[MAX_DATA_SIZE];
posOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, BlockPostingsFormat.POS_EXTENSION),
posOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene41PostingsFormat.POS_EXTENSION),
state.context);
CodecUtil.writeHeader(posOut, POS_CODEC, VERSION_CURRENT);
@ -148,7 +148,7 @@ final class BlockPostingsWriter extends PostingsWriterBase {
}
if (state.fieldInfos.hasPayloads() || state.fieldInfos.hasOffsets()) {
payOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, BlockPostingsFormat.PAY_EXTENSION),
payOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene41PostingsFormat.PAY_EXTENSION),
state.context);
CodecUtil.writeHeader(payOut, PAY_CODEC, VERSION_CURRENT);
}
@ -172,7 +172,7 @@ final class BlockPostingsWriter extends PostingsWriterBase {
freqBuffer = new int[MAX_DATA_SIZE];
// TODO: should we try skipping every 2/4 blocks...?
skipWriter = new BlockSkipWriter(maxSkipLevels,
skipWriter = new Lucene41SkipWriter(maxSkipLevels,
BLOCK_SIZE,
state.segmentInfo.getDocCount(),
docOut,
@ -182,7 +182,7 @@ final class BlockPostingsWriter extends PostingsWriterBase {
encoded = new byte[MAX_ENCODED_SIZE];
}
public BlockPostingsWriter(SegmentWriteState state) throws IOException {
public Lucene41PostingsWriter(SegmentWriteState state) throws IOException {
this(state, PackedInts.COMPACT);
}

View File

@ -1,4 +1,4 @@
package org.apache.lucene.codecs.block;
package org.apache.lucene.codecs.lucene41;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -35,12 +35,12 @@ import org.apache.lucene.store.IndexInput;
* 0 1 2 3 4 5
* d d d d d d (posting list)
* ^ ^ (skip point in MultiLeveSkipWriter)
* ^ (skip point in BlockSkipWriter)
* ^ (skip point in Lucene41SkipWriter)
*
* In this case, MultiLevelSkipListReader will use the last document as a skip point,
* while BlockSkipReader should assume no skip point will comes.
* while Lucene41SkipReader should assume no skip point will comes.
*
* If we use the interface directly in BlockSkipReader, it may silly try to read
* If we use the interface directly in Lucene41SkipReader, it may silly try to read
* another skip data after the only skip point is loaded.
*
* To illustrate this, we can call skipTo(d[5]), since skip point d[3] has smaller docId,
@ -50,8 +50,8 @@ import org.apache.lucene.store.IndexInput;
* Therefore, we'll trim df before passing it to the interface. see trim(int)
*
*/
final class BlockSkipReader extends MultiLevelSkipListReader {
// private boolean DEBUG = BlockPostingsReader.DEBUG;
final class Lucene41SkipReader extends MultiLevelSkipListReader {
// private boolean DEBUG = Lucene41PostingsReader.DEBUG;
private final int blockSize;
private long docPointer[];
@ -66,7 +66,7 @@ final class BlockSkipReader extends MultiLevelSkipListReader {
private long lastDocPointer;
private int lastPosBufferUpto;
public BlockSkipReader(IndexInput skipStream, int maxSkipLevels, int blockSize, boolean hasPos, boolean hasOffsets, boolean hasPayloads) {
public Lucene41SkipReader(IndexInput skipStream, int maxSkipLevels, int blockSize, boolean hasPos, boolean hasOffsets, boolean hasPayloads) {
super(skipStream, maxSkipLevels, blockSize, 8);
this.blockSize = blockSize;
docPointer = new long[maxSkipLevels];
@ -91,7 +91,7 @@ final class BlockSkipReader extends MultiLevelSkipListReader {
/**
* Trim original docFreq to tell skipReader read proper number of skip points.
*
* Since our definition in BlockSkip* is a little different from MultiLevelSkip*
* Since our definition in Lucene41Skip* is a little different from MultiLevelSkip*
* This trimmed docFreq will prevent skipReader from:
* 1. silly reading a non-existed skip point after the last block boundary
* 2. moving into the vInt block

View File

@ -1,4 +1,4 @@
package org.apache.lucene.codecs.block;
package org.apache.lucene.codecs.lucene41;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -43,8 +43,8 @@ import org.apache.lucene.codecs.MultiLevelSkipListWriter;
* 4. start offset.
*
*/
final class BlockSkipWriter extends MultiLevelSkipListWriter {
// private boolean DEBUG = BlockPostingsReader.DEBUG;
final class Lucene41SkipWriter extends MultiLevelSkipListWriter {
// private boolean DEBUG = Lucene41PostingsReader.DEBUG;
private int[] lastSkipDoc;
private long[] lastSkipDocPointer;
@ -66,7 +66,7 @@ final class BlockSkipWriter extends MultiLevelSkipListWriter {
private boolean fieldHasOffsets;
private boolean fieldHasPayloads;
public BlockSkipWriter(int maxSkipLevels, int blockSize, int docCount, IndexOutput docOut, IndexOutput posOut, IndexOutput payOut) {
public Lucene41SkipWriter(int maxSkipLevels, int blockSize, int docCount, IndexOutput docOut, IndexOutput posOut, IndexOutput payOut) {
super(blockSize, 8, maxSkipLevels, docCount);
this.docOut = docOut;
this.posOut = posOut;

View File

@ -0,0 +1,396 @@
<!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>
Lucene 4.1 file format.
<h1>Apache Lucene - Index File Formats</h1>
<div>
<ul>
<li><a href="#Introduction">Introduction</a></li>
<li><a href="#Definitions">Definitions</a>
<ul>
<li><a href="#Inverted_Indexing">Inverted Indexing</a></li>
<li><a href="#Types_of_Fields">Types of Fields</a></li>
<li><a href="#Segments">Segments</a></li>
<li><a href="#Document_Numbers">Document Numbers</a></li>
</ul>
</li>
<li><a href="#Overview">Index Structure Overview</a></li>
<li><a href="#File_Naming">File Naming</a></li>
<li><a href="#file-names">Summary of File Extensions</a></li>
<ul>
<li><a href="#Lock_File">Lock File</a></li>
<li><a href="#History">History</a></li>
<li><a href="#Limitations">Limitations</a></li>
</ul>
</div>
<a name="Introduction"></a>
<h2>Introduction</h2>
<div>
<p>This document defines the index file formats used in this version of Lucene.
If you are using a different version of Lucene, please consult the copy of
<code>docs/</code> that was distributed with
the version you are using.</p>
<p>Apache Lucene is written in Java, but several efforts are underway to write
<a href="http://wiki.apache.org/lucene-java/LuceneImplementations">versions of
Lucene in other programming languages</a>. If these versions are to remain
compatible with Apache Lucene, then a language-independent definition of the
Lucene index format is required. This document thus attempts to provide a
complete and independent definition of the Apache Lucene file formats.</p>
<p>As Lucene evolves, this document should evolve. Versions of Lucene in
different programming languages should endeavor to agree on file formats, and
generate new versions of this document.</p>
</div>
<a name="Definitions" id="Definitions"></a>
<h2>Definitions</h2>
<div>
<p>The fundamental concepts in Lucene are index, document, field and term.</p>
<p>An index contains a sequence of documents.</p>
<ul>
<li>A document is a sequence of fields.</li>
<li>A field is a named sequence of terms.</li>
<li>A term is a sequence of bytes.</li>
</ul>
<p>The same sequence of bytes in two different fields is considered a different
term. Thus terms are represented as a pair: the string naming the field, and the
bytes within the field.</p>
<a name="Inverted_Indexing"></a>
<h3>Inverted Indexing</h3>
<p>The index stores statistics about terms in order to make term-based search
more efficient. Lucene's index falls into the family of indexes known as an
<i>inverted index.</i> This is because it can list, for a term, the documents
that contain it. This is the inverse of the natural relationship, in which
documents list terms.</p>
<a name="Types_of_Fields"></a>
<h3>Types of Fields</h3>
<p>In Lucene, fields may be <i>stored</i>, in which case their text is stored
in the index literally, in a non-inverted manner. Fields that are inverted are
called <i>indexed</i>. A field may be both stored and indexed.</p>
<p>The text of a field may be <i>tokenized</i> into terms to be indexed, or the
text of a field may be used literally as a term to be indexed. Most fields are
tokenized, but sometimes it is useful for certain identifier fields to be
indexed literally.</p>
<p>See the {@link org.apache.lucene.document.Field Field}
java docs for more information on Fields.</p>
<a name="Segments" id="Segments"></a>
<h3>Segments</h3>
<p>Lucene indexes may be composed of multiple sub-indexes, or <i>segments</i>.
Each segment is a fully independent index, which could be searched separately.
Indexes evolve by:</p>
<ol>
<li>Creating new segments for newly added documents.</li>
<li>Merging existing segments.</li>
</ol>
<p>Searches may involve multiple segments and/or multiple indexes, each index
potentially composed of a set of segments.</p>
<a name="Document_Numbers"></a>
<h3>Document Numbers</h3>
<p>Internally, Lucene refers to documents by an integer <i>document number</i>.
The first document added to an index is numbered zero, and each subsequent
document added gets a number one greater than the previous.</p>
<p>Note that a document's number may change, so caution should be taken when
storing these numbers outside of Lucene. In particular, numbers may change in
the following situations:</p>
<ul>
<li>
<p>The numbers stored in each segment are unique only within the segment, and
must be converted before they can be used in a larger context. The standard
technique is to allocate each segment a range of values, based on the range of
numbers used in that segment. To convert a document number from a segment to an
external value, the segment's <i>base</i> document number is added. To convert
an external value back to a segment-specific value, the segment is identified
by the range that the external value is in, and the segment's base value is
subtracted. For example two five document segments might be combined, so that
the first segment has a base value of zero, and the second of five. Document
three from the second segment would have an external value of eight.</p>
</li>
<li>
<p>When documents are deleted, gaps are created in the numbering. These are
eventually removed as the index evolves through merging. Deleted documents are
dropped when segments are merged. A freshly-merged segment thus has no gaps in
its numbering.</p>
</li>
</ul>
</div>
<a name="Overview" id="Overview"></a>
<h2>Index Structure Overview</h2>
<div>
<p>Each segment index maintains the following:</p>
<ul>
<li>
{@link org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat Segment info}.
This contains metadata about a segment, such as the number of documents,
what files it uses,
</li>
<li>
{@link org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat Field names}.
This contains the set of field names used in the index.
</li>
<li>
{@link org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat Stored Field values}.
This contains, for each document, a list of attribute-value pairs, where the attributes
are field names. These are used to store auxiliary information about the document, such as
its title, url, or an identifier to access a database. The set of stored fields are what is
returned for each hit when searching. This is keyed by document number.
</li>
<li>
{@link org.apache.lucene.codecs.lucene41Lucene41PostingsFormat 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
proximity data.
</li>
<li>
{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Frequency data}.
For each term in the dictionary, the numbers of all the
documents that contain that term, and the frequency of the term in that
document, unless frequencies are omitted (IndexOptions.DOCS_ONLY)
</li>
<li>
{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Proximity data}.
For each term in the dictionary, the positions that the
term occurs in each document. Note that this will not exist if all fields in
all documents omit position data.
</li>
<li>
{@link org.apache.lucene.codecs.lucene40.Lucene40NormsFormat Normalization factors}.
For each field in each document, a value is stored
that is multiplied into the score for hits on that field.
</li>
<li>
{@link org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat Term Vectors}.
For each field in each document, the term vector (sometimes
called document vector) may be stored. A term vector consists of term text and
term frequency. To add Term Vectors to your index see the
{@link org.apache.lucene.document.Field Field} constructors
</li>
<li>
{@link org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat Per-document values}.
Like stored values, these are also keyed by document
number, but are generally intended to be loaded into main memory for fast
access. Whereas stored values are generally intended for summary results from
searches, per-document values are useful for things like scoring factors.
</li>
<li>
{@link org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat Deleted documents}.
An optional file indicating which documents are deleted.
</li>
</ul>
<p>Details on each of these are provided in their linked pages.</p>
</div>
<a name="File_Naming"></a>
<h2>File Naming</h2>
<div>
<p>All files belonging to a segment have the same name with varying extensions.
The extensions correspond to the different file formats described below. When
using the Compound File format (default in 1.4 and greater) these files (except
for the Segment info file, the Lock file, and Deleted documents file) are collapsed
into a single .cfs file (see below for details)</p>
<p>Typically, all segments in an index are stored in a single directory,
although this is not required.</p>
<p>As of version 2.1 (lock-less commits), file names are never re-used (there
is one exception, "segments.gen", see below). That is, when any file is saved
to the Directory it is given a never before used filename. This is achieved
using a simple generations approach. For example, the first segments file is
segments_1, then segments_2, etc. The generation is a sequential long integer
represented in alpha-numeric (base 36) form.</p>
</div>
<a name="file-names" id="file-names"></a>
<h2>Summary of File Extensions</h2>
<div>
<p>The following table summarizes the names and extensions of the files in
Lucene:</p>
<table cellspacing="1" cellpadding="4">
<tr>
<th>Name</th>
<th>Extension</th>
<th>Brief Description</th>
</tr>
<tr>
<td>{@link org.apache.lucene.index.SegmentInfos Segments File}</td>
<td>segments.gen, segments_N</td>
<td>Stores information about a commit point</td>
</tr>
<tr>
<td><a href="#Lock_File">Lock File</a></td>
<td>write.lock</td>
<td>The Write lock prevents multiple IndexWriters from writing to the same
file.</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat Segment Info}</td>
<td>.si</td>
<td>Stores metadata about a segment</td>
</tr>
<tr>
<td>{@link org.apache.lucene.store.CompoundFileDirectory Compound File}</td>
<td>.cfs, .cfe</td>
<td>An optional "virtual" file consisting of all the other index files for
systems that frequently run out of file handles.</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat Fields}</td>
<td>.fnm</td>
<td>Stores information about the fields</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat Field Index}</td>
<td>.fdx</td>
<td>Contains pointers to field data</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat Field Data}</td>
<td>.fdt</td>
<td>The stored fields for documents</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Dictionary}</td>
<td>.tim</td>
<td>The term dictionary, stores term info</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Index}</td>
<td>.tip</td>
<td>The index into the Term Dictionary</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Frequencies}</td>
<td>.doc</td>
<td>Contains the list of docs which contain each term along with frequency</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Positions}</td>
<td>.pos</td>
<td>Stores position information about where a term occurs in the index</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Payloads}</td>
<td>.pay</td>
<td>Stores additional per-position metadata information such as character offsets and user payloads</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40NormsFormat Norms}</td>
<td>.nrm.cfs, .nrm.cfe</td>
<td>Encodes length and boost factors for docs and fields</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat Per-Document Values}</td>
<td>.dv.cfs, .dv.cfe</td>
<td>Encodes additional scoring factors or other per-document information.</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat Term Vector Index}</td>
<td>.tvx</td>
<td>Stores offset into the document data file</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat Term Vector Documents}</td>
<td>.tvd</td>
<td>Contains information about each document that has term vectors</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat Term Vector Fields}</td>
<td>.tvf</td>
<td>The field level info about term vectors</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat Deleted Documents}</td>
<td>.del</td>
<td>Info about what files are deleted</td>
</tr>
</table>
</div>
<a name="Lock_File" id="Lock_File"></a>
<h2>Lock File</h2>
The write lock, which is stored in the index directory by default, is named
"write.lock". If the lock directory is different from the index directory then
the write lock will be named "XXXX-write.lock" where XXXX is a unique prefix
derived from the full path to the index directory. When this file is present, a
writer is currently modifying the index (adding or removing documents). This
lock file ensures that only one writer is modifying the index at a time.</p>
<a name="History"></a>
<h2>History</h2>
<p>Compatibility notes are provided in this document, describing how file
formats have changed from prior versions:</p>
<ul>
<li>In version 2.1, the file format was changed to allow lock-less commits (ie,
no more commit lock). The change is fully backwards compatible: you can open a
pre-2.1 index for searching or adding/deleting of docs. When the new segments
file is saved (committed), it will be written in the new file format (meaning
no specific "upgrade" process is needed). But note that once a commit has
occurred, pre-2.1 Lucene will not be able to read the index.</li>
<li>In version 2.3, the file format was changed to allow segments to share a
single set of doc store (vectors &amp; stored fields) files. This allows for
faster indexing in certain cases. The change is fully backwards compatible (in
the same way as the lock-less commits change in 2.1).</li>
<li>In version 2.4, Strings are now written as true UTF-8 byte sequence, not
Java's modified UTF-8. See <a href="http://issues.apache.org/jira/browse/LUCENE-510">
LUCENE-510</a> for details.</li>
<li>In version 2.9, an optional opaque Map&lt;String,String&gt; CommitUserData
may be passed to IndexWriter's commit methods (and later retrieved), which is
recorded in the segments_N file. See <a href="http://issues.apache.org/jira/browse/LUCENE-1382">
LUCENE-1382</a> for details. Also,
diagnostics were added to each segment written recording details about why it
was written (due to flush, merge; which OS/JRE was used; etc.). See issue
<a href="http://issues.apache.org/jira/browse/LUCENE-1654">LUCENE-1654</a> for details.</li>
<li>In version 3.0, compressed fields are no longer written to the index (they
can still be read, but on merge the new segment will write them, uncompressed).
See issue <a href="http://issues.apache.org/jira/browse/LUCENE-1960">LUCENE-1960</a>
for details.</li>
<li>In version 3.1, segments records the code version that created them. See
<a href="http://issues.apache.org/jira/browse/LUCENE-2720">LUCENE-2720</a> for details.
Additionally segments track explicitly whether or not they have term vectors.
See <a href="http://issues.apache.org/jira/browse/LUCENE-2811">LUCENE-2811</a>
for details.</li>
<li>In version 3.2, numeric fields are written as natively to stored fields
file, previously they were stored in text format only.</li>
<li>In version 3.4, fields can omit position data while still indexing term
frequencies.</li>
<li>In version 4.0, the format of the inverted index became extensible via
the {@link org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage
({@link org.apache.lucene.index.DocValues DocValues}) was introduced. Normalization
factors need no longer be a single byte, they can be any DocValues
{@link org.apache.lucene.index.DocValues.Type type}. Terms need not be unicode
strings, they can be any byte sequence. Term offsets can optionally be indexed
into the postings lists. Payloads can be stored in the term vectors.</li>
<li>In version 4.1, the format of the postings list changed to use either
of FOR compression or variable-byte encoding, depending upon the frequency
of the term.</li>
</ul>
<a name="Limitations" id="Limitations"></a>
<h2>Limitations</h2>
<div>
<p>When referring to term numbers, Lucene's current implementation uses a Java
<code>int</code> to hold the term index, which means the
maximum number of unique terms in any single index segment is ~2.1 billion
times the term index interval (default 128) = ~274 billion. This is technically
not a limitation of the index file format, just of Lucene's current
implementation.</p>
<p>Similarly, Lucene uses a Java <code>int</code> to refer to
document numbers, and the index file format uses an <code>Int32</code>
on-disk to store document numbers. This is a limitation
of both the index file format and the current implementation. Eventually these
should be replaced with either <code>UInt64</code> values, or
better yet, {@link org.apache.lucene.store.DataOutput#writeVInt VInt} values which have no limit.</p>
</div>
</body>
</html>

View File

@ -14,3 +14,4 @@
# limitations under the License.
org.apache.lucene.codecs.lucene40.Lucene40Codec
org.apache.lucene.codecs.lucene41.Lucene41Codec

View File

@ -14,3 +14,4 @@
# limitations under the License.
org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat
org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat

View File

@ -19,7 +19,7 @@ package org.apache.lucene;
import org.apache.lucene.analysis.*;
import org.apache.lucene.codecs.*;
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
import org.apache.lucene.codecs.lucene41.Lucene41Codec;
import org.apache.lucene.document.*;
import org.apache.lucene.index.*;
import org.apache.lucene.search.*;
@ -31,11 +31,11 @@ import org.apache.lucene.util.*;
public class TestExternalCodecs extends LuceneTestCase {
private static final class CustomPerFieldCodec extends Lucene40Codec {
private static final class CustomPerFieldCodec extends Lucene41Codec {
private final PostingsFormat ramFormat = PostingsFormat.forName("RAMOnly");
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene40");
private final PostingsFormat pulsingFormat = PostingsFormat.forName("Pulsing40");
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
private final PostingsFormat pulsingFormat = PostingsFormat.forName("Pulsing41");
@Override
public PostingsFormat getPostingsFormatForField(String field) {

View File

@ -1,4 +1,4 @@
package org.apache.lucene.codecs.block;
package org.apache.lucene.codecs.lucene41;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -18,22 +18,13 @@ package org.apache.lucene.codecs.block;
*/
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.block.BlockPostingsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
import org.apache.lucene.index.BasePostingsFormatTestCase;
/**
* Tests BlockPostingsFormat
*/
public class TestBlockPostingsFormat extends BasePostingsFormatTestCase {
private final PostingsFormat postings = new BlockPostingsFormat();
private final Codec codec = new Lucene40Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postings;
}
};
private final Codec codec = new Lucene41Codec();
@Override
protected Codec getCodec() {

View File

@ -1,4 +1,4 @@
package org.apache.lucene.codecs.block;
package org.apache.lucene.codecs.lucene41;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -19,7 +19,6 @@ package org.apache.lucene.codecs.block;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
@ -47,10 +46,10 @@ public class TestBlockPostingsFormat2 extends LuceneTestCase {
super.setUp();
dir = newFSDirectory(_TestUtil.getTempDir("testDFBlockSize"));
iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
iwc.setCodec(new Lucene40Codec() {
iwc.setCodec(new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return PostingsFormat.forName("Block");
return PostingsFormat.forName("Lucene41");
}
});
iw = new RandomIndexWriter(random(), dir, iwc);
@ -88,7 +87,7 @@ public class TestBlockPostingsFormat2 extends LuceneTestCase {
/** tests terms with df = blocksize */
public void testDFBlockSize() throws Exception {
Document doc = newDocument();
for (int i = 0; i < BlockPostingsFormat.BLOCK_SIZE; i++) {
for (int i = 0; i < Lucene41PostingsFormat.BLOCK_SIZE; i++) {
for (Field f : doc.getFields()) {
f.setStringValue(f.name() + " " + f.name() + "_2");
}
@ -99,7 +98,7 @@ public class TestBlockPostingsFormat2 extends LuceneTestCase {
/** tests terms with df % blocksize = 0 */
public void testDFBlockSizeMultiple() throws Exception {
Document doc = newDocument();
for (int i = 0; i < BlockPostingsFormat.BLOCK_SIZE * 16; i++) {
for (int i = 0; i < Lucene41PostingsFormat.BLOCK_SIZE * 16; i++) {
for (Field f : doc.getFields()) {
f.setStringValue(f.name() + " " + f.name() + "_2");
}
@ -110,7 +109,7 @@ public class TestBlockPostingsFormat2 extends LuceneTestCase {
/** tests terms with ttf = blocksize */
public void testTTFBlockSize() throws Exception {
Document doc = newDocument();
for (int i = 0; i < BlockPostingsFormat.BLOCK_SIZE/2; i++) {
for (int i = 0; i < Lucene41PostingsFormat.BLOCK_SIZE/2; i++) {
for (Field f : doc.getFields()) {
f.setStringValue(f.name() + " " + f.name() + " " + f.name() + "_2 " + f.name() + "_2");
}
@ -121,7 +120,7 @@ public class TestBlockPostingsFormat2 extends LuceneTestCase {
/** tests terms with ttf % blocksize = 0 */
public void testTTFBlockSizeMultiple() throws Exception {
Document doc = newDocument();
for (int i = 0; i < BlockPostingsFormat.BLOCK_SIZE/2; i++) {
for (int i = 0; i < Lucene41PostingsFormat.BLOCK_SIZE/2; i++) {
for (Field f : doc.getFields()) {
String proto = (f.name() + " " + f.name() + " " + f.name() + " " + f.name() + " "
+ f.name() + "_2 " + f.name() + "_2 " + f.name() + "_2 " + f.name() + "_2");

View File

@ -1,4 +1,4 @@
package org.apache.lucene.codecs.block;
package org.apache.lucene.codecs.lucene41;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -30,7 +30,7 @@ import org.apache.lucene.analysis.MockVariableLengthPayloadFilter;
import org.apache.lucene.analysis.TokenFilter;
import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
import org.apache.lucene.codecs.lucene41.Lucene41Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
@ -64,7 +64,7 @@ import org.apache.lucene.util.automaton.RegExp;
* Tests partial enumeration (only pulling a subset of the prox data)
*/
public class TestBlockPostingsFormat3 extends LuceneTestCase {
static final int MAXDOC = BlockPostingsFormat.BLOCK_SIZE * 20;
static final int MAXDOC = Lucene41PostingsFormat.BLOCK_SIZE * 20;
// creates 6 fields with different options and does "duels" of fields against each other
public void test() throws Exception {
@ -85,10 +85,10 @@ public class TestBlockPostingsFormat3 extends LuceneTestCase {
}
};
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
iwc.setCodec(new Lucene40Codec() {
iwc.setCodec(new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return PostingsFormat.forName("Block");
return PostingsFormat.forName("Lucene41");
// TODO: we could actually add more fields implemented with different PFs
}
});

View File

@ -1,4 +1,4 @@
package org.apache.lucene.codecs.block;
package org.apache.lucene.codecs.lucene41;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -17,9 +17,9 @@ package org.apache.lucene.codecs.block;
* limitations under the License.
*/
import static org.apache.lucene.codecs.block.BlockPostingsFormat.BLOCK_SIZE;
import static org.apache.lucene.codecs.block.ForUtil.MAX_DATA_SIZE;
import static org.apache.lucene.codecs.block.ForUtil.MAX_ENCODED_SIZE;
import static org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat.BLOCK_SIZE;
import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_DATA_SIZE;
import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_ENCODED_SIZE;
import java.io.IOException;
import java.util.Arrays;

View File

@ -21,10 +21,10 @@ import java.io.IOException;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41Codec;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
import org.apache.lucene.codecs.mocksep.MockSepPostingsFormat;
import org.apache.lucene.codecs.pulsing.Pulsing40PostingsFormat;
import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
import org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
@ -142,7 +142,7 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
assertQuery(new Term("content", "ccc"), dir, 10);
assertQuery(new Term("content", "aaa"), dir, 10);
Lucene40Codec codec = (Lucene40Codec)iwconf.getCodec();
Lucene41Codec codec = (Lucene41Codec)iwconf.getCodec();
iwconf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()))
.setOpenMode(OpenMode.APPEND).setCodec(codec);
@ -158,7 +158,7 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
}
addDocs2(writer, 10);
writer.commit();
codec = (Lucene40Codec)iwconf.getCodec();
codec = (Lucene41Codec)iwconf.getCodec();
assertEquals(30, writer.maxDoc());
assertQuery(new Term("content", "bbb"), dir, 10);
assertQuery(new Term("content", "ccc"), dir, 10); ////
@ -200,8 +200,8 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
}
public static class MockCodec extends Lucene40Codec {
final PostingsFormat lucene40 = new Lucene40PostingsFormat();
public static class MockCodec extends Lucene41Codec {
final PostingsFormat lucene40 = new Lucene41PostingsFormat();
final PostingsFormat simpleText = new SimpleTextPostingsFormat();
final PostingsFormat mockSep = new MockSepPostingsFormat();
@ -217,8 +217,8 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
}
}
public static class MockCodec2 extends Lucene40Codec {
final PostingsFormat lucene40 = new Lucene40PostingsFormat();
public static class MockCodec2 extends Lucene41Codec {
final PostingsFormat lucene40 = new Lucene41PostingsFormat();
final PostingsFormat simpleText = new SimpleTextPostingsFormat();
@Override
@ -268,13 +268,13 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
}
public void testSameCodecDifferentInstance() throws Exception {
Codec codec = new Lucene40Codec() {
Codec codec = new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
if ("id".equals(field)) {
return new Pulsing40PostingsFormat(1);
return new Pulsing41PostingsFormat(1);
} else if ("date".equals(field)) {
return new Pulsing40PostingsFormat(1);
return new Pulsing41PostingsFormat(1);
} else {
return super.getPostingsFormatForField(field);
}
@ -284,13 +284,13 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
}
public void testSameCodecDifferentParams() throws Exception {
Codec codec = new Lucene40Codec() {
Codec codec = new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
if ("id".equals(field)) {
return new Pulsing40PostingsFormat(1);
return new Pulsing41PostingsFormat(1);
} else if ("date".equals(field)) {
return new Pulsing40PostingsFormat(2);
return new Pulsing41PostingsFormat(2);
} else {
return super.getPostingsFormatForField(field);
}

View File

@ -27,8 +27,8 @@ import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
import org.apache.lucene.codecs.pulsing.Pulsing40PostingsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41Codec;
import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
@ -1058,9 +1058,9 @@ public class TestAddIndexes extends LuceneTestCase {
aux2.close();
}
private static final class CustomPerFieldCodec extends Lucene40Codec {
private static final class CustomPerFieldCodec extends Lucene41Codec {
private final PostingsFormat simpleTextFormat = PostingsFormat.forName("SimpleText");
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene40");
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
private final PostingsFormat mockSepFormat = PostingsFormat.forName("MockSep");
@Override
@ -1109,7 +1109,7 @@ public class TestAddIndexes extends LuceneTestCase {
private static final class UnRegisteredCodec extends FilterCodec {
public UnRegisteredCodec() {
super("NotRegistered", new Lucene40Codec());
super("NotRegistered", new Lucene41Codec());
}
}
@ -1138,7 +1138,7 @@ public class TestAddIndexes extends LuceneTestCase {
Directory dir = newDirectory();
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random()));
conf.setCodec(_TestUtil.alwaysPostingsFormat(new Pulsing40PostingsFormat(1 + random().nextInt(20))));
conf.setCodec(_TestUtil.alwaysPostingsFormat(new Pulsing41PostingsFormat(1 + random().nextInt(20))));
IndexWriter w = new IndexWriter(dir, conf);
try {
w.addIndexes(toAdd);

View File

@ -1,4 +1,4 @@
package org.apache.lucene.codecs.lucene40;
package org.apache.lucene.index;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -35,13 +35,13 @@ import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
/**
* Test that a plain Lucene40Codec puts codec headers in all files.
* Test that a plain default puts codec headers in all files.
*/
public class TestAllFilesHaveCodecHeader extends LuceneTestCase {
public void test() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
conf.setCodec(Codec.forName("Lucene40"));
conf.setCodec(Codec.forName("Lucene41"));
// riw should sometimes create docvalues fields, etc
RandomIndexWriter riw = new RandomIndexWriter(random(), dir, conf);
Document doc = new Document();

View File

@ -28,7 +28,7 @@ import java.util.Random;
import java.util.Set;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
@ -828,7 +828,7 @@ public void testFilesOpenClose() throws IOException {
// 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 Lucene40PostingsFormat())));
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));
@ -848,7 +848,7 @@ public void testFilesOpenClose() throws IOException {
writer = new IndexWriter(
dir,
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).
setCodec(_TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat())).
setCodec(_TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat())).
setMergePolicy(newLogMergePolicy(10))
);
writer.addDocument(doc);

View File

@ -110,7 +110,7 @@ public class TestDocTermOrds extends LuceneTestCase {
// Sometimes swap in codec that impls ord():
if (random().nextInt(10) == 7) {
// Make sure terms index has ords:
Codec codec = _TestUtil.alwaysPostingsFormat(PostingsFormat.forName("Lucene40WithOrds"));
Codec codec = _TestUtil.alwaysPostingsFormat(PostingsFormat.forName("Lucene41WithOrds"));
conf.setCodec(codec);
}
@ -207,7 +207,7 @@ public class TestDocTermOrds extends LuceneTestCase {
// Sometimes swap in codec that impls ord():
if (random().nextInt(10) == 7) {
Codec codec = _TestUtil.alwaysPostingsFormat(PostingsFormat.forName("Lucene40WithOrds"));
Codec codec = _TestUtil.alwaysPostingsFormat(PostingsFormat.forName("Lucene41WithOrds"));
conf.setCodec(codec);
}

View File

@ -61,7 +61,7 @@ public class TestDuelingCodecs extends LuceneTestCase {
public void setUp() throws Exception {
super.setUp();
// for now its SimpleText vs Lucene40(random postings format)
// for now its SimpleText vs Lucene41(random postings format)
// as this gives the best overall coverage. when we have more
// codecs we should probably pick 2 from Codec.availableCodecs()

View File

@ -19,7 +19,7 @@ package org.apache.lucene.index;
import org.apache.lucene.store.*;
import org.apache.lucene.analysis.*;
import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
import org.apache.lucene.document.*;
import org.apache.lucene.util.*;
@ -65,7 +65,7 @@ public class TestFlex extends LuceneTestCase {
public void testTermOrd() throws Exception {
Directory d = newDirectory();
IndexWriter w = new IndexWriter(d, newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random())).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat())));
new MockAnalyzer(random())).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat())));
Document doc = new Document();
doc.add(newTextField("f", "a b c", Field.Store.NO));
w.addDocument(doc);

View File

@ -23,7 +23,7 @@ import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.analysis.*;
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.store.Directory;
@ -69,7 +69,7 @@ public class TestMultiLevelSkipList extends LuceneTestCase {
public void testSimpleSkip() throws IOException {
Directory dir = new CountingRAMDirectory(new RAMDirectory());
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new PayloadAnalyzer()).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat())).setMergePolicy(newLogMergePolicy()));
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new PayloadAnalyzer()).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat())).setMergePolicy(newLogMergePolicy()));
Term term = new Term("test", "a");
for (int i = 0; i < 5000; i++) {
Document d1 = new Document();

View File

@ -24,7 +24,7 @@ import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util._TestUtil;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.store.Directory;
@ -75,7 +75,7 @@ public class TestSegmentTermEnum extends LuceneTestCase {
public void testPrevTermAtEnd() throws IOException
{
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat())));
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat())));
addDoc(writer, "aaa bbb");
writer.close();
SegmentReader reader = getOnlySegmentReader(DirectoryReader.open(dir));

View File

@ -25,8 +25,8 @@ import org.apache.lucene.codecs.Codec;
// enough to test the basics via Codec
public class TestNamedSPILoader extends LuceneTestCase {
public void testLookup() {
Codec codec = Codec.forName("Lucene40");
assertEquals("Lucene40", codec.getName());
Codec codec = Codec.forName("Lucene41");
assertEquals("Lucene41", codec.getName());
}
// we want an exception if its not found.
@ -39,6 +39,6 @@ public class TestNamedSPILoader extends LuceneTestCase {
public void testAvailableServices() {
Set<String> codecs = Codec.availableCodecs();
assertTrue(codecs.contains("Lucene40"));
assertTrue(codecs.contains("Lucene41"));
}
}

View File

@ -22,19 +22,19 @@ 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.lucene40.Lucene40PostingsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
/**
* A class used for testing {@link BloomFilteringPostingsFormat} with a concrete
* delegate (Lucene40). Creates a Bloom filter on ALL fields and with tiny
* delegate (Lucene41). Creates a Bloom filter on ALL fields and with tiny
* amounts of memory reserved for the filter. DO NOT USE IN A PRODUCTION
* APPLICATION This is not a realistic application of Bloom Filters as they
* ordinarily are larger and operate on only primary key type fields.
*/
public final class TestBloomFilteredLucene40Postings extends PostingsFormat {
public final class TestBloomFilteredLucene41Postings extends PostingsFormat {
private BloomFilteringPostingsFormat delegate;
@ -54,9 +54,9 @@ public final class TestBloomFilteredLucene40Postings extends PostingsFormat {
}
}
public TestBloomFilteredLucene40Postings() {
super("TestBloomFilteredLucene40Postings");
delegate = new BloomFilteringPostingsFormat(new Lucene40PostingsFormat(),
public TestBloomFilteredLucene41Postings() {
super("TestBloomFilteredLucene41Postings");
delegate = new BloomFilteringPostingsFormat(new Lucene41PostingsFormat(),
new LowMemoryBloomFactory());
}

View File

@ -1,4 +1,4 @@
package org.apache.lucene.codecs.lucene40ords;
package org.apache.lucene.codecs.lucene41ords;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -30,9 +30,9 @@ import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexReader;
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.lucene40.Lucene40Codec; // javadocs
import org.apache.lucene.codecs.lucene40.Lucene40PostingsReader;
import org.apache.lucene.codecs.lucene40.Lucene40PostingsWriter;
import org.apache.lucene.codecs.lucene41.Lucene41Codec; // 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;
import org.apache.lucene.util.BytesRef;
@ -41,18 +41,18 @@ import org.apache.lucene.util.BytesRef;
// any PostingsBaseFormat and make it ord-able...
/**
* Customized version of {@link Lucene40Codec} that uses
* Customized version of {@link Lucene41Codec} that uses
* {@link FixedGapTermsIndexWriter}.
*/
public final class Lucene40WithOrds extends PostingsFormat {
public final class Lucene41WithOrds extends PostingsFormat {
public Lucene40WithOrds() {
super("Lucene40WithOrds");
public Lucene41WithOrds() {
super("Lucene41WithOrds");
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase docs = new Lucene40PostingsWriter(state);
PostingsWriterBase docs = new Lucene41PostingsWriter(state);
// TODO: should we make the terms index more easily
// pluggable? Ie so that this codec would record which
@ -91,7 +91,7 @@ public final class Lucene40WithOrds extends PostingsFormat {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
PostingsReaderBase postings = new Lucene41PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
TermsIndexReaderBase indexReader;
boolean success = false;

View File

@ -29,11 +29,10 @@ import java.util.Set;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.asserting.AssertingPostingsFormat;
import org.apache.lucene.codecs.block.BlockPostingsFormat;
import org.apache.lucene.codecs.bloom.TestBloomFilteredLucene40Postings;
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
import org.apache.lucene.codecs.lucene40ords.Lucene40WithOrds;
import org.apache.lucene.codecs.lucene41.Lucene41Codec;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
import org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings;
import org.apache.lucene.codecs.memory.DirectPostingsFormat;
import org.apache.lucene.codecs.memory.MemoryPostingsFormat;
import org.apache.lucene.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
@ -41,7 +40,7 @@ import org.apache.lucene.codecs.mockintblock.MockVariableIntBlockPostingsFormat;
import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
import org.apache.lucene.codecs.mocksep.MockSepPostingsFormat;
import org.apache.lucene.codecs.nestedpulsing.NestedPulsingPostingsFormat;
import org.apache.lucene.codecs.pulsing.Pulsing40PostingsFormat;
import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
import org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
@ -55,7 +54,7 @@ import org.apache.lucene.util._TestUtil;
* documents in different orders and the test will still be deterministic
* and reproducable.
*/
public class RandomCodec extends Lucene40Codec {
public class RandomCodec extends Lucene41Codec {
/** Shuffled list of postings formats to use for new mappings */
private List<PostingsFormat> formats = new ArrayList<PostingsFormat>();
@ -94,23 +93,22 @@ public class RandomCodec extends Lucene40Codec {
int lowFreqCutoff = _TestUtil.nextInt(random, 2, 100);
add(avoidCodecs,
new Lucene40PostingsFormat(minItemsPerBlock, maxItemsPerBlock),
new BlockPostingsFormat(minItemsPerBlock, maxItemsPerBlock),
new Lucene41PostingsFormat(minItemsPerBlock, maxItemsPerBlock),
new DirectPostingsFormat(LuceneTestCase.rarely(random) ? 1 : (LuceneTestCase.rarely(random) ? Integer.MAX_VALUE : maxItemsPerBlock),
LuceneTestCase.rarely(random) ? 1 : (LuceneTestCase.rarely(random) ? Integer.MAX_VALUE : lowFreqCutoff)),
new Pulsing40PostingsFormat(1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock),
new Pulsing41PostingsFormat(1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock),
// add pulsing again with (usually) different parameters
new Pulsing40PostingsFormat(1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock),
new Pulsing41PostingsFormat(1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock),
//TODO as a PostingsFormat which wraps others, we should allow TestBloomFilteredLucene40Postings 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 TestBloomFilteredLucene40Postings(),
new TestBloomFilteredLucene41Postings(),
new MockSepPostingsFormat(),
new MockFixedIntBlockPostingsFormat(_TestUtil.nextInt(random, 1, 2000)),
new MockVariableIntBlockPostingsFormat( _TestUtil.nextInt(random, 1, 127)),
new MockRandomPostingsFormat(random),
new NestedPulsingPostingsFormat(),
new Lucene40WithOrds(),
new Lucene41WithOrds(),
new SimpleTextPostingsFormat(),
new AssertingPostingsFormat(),
new MemoryPostingsFormat(true, random.nextFloat()),

View File

@ -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.lucene41.Lucene41Codec;
import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
import org.apache.lucene.index.RandomCodec;
@ -129,26 +130,24 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule {
avoidCodecs.addAll(Arrays.asList(a.value()));
}
PREFLEX_IMPERSONATION_IS_ACTIVE = false;
savedCodec = Codec.getDefault();
int randomVal = random.nextInt(10);
/* note: re-enable this if we make a 4.x impersonator
if ("Lucene3x".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) &&
if ("Lucene40".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) &&
"random".equals(TEST_POSTINGSFORMAT) &&
randomVal < 2 &&
!shouldAvoidCodec("Lucene3x"))) { // preflex-only setup
codec = Codec.forName("Lucene3x");
assert (codec instanceof PreFlexRWCodec) : "fix your classpath to have tests-framework.jar before lucene-core.jar";
PREFLEX_IMPERSONATION_IS_ACTIVE = true;
} else */ if (!"random".equals(TEST_POSTINGSFORMAT)) {
!shouldAvoidCodec("Lucene40"))) {
codec = Codec.forName("Lucene40");
// nocommit: assert (codec instanceof PreFlexRWCodec) : "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)) {
format = new MockRandomPostingsFormat(random);
} else {
format = PostingsFormat.forName(TEST_POSTINGSFORMAT);
}
codec = new Lucene40Codec() {
codec = new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return format;

View File

@ -19,7 +19,7 @@ org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat
org.apache.lucene.codecs.mocksep.MockSepPostingsFormat
org.apache.lucene.codecs.nestedpulsing.NestedPulsingPostingsFormat
org.apache.lucene.codecs.ramonly.RAMOnlyPostingsFormat
org.apache.lucene.codecs.lucene40ords.Lucene40WithOrds
org.apache.lucene.codecs.bloom.TestBloomFilteredLucene40Postings
org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds
org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings
org.apache.lucene.codecs.asserting.AssertingPostingsFormat