diff --git a/dev-tools/scripts/smokeTestRelease.py b/dev-tools/scripts/smokeTestRelease.py
index dba87586f70..cc72aabf147 100644
--- a/dev-tools/scripts/smokeTestRelease.py
+++ b/dev-tools/scripts/smokeTestRelease.py
@@ -578,7 +578,7 @@ def verifyUnpacked(project, artifact, unpackPath, version, tmpDir):
l = os.listdir(unpackPath)
textFiles = ['LICENSE', 'NOTICE', 'README']
if project == 'lucene':
- textFiles.extend(('JRE_VERSION_MIGRATION', 'CHANGES', 'MIGRATE'))
+ textFiles.extend(('JRE_VERSION_MIGRATION', 'CHANGES', 'MIGRATE', 'SYSTEM_REQUIREMENTS'))
if isSrc:
textFiles.append('BUILD')
for fileName in textFiles:
@@ -629,6 +629,10 @@ def verifyUnpacked(project, artifact, unpackPath, version, tmpDir):
if project == 'lucene':
if len(l) > 0:
raise RuntimeError('%s: unexpected files/dirs in artifact %s: %s' % (project, artifact, l))
+ else:
+ # TODO: re-enable this check
+ if False and not os.path.exists('%s/solr/SYSTEM_REQUIREMENTS.txt' % unpackPath):
+ raise RuntimeError('%s: solr/SYSTEM_REQUIREMENTS.txt does not exist in artifact %s' % (project, artifact))
if isSrc:
print(' make sure no JARs/WARs in src dist...')
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index d9863e204f4..24f04b348fb 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -56,6 +56,12 @@ Bug Fixes
* LUCENE-4468: Fix rareish integer overflows in Block and Lucene40 postings
formats (Robert Muir)
+
+* LUCENE-4486: Add support for ConstantScoreQuery in Highlighter.
+ (Simon Willnauer)
+
+* LUCENE-4485: When CheckIndex terms, terms/docs pairs and tokens,
+ these counts now all exclude deleted documents. (Mike McCandless)
Optimizations
diff --git a/lucene/SYSTEM_REQUIREMENTS.txt b/lucene/SYSTEM_REQUIREMENTS.txt
new file mode 100644
index 00000000000..d5edcc6766e
--- /dev/null
+++ b/lucene/SYSTEM_REQUIREMENTS.txt
@@ -0,0 +1,16 @@
+# System Requirements
+
+Apache Lucene runs of Java 6 or greater. When using Java 7, be sure to
+install at least Update 1! With all Java versions it is strongly
+recommended to not use experimental `-XX` JVM options. It is also
+recommended to always use the latest update version of your Java VM,
+because bugs may affect Lucene. An overview of known JVM bugs can be
+found on http://wiki.apache.org/lucene-java/SunJavaBugs.
+
+CPU, disk and memory requirements are based on the many choices made in
+implementing Lucene (document size, number of documents, and number of
+hits retrieved to name a few). The benchmarks page has some information
+related to performance on particular platforms.
+
+*To build Apache Lucene from source, refer to the `BUILD.txt` file in
+the distribution directory.*
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountFilterFactory.java
index 83d60671d4c..346d9afb95e 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneous/LimitTokenCountFilterFactory.java
@@ -36,16 +36,13 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
*/
public class LimitTokenCountFilterFactory extends TokenFilterFactory {
+ public static final String MAX_TOKEN_COUNT_KEY = "maxTokenCount";
int maxTokenCount;
@Override
public void init(Map args) {
super.init( args );
- String maxTokenCountArg = args.get("maxTokenCount");
- if (maxTokenCountArg == null) {
- throw new IllegalArgumentException("maxTokenCount is mandatory.");
- }
- maxTokenCount = Integer.parseInt(args.get(maxTokenCountArg));
+ maxTokenCount = getInt(MAX_TOKEN_COUNT_KEY);
}
@Override
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenCountFilterFactory.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenCountFilterFactory.java
new file mode 100644
index 00000000000..9296d0d86ce
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestLimitTokenCountFilterFactory.java
@@ -0,0 +1,55 @@
+package org.apache.lucene.analysis.miscellaneous;
+
+/**
+ * Copyright 2004 The Apache Software Foundation
+ *
+ * Licensed 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 java.io.StringReader;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.analysis.TokenStream;
+
+public class TestLimitTokenCountFilterFactory extends BaseTokenStreamTestCase {
+
+ public void test() throws IOException {
+ LimitTokenCountFilterFactory factory = new LimitTokenCountFilterFactory();
+ Map args = new HashMap();
+ args.put(LimitTokenCountFilterFactory.MAX_TOKEN_COUNT_KEY, "3");
+ factory.init(args);
+ String test = "A1 B2 C3 D4 E5 F6";
+ MockTokenizer tok = new MockTokenizer(new StringReader(test), MockTokenizer.WHITESPACE, false);
+ // LimitTokenCountFilter doesn't consume the entire stream that it wraps
+ tok.setEnableChecks(false);
+ TokenStream stream = factory.create(tok);
+ assertTokenStreamContents(stream, new String[] { "A1", "B2", "C3" });
+
+ // param is required
+ factory = new LimitTokenCountFilterFactory();
+ args = new HashMap();
+ IllegalArgumentException iae = null;
+ try {
+ factory.init(args);
+ } catch (IllegalArgumentException e) {
+ assertTrue("exception doesn't mention param: " + e.getMessage(),
+ 0 < e.getMessage().indexOf(LimitTokenCountFilterFactory.MAX_TOKEN_COUNT_KEY));
+ iae = e;
+ }
+ assertNotNull("no exception thrown", iae);
+ }
+}
diff --git a/lucene/build.xml b/lucene/build.xml
index b26d455ec79..d8d3f0fb440 100644
--- a/lucene/build.xml
+++ b/lucene/build.xml
@@ -33,6 +33,7 @@
-
+
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java
index a3b27a2890e..367e45e2b1e 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java
@@ -69,7 +69,27 @@ public class BlockTermsWriter extends FieldsConsumer {
final FieldInfos fieldInfos;
FieldInfo currentField;
private final TermsIndexWriterBase termsIndexWriter;
- private final List fields = new ArrayList();
+
+ private static class FieldMetaData {
+ public final FieldInfo fieldInfo;
+ public final long numTerms;
+ public final long termsStartPointer;
+ public final long sumTotalTermFreq;
+ public final long sumDocFreq;
+ public final int docCount;
+
+ public FieldMetaData(FieldInfo fieldInfo, long numTerms, long termsStartPointer, long sumTotalTermFreq, long sumDocFreq, int docCount) {
+ assert numTerms > 0;
+ this.fieldInfo = fieldInfo;
+ this.termsStartPointer = termsStartPointer;
+ this.numTerms = numTerms;
+ this.sumTotalTermFreq = sumTotalTermFreq;
+ this.sumDocFreq = sumDocFreq;
+ this.docCount = docCount;
+ }
+ }
+
+ private final List fields = new ArrayList();
// private final String segment;
@@ -108,9 +128,7 @@ public class BlockTermsWriter extends FieldsConsumer {
assert currentField == null || currentField.name.compareTo(field.name) < 0;
currentField = field;
TermsIndexWriterBase.FieldWriter fieldIndexWriter = termsIndexWriter.addField(field, out.getFilePointer());
- final TermsWriter terms = new TermsWriter(fieldIndexWriter, field, postingsWriter);
- fields.add(terms);
- return terms;
+ return new TermsWriter(fieldIndexWriter, field, postingsWriter);
}
@Override
@@ -118,27 +136,18 @@ public class BlockTermsWriter extends FieldsConsumer {
try {
- int nonZeroCount = 0;
- for(TermsWriter field : fields) {
- if (field.numTerms > 0) {
- nonZeroCount++;
- }
- }
-
final long dirStart = out.getFilePointer();
- out.writeVInt(nonZeroCount);
- for(TermsWriter field : fields) {
- if (field.numTerms > 0) {
- out.writeVInt(field.fieldInfo.number);
- out.writeVLong(field.numTerms);
- out.writeVLong(field.termsStartPointer);
- if (field.fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
- out.writeVLong(field.sumTotalTermFreq);
- }
- out.writeVLong(field.sumDocFreq);
- out.writeVInt(field.docCount);
+ out.writeVInt(fields.size());
+ for(FieldMetaData field : fields) {
+ out.writeVInt(field.fieldInfo.number);
+ out.writeVLong(field.numTerms);
+ out.writeVLong(field.termsStartPointer);
+ if (field.fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
+ out.writeVLong(field.sumTotalTermFreq);
}
+ out.writeVLong(field.sumDocFreq);
+ out.writeVInt(field.docCount);
}
writeTrailer(dirStart);
} finally {
@@ -249,6 +258,14 @@ public class BlockTermsWriter extends FieldsConsumer {
this.sumDocFreq = sumDocFreq;
this.docCount = docCount;
fieldIndexWriter.finish(out.getFilePointer());
+ if (numTerms > 0) {
+ fields.add(new FieldMetaData(fieldInfo,
+ numTerms,
+ termsStartPointer,
+ sumTotalTermFreq,
+ sumDocFreq,
+ docCount));
+ }
}
private int sharedPrefix(BytesRef term1, BytesRef term2) {
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java
index 97e22f4a03d..936d4ed5d61 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java
@@ -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 {
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing40PostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing41PostingsFormat.java
similarity index 70%
rename from lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing40PostingsFormat.java
rename to lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing41PostingsFormat.java
index faf8df2bbf8..9946062e09b 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing40PostingsFormat.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing41PostingsFormat.java
@@ -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() {
+ /** Inlines docFreq=1 terms, otherwise uses the normal "Lucene41" format. */
+ public Pulsing41PostingsFormat() {
this(1);
}
- /** Inlines docFreq=freqCutoff
terms, otherwise uses the normal "Lucene40" format. */
- public Pulsing40PostingsFormat(int freqCutoff) {
+ /** Inlines docFreq=freqCutoff
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=freqCutoff
terms, otherwise uses the normal "Lucene40" format. */
- public Pulsing40PostingsFormat(int freqCutoff, int minBlockSize, int maxBlockSize) {
- super("Pulsing40", new Lucene40PostingsBaseFormat(), freqCutoff, minBlockSize, maxBlockSize);
+ /** Inlines docFreq=freqCutoff
terms, otherwise uses the normal "Lucene41" format. */
+ public Pulsing41PostingsFormat(int freqCutoff, int minBlockSize, int maxBlockSize) {
+ super("Pulsing41", new Lucene41PostingsBaseFormat(), freqCutoff, minBlockSize, maxBlockSize);
}
}
diff --git a/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat b/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
index 72b05c5e74e..22062983d91 100644
--- a/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
+++ b/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
@@ -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
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestFixedGapPostingsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestFixedGapPostingsFormat.java
index d45b6828d1a..141ff99f5fe 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestFixedGapPostingsFormat.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/blockterms/TestFixedGapPostingsFormat.java
@@ -19,8 +19,8 @@ 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.lucene41.Lucene41Codec;
+import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
import org.apache.lucene.index.BasePostingsFormatTestCase;
/**
@@ -29,8 +29,8 @@ 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 Codec codec = new Lucene40Codec() {
+ private final PostingsFormat postings = new Lucene41WithOrds();
+ private final Codec codec = new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postings;
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/bloom/TestBloomPostingsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/bloom/TestBloomPostingsFormat.java
index 3bd9a90935c..6c3034c5fd3 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/bloom/TestBloomPostingsFormat.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/bloom/TestBloomPostingsFormat.java
@@ -19,15 +19,15 @@ package org.apache.lucene.codecs.bloom;
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;
/**
* Basic tests for BloomPostingsFormat
*/
public class TestBloomPostingsFormat extends BasePostingsFormatTestCase {
- private final PostingsFormat postings = new TestBloomFilteredLucene40Postings();
- private final Codec codec = new Lucene40Codec() {
+ private final PostingsFormat postings = new TestBloomFilteredLucene41Postings();
+ private final Codec codec = new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postings;
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java
index 9b25a4c1090..bb3a4824589 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java
@@ -23,7 +23,7 @@ import java.util.Collections;
import java.util.List;
import org.apache.lucene.analysis.MockAnalyzer;
-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.DoubleField;
import org.apache.lucene.document.Field;
@@ -90,10 +90,10 @@ public class TestCompressingStoredFieldsFormat extends LuceneTestCase {
if (random().nextBoolean() && (i % (data.length / 10) == 0)) {
iw.w.close();
// switch codecs
- if (iwConf.getCodec() instanceof Lucene40Codec) {
+ if (iwConf.getCodec() instanceof Lucene41Codec) {
iwConf.setCodec(CompressingCodec.randomInstance(random()));
} else {
- iwConf.setCodec(new Lucene40Codec());
+ iwConf.setCodec(new Lucene41Codec());
}
iw = new RandomIndexWriter(random(), dir, iwConf);
}
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/intblock/TestFixedIntBlockPostingsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/intblock/TestFixedIntBlockPostingsFormat.java
index e6338a0a674..93a1b548a62 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/intblock/TestFixedIntBlockPostingsFormat.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/intblock/TestFixedIntBlockPostingsFormat.java
@@ -19,7 +19,7 @@ package org.apache.lucene.codecs.intblock;
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.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
import org.apache.lucene.index.BasePostingsFormatTestCase;
@@ -29,7 +29,7 @@ import org.apache.lucene.index.BasePostingsFormatTestCase;
public class TestFixedIntBlockPostingsFormat extends BasePostingsFormatTestCase {
// TODO: randomize blocksize
private final PostingsFormat postings = new MockFixedIntBlockPostingsFormat();
- private final Codec codec = new Lucene40Codec() {
+ private final Codec codec = new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postings;
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/intblock/TestVariableIntBlockPostingsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/intblock/TestVariableIntBlockPostingsFormat.java
index c7955acc9f5..156f91840b0 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/intblock/TestVariableIntBlockPostingsFormat.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/intblock/TestVariableIntBlockPostingsFormat.java
@@ -19,7 +19,7 @@ package org.apache.lucene.codecs.intblock;
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.codecs.mockintblock.MockVariableIntBlockPostingsFormat;
import org.apache.lucene.index.BasePostingsFormatTestCase;
@@ -29,7 +29,7 @@ import org.apache.lucene.index.BasePostingsFormatTestCase;
public class TestVariableIntBlockPostingsFormat extends BasePostingsFormatTestCase {
// TODO: randomize blocksize
private final PostingsFormat postings = new MockVariableIntBlockPostingsFormat();
- private final Codec codec = new Lucene40Codec() {
+ private final Codec codec = new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postings;
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestDirectPostingsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestDirectPostingsFormat.java
index caf55a83084..bab45bcc366 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestDirectPostingsFormat.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestDirectPostingsFormat.java
@@ -19,7 +19,7 @@ package org.apache.lucene.codecs.memory;
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;
/**
@@ -29,7 +29,7 @@ public class TestDirectPostingsFormat extends BasePostingsFormatTestCase {
// TODO: randomize parameters
private final PostingsFormat postings = new DirectPostingsFormat();
- private final Codec codec = new Lucene40Codec() {
+ private final Codec codec = new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postings;
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestMemoryPostingsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestMemoryPostingsFormat.java
index ca07382b3a9..93892c7b640 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestMemoryPostingsFormat.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestMemoryPostingsFormat.java
@@ -19,7 +19,7 @@ package org.apache.lucene.codecs.memory;
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;
/**
@@ -28,7 +28,7 @@ import org.apache.lucene.index.BasePostingsFormatTestCase;
public class TestMemoryPostingsFormat extends BasePostingsFormatTestCase {
// TODO: randomize doPack
private final PostingsFormat postings = new MemoryPostingsFormat();
- private final Codec codec = new Lucene40Codec() {
+ private final Codec codec = new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postings;
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/pulsing/Test10KPulsings.java b/lucene/codecs/src/test/org/apache/lucene/codecs/pulsing/Test10KPulsings.java
index 3e47dc549cc..6e53a631ea3 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/pulsing/Test10KPulsings.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/pulsing/Test10KPulsings.java
@@ -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);
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/pulsing/TestPulsingPostingsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/pulsing/TestPulsingPostingsFormat.java
index 3156323df61..75271d33d17 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/pulsing/TestPulsingPostingsFormat.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/pulsing/TestPulsingPostingsFormat.java
@@ -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;
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/pulsing/TestPulsingReuse.java b/lucene/codecs/src/test/org/apache/lucene/codecs/pulsing/TestPulsingReuse.java
index 488fca34baf..cfa520aef02 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/pulsing/TestPulsingReuse.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/pulsing/TestPulsingReuse.java
@@ -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));
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/sep/TestSepPostingsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/sep/TestSepPostingsFormat.java
index 318822ce19d..8c6df1d93fd 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/sep/TestSepPostingsFormat.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/sep/TestSepPostingsFormat.java
@@ -19,7 +19,7 @@ package org.apache.lucene.codecs.sep;
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.codecs.mocksep.MockSepPostingsFormat;
import org.apache.lucene.index.BasePostingsFormatTestCase;
@@ -29,7 +29,7 @@ import org.apache.lucene.index.BasePostingsFormatTestCase;
public class TestSepPostingsFormat extends BasePostingsFormatTestCase {
// TODO: randomize cutoff
private final PostingsFormat postings = new MockSepPostingsFormat();
- private final Codec codec = new Lucene40Codec() {
+ private final Codec codec = new Lucene41Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postings;
diff --git a/lucene/common-build.xml b/lucene/common-build.xml
index 1f9c09e8a0a..1f1b5c6cd74 100644
--- a/lucene/common-build.xml
+++ b/lucene/common-build.xml
@@ -771,6 +771,7 @@
+
@@ -808,7 +809,7 @@
dynamicAssignmentRatio="${tests.dynamicAssignmentRatio}"
shuffleOnSlave="true"
- leaveTemporary="false"
+ leaveTemporary="${tests.leaveTemporary}"
seed="${tests.seed}"
heartbeat="${tests.heartbeat}"
@@ -1116,6 +1117,10 @@ ant -Dtests.timestamps=on ...
# Override forked JVM file.encoding
ant -Dtests.file.encoding=XXX ...
+# Don't remove temporary files under slave directories, even if
+# the test passes.
+ant -Dtests.leaveTemporary=true
+
# Output test files and reports.
${tests-output}/tests-report.txt - full ASCII tests report
${tests-output}/tests-failures.txt - failures only (if any)
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java
index c3c4c8cf5aa..39ced1d8e44 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java
@@ -228,7 +228,30 @@ public class BlockTreeTermsWriter extends FieldsConsumer {
final PostingsWriterBase postingsWriter;
final FieldInfos fieldInfos;
FieldInfo currentField;
- private final List fields = new ArrayList();
+
+ private static class FieldMetaData {
+ public final FieldInfo fieldInfo;
+ public final BytesRef rootCode;
+ public final long numTerms;
+ public final long indexStartFP;
+ public final long sumTotalTermFreq;
+ public final long sumDocFreq;
+ public final int docCount;
+
+ public FieldMetaData(FieldInfo fieldInfo, BytesRef rootCode, long numTerms, long indexStartFP, long sumTotalTermFreq, long sumDocFreq, int docCount) {
+ assert numTerms > 0;
+ this.fieldInfo = fieldInfo;
+ assert rootCode != null: "field=" + fieldInfo.name + " numTerms=" + numTerms;
+ this.rootCode = rootCode;
+ this.indexStartFP = indexStartFP;
+ this.numTerms = numTerms;
+ this.sumTotalTermFreq = sumTotalTermFreq;
+ this.sumDocFreq = sumDocFreq;
+ this.docCount = docCount;
+ }
+ }
+
+ private final List fields = new ArrayList();
// private final String segment;
/** Create a new writer. The number of items (terms or
@@ -313,9 +336,7 @@ public class BlockTreeTermsWriter extends FieldsConsumer {
//if (DEBUG) System.out.println("\nBTTW.addField seg=" + segment + " field=" + field.name);
assert currentField == null || currentField.name.compareTo(field.name) < 0;
currentField = field;
- final TermsWriter terms = new TermsWriter(field);
- fields.add(terms);
- return terms;
+ return new TermsWriter(field);
}
static long encodeOutput(long fp, boolean hasTerms, boolean isFloor) {
@@ -1007,6 +1028,14 @@ public class BlockTreeTermsWriter extends FieldsConsumer {
// System.out.println("SAVED to " + dotFileName);
// w.close();
// }
+
+ fields.add(new FieldMetaData(fieldInfo,
+ ((PendingBlock) pending.get(0)).index.getEmptyOutput(),
+ numTerms,
+ indexStartFP,
+ sumTotalTermFreq,
+ sumDocFreq,
+ docCount));
} else {
assert sumTotalTermFreq == 0 || fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY && sumTotalTermFreq == -1;
assert sumDocFreq == 0;
@@ -1024,34 +1053,23 @@ public class BlockTreeTermsWriter extends FieldsConsumer {
IOException ioe = null;
try {
- int nonZeroCount = 0;
- for(TermsWriter field : fields) {
- if (field.numTerms > 0) {
- nonZeroCount++;
- }
- }
-
final long dirStart = out.getFilePointer();
final long indexDirStart = indexOut.getFilePointer();
- out.writeVInt(nonZeroCount);
+ out.writeVInt(fields.size());
- for(TermsWriter field : fields) {
- if (field.numTerms > 0) {
- //System.out.println(" field " + field.fieldInfo.name + " " + field.numTerms + " terms");
- out.writeVInt(field.fieldInfo.number);
- out.writeVLong(field.numTerms);
- final BytesRef rootCode = ((PendingBlock) field.pending.get(0)).index.getEmptyOutput();
- assert rootCode != null: "field=" + field.fieldInfo.name + " numTerms=" + field.numTerms;
- out.writeVInt(rootCode.length);
- out.writeBytes(rootCode.bytes, rootCode.offset, rootCode.length);
- if (field.fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
- out.writeVLong(field.sumTotalTermFreq);
- }
- out.writeVLong(field.sumDocFreq);
- out.writeVInt(field.docCount);
- indexOut.writeVLong(field.indexStartFP);
+ for(FieldMetaData field : fields) {
+ //System.out.println(" field " + field.fieldInfo.name + " " + field.numTerms + " terms");
+ out.writeVInt(field.fieldInfo.number);
+ out.writeVLong(field.numTerms);
+ out.writeVInt(field.rootCode.length);
+ out.writeBytes(field.rootCode.bytes, field.rootCode.offset, field.rootCode.length);
+ if (field.fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
+ out.writeVLong(field.sumTotalTermFreq);
}
+ out.writeVLong(field.sumDocFreq);
+ out.writeVInt(field.docCount);
+ indexOut.writeVLong(field.indexStartFP);
}
writeTrailer(out, dirStart);
writeIndexTrailer(indexOut, indexDirStart);
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
index 1892df6d300..7a473a3ed38 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
@@ -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.
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java b/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java
index 4dfae68232a..12f17197d2b 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java
@@ -21,13 +21,13 @@ package org.apache.lucene.codecs;
* A codec that forwards all its method calls to another codec.
*
* 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}:
*
* public final class CustomCodec extends FilterCodec {
*
* public CustomCodec() {
- * super("CustomCodec", new Lucene40Codec());
+ * super("CustomCodec", new Lucene41Codec());
* }
*
* public LiveDocsFormat liveDocsFormat() {
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java
index b98205ec8cd..a0d66af61d0 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java
@@ -36,12 +36,13 @@ 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).
-public class Lucene40Codec extends Codec {
+@Deprecated
+public final class Lucene40Codec extends Codec {
private final StoredFieldsFormat fieldsFormat = new Lucene40StoredFieldsFormat();
private final TermVectorsFormat vectorsFormat = new Lucene40TermVectorsFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene40FieldInfosFormat();
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsBaseFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsBaseFormat.java
index df6611922e2..eaf452d6252 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsBaseFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsBaseFormat.java
@@ -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");
}
}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java
index 16d9c47ed97..1f9c28efdb4 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java
@@ -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}.
*
- * @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
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java
index 64d2e49b1ff..a3729e2f1da 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java
@@ -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();
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SkipListReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SkipListReader.java
index 4cef37a5977..1580a390ba9 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SkipListReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SkipListReader.java
@@ -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;
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/block/ForUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/ForUtil.java
similarity index 98%
rename from lucene/codecs/src/java/org/apache/lucene/codecs/block/ForUtil.java
rename to lucene/core/src/java/org/apache/lucene/codecs/lucene41/ForUtil.java
index fc52520f551..88f70a249c1 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/block/ForUtil.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/ForUtil.java
@@ -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,
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java
new file mode 100644
index 00000000000..48219582b75
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java
@@ -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.
+ *
+ * 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 field
.
+ *
+ * The default implementation always returns "Lucene41"
+ */
+ public PostingsFormat getPostingsFormatForField(String field) {
+ return defaultFormat;
+ }
+
+ private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
+}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsBaseFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsBaseFormat.java
new file mode 100644
index 00000000000..0360c0d2709
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsBaseFormat.java
@@ -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);
+ }
+}
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java
similarity index 95%
rename from lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsFormat.java
rename to lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java
index 73ce8df62c4..3cbc9653bed 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java
@@ -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.
*
*
NOTE: this format is still experimental and
@@ -58,7 +58,7 @@ import org.apache.lucene.util.packed.PackedInts;
*
*
* Block structure:
- * When the postings are long enough, BlockPostingsFormat will try to encode most integer data
+ *
When the postings are long enough, Lucene41PostingsFormat will try to encode most integer data
* as a packed block.
* 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.
@@ -161,7 +161,7 @@ import org.apache.lucene.util.packed.PackedInts;
* 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).
+ * (i.e. 8 in Lucene41PostingsFormat).
*
*
*
@@ -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
* skipIntervalth, 2*skipIntervalth ... posting in the list. However,
- * in BlockPostingsFormat, the skip data is saved for skipInterval+1th,
+ * in Lucene41PostingsFormat, the skip data is saved for skipInterval+1th,
* 2*skipInterval+1th ... posting (skipInterval==PackedBlockSize in this case).
* When DocFreq is multiple of PackedBlockSize, MultiLevelSkipListWriter will expect one
- * more skip data than BlockSkipWriter.
+ * more skip data than Lucene41SkipWriter.
* SkipDatum is the metadata of one skip entry.
* For the first block (no matter packed or VInt), it is omitted.
* DocSkip records the document number of every PackedBlockSizeth 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: Frequencies and Skip Data
@@ -380,12 +380,18 @@ 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() {
+ /** Creates {@code Lucene41PostingsFormat} with default
+ * settings. */
+ public Lucene41PostingsFormat() {
this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
}
- public BlockPostingsFormat(int minTermBlockSize, int maxTermBlockSize) {
- super("Block");
+ /** 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;
assert minTermBlockSize > 1;
this.maxTermBlockSize = maxTermBlockSize;
@@ -399,7 +405,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 +424,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,
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java
similarity index 95%
rename from lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsReader.java
rename to lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java
index 77b766743f0..6292b18e6a7 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java
@@ -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,36 @@ final class BlockPostingsReader extends PostingsReaderBase {
// public static boolean DEBUG = false;
- public BlockPostingsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo segmentInfo, IOContext ioContext, String segmentSuffix) throws IOException {
+ /** Sole constructor. */
+ 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 +110,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 +322,7 @@ final class BlockPostingsReader extends PostingsReaderBase {
private int docBufferUpto;
- private BlockSkipReader skipper;
+ private Lucene41SkipReader skipper;
private boolean skipped;
final IndexInput startDocIn;
@@ -353,7 +354,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 +487,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 +503,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 +578,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 +629,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 +798,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 +988,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 +1045,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 +1283,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,
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java
similarity index 92%
rename from lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsWriter.java
rename to lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java
index 305e1f38d51..19391afcaa7 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockPostingsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java
@@ -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,15 +58,14 @@ 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
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;
@@ -112,12 +111,14 @@ 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 {
+ /** Creates a postings writer with the specified PackedInts overhead ratio */
+ // TODO: does this ctor even make sense?
+ 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 +128,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 +149,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 +173,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 +183,8 @@ final class BlockPostingsWriter extends PostingsWriterBase {
encoded = new byte[MAX_ENCODED_SIZE];
}
- public BlockPostingsWriter(SegmentWriteState state) throws IOException {
+ /** Creates a postings writer with PackedInts.COMPACT
*/
+ public Lucene41PostingsWriter(SegmentWriteState state) throws IOException {
this(state, PackedInts.COMPACT);
}
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockSkipReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipReader.java
similarity index 91%
rename from lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockSkipReader.java
rename to lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipReader.java
index e5803fd9696..483b0ec21df 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockSkipReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipReader.java
@@ -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
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockSkipWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipWriter.java
similarity index 94%
rename from lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockSkipWriter.java
rename to lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipWriter.java
index 409930c6fed..1bd082859d9 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/block/BlockSkipWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SkipWriter.java
@@ -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;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html
new file mode 100644
index 00000000000..14782803a1c
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html
@@ -0,0 +1,396 @@
+
+
+
+
+
+
+
+Lucene 4.1 file format.
+
+Apache Lucene - Index File Formats
+
+
+Introduction
+
+
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
+docs/
that was distributed with
+the version you are using.
+
Apache Lucene is written in Java, but several efforts are underway to write
+versions of
+Lucene in other programming languages. 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.
+
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.
+
+
+Definitions
+
+
The fundamental concepts in Lucene are index, document, field and term.
+
An index contains a sequence of documents.
+
+- A document is a sequence of fields.
+- A field is a named sequence of terms.
+- A term is a sequence of bytes.
+
+
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.
+
+
Inverted Indexing
+
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
+inverted index. 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.
+
+
Types of Fields
+
In Lucene, fields may be stored, in which case their text is stored
+in the index literally, in a non-inverted manner. Fields that are inverted are
+called indexed. A field may be both stored and indexed.
+
The text of a field may be tokenized 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.
+
See the {@link org.apache.lucene.document.Field Field}
+java docs for more information on Fields.
+
+
Segments
+
Lucene indexes may be composed of multiple sub-indexes, or segments.
+Each segment is a fully independent index, which could be searched separately.
+Indexes evolve by:
+
+- Creating new segments for newly added documents.
+- Merging existing segments.
+
+
Searches may involve multiple segments and/or multiple indexes, each index
+potentially composed of a set of segments.
+
+
Document Numbers
+
Internally, Lucene refers to documents by an integer document number.
+The first document added to an index is numbered zero, and each subsequent
+document added gets a number one greater than the previous.
+
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:
+
+-
+
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 base 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.
+
+-
+
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.
+
+
+
+
+Index Structure Overview
+
+
Each segment index maintains the following:
+
+-
+{@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,
+
+-
+{@link org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat Field names}.
+ This contains the set of field names used in the index.
+
+-
+{@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.
+
+-
+{@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
+proximity data.
+
+-
+{@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)
+
+-
+{@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.
+
+-
+{@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.
+
+-
+{@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
+
+-
+{@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.
+
+-
+{@link org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat Deleted documents}.
+An optional file indicating which documents are deleted.
+
+
+
Details on each of these are provided in their linked pages.
+
+
+File Naming
+
+
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)
+
Typically, all segments in an index are stored in a single directory,
+although this is not required.
+
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.
+
+
+Summary of File Extensions
+
+
The following table summarizes the names and extensions of the files in
+Lucene:
+
+
+Name |
+Extension |
+Brief Description |
+
+
+{@link org.apache.lucene.index.SegmentInfos Segments File} |
+segments.gen, segments_N |
+Stores information about a commit point |
+
+
+Lock File |
+write.lock |
+The Write lock prevents multiple IndexWriters from writing to the same
+file. |
+
+
+{@link org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat Segment Info} |
+.si |
+Stores metadata about a segment |
+
+
+{@link org.apache.lucene.store.CompoundFileDirectory Compound File} |
+.cfs, .cfe |
+An optional "virtual" file consisting of all the other index files for
+systems that frequently run out of file handles. |
+
+
+{@link org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat Fields} |
+.fnm |
+Stores information about the fields |
+
+
+{@link org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat Field Index} |
+.fdx |
+Contains pointers to field data |
+
+
+{@link org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat Field Data} |
+.fdt |
+The stored fields for documents |
+
+
+{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Dictionary} |
+.tim |
+The term dictionary, stores term info |
+
+
+{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Index} |
+.tip |
+The index into the Term Dictionary |
+
+
+{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Frequencies} |
+.doc |
+Contains the list of docs which contain each term along with frequency |
+
+
+{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Positions} |
+.pos |
+Stores position information about where a term occurs in the index |
+
+
+{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Payloads} |
+.pay |
+Stores additional per-position metadata information such as character offsets and user payloads |
+
+
+{@link org.apache.lucene.codecs.lucene40.Lucene40NormsFormat Norms} |
+.nrm.cfs, .nrm.cfe |
+Encodes length and boost factors for docs and fields |
+
+
+{@link org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat Per-Document Values} |
+.dv.cfs, .dv.cfe |
+Encodes additional scoring factors or other per-document information. |
+
+
+{@link org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat Term Vector Index} |
+.tvx |
+Stores offset into the document data file |
+
+
+{@link org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat Term Vector Documents} |
+.tvd |
+Contains information about each document that has term vectors |
+
+
+{@link org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat Term Vector Fields} |
+.tvf |
+The field level info about term vectors |
+
+
+{@link org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat Deleted Documents} |
+.del |
+Info about what files are deleted |
+
+
+
+
+Lock File
+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.
+
+History
+Compatibility notes are provided in this document, describing how file
+formats have changed from prior versions:
+
+- 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.
+- In version 2.3, the file format was changed to allow segments to share a
+single set of doc store (vectors & 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).
+- In version 2.4, Strings are now written as true UTF-8 byte sequence, not
+Java's modified UTF-8. See
+LUCENE-510 for details.
+- In version 2.9, an optional opaque Map<String,String> CommitUserData
+may be passed to IndexWriter's commit methods (and later retrieved), which is
+recorded in the segments_N file. See
+LUCENE-1382 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
+LUCENE-1654 for details.
+- 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 LUCENE-1960
+for details.
+- In version 3.1, segments records the code version that created them. See
+LUCENE-2720 for details.
+Additionally segments track explicitly whether or not they have term vectors.
+See LUCENE-2811
+for details.
+- In version 3.2, numeric fields are written as natively to stored fields
+file, previously they were stored in text format only.
+- In version 3.4, fields can omit position data while still indexing term
+frequencies.
+- 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.
+- 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.
+
+
+Limitations
+
+
When referring to term numbers, Lucene's current implementation uses a Java
+int
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.
+
Similarly, Lucene uses a Java int
to refer to
+document numbers, and the index file format uses an Int32
+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 UInt64
values, or
+better yet, {@link org.apache.lucene.store.DataOutput#writeVInt VInt} values which have no limit.
+
+
+
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/package.html b/lucene/core/src/java/org/apache/lucene/codecs/package.html
index e6de64d057b..91a65458ac1 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/package.html
+++ b/lucene/core/src/java/org/apache/lucene/codecs/package.html
@@ -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.