mirror of https://github.com/apache/lucene.git
catch up with trunk
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene3846@1399701 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
commit
0c32e16186
|
@ -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...')
|
||||
|
|
|
@ -57,6 +57,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
|
||||
|
||||
* LUCENE-4443: BlockPostingsFormat no longer writes unnecessary offsets
|
||||
|
|
|
@ -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.*
|
|
@ -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<String, String> 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
|
||||
|
|
|
@ -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<String, String> args = new HashMap<String, String>();
|
||||
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<String, String>();
|
||||
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);
|
||||
}
|
||||
}
|
|
@ -33,6 +33,7 @@
|
|||
<patternset id="binary.root.dist.patterns"
|
||||
includes="LICENSE.txt,NOTICE.txt,README.txt,
|
||||
MIGRATE.txt,JRE_VERSION_MIGRATION.txt,
|
||||
SYSTEM_REQUIREMENTS.txt,
|
||||
CHANGES.txt,
|
||||
**/lib/*.jar,
|
||||
licenses/**,
|
||||
|
@ -297,7 +298,7 @@
|
|||
</xslt>
|
||||
|
||||
<pegdown todir="${javadoc.dir}">
|
||||
<fileset dir="." includes="MIGRATE.txt,JRE_VERSION_MIGRATION.txt"/>
|
||||
<fileset dir="." includes="MIGRATE.txt,JRE_VERSION_MIGRATION.txt,SYSTEM_REQUIREMENTS.txt"/>
|
||||
<globmapper from="*.txt" to="*.html"/>
|
||||
</pegdown>
|
||||
|
||||
|
|
|
@ -69,7 +69,27 @@ public class BlockTermsWriter extends FieldsConsumer {
|
|||
final FieldInfos fieldInfos;
|
||||
FieldInfo currentField;
|
||||
private final TermsIndexWriterBase termsIndexWriter;
|
||||
private final List<TermsWriter> fields = new ArrayList<TermsWriter>();
|
||||
|
||||
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<FieldMetaData> fields = new ArrayList<FieldMetaData>();
|
||||
|
||||
// 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) {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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=<code>freqCutoff</code> terms, otherwise uses the normal "Lucene40" format. */
|
||||
public Pulsing40PostingsFormat(int freqCutoff) {
|
||||
/** Inlines docFreq=<code>freqCutoff</code> terms, otherwise uses the normal "Lucene41" format. */
|
||||
public Pulsing41PostingsFormat(int freqCutoff) {
|
||||
this(freqCutoff, BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
|
||||
}
|
||||
|
||||
/** Inlines docFreq=<code>freqCutoff</code> terms, otherwise uses the normal "Lucene40" format. */
|
||||
public Pulsing40PostingsFormat(int freqCutoff, int minBlockSize, int maxBlockSize) {
|
||||
super("Pulsing40", new Lucene40PostingsBaseFormat(), freqCutoff, minBlockSize, maxBlockSize);
|
||||
/** Inlines docFreq=<code>freqCutoff</code> terms, otherwise uses the normal "Lucene41" format. */
|
||||
public Pulsing41PostingsFormat(int freqCutoff, int minBlockSize, int maxBlockSize) {
|
||||
super("Pulsing41", new Lucene41PostingsBaseFormat(), freqCutoff, minBlockSize, maxBlockSize);
|
||||
}
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -771,6 +771,7 @@
|
|||
<property name="tests.method" value="" />
|
||||
<property name="tests.dynamicAssignmentRatio" value="0.50" /> <!-- 50% of suites -->
|
||||
<property name="tests.haltonfailure" value="true" />
|
||||
<property name="tests.leaveTemporary" value="false" />
|
||||
<property name="tests.iters" value="" />
|
||||
<property name="tests.dups" value="1" />
|
||||
|
||||
|
@ -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)
|
||||
|
|
|
@ -228,7 +228,30 @@ public class BlockTreeTermsWriter extends FieldsConsumer {
|
|||
final PostingsWriterBase postingsWriter;
|
||||
final FieldInfos fieldInfos;
|
||||
FieldInfo currentField;
|
||||
private final List<TermsWriter> fields = new ArrayList<TermsWriter>();
|
||||
|
||||
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<FieldMetaData> fields = new ArrayList<FieldMetaData>();
|
||||
// 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);
|
||||
|
|
|
@ -119,7 +119,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
|
|||
loader.reload(classloader);
|
||||
}
|
||||
|
||||
private static Codec defaultCodec = Codec.forName("Lucene40");
|
||||
private static Codec defaultCodec = Codec.forName("Lucene41");
|
||||
|
||||
/** expert: returns the default codec used for newly created
|
||||
* {@link IndexWriterConfig}s.
|
||||
|
|
|
@ -21,13 +21,13 @@ package org.apache.lucene.codecs;
|
|||
* A codec that forwards all its method calls to another codec.
|
||||
* <p>
|
||||
* Extend this class when you need to reuse the functionality of an existing
|
||||
* codec. For example, if you want to build a codec that redefines Lucene40's
|
||||
* codec. For example, if you want to build a codec that redefines Lucene41's
|
||||
* {@link LiveDocsFormat}:
|
||||
* <pre class="prettyprint">
|
||||
* public final class CustomCodec extends FilterCodec {
|
||||
*
|
||||
* public CustomCodec() {
|
||||
* super("CustomCodec", new Lucene40Codec());
|
||||
* super("CustomCodec", new Lucene41Codec());
|
||||
* }
|
||||
*
|
||||
* public LiveDocsFormat liveDocsFormat() {
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -29,9 +29,10 @@ import org.apache.lucene.index.SegmentWriteState;
|
|||
* Provides a {@link PostingsReaderBase} and {@link
|
||||
* PostingsWriterBase}.
|
||||
*
|
||||
* @lucene.experimental */
|
||||
* @deprecated Only for reading old 4.0 segments */
|
||||
|
||||
// TODO: should these also be named / looked up via SPI?
|
||||
@Deprecated
|
||||
public final class Lucene40PostingsBaseFormat extends PostingsBaseFormat {
|
||||
|
||||
/** Sole constructor. */
|
||||
|
@ -46,6 +47,6 @@ public final class Lucene40PostingsBaseFormat extends PostingsBaseFormat {
|
|||
|
||||
@Override
|
||||
public PostingsWriterBase postingsWriterBase(SegmentWriteState state) throws IOException {
|
||||
return new Lucene40PostingsWriter(state);
|
||||
throw new UnsupportedOperationException("this codec can only be used for reading");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -211,15 +211,18 @@ import org.apache.lucene.util.fst.FST; // javadocs
|
|||
* previous occurrence and an OffsetLength follows. Offset data is only written for
|
||||
* {@link IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS}.</p>
|
||||
*
|
||||
* @lucene.experimental */
|
||||
* @deprecated Only for reading old 4.0 segments */
|
||||
|
||||
// TODO: this class could be created by wrapping
|
||||
// BlockTreeTermsDict around Lucene40PostingsBaseFormat; ie
|
||||
// we should not duplicate the code from that class here:
|
||||
public final class Lucene40PostingsFormat extends PostingsFormat {
|
||||
@Deprecated
|
||||
public class Lucene40PostingsFormat extends PostingsFormat {
|
||||
|
||||
private final int minBlockSize;
|
||||
private final int maxBlockSize;
|
||||
/** minimum items (terms or sub-blocks) per block for BlockTree */
|
||||
protected final int minBlockSize;
|
||||
/** maximum items (terms or sub-blocks) per block for BlockTree */
|
||||
protected final int maxBlockSize;
|
||||
|
||||
/** Creates {@code Lucene40PostingsFormat} with default
|
||||
* settings. */
|
||||
|
@ -231,7 +234,7 @@ public final class Lucene40PostingsFormat extends PostingsFormat {
|
|||
* values for {@code minBlockSize} and {@code
|
||||
* maxBlockSize} passed to block terms dictionary.
|
||||
* @see BlockTreeTermsWriter#BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int) */
|
||||
public Lucene40PostingsFormat(int minBlockSize, int maxBlockSize) {
|
||||
private Lucene40PostingsFormat(int minBlockSize, int maxBlockSize) {
|
||||
super("Lucene40");
|
||||
this.minBlockSize = minBlockSize;
|
||||
assert minBlockSize > 1;
|
||||
|
@ -240,22 +243,7 @@ public final class Lucene40PostingsFormat extends PostingsFormat {
|
|||
|
||||
@Override
|
||||
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
PostingsWriterBase docs = new Lucene40PostingsWriter(state);
|
||||
|
||||
// TODO: should we make the terms index more easily
|
||||
// pluggable? Ie so that this codec would record which
|
||||
// index impl was used, and switch on loading?
|
||||
// Or... you must make a new Codec for this?
|
||||
boolean success = false;
|
||||
try {
|
||||
FieldsConsumer ret = new BlockTreeTermsWriter(state, docs, minBlockSize, maxBlockSize);
|
||||
success = true;
|
||||
return ret;
|
||||
} finally {
|
||||
if (!success) {
|
||||
docs.close();
|
||||
}
|
||||
}
|
||||
throw new UnsupportedOperationException("this codec can only be used for reading");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -45,10 +45,21 @@ import org.apache.lucene.util.IOUtils;
|
|||
* postings format.
|
||||
*
|
||||
* @see Lucene40PostingsFormat
|
||||
* @lucene.experimental */
|
||||
|
||||
* @deprecated Only for reading old 4.0 segments */
|
||||
@Deprecated
|
||||
public class Lucene40PostingsReader extends PostingsReaderBase {
|
||||
|
||||
final static String TERMS_CODEC = "Lucene40PostingsWriterTerms";
|
||||
final static String FRQ_CODEC = "Lucene40PostingsWriterFrq";
|
||||
final static String PRX_CODEC = "Lucene40PostingsWriterPrx";
|
||||
|
||||
//private static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
|
||||
|
||||
// Increment version to change it:
|
||||
final static int VERSION_START = 0;
|
||||
final static int VERSION_LONG_SKIP = 1;
|
||||
final static int VERSION_CURRENT = VERSION_LONG_SKIP;
|
||||
|
||||
private final IndexInput freqIn;
|
||||
private final IndexInput proxIn;
|
||||
// public static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
|
||||
|
@ -67,7 +78,7 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
|
|||
try {
|
||||
freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.FREQ_EXTENSION),
|
||||
ioContext);
|
||||
CodecUtil.checkHeader(freqIn, Lucene40PostingsWriter.FRQ_CODEC, Lucene40PostingsWriter.VERSION_START,Lucene40PostingsWriter.VERSION_CURRENT);
|
||||
CodecUtil.checkHeader(freqIn, FRQ_CODEC, VERSION_START, VERSION_CURRENT);
|
||||
// TODO: hasProx should (somehow!) become codec private,
|
||||
// but it's tricky because 1) FIS.hasProx is global (it
|
||||
// could be all fields that have prox are written by a
|
||||
|
@ -79,7 +90,7 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
|
|||
if (fieldInfos.hasProx()) {
|
||||
proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.PROX_EXTENSION),
|
||||
ioContext);
|
||||
CodecUtil.checkHeader(proxIn, Lucene40PostingsWriter.PRX_CODEC, Lucene40PostingsWriter.VERSION_START,Lucene40PostingsWriter.VERSION_CURRENT);
|
||||
CodecUtil.checkHeader(proxIn, PRX_CODEC, VERSION_START, VERSION_CURRENT);
|
||||
} else {
|
||||
proxIn = null;
|
||||
}
|
||||
|
@ -97,8 +108,7 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
|
|||
public void init(IndexInput termsIn) throws IOException {
|
||||
|
||||
// Make sure we are talking to the matching past writer
|
||||
CodecUtil.checkHeader(termsIn, Lucene40PostingsWriter.TERMS_CODEC,
|
||||
Lucene40PostingsWriter.VERSION_START, Lucene40PostingsWriter.VERSION_CURRENT);
|
||||
CodecUtil.checkHeader(termsIn, TERMS_CODEC, VERSION_START, VERSION_CURRENT);
|
||||
|
||||
skipInterval = termsIn.readInt();
|
||||
maxSkipLevels = termsIn.readInt();
|
||||
|
|
|
@ -28,8 +28,9 @@ import org.apache.lucene.store.IndexInput;
|
|||
* that stores positions and payloads.
|
||||
*
|
||||
* @see Lucene40PostingsFormat
|
||||
* @lucene.experimental
|
||||
* @deprecated Only for reading old 4.0 segments
|
||||
*/
|
||||
@Deprecated
|
||||
public class Lucene40SkipListReader extends MultiLevelSkipListReader {
|
||||
private boolean currentFieldStoresPayloads;
|
||||
private boolean currentFieldStoresOffsets;
|
||||
|
|
|
@ -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,
|
|
@ -0,0 +1,122 @@
|
|||
package org.apache.lucene.codecs.lucene41;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.LiveDocsFormat;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.SegmentInfoFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40NormsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
||||
|
||||
/**
|
||||
* Implements the Lucene 4.1 index format, with configurable per-field postings formats.
|
||||
* <p>
|
||||
* If you want to reuse functionality of this codec in another codec, extend
|
||||
* {@link FilterCodec}.
|
||||
*
|
||||
* @see org.apache.lucene.codecs.lucene41 package documentation for file format details.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
// NOTE: if we make largish changes in a minor release, easier to just make Lucene42Codec or whatever
|
||||
// if they are backwards compatible or smallish we can probably do the backwards in the postingsreader
|
||||
// (it writes a minor version, etc).
|
||||
public class Lucene41Codec extends Codec {
|
||||
private final StoredFieldsFormat fieldsFormat = new Lucene40StoredFieldsFormat();
|
||||
private final TermVectorsFormat vectorsFormat = new Lucene40TermVectorsFormat();
|
||||
private final FieldInfosFormat fieldInfosFormat = new Lucene40FieldInfosFormat();
|
||||
private final DocValuesFormat docValuesFormat = new Lucene40DocValuesFormat();
|
||||
private final SegmentInfoFormat infosFormat = new Lucene40SegmentInfoFormat();
|
||||
private final NormsFormat normsFormat = new Lucene40NormsFormat();
|
||||
private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
|
||||
|
||||
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return Lucene41Codec.this.getPostingsFormatForField(field);
|
||||
}
|
||||
};
|
||||
|
||||
/** Sole constructor. */
|
||||
public Lucene41Codec() {
|
||||
super("Lucene41");
|
||||
}
|
||||
|
||||
@Override
|
||||
public final StoredFieldsFormat storedFieldsFormat() {
|
||||
return fieldsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final TermVectorsFormat termVectorsFormat() {
|
||||
return vectorsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final DocValuesFormat docValuesFormat() {
|
||||
return docValuesFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final PostingsFormat postingsFormat() {
|
||||
return postingsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final FieldInfosFormat fieldInfosFormat() {
|
||||
return fieldInfosFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final SegmentInfoFormat segmentInfoFormat() {
|
||||
return infosFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final NormsFormat normsFormat() {
|
||||
return normsFormat;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final LiveDocsFormat liveDocsFormat() {
|
||||
return liveDocsFormat;
|
||||
}
|
||||
|
||||
/** Returns the postings format that should be used for writing
|
||||
* new segments of <code>field</code>.
|
||||
*
|
||||
* The default implementation always returns "Lucene41"
|
||||
*/
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return defaultFormat;
|
||||
}
|
||||
|
||||
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.codecs.block;
|
||||
package org.apache.lucene.codecs.lucene41;
|
||||
|
||||
|
||||
/*
|
||||
|
@ -38,7 +38,7 @@ import org.apache.lucene.util.IOUtils;
|
|||
import org.apache.lucene.util.packed.PackedInts;
|
||||
|
||||
/**
|
||||
* Block postings format, which encodes postings in packed integer blocks
|
||||
* Lucene 4.1 postings format, which encodes postings in packed integer blocks
|
||||
* for fast decode.
|
||||
*
|
||||
* <p><b>NOTE</b>: this format is still experimental and
|
||||
|
@ -58,7 +58,7 @@ import org.apache.lucene.util.packed.PackedInts;
|
|||
*
|
||||
* <li>
|
||||
* <b>Block structure</b>:
|
||||
* <p>When the postings are long enough, BlockPostingsFormat will try to encode most integer data
|
||||
* <p>When the postings are long enough, Lucene41PostingsFormat will try to encode most integer data
|
||||
* as a packed block.</p>
|
||||
* <p>Take a term with 259 documents as an example, the first 256 document ids are encoded as two packed
|
||||
* blocks, while the remaining 3 are encoded as one VInt block. </p>
|
||||
|
@ -161,7 +161,7 @@ import org.apache.lucene.util.packed.PackedInts;
|
|||
* <li>SkipFPDelta determines the position of this term's SkipData within the .doc
|
||||
* file. In particular, it is the length of the TermFreq data.
|
||||
* SkipDelta is only stored if DocFreq is not smaller than SkipMinimum
|
||||
* (i.e. 8 in BlockPostingsFormat).</li>
|
||||
* (i.e. 8 in Lucene41PostingsFormat).</li>
|
||||
* </ul>
|
||||
* </dd>
|
||||
* </dl>
|
||||
|
@ -238,10 +238,10 @@ import org.apache.lucene.util.packed.PackedInts;
|
|||
* We use this trick since the definition of skip entry is a little different from base interface.
|
||||
* In {@link MultiLevelSkipListWriter}, skip data is assumed to be saved for
|
||||
* skipInterval<sup>th</sup>, 2*skipInterval<sup>th</sup> ... posting in the list. However,
|
||||
* in BlockPostingsFormat, the skip data is saved for skipInterval+1<sup>th</sup>,
|
||||
* in Lucene41PostingsFormat, the skip data is saved for skipInterval+1<sup>th</sup>,
|
||||
* 2*skipInterval+1<sup>th</sup> ... posting (skipInterval==PackedBlockSize in this case).
|
||||
* When DocFreq is multiple of PackedBlockSize, MultiLevelSkipListWriter will expect one
|
||||
* more skip data than BlockSkipWriter. </li>
|
||||
* more skip data than Lucene41SkipWriter. </li>
|
||||
* <li>SkipDatum is the metadata of one skip entry.
|
||||
* For the first block (no matter packed or VInt), it is omitted.</li>
|
||||
* <li>DocSkip records the document number of every PackedBlockSize<sup>th</sup> document number in
|
||||
|
@ -351,7 +351,7 @@ import org.apache.lucene.util.packed.PackedInts;
|
|||
* @lucene.experimental
|
||||
*/
|
||||
|
||||
public final class BlockPostingsFormat extends PostingsFormat {
|
||||
public final class Lucene41PostingsFormat extends PostingsFormat {
|
||||
/**
|
||||
* Filename extension for document number, frequencies, and skip data.
|
||||
* See chapter: <a href="#Frequencies">Frequencies and Skip Data</a>
|
||||
|
@ -380,12 +380,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,
|
|
@ -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,
|
|
@ -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 <code>PackedInts.COMPACT</code> */
|
||||
public Lucene41PostingsWriter(SegmentWriteState state) throws IOException {
|
||||
this(state, PackedInts.COMPACT);
|
||||
}
|
||||
|
|
@ -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
|
|
@ -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;
|
|
@ -0,0 +1,396 @@
|
|||
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
|
||||
<!--
|
||||
Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
contributor license agreements. See the NOTICE file distributed with
|
||||
this work for additional information regarding copyright ownership.
|
||||
The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
(the "License"); you may not use this file except in compliance with
|
||||
the License. You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
-->
|
||||
<html>
|
||||
<head>
|
||||
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
|
||||
</head>
|
||||
<body>
|
||||
Lucene 4.1 file format.
|
||||
|
||||
<h1>Apache Lucene - Index File Formats</h1>
|
||||
<div>
|
||||
<ul>
|
||||
<li><a href="#Introduction">Introduction</a></li>
|
||||
<li><a href="#Definitions">Definitions</a>
|
||||
<ul>
|
||||
<li><a href="#Inverted_Indexing">Inverted Indexing</a></li>
|
||||
<li><a href="#Types_of_Fields">Types of Fields</a></li>
|
||||
<li><a href="#Segments">Segments</a></li>
|
||||
<li><a href="#Document_Numbers">Document Numbers</a></li>
|
||||
</ul>
|
||||
</li>
|
||||
<li><a href="#Overview">Index Structure Overview</a></li>
|
||||
<li><a href="#File_Naming">File Naming</a></li>
|
||||
<li><a href="#file-names">Summary of File Extensions</a></li>
|
||||
<ul>
|
||||
<li><a href="#Lock_File">Lock File</a></li>
|
||||
<li><a href="#History">History</a></li>
|
||||
<li><a href="#Limitations">Limitations</a></li>
|
||||
</ul>
|
||||
</div>
|
||||
<a name="Introduction"></a>
|
||||
<h2>Introduction</h2>
|
||||
<div>
|
||||
<p>This document defines the index file formats used in this version of Lucene.
|
||||
If you are using a different version of Lucene, please consult the copy of
|
||||
<code>docs/</code> that was distributed with
|
||||
the version you are using.</p>
|
||||
<p>Apache Lucene is written in Java, but several efforts are underway to write
|
||||
<a href="http://wiki.apache.org/lucene-java/LuceneImplementations">versions of
|
||||
Lucene in other programming languages</a>. If these versions are to remain
|
||||
compatible with Apache Lucene, then a language-independent definition of the
|
||||
Lucene index format is required. This document thus attempts to provide a
|
||||
complete and independent definition of the Apache Lucene file formats.</p>
|
||||
<p>As Lucene evolves, this document should evolve. Versions of Lucene in
|
||||
different programming languages should endeavor to agree on file formats, and
|
||||
generate new versions of this document.</p>
|
||||
</div>
|
||||
<a name="Definitions" id="Definitions"></a>
|
||||
<h2>Definitions</h2>
|
||||
<div>
|
||||
<p>The fundamental concepts in Lucene are index, document, field and term.</p>
|
||||
<p>An index contains a sequence of documents.</p>
|
||||
<ul>
|
||||
<li>A document is a sequence of fields.</li>
|
||||
<li>A field is a named sequence of terms.</li>
|
||||
<li>A term is a sequence of bytes.</li>
|
||||
</ul>
|
||||
<p>The same sequence of bytes in two different fields is considered a different
|
||||
term. Thus terms are represented as a pair: the string naming the field, and the
|
||||
bytes within the field.</p>
|
||||
<a name="Inverted_Indexing"></a>
|
||||
<h3>Inverted Indexing</h3>
|
||||
<p>The index stores statistics about terms in order to make term-based search
|
||||
more efficient. Lucene's index falls into the family of indexes known as an
|
||||
<i>inverted index.</i> This is because it can list, for a term, the documents
|
||||
that contain it. This is the inverse of the natural relationship, in which
|
||||
documents list terms.</p>
|
||||
<a name="Types_of_Fields"></a>
|
||||
<h3>Types of Fields</h3>
|
||||
<p>In Lucene, fields may be <i>stored</i>, in which case their text is stored
|
||||
in the index literally, in a non-inverted manner. Fields that are inverted are
|
||||
called <i>indexed</i>. A field may be both stored and indexed.</p>
|
||||
<p>The text of a field may be <i>tokenized</i> into terms to be indexed, or the
|
||||
text of a field may be used literally as a term to be indexed. Most fields are
|
||||
tokenized, but sometimes it is useful for certain identifier fields to be
|
||||
indexed literally.</p>
|
||||
<p>See the {@link org.apache.lucene.document.Field Field}
|
||||
java docs for more information on Fields.</p>
|
||||
<a name="Segments" id="Segments"></a>
|
||||
<h3>Segments</h3>
|
||||
<p>Lucene indexes may be composed of multiple sub-indexes, or <i>segments</i>.
|
||||
Each segment is a fully independent index, which could be searched separately.
|
||||
Indexes evolve by:</p>
|
||||
<ol>
|
||||
<li>Creating new segments for newly added documents.</li>
|
||||
<li>Merging existing segments.</li>
|
||||
</ol>
|
||||
<p>Searches may involve multiple segments and/or multiple indexes, each index
|
||||
potentially composed of a set of segments.</p>
|
||||
<a name="Document_Numbers"></a>
|
||||
<h3>Document Numbers</h3>
|
||||
<p>Internally, Lucene refers to documents by an integer <i>document number</i>.
|
||||
The first document added to an index is numbered zero, and each subsequent
|
||||
document added gets a number one greater than the previous.</p>
|
||||
<p>Note that a document's number may change, so caution should be taken when
|
||||
storing these numbers outside of Lucene. In particular, numbers may change in
|
||||
the following situations:</p>
|
||||
<ul>
|
||||
<li>
|
||||
<p>The numbers stored in each segment are unique only within the segment, and
|
||||
must be converted before they can be used in a larger context. The standard
|
||||
technique is to allocate each segment a range of values, based on the range of
|
||||
numbers used in that segment. To convert a document number from a segment to an
|
||||
external value, the segment's <i>base</i> document number is added. To convert
|
||||
an external value back to a segment-specific value, the segment is identified
|
||||
by the range that the external value is in, and the segment's base value is
|
||||
subtracted. For example two five document segments might be combined, so that
|
||||
the first segment has a base value of zero, and the second of five. Document
|
||||
three from the second segment would have an external value of eight.</p>
|
||||
</li>
|
||||
<li>
|
||||
<p>When documents are deleted, gaps are created in the numbering. These are
|
||||
eventually removed as the index evolves through merging. Deleted documents are
|
||||
dropped when segments are merged. A freshly-merged segment thus has no gaps in
|
||||
its numbering.</p>
|
||||
</li>
|
||||
</ul>
|
||||
</div>
|
||||
<a name="Overview" id="Overview"></a>
|
||||
<h2>Index Structure Overview</h2>
|
||||
<div>
|
||||
<p>Each segment index maintains the following:</p>
|
||||
<ul>
|
||||
<li>
|
||||
{@link org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat Segment info}.
|
||||
This contains metadata about a segment, such as the number of documents,
|
||||
what files it uses,
|
||||
</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat Field names}.
|
||||
This contains the set of field names used in the index.
|
||||
</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat Stored Field values}.
|
||||
This contains, for each document, a list of attribute-value pairs, where the attributes
|
||||
are field names. These are used to store auxiliary information about the document, such as
|
||||
its title, url, or an identifier to access a database. The set of stored fields are what is
|
||||
returned for each hit when searching. This is keyed by document number.
|
||||
</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.codecs.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.
|
||||
</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Frequency data}.
|
||||
For each term in the dictionary, the numbers of all the
|
||||
documents that contain that term, and the frequency of the term in that
|
||||
document, unless frequencies are omitted (IndexOptions.DOCS_ONLY)
|
||||
</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Proximity data}.
|
||||
For each term in the dictionary, the positions that the
|
||||
term occurs in each document. Note that this will not exist if all fields in
|
||||
all documents omit position data.
|
||||
</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.codecs.lucene40.Lucene40NormsFormat Normalization factors}.
|
||||
For each field in each document, a value is stored
|
||||
that is multiplied into the score for hits on that field.
|
||||
</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat Term Vectors}.
|
||||
For each field in each document, the term vector (sometimes
|
||||
called document vector) may be stored. A term vector consists of term text and
|
||||
term frequency. To add Term Vectors to your index see the
|
||||
{@link org.apache.lucene.document.Field Field} constructors
|
||||
</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat Per-document values}.
|
||||
Like stored values, these are also keyed by document
|
||||
number, but are generally intended to be loaded into main memory for fast
|
||||
access. Whereas stored values are generally intended for summary results from
|
||||
searches, per-document values are useful for things like scoring factors.
|
||||
</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat Deleted documents}.
|
||||
An optional file indicating which documents are deleted.
|
||||
</li>
|
||||
</ul>
|
||||
<p>Details on each of these are provided in their linked pages.</p>
|
||||
</div>
|
||||
<a name="File_Naming"></a>
|
||||
<h2>File Naming</h2>
|
||||
<div>
|
||||
<p>All files belonging to a segment have the same name with varying extensions.
|
||||
The extensions correspond to the different file formats described below. When
|
||||
using the Compound File format (default in 1.4 and greater) these files (except
|
||||
for the Segment info file, the Lock file, and Deleted documents file) are collapsed
|
||||
into a single .cfs file (see below for details)</p>
|
||||
<p>Typically, all segments in an index are stored in a single directory,
|
||||
although this is not required.</p>
|
||||
<p>As of version 2.1 (lock-less commits), file names are never re-used (there
|
||||
is one exception, "segments.gen", see below). That is, when any file is saved
|
||||
to the Directory it is given a never before used filename. This is achieved
|
||||
using a simple generations approach. For example, the first segments file is
|
||||
segments_1, then segments_2, etc. The generation is a sequential long integer
|
||||
represented in alpha-numeric (base 36) form.</p>
|
||||
</div>
|
||||
<a name="file-names" id="file-names"></a>
|
||||
<h2>Summary of File Extensions</h2>
|
||||
<div>
|
||||
<p>The following table summarizes the names and extensions of the files in
|
||||
Lucene:</p>
|
||||
<table cellspacing="1" cellpadding="4">
|
||||
<tr>
|
||||
<th>Name</th>
|
||||
<th>Extension</th>
|
||||
<th>Brief Description</th>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.index.SegmentInfos Segments File}</td>
|
||||
<td>segments.gen, segments_N</td>
|
||||
<td>Stores information about a commit point</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td><a href="#Lock_File">Lock File</a></td>
|
||||
<td>write.lock</td>
|
||||
<td>The Write lock prevents multiple IndexWriters from writing to the same
|
||||
file.</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat Segment Info}</td>
|
||||
<td>.si</td>
|
||||
<td>Stores metadata about a segment</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.store.CompoundFileDirectory Compound File}</td>
|
||||
<td>.cfs, .cfe</td>
|
||||
<td>An optional "virtual" file consisting of all the other index files for
|
||||
systems that frequently run out of file handles.</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat Fields}</td>
|
||||
<td>.fnm</td>
|
||||
<td>Stores information about the fields</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat Field Index}</td>
|
||||
<td>.fdx</td>
|
||||
<td>Contains pointers to field data</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat Field Data}</td>
|
||||
<td>.fdt</td>
|
||||
<td>The stored fields for documents</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Dictionary}</td>
|
||||
<td>.tim</td>
|
||||
<td>The term dictionary, stores term info</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Term Index}</td>
|
||||
<td>.tip</td>
|
||||
<td>The index into the Term Dictionary</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Frequencies}</td>
|
||||
<td>.doc</td>
|
||||
<td>Contains the list of docs which contain each term along with frequency</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Positions}</td>
|
||||
<td>.pos</td>
|
||||
<td>Stores position information about where a term occurs in the index</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat Payloads}</td>
|
||||
<td>.pay</td>
|
||||
<td>Stores additional per-position metadata information such as character offsets and user payloads</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40NormsFormat Norms}</td>
|
||||
<td>.nrm.cfs, .nrm.cfe</td>
|
||||
<td>Encodes length and boost factors for docs and fields</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat Per-Document Values}</td>
|
||||
<td>.dv.cfs, .dv.cfe</td>
|
||||
<td>Encodes additional scoring factors or other per-document information.</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat Term Vector Index}</td>
|
||||
<td>.tvx</td>
|
||||
<td>Stores offset into the document data file</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat Term Vector Documents}</td>
|
||||
<td>.tvd</td>
|
||||
<td>Contains information about each document that has term vectors</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat Term Vector Fields}</td>
|
||||
<td>.tvf</td>
|
||||
<td>The field level info about term vectors</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat Deleted Documents}</td>
|
||||
<td>.del</td>
|
||||
<td>Info about what files are deleted</td>
|
||||
</tr>
|
||||
</table>
|
||||
</div>
|
||||
<a name="Lock_File" id="Lock_File"></a>
|
||||
<h2>Lock File</h2>
|
||||
The write lock, which is stored in the index directory by default, is named
|
||||
"write.lock". If the lock directory is different from the index directory then
|
||||
the write lock will be named "XXXX-write.lock" where XXXX is a unique prefix
|
||||
derived from the full path to the index directory. When this file is present, a
|
||||
writer is currently modifying the index (adding or removing documents). This
|
||||
lock file ensures that only one writer is modifying the index at a time.</p>
|
||||
<a name="History"></a>
|
||||
<h2>History</h2>
|
||||
<p>Compatibility notes are provided in this document, describing how file
|
||||
formats have changed from prior versions:</p>
|
||||
<ul>
|
||||
<li>In version 2.1, the file format was changed to allow lock-less commits (ie,
|
||||
no more commit lock). The change is fully backwards compatible: you can open a
|
||||
pre-2.1 index for searching or adding/deleting of docs. When the new segments
|
||||
file is saved (committed), it will be written in the new file format (meaning
|
||||
no specific "upgrade" process is needed). But note that once a commit has
|
||||
occurred, pre-2.1 Lucene will not be able to read the index.</li>
|
||||
<li>In version 2.3, the file format was changed to allow segments to share a
|
||||
single set of doc store (vectors & stored fields) files. This allows for
|
||||
faster indexing in certain cases. The change is fully backwards compatible (in
|
||||
the same way as the lock-less commits change in 2.1).</li>
|
||||
<li>In version 2.4, Strings are now written as true UTF-8 byte sequence, not
|
||||
Java's modified UTF-8. See <a href="http://issues.apache.org/jira/browse/LUCENE-510">
|
||||
LUCENE-510</a> for details.</li>
|
||||
<li>In version 2.9, an optional opaque Map<String,String> CommitUserData
|
||||
may be passed to IndexWriter's commit methods (and later retrieved), which is
|
||||
recorded in the segments_N file. See <a href="http://issues.apache.org/jira/browse/LUCENE-1382">
|
||||
LUCENE-1382</a> for details. Also,
|
||||
diagnostics were added to each segment written recording details about why it
|
||||
was written (due to flush, merge; which OS/JRE was used; etc.). See issue
|
||||
<a href="http://issues.apache.org/jira/browse/LUCENE-1654">LUCENE-1654</a> for details.</li>
|
||||
<li>In version 3.0, compressed fields are no longer written to the index (they
|
||||
can still be read, but on merge the new segment will write them, uncompressed).
|
||||
See issue <a href="http://issues.apache.org/jira/browse/LUCENE-1960">LUCENE-1960</a>
|
||||
for details.</li>
|
||||
<li>In version 3.1, segments records the code version that created them. See
|
||||
<a href="http://issues.apache.org/jira/browse/LUCENE-2720">LUCENE-2720</a> for details.
|
||||
Additionally segments track explicitly whether or not they have term vectors.
|
||||
See <a href="http://issues.apache.org/jira/browse/LUCENE-2811">LUCENE-2811</a>
|
||||
for details.</li>
|
||||
<li>In version 3.2, numeric fields are written as natively to stored fields
|
||||
file, previously they were stored in text format only.</li>
|
||||
<li>In version 3.4, fields can omit position data while still indexing term
|
||||
frequencies.</li>
|
||||
<li>In version 4.0, the format of the inverted index became extensible via
|
||||
the {@link org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage
|
||||
({@link org.apache.lucene.index.DocValues DocValues}) was introduced. Normalization
|
||||
factors need no longer be a single byte, they can be any DocValues
|
||||
{@link org.apache.lucene.index.DocValues.Type type}. Terms need not be unicode
|
||||
strings, they can be any byte sequence. Term offsets can optionally be indexed
|
||||
into the postings lists. Payloads can be stored in the term vectors.</li>
|
||||
<li>In version 4.1, the format of the postings list changed to use either
|
||||
of FOR compression or variable-byte encoding, depending upon the frequency
|
||||
of the term.</li>
|
||||
</ul>
|
||||
<a name="Limitations" id="Limitations"></a>
|
||||
<h2>Limitations</h2>
|
||||
<div>
|
||||
<p>When referring to term numbers, Lucene's current implementation uses a Java
|
||||
<code>int</code> to hold the term index, which means the
|
||||
maximum number of unique terms in any single index segment is ~2.1 billion
|
||||
times the term index interval (default 128) = ~274 billion. This is technically
|
||||
not a limitation of the index file format, just of Lucene's current
|
||||
implementation.</p>
|
||||
<p>Similarly, Lucene uses a Java <code>int</code> to refer to
|
||||
document numbers, and the index file format uses an <code>Int32</code>
|
||||
on-disk to store document numbers. This is a limitation
|
||||
of both the index file format and the current implementation. Eventually these
|
||||
should be replaced with either <code>UInt64</code> values, or
|
||||
better yet, {@link org.apache.lucene.store.DataOutput#writeVInt VInt} values which have no limit.</p>
|
||||
</div>
|
||||
</body>
|
||||
</html>
|
|
@ -61,8 +61,8 @@ name of your codec.
|
|||
If you just want to customise the {@link org.apache.lucene.codecs.PostingsFormat}, or use different postings
|
||||
formats for different fields, then you can register your custom postings format in the same way (in
|
||||
META-INF/services/org.apache.lucene.codecs.PostingsFormat), and then extend the default
|
||||
{@link org.apache.lucene.codecs.lucene40.Lucene40Codec} and override
|
||||
{@link org.apache.lucene.codecs.lucene40.Lucene40Codec#getPostingsFormatForField(String)} to return your custom
|
||||
{@link org.apache.lucene.codecs.lucene41.Lucene41Codec} and override
|
||||
{@link org.apache.lucene.codecs.lucene41.Lucene41Codec#getPostingsFormatForField(String)} to return your custom
|
||||
postings format.
|
||||
</p>
|
||||
</body>
|
||||
|
|
|
@ -233,9 +233,12 @@ public class CheckIndex {
|
|||
TermIndexStatus() {
|
||||
}
|
||||
|
||||
/** Total term count */
|
||||
/** Number of terms with at least one live doc. */
|
||||
public long termCount = 0L;
|
||||
|
||||
/** Number of terms with zero live docs docs. */
|
||||
public long delTermCount = 0L;
|
||||
|
||||
/** Total frequency across all terms. */
|
||||
public long totFreq = 0L;
|
||||
|
||||
|
@ -750,7 +753,7 @@ public class CheckIndex {
|
|||
final TermsEnum termsEnum = terms.iterator(null);
|
||||
|
||||
boolean hasOrd = true;
|
||||
final long termCountStart = status.termCount;
|
||||
final long termCountStart = status.delTermCount + status.termCount;
|
||||
|
||||
BytesRef lastTerm = null;
|
||||
|
||||
|
@ -781,7 +784,6 @@ public class CheckIndex {
|
|||
if (docFreq <= 0) {
|
||||
throw new RuntimeException("docfreq: " + docFreq + " is out of bounds");
|
||||
}
|
||||
status.totFreq += docFreq;
|
||||
sumDocFreq += docFreq;
|
||||
|
||||
docs = termsEnum.docs(liveDocs, docs);
|
||||
|
@ -796,15 +798,13 @@ public class CheckIndex {
|
|||
}
|
||||
|
||||
if (hasOrd) {
|
||||
final long ordExpected = status.termCount - termCountStart;
|
||||
final long ordExpected = status.delTermCount + status.termCount - termCountStart;
|
||||
if (ord != ordExpected) {
|
||||
throw new RuntimeException("ord mismatch: TermsEnum has ord=" + ord + " vs actual=" + ordExpected);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
status.termCount++;
|
||||
|
||||
final DocsEnum docs2;
|
||||
if (postings != null) {
|
||||
docs2 = postings;
|
||||
|
@ -820,6 +820,7 @@ public class CheckIndex {
|
|||
if (doc == DocIdSetIterator.NO_MORE_DOCS) {
|
||||
break;
|
||||
}
|
||||
status.totFreq++;
|
||||
visitedDocs.set(doc);
|
||||
int freq = -1;
|
||||
if (hasFreqs) {
|
||||
|
@ -883,6 +884,12 @@ public class CheckIndex {
|
|||
}
|
||||
}
|
||||
|
||||
if (docCount != 0) {
|
||||
status.termCount++;
|
||||
} else {
|
||||
status.delTermCount++;
|
||||
}
|
||||
|
||||
final long totalTermFreq2 = termsEnum.totalTermFreq();
|
||||
final boolean hasTotalTermFreq = hasFreqs && totalTermFreq2 != -1;
|
||||
|
||||
|
@ -1063,11 +1070,11 @@ public class CheckIndex {
|
|||
// check unique term count
|
||||
long termCount = -1;
|
||||
|
||||
if (status.termCount-termCountStart > 0) {
|
||||
if ((status.delTermCount+status.termCount)-termCountStart > 0) {
|
||||
termCount = fields.terms(field).size();
|
||||
|
||||
if (termCount != -1 && termCount != status.termCount - termCountStart) {
|
||||
throw new RuntimeException("termCount mismatch " + termCount + " vs " + (status.termCount - termCountStart));
|
||||
if (termCount != -1 && termCount != status.delTermCount + status.termCount - termCountStart) {
|
||||
throw new RuntimeException("termCount mismatch " + (status.delTermCount + termCount) + " vs " + (status.termCount - termCountStart));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -34,7 +34,6 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.index.DocumentsWriterPerThread.FlushedSegment;
|
||||
import org.apache.lucene.index.FieldInfos.FieldNumbers;
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.index.MergeState.CheckAbort;
|
||||
|
@ -42,7 +41,6 @@ import org.apache.lucene.search.Query;
|
|||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.FlushInfo;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.Lock;
|
||||
import org.apache.lucene.store.LockObtainFailedException;
|
||||
|
@ -52,7 +50,6 @@ import org.apache.lucene.util.Bits;
|
|||
import org.apache.lucene.util.Constants;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
import org.apache.lucene.util.MutableBits;
|
||||
import org.apache.lucene.util.ThreadInterruptedException;
|
||||
|
||||
/**
|
||||
|
@ -3118,13 +3115,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
mergedDeletes.dropChanges();
|
||||
}
|
||||
readerPool.release(mergedDeletes);
|
||||
if (dropSegment) {
|
||||
readerPool.drop(mergedDeletes.info);
|
||||
}
|
||||
}
|
||||
|
||||
if (dropSegment) {
|
||||
assert !segmentInfos.contains(merge.info);
|
||||
readerPool.drop(merge.info);
|
||||
deleter.deleteNewFiles(merge.info.files());
|
||||
}
|
||||
|
||||
|
@ -3736,8 +3731,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
}
|
||||
|
||||
final IndexReaderWarmer mergedSegmentWarmer = config.getMergedSegmentWarmer();
|
||||
|
||||
if (poolReaders && mergedSegmentWarmer != null) {
|
||||
if (poolReaders && mergedSegmentWarmer != null && merge.info.info.getDocCount() != 0) {
|
||||
final ReadersAndLiveDocs rld = readerPool.get(merge.info, true);
|
||||
final SegmentReader sr = rld.getReader(IOContext.READ);
|
||||
try {
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.index;
|
|||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat; // javadocs
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; // javadocs
|
||||
import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain;
|
||||
import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
|
@ -186,14 +186,14 @@ public class LiveIndexWriterConfig {
|
|||
* <b>NOTE:</b> This parameter does not apply to all PostingsFormat implementations,
|
||||
* including the default one in this release. It only makes sense for term indexes
|
||||
* that are implemented as a fixed gap between terms. For example,
|
||||
* {@link Lucene40PostingsFormat} implements the term index instead based upon how
|
||||
* {@link Lucene41PostingsFormat} implements the term index instead based upon how
|
||||
* terms share prefixes. To configure its parameters (the minimum and maximum size
|
||||
* for a block), you would instead use {@link Lucene40PostingsFormat#Lucene40PostingsFormat(int, int)}.
|
||||
* for a block), you would instead use {@link Lucene41PostingsFormat#Lucene41PostingsFormat(int, int)}.
|
||||
* which can also be configured on a per-field basis:
|
||||
* <pre class="prettyprint">
|
||||
* //customize Lucene40PostingsFormat, passing minBlockSize=50, maxBlockSize=100
|
||||
* final PostingsFormat tweakedPostings = new Lucene40PostingsFormat(50, 100);
|
||||
* iwc.setCodec(new Lucene40Codec() {
|
||||
* //customize Lucene41PostingsFormat, passing minBlockSize=50, maxBlockSize=100
|
||||
* final PostingsFormat tweakedPostings = new Lucene41PostingsFormat(50, 100);
|
||||
* iwc.setCodec(new Lucene41Codec() {
|
||||
* @Override
|
||||
* public PostingsFormat getPostingsFormatForField(String field) {
|
||||
* if (field.equals("fieldWithTonsOfTerms"))
|
||||
|
|
|
@ -14,3 +14,4 @@
|
|||
# limitations under the License.
|
||||
|
||||
org.apache.lucene.codecs.lucene40.Lucene40Codec
|
||||
org.apache.lucene.codecs.lucene41.Lucene41Codec
|
||||
|
|
|
@ -14,3 +14,4 @@
|
|||
# limitations under the License.
|
||||
|
||||
org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat
|
||||
org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene;
|
|||
|
||||
import org.apache.lucene.analysis.*;
|
||||
import org.apache.lucene.codecs.*;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41Codec;
|
||||
import org.apache.lucene.document.*;
|
||||
import org.apache.lucene.index.*;
|
||||
import org.apache.lucene.search.*;
|
||||
|
@ -31,11 +31,11 @@ import org.apache.lucene.util.*;
|
|||
|
||||
public class TestExternalCodecs extends LuceneTestCase {
|
||||
|
||||
private static final class CustomPerFieldCodec extends Lucene40Codec {
|
||||
private static final class CustomPerFieldCodec extends Lucene41Codec {
|
||||
|
||||
private final PostingsFormat ramFormat = PostingsFormat.forName("RAMOnly");
|
||||
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene40");
|
||||
private final PostingsFormat pulsingFormat = PostingsFormat.forName("Pulsing40");
|
||||
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
|
||||
private final PostingsFormat pulsingFormat = PostingsFormat.forName("Pulsing41");
|
||||
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
|
|
|
@ -38,11 +38,12 @@ import org.apache.lucene.util.LineFileDocs;
|
|||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util._TestUtil;
|
||||
|
||||
// TODO: really this should be in BaseTestPF or somewhere else? useful test!
|
||||
public class TestReuseDocsEnum extends LuceneTestCase {
|
||||
|
||||
public void testReuseDocsEnumNoReuse() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
Codec cp = _TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat());
|
||||
Codec cp = _TestUtil.alwaysPostingsFormat(new Lucene40RWPostingsFormat());
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir,
|
||||
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setCodec(cp));
|
||||
int numdocs = atLeast(20);
|
||||
|
@ -69,7 +70,7 @@ public class TestReuseDocsEnum extends LuceneTestCase {
|
|||
// tests for reuse only if bits are the same either null or the same instance
|
||||
public void testReuseDocsEnumSameBitsOrNull() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
Codec cp = _TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat());
|
||||
Codec cp = _TestUtil.alwaysPostingsFormat(new Lucene40RWPostingsFormat());
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir,
|
||||
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setCodec(cp));
|
||||
int numdocs = atLeast(20);
|
||||
|
@ -113,7 +114,7 @@ public class TestReuseDocsEnum extends LuceneTestCase {
|
|||
// make sure we never reuse from another reader even if it is the same field & codec etc
|
||||
public void testReuseDocsEnumDifferentReader() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
Codec cp = _TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat());
|
||||
Codec cp = _TestUtil.alwaysPostingsFormat(new Lucene40RWPostingsFormat());
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir,
|
||||
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setCodec(cp));
|
||||
int numdocs = atLeast(20);
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.codecs.block;
|
||||
package org.apache.lucene.codecs.lucene41;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -18,22 +18,13 @@ package org.apache.lucene.codecs.block;
|
|||
*/
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.block.BlockPostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
|
||||
import org.apache.lucene.index.BasePostingsFormatTestCase;
|
||||
|
||||
/**
|
||||
* Tests BlockPostingsFormat
|
||||
*/
|
||||
public class TestBlockPostingsFormat extends BasePostingsFormatTestCase {
|
||||
private final PostingsFormat postings = new BlockPostingsFormat();
|
||||
private final Codec codec = new Lucene40Codec() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return postings;
|
||||
}
|
||||
};
|
||||
private final Codec codec = new Lucene41Codec();
|
||||
|
||||
@Override
|
||||
protected Codec getCodec() {
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.codecs.block;
|
||||
package org.apache.lucene.codecs.lucene41;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -19,7 +19,6 @@ package org.apache.lucene.codecs.block;
|
|||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
|
@ -47,10 +46,10 @@ public class TestBlockPostingsFormat2 extends LuceneTestCase {
|
|||
super.setUp();
|
||||
dir = newFSDirectory(_TestUtil.getTempDir("testDFBlockSize"));
|
||||
iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
|
||||
iwc.setCodec(new Lucene40Codec() {
|
||||
iwc.setCodec(new Lucene41Codec() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return PostingsFormat.forName("Block");
|
||||
return PostingsFormat.forName("Lucene41");
|
||||
}
|
||||
});
|
||||
iw = new RandomIndexWriter(random(), dir, iwc);
|
||||
|
@ -88,7 +87,7 @@ public class TestBlockPostingsFormat2 extends LuceneTestCase {
|
|||
/** tests terms with df = blocksize */
|
||||
public void testDFBlockSize() throws Exception {
|
||||
Document doc = newDocument();
|
||||
for (int i = 0; i < BlockPostingsFormat.BLOCK_SIZE; i++) {
|
||||
for (int i = 0; i < Lucene41PostingsFormat.BLOCK_SIZE; i++) {
|
||||
for (Field f : doc.getFields()) {
|
||||
f.setStringValue(f.name() + " " + f.name() + "_2");
|
||||
}
|
||||
|
@ -99,7 +98,7 @@ public class TestBlockPostingsFormat2 extends LuceneTestCase {
|
|||
/** tests terms with df % blocksize = 0 */
|
||||
public void testDFBlockSizeMultiple() throws Exception {
|
||||
Document doc = newDocument();
|
||||
for (int i = 0; i < BlockPostingsFormat.BLOCK_SIZE * 16; i++) {
|
||||
for (int i = 0; i < Lucene41PostingsFormat.BLOCK_SIZE * 16; i++) {
|
||||
for (Field f : doc.getFields()) {
|
||||
f.setStringValue(f.name() + " " + f.name() + "_2");
|
||||
}
|
||||
|
@ -110,7 +109,7 @@ public class TestBlockPostingsFormat2 extends LuceneTestCase {
|
|||
/** tests terms with ttf = blocksize */
|
||||
public void testTTFBlockSize() throws Exception {
|
||||
Document doc = newDocument();
|
||||
for (int i = 0; i < BlockPostingsFormat.BLOCK_SIZE/2; i++) {
|
||||
for (int i = 0; i < Lucene41PostingsFormat.BLOCK_SIZE/2; i++) {
|
||||
for (Field f : doc.getFields()) {
|
||||
f.setStringValue(f.name() + " " + f.name() + " " + f.name() + "_2 " + f.name() + "_2");
|
||||
}
|
||||
|
@ -121,7 +120,7 @@ public class TestBlockPostingsFormat2 extends LuceneTestCase {
|
|||
/** tests terms with ttf % blocksize = 0 */
|
||||
public void testTTFBlockSizeMultiple() throws Exception {
|
||||
Document doc = newDocument();
|
||||
for (int i = 0; i < BlockPostingsFormat.BLOCK_SIZE/2; i++) {
|
||||
for (int i = 0; i < Lucene41PostingsFormat.BLOCK_SIZE/2; i++) {
|
||||
for (Field f : doc.getFields()) {
|
||||
String proto = (f.name() + " " + f.name() + " " + f.name() + " " + f.name() + " "
|
||||
+ f.name() + "_2 " + f.name() + "_2 " + f.name() + "_2 " + f.name() + "_2");
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.codecs.block;
|
||||
package org.apache.lucene.codecs.lucene41;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -30,7 +30,7 @@ import org.apache.lucene.analysis.MockVariableLengthPayloadFilter;
|
|||
import org.apache.lucene.analysis.TokenFilter;
|
||||
import org.apache.lucene.analysis.Tokenizer;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41Codec;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
|
@ -64,7 +64,7 @@ import org.apache.lucene.util.automaton.RegExp;
|
|||
* Tests partial enumeration (only pulling a subset of the prox data)
|
||||
*/
|
||||
public class TestBlockPostingsFormat3 extends LuceneTestCase {
|
||||
static final int MAXDOC = BlockPostingsFormat.BLOCK_SIZE * 20;
|
||||
static final int MAXDOC = Lucene41PostingsFormat.BLOCK_SIZE * 20;
|
||||
|
||||
// creates 6 fields with different options and does "duels" of fields against each other
|
||||
public void test() throws Exception {
|
||||
|
@ -85,10 +85,10 @@ public class TestBlockPostingsFormat3 extends LuceneTestCase {
|
|||
}
|
||||
};
|
||||
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
|
||||
iwc.setCodec(new Lucene40Codec() {
|
||||
iwc.setCodec(new Lucene41Codec() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return PostingsFormat.forName("Block");
|
||||
return PostingsFormat.forName("Lucene41");
|
||||
// TODO: we could actually add more fields implemented with different PFs
|
||||
}
|
||||
});
|
|
@ -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;
|
|
@ -21,10 +21,10 @@ import java.io.IOException;
|
|||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41Codec;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
|
||||
import org.apache.lucene.codecs.mocksep.MockSepPostingsFormat;
|
||||
import org.apache.lucene.codecs.pulsing.Pulsing40PostingsFormat;
|
||||
import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
|
||||
import org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
|
@ -142,7 +142,7 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
|
|||
|
||||
assertQuery(new Term("content", "ccc"), dir, 10);
|
||||
assertQuery(new Term("content", "aaa"), dir, 10);
|
||||
Lucene40Codec codec = (Lucene40Codec)iwconf.getCodec();
|
||||
Lucene41Codec codec = (Lucene41Codec)iwconf.getCodec();
|
||||
|
||||
iwconf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()))
|
||||
.setOpenMode(OpenMode.APPEND).setCodec(codec);
|
||||
|
@ -158,7 +158,7 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
|
|||
}
|
||||
addDocs2(writer, 10);
|
||||
writer.commit();
|
||||
codec = (Lucene40Codec)iwconf.getCodec();
|
||||
codec = (Lucene41Codec)iwconf.getCodec();
|
||||
assertEquals(30, writer.maxDoc());
|
||||
assertQuery(new Term("content", "bbb"), dir, 10);
|
||||
assertQuery(new Term("content", "ccc"), dir, 10); ////
|
||||
|
@ -200,8 +200,8 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
|
|||
|
||||
}
|
||||
|
||||
public static class MockCodec extends Lucene40Codec {
|
||||
final PostingsFormat lucene40 = new Lucene40PostingsFormat();
|
||||
public static class MockCodec extends Lucene41Codec {
|
||||
final PostingsFormat lucene40 = new Lucene41PostingsFormat();
|
||||
final PostingsFormat simpleText = new SimpleTextPostingsFormat();
|
||||
final PostingsFormat mockSep = new MockSepPostingsFormat();
|
||||
|
||||
|
@ -217,8 +217,8 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public static class MockCodec2 extends Lucene40Codec {
|
||||
final PostingsFormat lucene40 = new Lucene40PostingsFormat();
|
||||
public static class MockCodec2 extends Lucene41Codec {
|
||||
final PostingsFormat lucene40 = new Lucene41PostingsFormat();
|
||||
final PostingsFormat simpleText = new SimpleTextPostingsFormat();
|
||||
|
||||
@Override
|
||||
|
@ -268,13 +268,13 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
|
|||
}
|
||||
|
||||
public void testSameCodecDifferentInstance() throws Exception {
|
||||
Codec codec = new Lucene40Codec() {
|
||||
Codec codec = new Lucene41Codec() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
if ("id".equals(field)) {
|
||||
return new Pulsing40PostingsFormat(1);
|
||||
return new Pulsing41PostingsFormat(1);
|
||||
} else if ("date".equals(field)) {
|
||||
return new Pulsing40PostingsFormat(1);
|
||||
return new Pulsing41PostingsFormat(1);
|
||||
} else {
|
||||
return super.getPostingsFormatForField(field);
|
||||
}
|
||||
|
@ -284,13 +284,13 @@ public class TestPerFieldPostingsFormat2 extends LuceneTestCase {
|
|||
}
|
||||
|
||||
public void testSameCodecDifferentParams() throws Exception {
|
||||
Codec codec = new Lucene40Codec() {
|
||||
Codec codec = new Lucene41Codec() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
if ("id".equals(field)) {
|
||||
return new Pulsing40PostingsFormat(1);
|
||||
return new Pulsing41PostingsFormat(1);
|
||||
} else if ("date".equals(field)) {
|
||||
return new Pulsing40PostingsFormat(2);
|
||||
return new Pulsing41PostingsFormat(2);
|
||||
} else {
|
||||
return super.getPostingsFormatForField(field);
|
||||
}
|
||||
|
|
|
@ -27,8 +27,8 @@ import org.apache.lucene.analysis.MockAnalyzer;
|
|||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
|
||||
import org.apache.lucene.codecs.pulsing.Pulsing40PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41Codec;
|
||||
import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
|
@ -1058,9 +1058,9 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
aux2.close();
|
||||
}
|
||||
|
||||
private static final class CustomPerFieldCodec extends Lucene40Codec {
|
||||
private static final class CustomPerFieldCodec extends Lucene41Codec {
|
||||
private final PostingsFormat simpleTextFormat = PostingsFormat.forName("SimpleText");
|
||||
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene40");
|
||||
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
|
||||
private final PostingsFormat mockSepFormat = PostingsFormat.forName("MockSep");
|
||||
|
||||
@Override
|
||||
|
@ -1109,7 +1109,7 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
|
||||
private static final class UnRegisteredCodec extends FilterCodec {
|
||||
public UnRegisteredCodec() {
|
||||
super("NotRegistered", new Lucene40Codec());
|
||||
super("NotRegistered", new Lucene41Codec());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1138,7 +1138,7 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT,
|
||||
new MockAnalyzer(random()));
|
||||
conf.setCodec(_TestUtil.alwaysPostingsFormat(new Pulsing40PostingsFormat(1 + random().nextInt(20))));
|
||||
conf.setCodec(_TestUtil.alwaysPostingsFormat(new Pulsing41PostingsFormat(1 + random().nextInt(20))));
|
||||
IndexWriter w = new IndexWriter(dir, conf);
|
||||
try {
|
||||
w.addIndexes(toAdd);
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.codecs.lucene40;
|
||||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -35,13 +35,13 @@ import org.apache.lucene.util.LuceneTestCase;
|
|||
import org.apache.lucene.util._TestUtil;
|
||||
|
||||
/**
|
||||
* Test that a plain Lucene40Codec puts codec headers in all files.
|
||||
* Test that a plain default puts codec headers in all files.
|
||||
*/
|
||||
public class TestAllFilesHaveCodecHeader extends LuceneTestCase {
|
||||
public void test() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
|
||||
conf.setCodec(Codec.forName("Lucene40"));
|
||||
conf.setCodec(Codec.forName("Lucene41"));
|
||||
// riw should sometimes create docvalues fields, etc
|
||||
RandomIndexWriter riw = new RandomIndexWriter(random(), dir, conf);
|
||||
Document doc = new Document();
|
|
@ -75,8 +75,8 @@ public class TestCheckIndex extends LuceneTestCase {
|
|||
|
||||
assertNotNull(seg.termIndexStatus);
|
||||
assertNull(seg.termIndexStatus.error);
|
||||
assertEquals(19, seg.termIndexStatus.termCount);
|
||||
assertEquals(19, seg.termIndexStatus.totFreq);
|
||||
assertEquals(18, seg.termIndexStatus.termCount);
|
||||
assertEquals(18, seg.termIndexStatus.totFreq);
|
||||
assertEquals(18, seg.termIndexStatus.totPos);
|
||||
|
||||
assertNotNull(seg.storedFieldStatus);
|
||||
|
|
|
@ -28,7 +28,7 @@ import java.util.Random;
|
|||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
|
@ -828,7 +828,7 @@ public void testFilesOpenClose() throws IOException {
|
|||
// LUCENE-1609: don't load terms index
|
||||
public void testNoTermsIndex() throws Throwable {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat())));
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat())));
|
||||
Document doc = new Document();
|
||||
doc.add(newTextField("field", "a b c d e f g h i j k l m n o p q r s t u v w x y z", Field.Store.NO));
|
||||
doc.add(newTextField("number", "0 1 2 3 4 5 6 7 8 9", Field.Store.NO));
|
||||
|
@ -848,7 +848,7 @@ public void testFilesOpenClose() throws IOException {
|
|||
writer = new IndexWriter(
|
||||
dir,
|
||||
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).
|
||||
setCodec(_TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat())).
|
||||
setCodec(_TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat())).
|
||||
setMergePolicy(newLogMergePolicy(10))
|
||||
);
|
||||
writer.addDocument(doc);
|
||||
|
|
|
@ -110,7 +110,7 @@ public class TestDocTermOrds extends LuceneTestCase {
|
|||
// Sometimes swap in codec that impls ord():
|
||||
if (random().nextInt(10) == 7) {
|
||||
// Make sure terms index has ords:
|
||||
Codec codec = _TestUtil.alwaysPostingsFormat(PostingsFormat.forName("Lucene40WithOrds"));
|
||||
Codec codec = _TestUtil.alwaysPostingsFormat(PostingsFormat.forName("Lucene41WithOrds"));
|
||||
conf.setCodec(codec);
|
||||
}
|
||||
|
||||
|
@ -207,7 +207,7 @@ public class TestDocTermOrds extends LuceneTestCase {
|
|||
|
||||
// Sometimes swap in codec that impls ord():
|
||||
if (random().nextInt(10) == 7) {
|
||||
Codec codec = _TestUtil.alwaysPostingsFormat(PostingsFormat.forName("Lucene40WithOrds"));
|
||||
Codec codec = _TestUtil.alwaysPostingsFormat(PostingsFormat.forName("Lucene41WithOrds"));
|
||||
conf.setCodec(codec);
|
||||
}
|
||||
|
||||
|
|
|
@ -61,7 +61,7 @@ public class TestDuelingCodecs extends LuceneTestCase {
|
|||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
|
||||
// for now its SimpleText vs Lucene40(random postings format)
|
||||
// for now its SimpleText vs Lucene41(random postings format)
|
||||
// as this gives the best overall coverage. when we have more
|
||||
// codecs we should probably pick 2 from Codec.availableCodecs()
|
||||
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.index;
|
|||
|
||||
import org.apache.lucene.store.*;
|
||||
import org.apache.lucene.analysis.*;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
|
||||
import org.apache.lucene.document.*;
|
||||
import org.apache.lucene.util.*;
|
||||
|
||||
|
@ -65,7 +65,7 @@ public class TestFlex extends LuceneTestCase {
|
|||
public void testTermOrd() throws Exception {
|
||||
Directory d = newDirectory();
|
||||
IndexWriter w = new IndexWriter(d, newIndexWriterConfig(TEST_VERSION_CURRENT,
|
||||
new MockAnalyzer(random())).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat())));
|
||||
new MockAnalyzer(random())).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat())));
|
||||
Document doc = new Document();
|
||||
doc.add(newTextField("f", "a b c", Field.Store.NO));
|
||||
w.addDocument(doc);
|
||||
|
|
|
@ -23,7 +23,7 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
|
||||
import org.apache.lucene.analysis.*;
|
||||
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
@ -69,7 +69,7 @@ public class TestMultiLevelSkipList extends LuceneTestCase {
|
|||
|
||||
public void testSimpleSkip() throws IOException {
|
||||
Directory dir = new CountingRAMDirectory(new RAMDirectory());
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new PayloadAnalyzer()).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat())).setMergePolicy(newLogMergePolicy()));
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new PayloadAnalyzer()).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat())).setMergePolicy(newLogMergePolicy()));
|
||||
Term term = new Term("test", "a");
|
||||
for (int i = 0; i < 5000; i++) {
|
||||
Document d1 = new Document();
|
||||
|
|
|
@ -24,7 +24,7 @@ import org.apache.lucene.util.LuceneTestCase;
|
|||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util._TestUtil;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
@ -75,7 +75,7 @@ public class TestSegmentTermEnum extends LuceneTestCase {
|
|||
|
||||
public void testPrevTermAtEnd() throws IOException
|
||||
{
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat())));
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat())));
|
||||
addDoc(writer, "aaa bbb");
|
||||
writer.close();
|
||||
SegmentReader reader = getOnlySegmentReader(DirectoryReader.open(dir));
|
||||
|
|
|
@ -25,8 +25,8 @@ import org.apache.lucene.codecs.Codec;
|
|||
// enough to test the basics via Codec
|
||||
public class TestNamedSPILoader extends LuceneTestCase {
|
||||
public void testLookup() {
|
||||
Codec codec = Codec.forName("Lucene40");
|
||||
assertEquals("Lucene40", codec.getName());
|
||||
Codec codec = Codec.forName("Lucene41");
|
||||
assertEquals("Lucene41", codec.getName());
|
||||
}
|
||||
|
||||
// we want an exception if its not found.
|
||||
|
@ -39,6 +39,6 @@ public class TestNamedSPILoader extends LuceneTestCase {
|
|||
|
||||
public void testAvailableServices() {
|
||||
Set<String> codecs = Codec.availableCodecs();
|
||||
assertTrue(codecs.contains("Lucene40"));
|
||||
assertTrue(codecs.contains("Lucene41"));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -141,6 +141,11 @@ public class WeightedSpanTermExtractor {
|
|||
extractWeightedSpanTerms(terms, (SpanQuery) query);
|
||||
} else if (query instanceof FilteredQuery) {
|
||||
extract(((FilteredQuery) query).getQuery(), terms);
|
||||
} else if (query instanceof ConstantScoreQuery) {
|
||||
final Query q = ((ConstantScoreQuery) query).getQuery();
|
||||
if (q != null) {
|
||||
extract(q, terms);
|
||||
}
|
||||
} else if (query instanceof DisjunctionMaxQuery) {
|
||||
for (Iterator<Query> iterator = ((DisjunctionMaxQuery) query).iterator(); iterator.hasNext();) {
|
||||
extract(iterator.next(), terms);
|
||||
|
|
|
@ -664,6 +664,31 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
|
|||
helper.start();
|
||||
}
|
||||
|
||||
public void testGetBestFragmentsConstantScore() throws Exception {
|
||||
TestHighlightRunner helper = new TestHighlightRunner() {
|
||||
|
||||
@Override
|
||||
public void run() throws Exception {
|
||||
numHighlights = 0;
|
||||
if (random().nextBoolean()) {
|
||||
BooleanQuery bq = new BooleanQuery();
|
||||
bq.add(new ConstantScoreQuery(new QueryWrapperFilter(new TermQuery(
|
||||
new Term(FIELD_NAME, "kennedy")))), Occur.MUST);
|
||||
bq.add(new ConstantScoreQuery(new TermQuery(new Term(FIELD_NAME, "kennedy"))), Occur.MUST);
|
||||
doSearching(bq);
|
||||
} else {
|
||||
doSearching(new ConstantScoreQuery(new TermQuery(new Term(FIELD_NAME,
|
||||
"kennedy"))));
|
||||
}
|
||||
doStandardHighlights(analyzer, searcher, hits, query, HighlighterTest.this);
|
||||
assertTrue("Failed to find correct number of highlights " + numHighlights + " found",
|
||||
numHighlights == 4);
|
||||
}
|
||||
};
|
||||
|
||||
helper.start();
|
||||
}
|
||||
|
||||
public void testGetFuzzyFragments() throws Exception {
|
||||
TestHighlightRunner helper = new TestHighlightRunner() {
|
||||
|
||||
|
|
|
@ -30,7 +30,7 @@ import org.apache.lucene.analysis.BaseTokenStreamTestCase;
|
|||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.analysis.MockTokenFilter;
|
||||
import org.apache.lucene.analysis.MockTokenizer;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
|
@ -123,7 +123,7 @@ public class MemoryIndexTest extends BaseTokenStreamTestCase {
|
|||
Directory ramdir = new RAMDirectory();
|
||||
Analyzer analyzer = randomAnalyzer();
|
||||
IndexWriter writer = new IndexWriter(ramdir,
|
||||
new IndexWriterConfig(TEST_VERSION_CURRENT, analyzer).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene40PostingsFormat())));
|
||||
new IndexWriterConfig(TEST_VERSION_CURRENT, analyzer).setCodec(_TestUtil.alwaysPostingsFormat(new Lucene41PostingsFormat())));
|
||||
Document doc = new Document();
|
||||
Field field1 = newTextField("foo", fooField.toString(), Field.Store.NO);
|
||||
Field field2 = newTextField("term", termField.toString(), Field.Store.NO);
|
||||
|
|
|
@ -63,9 +63,10 @@
|
|||
<h2>Reference Documents</h2>
|
||||
<ul>
|
||||
<li><a href="changes/Changes.html">Changes</a>: List of changes in this release.</li>
|
||||
<li><a href="SYSTEM_REQUIREMENTS.html">System Requirements</a>: Minimum and supported Java versions.</li>
|
||||
<li><a href="MIGRATE.html">Migration Guide</a>: What changed in Lucene 4; how to migrate code from Lucene 3.x.</li>
|
||||
<li><a href="JRE_VERSION_MIGRATION.html">JRE Version Migration</a>: Information about upgrading between major JRE versions.</li>
|
||||
<li><a href="core/org/apache/lucene/codecs/lucene40/package-summary.html#package_description">File Formats</a>: Guide to the supported index format used by Lucene. This can be customized by using <a href="core/org/apache/lucene/codecs/package-summary.html#package_description">an alternate codec</a>.</li>
|
||||
<li><a href="core/org/apache/lucene/codecs/lucene41/package-summary.html#package_description">File Formats</a>: Guide to the supported index format used by Lucene. This can be customized by using <a href="core/org/apache/lucene/codecs/package-summary.html#package_description">an alternate codec</a>.</li>
|
||||
<li><a href="core/org/apache/lucene/search/package-summary.html#package_description">Search and Scoring in Lucene</a>: Introduction to how Lucene scores documents.</li>
|
||||
<li><a href="core/org/apache/lucene/search/similarities/TFIDFSimilarity.html">Classic Scoring Formula</a>: Formula of Lucene's classic <a href="http://en.wikipedia.org/wiki/Vector_Space_Model">Vector Space</a> implementation. (look <a href="core/org/apache/lucene/search/similarities/package-summary.html#package_description">here</a> for other models)</li>
|
||||
<li><a href="queryparser/org/apache/lucene/queryparser/classic/package-summary.html#package_description">Classic QueryParser Syntax</a>: Overview of the Classic QueryParser's syntax and features.</li>
|
||||
|
|
File diff suppressed because one or more lines are too long
|
@ -34,6 +34,7 @@ import org.apache.lucene.search.TopDocs;
|
|||
import org.apache.lucene.spatial.query.SpatialArgsParser;
|
||||
import org.junit.Assert;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.ArrayList;
|
||||
|
@ -109,8 +110,11 @@ public abstract class StrategyTestCase extends SpatialTestCase {
|
|||
}
|
||||
|
||||
protected Iterator<SampleData> getSampleData(String testDataFile) throws IOException {
|
||||
return new SampleDataReader(
|
||||
getClass().getClassLoader().getResourceAsStream("data/"+testDataFile) );
|
||||
String path = "data/" + testDataFile;
|
||||
InputStream stream = getClass().getClassLoader().getResourceAsStream(path);
|
||||
if (stream == null)
|
||||
throw new FileNotFoundException("classpath resource not found: "+path);
|
||||
return new SampleDataReader(stream);
|
||||
}
|
||||
|
||||
protected Iterator<SpatialTestQuery> getTestQueries(String testQueryFile, SpatialContext ctx) throws IOException {
|
||||
|
|
|
@ -0,0 +1,73 @@
|
|||
package org.apache.lucene.spatial.prefix;
|
||||
|
||||
/*
|
||||
* 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 com.spatial4j.core.context.SpatialContextFactory;
|
||||
import com.spatial4j.core.shape.Shape;
|
||||
import org.apache.lucene.spatial.StrategyTestCase;
|
||||
import org.apache.lucene.spatial.prefix.tree.GeohashPrefixTree;
|
||||
import org.apache.lucene.spatial.query.SpatialArgs;
|
||||
import org.apache.lucene.spatial.query.SpatialOperation;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
|
||||
public class JtsPolygonTest extends StrategyTestCase {
|
||||
|
||||
private static final double LUCENE_4464_distErrPct = SpatialArgs.DEFAULT_DISTERRPCT;//DEFAULT 2.5%
|
||||
|
||||
public JtsPolygonTest() {
|
||||
try {
|
||||
HashMap<String, String> args = new HashMap<String, String>();
|
||||
args.put("spatialContextFactory",
|
||||
"com.spatial4j.core.context.jts.JtsSpatialContextFactory");
|
||||
ctx = SpatialContextFactory.makeSpatialContext(args, getClass().getClassLoader());
|
||||
} catch (NoClassDefFoundError e) {
|
||||
assumeTrue("This test requires JTS jar: "+e, false);
|
||||
}
|
||||
|
||||
GeohashPrefixTree grid = new GeohashPrefixTree(ctx, 11);//< 1 meter == 11 maxLevels
|
||||
this.strategy = new RecursivePrefixTreeStrategy(grid, getClass().getSimpleName());
|
||||
((RecursivePrefixTreeStrategy)this.strategy).setDistErrPct(LUCENE_4464_distErrPct);//1% radius (small!)
|
||||
}
|
||||
|
||||
@Test
|
||||
/** LUCENE-4464 */
|
||||
public void testCloseButNoMatch() throws IOException {
|
||||
getAddAndVerifyIndexedDocuments("LUCENE-4464.txt");
|
||||
SpatialArgs args = q(
|
||||
"POLYGON((-93.18100824442227 45.25676372469945," +
|
||||
"-93.23182001200654 45.21421290799412," +
|
||||
"-93.16315546122038 45.23742639412364," +
|
||||
"-93.18100824442227 45.25676372469945))",
|
||||
LUCENE_4464_distErrPct);
|
||||
SearchResults got = executeQuery(strategy.makeQuery(args), 100);
|
||||
assertEquals(1, got.numFound);
|
||||
assertEquals("poly2", got.results.get(0).document.get("id"));
|
||||
//did not find poly 1 !
|
||||
}
|
||||
|
||||
private SpatialArgs q(String shapeStr, double distErrPct) {
|
||||
Shape shape = ctx.readShape(shapeStr);
|
||||
SpatialArgs args = new SpatialArgs(SpatialOperation.Intersects, shape);
|
||||
args.setDistErrPct(distErrPct);
|
||||
return args;
|
||||
}
|
||||
|
||||
}
|
|
@ -20,10 +20,10 @@ package org.apache.lucene.codecs.asserting;
|
|||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41Codec;
|
||||
|
||||
/**
|
||||
* Acts like {@link Lucene40Codec} but with additional asserts.
|
||||
* Acts like {@link Lucene41Codec} but with additional asserts.
|
||||
*/
|
||||
public final class AssertingCodec extends FilterCodec {
|
||||
|
||||
|
@ -31,7 +31,7 @@ public final class AssertingCodec extends FilterCodec {
|
|||
private final TermVectorsFormat vectors = new AssertingTermVectorsFormat();
|
||||
|
||||
public AssertingCodec() {
|
||||
super("Asserting", new Lucene40Codec());
|
||||
super("Asserting", new Lucene41Codec());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -27,7 +27,7 @@ import org.apache.lucene.codecs.PostingsConsumer;
|
|||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.TermStats;
|
||||
import org.apache.lucene.codecs.TermsConsumer;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
|
||||
import org.apache.lucene.index.AssertingAtomicReader;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
|
@ -38,10 +38,10 @@ import org.apache.lucene.util.BytesRef;
|
|||
import org.apache.lucene.util.OpenBitSet;
|
||||
|
||||
/**
|
||||
* Just like {@link Lucene40PostingsFormat} but with additional asserts.
|
||||
* Just like {@link Lucene41PostingsFormat} but with additional asserts.
|
||||
*/
|
||||
public final class AssertingPostingsFormat extends PostingsFormat {
|
||||
private final PostingsFormat in = new Lucene40PostingsFormat();
|
||||
private final PostingsFormat in = new Lucene41PostingsFormat();
|
||||
|
||||
public AssertingPostingsFormat() {
|
||||
super("Asserting");
|
||||
|
|
|
@ -22,19 +22,19 @@ import java.io.IOException;
|
|||
import org.apache.lucene.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
||||
/**
|
||||
* A class used for testing {@link BloomFilteringPostingsFormat} with a concrete
|
||||
* delegate (Lucene40). Creates a Bloom filter on ALL fields and with tiny
|
||||
* delegate (Lucene41). Creates a Bloom filter on ALL fields and with tiny
|
||||
* amounts of memory reserved for the filter. DO NOT USE IN A PRODUCTION
|
||||
* APPLICATION This is not a realistic application of Bloom Filters as they
|
||||
* ordinarily are larger and operate on only primary key type fields.
|
||||
*/
|
||||
public final class TestBloomFilteredLucene40Postings extends PostingsFormat {
|
||||
public final class TestBloomFilteredLucene41Postings extends PostingsFormat {
|
||||
|
||||
private BloomFilteringPostingsFormat delegate;
|
||||
|
||||
|
@ -54,9 +54,9 @@ public final class TestBloomFilteredLucene40Postings extends PostingsFormat {
|
|||
}
|
||||
}
|
||||
|
||||
public TestBloomFilteredLucene40Postings() {
|
||||
super("TestBloomFilteredLucene40Postings");
|
||||
delegate = new BloomFilteringPostingsFormat(new Lucene40PostingsFormat(),
|
||||
public TestBloomFilteredLucene41Postings() {
|
||||
super("TestBloomFilteredLucene41Postings");
|
||||
delegate = new BloomFilteringPostingsFormat(new Lucene41PostingsFormat(),
|
||||
new LowMemoryBloomFactory());
|
||||
}
|
||||
|
|
@ -21,14 +21,14 @@ import java.util.Random;
|
|||
|
||||
import org.apache.lucene.codecs.FilterCodec;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41Codec;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.generators.RandomInts;
|
||||
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
|
||||
|
||||
/**
|
||||
* A codec that uses {@link CompressingStoredFieldsFormat} for its stored
|
||||
* fields and delegates to {@link Lucene40Codec} for everything else.
|
||||
* fields and delegates to {@link Lucene41Codec} for everything else.
|
||||
*/
|
||||
public class CompressingCodec extends FilterCodec {
|
||||
|
||||
|
@ -49,7 +49,7 @@ public class CompressingCodec extends FilterCodec {
|
|||
*/
|
||||
public CompressingCodec(CompressionMode compressionMode, int chunkSize,
|
||||
CompressingStoredFieldsIndex storedFieldsIndexFormat) {
|
||||
super("Compressing", new Lucene40Codec());
|
||||
super("Compressing", new Lucene41Codec());
|
||||
this.storedFieldsFormat = new CompressingStoredFieldsFormat(compressionMode, chunkSize, storedFieldsIndexFormat);
|
||||
}
|
||||
|
||||
|
|
|
@ -45,16 +45,6 @@ import org.apache.lucene.util.IOUtils;
|
|||
* @lucene.experimental
|
||||
*/
|
||||
public final class Lucene40PostingsWriter extends PostingsWriterBase {
|
||||
final static String TERMS_CODEC = "Lucene40PostingsWriterTerms";
|
||||
final static String FRQ_CODEC = "Lucene40PostingsWriterFrq";
|
||||
final static String PRX_CODEC = "Lucene40PostingsWriterPrx";
|
||||
|
||||
//private static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
|
||||
|
||||
// Increment version to change it:
|
||||
final static int VERSION_START = 0;
|
||||
final static int VERSION_LONG_SKIP = 1;
|
||||
final static int VERSION_CURRENT = VERSION_LONG_SKIP;
|
||||
|
||||
final IndexOutput freqOut;
|
||||
final IndexOutput proxOut;
|
||||
|
@ -111,7 +101,7 @@ public final class Lucene40PostingsWriter extends PostingsWriterBase {
|
|||
boolean success = false;
|
||||
IndexOutput proxOut = null;
|
||||
try {
|
||||
CodecUtil.writeHeader(freqOut, FRQ_CODEC, VERSION_CURRENT);
|
||||
CodecUtil.writeHeader(freqOut, Lucene40PostingsReader.FRQ_CODEC, Lucene40PostingsReader.VERSION_CURRENT);
|
||||
// TODO: this is a best effort, if one of these fields has no postings
|
||||
// then we make an empty prx file, same as if we are wrapped in
|
||||
// per-field postingsformat. maybe... we shouldn't
|
||||
|
@ -121,7 +111,7 @@ public final class Lucene40PostingsWriter extends PostingsWriterBase {
|
|||
// prox file
|
||||
fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene40PostingsFormat.PROX_EXTENSION);
|
||||
proxOut = state.directory.createOutput(fileName, state.context);
|
||||
CodecUtil.writeHeader(proxOut, PRX_CODEC, VERSION_CURRENT);
|
||||
CodecUtil.writeHeader(proxOut, Lucene40PostingsReader.PRX_CODEC, Lucene40PostingsReader.VERSION_CURRENT);
|
||||
} else {
|
||||
// Every field omits TF so we will write no prox file
|
||||
proxOut = null;
|
||||
|
@ -146,7 +136,7 @@ public final class Lucene40PostingsWriter extends PostingsWriterBase {
|
|||
@Override
|
||||
public void start(IndexOutput termsOut) throws IOException {
|
||||
this.termsOut = termsOut;
|
||||
CodecUtil.writeHeader(termsOut, TERMS_CODEC, VERSION_CURRENT);
|
||||
CodecUtil.writeHeader(termsOut, Lucene40PostingsReader.TERMS_CODEC, Lucene40PostingsReader.VERSION_CURRENT);
|
||||
termsOut.writeInt(skipInterval); // write skipInterval
|
||||
termsOut.writeInt(maxSkipLevels); // write maxSkipLevels
|
||||
termsOut.writeInt(skipMinimum); // write skipMinimum
|
|
@ -0,0 +1,50 @@
|
|||
package org.apache.lucene.codecs.lucene40;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.BlockTreeTermsWriter;
|
||||
import org.apache.lucene.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.codecs.PostingsWriterBase;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Read-write version of {@link Lucene40PostingsFormat} for testing.
|
||||
*/
|
||||
public class Lucene40RWPostingsFormat extends Lucene40PostingsFormat {
|
||||
@Override
|
||||
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
PostingsWriterBase docs = new Lucene40PostingsWriter(state);
|
||||
|
||||
// TODO: should we make the terms index more easily
|
||||
// pluggable? Ie so that this codec would record which
|
||||
// index impl was used, and switch on loading?
|
||||
// Or... you must make a new Codec for this?
|
||||
boolean success = false;
|
||||
try {
|
||||
FieldsConsumer ret = new BlockTreeTermsWriter(state, docs, minBlockSize, maxBlockSize);
|
||||
success = true;
|
||||
return ret;
|
||||
} finally {
|
||||
if (!success) {
|
||||
docs.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -29,8 +29,9 @@ import org.apache.lucene.codecs.MultiLevelSkipListWriter;
|
|||
* that stores positions and payloads.
|
||||
*
|
||||
* @see Lucene40PostingsFormat
|
||||
* @lucene.experimental
|
||||
* @deprecated Only for reading old 4.0 segments
|
||||
*/
|
||||
@Deprecated
|
||||
public class Lucene40SkipListWriter extends MultiLevelSkipListWriter {
|
||||
private int[] lastSkipDoc;
|
||||
private int[] lastSkipPayloadLength;
|
|
@ -20,6 +20,6 @@
|
|||
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
|
||||
</head>
|
||||
<body>
|
||||
BlockPostingsFormat file format.
|
||||
Support for testing {@link org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat}.
|
||||
</body>
|
||||
</html>
|
|
@ -1,4 +1,4 @@
|
|||
package org.apache.lucene.codecs.lucene40ords;
|
||||
package org.apache.lucene.codecs.lucene41ords;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -30,9 +30,9 @@ import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexReader;
|
|||
import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexWriter;
|
||||
import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase;
|
||||
import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40Codec; // javadocs
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40PostingsReader;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40PostingsWriter;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41Codec; // javadocs
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -41,18 +41,18 @@ import org.apache.lucene.util.BytesRef;
|
|||
// any PostingsBaseFormat and make it ord-able...
|
||||
|
||||
/**
|
||||
* Customized version of {@link Lucene40Codec} that uses
|
||||
* Customized version of {@link Lucene41Codec} that uses
|
||||
* {@link FixedGapTermsIndexWriter}.
|
||||
*/
|
||||
public final class Lucene40WithOrds extends PostingsFormat {
|
||||
public final class Lucene41WithOrds extends PostingsFormat {
|
||||
|
||||
public Lucene40WithOrds() {
|
||||
super("Lucene40WithOrds");
|
||||
public Lucene41WithOrds() {
|
||||
super("Lucene41WithOrds");
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
PostingsWriterBase docs = new Lucene40PostingsWriter(state);
|
||||
PostingsWriterBase docs = new Lucene41PostingsWriter(state);
|
||||
|
||||
// TODO: should we make the terms index more easily
|
||||
// pluggable? Ie so that this codec would record which
|
||||
|
@ -91,7 +91,7 @@ public final class Lucene40WithOrds extends PostingsFormat {
|
|||
|
||||
@Override
|
||||
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
|
||||
PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
|
||||
PostingsReaderBase postings = new Lucene41PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
|
||||
TermsIndexReaderBase indexReader;
|
||||
|
||||
boolean success = false;
|
|
@ -38,8 +38,8 @@ import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase;
|
|||
import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase;
|
||||
import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexReader;
|
||||
import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexWriter;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40PostingsReader;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40PostingsWriter;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
|
||||
import org.apache.lucene.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
|
||||
import org.apache.lucene.codecs.mockintblock.MockVariableIntBlockPostingsFormat;
|
||||
import org.apache.lucene.codecs.mocksep.MockSingleIntFactory;
|
||||
|
@ -174,7 +174,8 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
|
|||
if (LuceneTestCase.VERBOSE) {
|
||||
System.out.println("MockRandomCodec: writing Standard postings");
|
||||
}
|
||||
postingsWriter = new Lucene40PostingsWriter(state, skipInterval);
|
||||
// TODO: randomize variables like acceptibleOverHead?!
|
||||
postingsWriter = new Lucene41PostingsWriter(state, skipInterval);
|
||||
}
|
||||
|
||||
if (random.nextBoolean()) {
|
||||
|
@ -313,7 +314,7 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
|
|||
if (LuceneTestCase.VERBOSE) {
|
||||
System.out.println("MockRandomCodec: reading Standard postings");
|
||||
}
|
||||
postingsReader = new Lucene40PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
|
||||
postingsReader = new Lucene41PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
|
||||
}
|
||||
|
||||
if (random.nextBoolean()) {
|
||||
|
|
|
@ -26,8 +26,8 @@ import org.apache.lucene.codecs.FieldsProducer;
|
|||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.PostingsReaderBase;
|
||||
import org.apache.lucene.codecs.PostingsWriterBase;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40PostingsReader;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40PostingsWriter;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
|
||||
import org.apache.lucene.codecs.pulsing.PulsingPostingsReader;
|
||||
import org.apache.lucene.codecs.pulsing.PulsingPostingsWriter;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
|
@ -35,7 +35,7 @@ import org.apache.lucene.index.SegmentWriteState;
|
|||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/**
|
||||
* Pulsing(1, Pulsing(2, Lucene40))
|
||||
* Pulsing(1, Pulsing(2, Lucene41))
|
||||
* @lucene.experimental
|
||||
*/
|
||||
// TODO: if we create PulsingPostingsBaseFormat then we
|
||||
|
@ -55,7 +55,7 @@ public final class NestedPulsingPostingsFormat extends PostingsFormat {
|
|||
// Terms dict
|
||||
boolean success = false;
|
||||
try {
|
||||
docsWriter = new Lucene40PostingsWriter(state);
|
||||
docsWriter = new Lucene41PostingsWriter(state);
|
||||
|
||||
pulsingWriterInner = new PulsingPostingsWriter(2, docsWriter);
|
||||
pulsingWriter = new PulsingPostingsWriter(1, pulsingWriterInner);
|
||||
|
@ -77,7 +77,7 @@ public final class NestedPulsingPostingsFormat extends PostingsFormat {
|
|||
PostingsReaderBase pulsingReader = null;
|
||||
boolean success = false;
|
||||
try {
|
||||
docsReader = new Lucene40PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
|
||||
docsReader = new Lucene41PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
|
||||
pulsingReaderInner = new PulsingPostingsReader(docsReader);
|
||||
pulsingReader = new PulsingPostingsReader(pulsingReaderInner);
|
||||
FieldsProducer ret = new BlockTreeTermsReader(
|
||||
|
|
|
@ -29,11 +29,10 @@ import java.util.Set;
|
|||
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.asserting.AssertingPostingsFormat;
|
||||
import org.apache.lucene.codecs.block.BlockPostingsFormat;
|
||||
import org.apache.lucene.codecs.bloom.TestBloomFilteredLucene40Postings;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene40ords.Lucene40WithOrds;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41Codec;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
|
||||
import org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings;
|
||||
import org.apache.lucene.codecs.memory.DirectPostingsFormat;
|
||||
import org.apache.lucene.codecs.memory.MemoryPostingsFormat;
|
||||
import org.apache.lucene.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
|
||||
|
@ -41,7 +40,7 @@ import org.apache.lucene.codecs.mockintblock.MockVariableIntBlockPostingsFormat;
|
|||
import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
|
||||
import org.apache.lucene.codecs.mocksep.MockSepPostingsFormat;
|
||||
import org.apache.lucene.codecs.nestedpulsing.NestedPulsingPostingsFormat;
|
||||
import org.apache.lucene.codecs.pulsing.Pulsing40PostingsFormat;
|
||||
import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
|
||||
import org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util._TestUtil;
|
||||
|
@ -55,7 +54,7 @@ import org.apache.lucene.util._TestUtil;
|
|||
* documents in different orders and the test will still be deterministic
|
||||
* and reproducable.
|
||||
*/
|
||||
public class RandomCodec extends Lucene40Codec {
|
||||
public class RandomCodec extends Lucene41Codec {
|
||||
/** Shuffled list of postings formats to use for new mappings */
|
||||
private List<PostingsFormat> formats = new ArrayList<PostingsFormat>();
|
||||
|
||||
|
@ -94,23 +93,22 @@ public class RandomCodec extends Lucene40Codec {
|
|||
int lowFreqCutoff = _TestUtil.nextInt(random, 2, 100);
|
||||
|
||||
add(avoidCodecs,
|
||||
new Lucene40PostingsFormat(minItemsPerBlock, maxItemsPerBlock),
|
||||
new BlockPostingsFormat(minItemsPerBlock, maxItemsPerBlock),
|
||||
new Lucene41PostingsFormat(minItemsPerBlock, maxItemsPerBlock),
|
||||
new DirectPostingsFormat(LuceneTestCase.rarely(random) ? 1 : (LuceneTestCase.rarely(random) ? Integer.MAX_VALUE : maxItemsPerBlock),
|
||||
LuceneTestCase.rarely(random) ? 1 : (LuceneTestCase.rarely(random) ? Integer.MAX_VALUE : lowFreqCutoff)),
|
||||
new Pulsing40PostingsFormat(1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock),
|
||||
new Pulsing41PostingsFormat(1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock),
|
||||
// add pulsing again with (usually) different parameters
|
||||
new Pulsing40PostingsFormat(1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock),
|
||||
//TODO as a PostingsFormat which wraps others, we should allow TestBloomFilteredLucene40Postings to be constructed
|
||||
new Pulsing41PostingsFormat(1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock),
|
||||
//TODO as a PostingsFormat which wraps others, we should allow TestBloomFilteredLucene41Postings to be constructed
|
||||
//with a choice of concrete PostingsFormats. Maybe useful to have a generic means of marking and dealing
|
||||
//with such "wrapper" classes?
|
||||
new TestBloomFilteredLucene40Postings(),
|
||||
new TestBloomFilteredLucene41Postings(),
|
||||
new MockSepPostingsFormat(),
|
||||
new MockFixedIntBlockPostingsFormat(_TestUtil.nextInt(random, 1, 2000)),
|
||||
new MockVariableIntBlockPostingsFormat( _TestUtil.nextInt(random, 1, 127)),
|
||||
new MockRandomPostingsFormat(random),
|
||||
new NestedPulsingPostingsFormat(),
|
||||
new Lucene40WithOrds(),
|
||||
new Lucene41WithOrds(),
|
||||
new SimpleTextPostingsFormat(),
|
||||
new AssertingPostingsFormat(),
|
||||
new MemoryPostingsFormat(true, random.nextFloat()),
|
||||
|
|
|
@ -32,6 +32,8 @@ import org.apache.lucene.codecs.PostingsFormat;
|
|||
import org.apache.lucene.codecs.asserting.AssertingCodec;
|
||||
import org.apache.lucene.codecs.compressing.CompressingCodec;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40RWPostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41Codec;
|
||||
import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
|
||||
import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
|
||||
import org.apache.lucene.index.RandomCodec;
|
||||
|
@ -129,26 +131,23 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule {
|
|||
avoidCodecs.addAll(Arrays.asList(a.value()));
|
||||
}
|
||||
|
||||
PREFLEX_IMPERSONATION_IS_ACTIVE = false;
|
||||
savedCodec = Codec.getDefault();
|
||||
int randomVal = random.nextInt(10);
|
||||
|
||||
/* note: re-enable this if we make a 4.x impersonator
|
||||
if ("Lucene3x".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) &&
|
||||
if ("Lucene40".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) &&
|
||||
"random".equals(TEST_POSTINGSFORMAT) &&
|
||||
randomVal < 2 &&
|
||||
!shouldAvoidCodec("Lucene3x"))) { // preflex-only setup
|
||||
codec = Codec.forName("Lucene3x");
|
||||
assert (codec instanceof PreFlexRWCodec) : "fix your classpath to have tests-framework.jar before lucene-core.jar";
|
||||
PREFLEX_IMPERSONATION_IS_ACTIVE = true;
|
||||
} else */ if (!"random".equals(TEST_POSTINGSFORMAT)) {
|
||||
!shouldAvoidCodec("Lucene40"))) {
|
||||
codec = Codec.forName("Lucene40");
|
||||
assert (PostingsFormat.forName("Lucene40") instanceof Lucene40RWPostingsFormat) : "fix your classpath to have tests-framework.jar before lucene-core.jar";
|
||||
} else if (!"random".equals(TEST_POSTINGSFORMAT)) {
|
||||
final PostingsFormat format;
|
||||
if ("MockRandom".equals(TEST_POSTINGSFORMAT)) {
|
||||
format = new MockRandomPostingsFormat(random);
|
||||
} else {
|
||||
format = PostingsFormat.forName(TEST_POSTINGSFORMAT);
|
||||
}
|
||||
codec = new Lucene40Codec() {
|
||||
codec = new Lucene41Codec() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return format;
|
||||
|
|
|
@ -44,7 +44,7 @@ import java.util.zip.ZipFile;
|
|||
|
||||
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.perfield.PerFieldPostingsFormat;
|
||||
import org.apache.lucene.document.ByteDocValuesField;
|
||||
import org.apache.lucene.document.DerefBytesDocValuesField;
|
||||
|
@ -651,7 +651,7 @@ public class _TestUtil {
|
|||
if (LuceneTestCase.VERBOSE) {
|
||||
System.out.println("forcing postings format to:" + format);
|
||||
}
|
||||
return new Lucene40Codec() {
|
||||
return new Lucene41Codec() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
return format;
|
||||
|
|
|
@ -19,7 +19,7 @@ org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat
|
|||
org.apache.lucene.codecs.mocksep.MockSepPostingsFormat
|
||||
org.apache.lucene.codecs.nestedpulsing.NestedPulsingPostingsFormat
|
||||
org.apache.lucene.codecs.ramonly.RAMOnlyPostingsFormat
|
||||
org.apache.lucene.codecs.lucene40ords.Lucene40WithOrds
|
||||
org.apache.lucene.codecs.bloom.TestBloomFilteredLucene40Postings
|
||||
org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds
|
||||
org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings
|
||||
org.apache.lucene.codecs.asserting.AssertingPostingsFormat
|
||||
|
||||
org.apache.lucene.codecs.lucene40.Lucene40RWPostingsFormat
|
||||
|
|
|
@ -42,9 +42,7 @@ New Features
|
|||
values of a multiValued field in their original order when highlighting.
|
||||
(Joel Bernstein via yonik)
|
||||
|
||||
* SOLR-3929
|
||||
support configuring IndexWriter max thread count in solrconfig
|
||||
|
||||
* SOLR-3929: Support configuring IndexWriter max thread count in solrconfig.
|
||||
(phunt via Mark Miller)
|
||||
|
||||
Optimizations
|
||||
|
@ -63,6 +61,9 @@ Optimizations
|
|||
* SOLR-3734: Improve Schema-Browser Handling for CopyField using
|
||||
dynamicField's (steffkes)
|
||||
|
||||
* SOLR-3941: The "commitOnLeader" part of distributed recovery can use
|
||||
openSearcher=false. (Tomas Fernandez Lobbe via Mark Miller)
|
||||
|
||||
Bug Fixes
|
||||
----------------------
|
||||
|
||||
|
@ -74,6 +75,15 @@ Bug Fixes
|
|||
* SOLR-3917: Partial State on Schema-Browser UI is not defined for Dynamic
|
||||
Fields & Types (steffkes)
|
||||
|
||||
* SOLR-3939: Consider a sync attempt from leader to replica that fails due
|
||||
to 404 a success. (Mark Miller, Joel Bernstein)
|
||||
|
||||
* SOLR-3940: Rejoining the leader election incorrectly triggers the code path
|
||||
for a fresh cluster start rather than fail over. (Mark Miller)
|
||||
|
||||
* SOLR-3961: Fixed error using LimitTokenCountFilterFactory
|
||||
(Jack Krupansky, hossman)
|
||||
|
||||
Other Changes
|
||||
----------------------
|
||||
|
||||
|
|
|
@ -0,0 +1,16 @@
|
|||
# System Requirements
|
||||
|
||||
Apache Solr 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 Solr. 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 Solr (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 Solr from source, refer to the `BUILD.txt` file in
|
||||
the distribution directory.*
|
|
@ -144,7 +144,7 @@
|
|||
depends="javadocs,changes-to-html,process-webpages"/>
|
||||
<target name="compile-core" depends="compile-solr-core" unless="solr.core.compiled"/>
|
||||
|
||||
<target name="process-webpages" depends="define-lucene-javadoc-url"><!--depends="resolve-groovy,resolve-pegdown">-->
|
||||
<target name="process-webpages" depends="define-lucene-javadoc-url,resolve-groovy,resolve-pegdown">
|
||||
<makeurl property="process-webpages.buildfiles" separator="|">
|
||||
<fileset dir="." includes="core/build.xml,test-framework/build.xml,solrj/build.xml,contrib/**/build.xml"/>
|
||||
</makeurl>
|
||||
|
@ -163,12 +163,10 @@
|
|||
<param name="luceneJavadocUrl" expression="${lucene.javadoc.url}"/>
|
||||
</xslt>
|
||||
|
||||
<!--nothing at the moment:
|
||||
<pegdown todir="${javadoc.dir}">
|
||||
<fileset dir="." includes="MIGRATE.txt,JRE_VERSION_MIGRATION.txt"/>
|
||||
<fileset dir="." includes="SYSTEM_REQUIREMENTS.txt"/>
|
||||
<globmapper from="*.txt" to="*.html"/>
|
||||
</pegdown>
|
||||
-->
|
||||
|
||||
<copy todir="${javadoc.dir}">
|
||||
<fileset dir="site/html" includes="**/*"/>
|
||||
|
@ -439,8 +437,8 @@
|
|||
<tar destfile="${package.dir}/${fullnamever}.tgz" compression="gzip" longfile="gnu">
|
||||
<tarfileset dir="."
|
||||
prefix="${fullnamever}"
|
||||
includes="LICENSE.txt NOTICE.txt CHANGES.txt README.txt example/**
|
||||
client/README.txt client/ruby/solr-ruby/**
|
||||
includes="LICENSE.txt NOTICE.txt CHANGES.txt README.txt SYSTEM_REQUIREMENTS.txt
|
||||
example/** client/README.txt client/ruby/solr-ruby/**
|
||||
contrib/**/lib/** contrib/**/README.txt
|
||||
licenses/**"
|
||||
excludes="lib/README.committers.txt **/data/ **/logs/*
|
||||
|
|
|
@ -324,7 +324,7 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
|
|||
SolrException.log(log, "Error trying to start recovery", t);
|
||||
}
|
||||
|
||||
leaderElector.joinElection(this);
|
||||
leaderElector.joinElection(this, true);
|
||||
}
|
||||
|
||||
private boolean shouldIBeLeader(ZkNodeProps leaderProps, SolrCore core) {
|
||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.solr.cloud;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.UnsupportedEncodingException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
|
@ -43,7 +42,7 @@ import org.slf4j.LoggerFactory;
|
|||
* Leader Election process. This class contains the logic by which a
|
||||
* leader is chosen. First call * {@link #setup(ElectionContext)} to ensure
|
||||
* the election process is init'd. Next call
|
||||
* {@link #joinElection(ElectionContext)} to start the leader election.
|
||||
* {@link #joinElection(ElectionContext, boolean)} to start the leader election.
|
||||
*
|
||||
* The implementation follows the classic ZooKeeper recipe of creating an
|
||||
* ephemeral, sequential node for each candidate and then looking at the set
|
||||
|
@ -203,7 +202,7 @@ public class LeaderElector {
|
|||
*
|
||||
* @return sequential node number
|
||||
*/
|
||||
public int joinElection(ElectionContext context) throws KeeperException, InterruptedException, IOException {
|
||||
public int joinElection(ElectionContext context, boolean replacement) throws KeeperException, InterruptedException, IOException {
|
||||
final String shardsElectZkPath = context.electionPath + LeaderElector.ELECTION_NODE;
|
||||
|
||||
long sessionId = zkClient.getSolrZooKeeper().getSessionId();
|
||||
|
@ -259,7 +258,7 @@ public class LeaderElector {
|
|||
}
|
||||
}
|
||||
int seq = getSeq(leaderSeqPath);
|
||||
checkIfIamLeader(seq, context, false);
|
||||
checkIfIamLeader(seq, context, replacement);
|
||||
|
||||
return seq;
|
||||
}
|
||||
|
|
|
@ -37,6 +37,7 @@ import org.apache.solr.common.cloud.ZkNodeProps;
|
|||
import org.apache.solr.common.cloud.ZkStateReader;
|
||||
import org.apache.solr.common.cloud.ZooKeeperException;
|
||||
import org.apache.solr.common.params.ModifiableSolrParams;
|
||||
import org.apache.solr.common.params.UpdateParams;
|
||||
import org.apache.solr.core.CoreContainer;
|
||||
import org.apache.solr.core.CoreDescriptor;
|
||||
import org.apache.solr.core.RequestHandlers.LazyRequestHandlerWrapper;
|
||||
|
@ -177,6 +178,7 @@ public class RecoveryStrategy extends Thread implements ClosableThread {
|
|||
UpdateRequest ureq = new UpdateRequest();
|
||||
ureq.setParams(new ModifiableSolrParams());
|
||||
ureq.getParams().set(DistributedUpdateProcessor.COMMIT_END_POINT, true);
|
||||
ureq.getParams().set(UpdateParams.OPEN_SEARCHER, false);
|
||||
ureq.setAction(AbstractUpdateRequest.ACTION.COMMIT, false, true).process(
|
||||
server);
|
||||
server.shutdown();
|
||||
|
|
|
@ -191,7 +191,7 @@ public final class ZkController {
|
|||
|
||||
ZkController.this.overseer = new Overseer(shardHandler, adminPath, zkStateReader);
|
||||
ElectionContext context = new OverseerElectionContext(zkClient, overseer, getNodeName());
|
||||
overseerElector.joinElection(context);
|
||||
overseerElector.joinElection(context, true);
|
||||
zkStateReader.createClusterStateWatchersAndUpdate();
|
||||
|
||||
// cc.newCmdDistribExecutor();
|
||||
|
@ -422,7 +422,7 @@ public final class ZkController {
|
|||
this.overseer = new Overseer(shardHandler, adminPath, zkStateReader);
|
||||
ElectionContext context = new OverseerElectionContext(zkClient, overseer, getNodeName());
|
||||
overseerElector.setup(context);
|
||||
overseerElector.joinElection(context);
|
||||
overseerElector.joinElection(context, false);
|
||||
zkStateReader.createClusterStateWatchersAndUpdate();
|
||||
|
||||
} catch (IOException e) {
|
||||
|
@ -730,7 +730,7 @@ public final class ZkController {
|
|||
|
||||
leaderElector.setup(context);
|
||||
electionContexts.put(coreZkNodeName, context);
|
||||
leaderElector.joinElection(context);
|
||||
leaderElector.joinElection(context, false);
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -2,7 +2,7 @@ package org.apache.solr.core;
|
|||
|
||||
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.solr.schema.IndexSchema;
|
||||
import org.apache.solr.schema.SchemaAware;
|
||||
import org.apache.solr.schema.SchemaField;
|
||||
|
@ -42,7 +42,7 @@ public class SchemaCodecFactory extends CodecFactory implements SchemaAware {
|
|||
|
||||
@Override
|
||||
public void inform(final IndexSchema schema) {
|
||||
codec = new Lucene40Codec() {
|
||||
codec = new Lucene41Codec() {
|
||||
@Override
|
||||
public PostingsFormat getPostingsFormatForField(String field) {
|
||||
final SchemaField fieldOrNull = schema.getFieldOrNull(field);
|
||||
|
|
|
@ -312,6 +312,11 @@ public class PeerSync {
|
|||
log.warn(msg() + " got a 503 from " + srsp.getShardAddress() + ", counting as success");
|
||||
return true;
|
||||
}
|
||||
|
||||
if (cantReachIsSuccess && sreq.purpose == 1 && srsp.getException() instanceof SolrException && ((SolrException) srsp.getException()).code() == 404) {
|
||||
log.warn(msg() + " got a 404 from " + srsp.getShardAddress() + ", counting as success");
|
||||
return true;
|
||||
}
|
||||
// TODO: at least log???
|
||||
// srsp.getException().printStackTrace(System.out);
|
||||
|
||||
|
|
|
@ -269,6 +269,12 @@
|
|||
<filter class="solr.LengthFilterFactory" min="2" max="5"/>
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
<fieldtype name="limitfilt" class="solr.TextField">
|
||||
<analyzer>
|
||||
<tokenizer class="solr.MockTokenizerFactory"/>
|
||||
<filter class="solr.LimitTokenCountFilterFactory" maxTokenCount="100" />
|
||||
</analyzer>
|
||||
</fieldtype>
|
||||
|
||||
<fieldtype name="subword" class="solr.TextField" multiValued="true" positionIncrementGap="100">
|
||||
<analyzer type="index">
|
||||
|
|
|
@ -17,9 +17,9 @@
|
|||
-->
|
||||
<schema name="codec" version="1.2">
|
||||
<types>
|
||||
<fieldType name="string_pulsing" class="solr.StrField" postingsFormat="Pulsing40"/>
|
||||
<fieldType name="string_pulsing" class="solr.StrField" postingsFormat="Pulsing41"/>
|
||||
<fieldType name="string_simpletext" class="solr.StrField" postingsFormat="SimpleText"/>
|
||||
<fieldType name="string_standard" class="solr.StrField" postingsFormat="Lucene40"/>
|
||||
<fieldType name="string_standard" class="solr.StrField" postingsFormat="Lucene41"/>
|
||||
<fieldType name="string" class="solr.StrField" />
|
||||
|
||||
</types>
|
||||
|
|
|
@ -49,6 +49,7 @@ import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
|
|||
import org.apache.solr.client.solrj.request.ContentStreamUpdateRequest;
|
||||
import org.apache.solr.client.solrj.request.CoreAdminRequest;
|
||||
import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
|
||||
import org.apache.solr.client.solrj.request.CoreAdminRequest.Unload;
|
||||
import org.apache.solr.client.solrj.request.QueryRequest;
|
||||
import org.apache.solr.client.solrj.response.CoreAdminResponse;
|
||||
import org.apache.solr.client.solrj.response.QueryResponse;
|
||||
|
@ -742,10 +743,10 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
|
|||
0,
|
||||
((HttpSolrServer) client).getBaseURL().length()
|
||||
- DEFAULT_COLLECTION.length() - 1);
|
||||
createCollection(oneInstanceCollection2, collectionClients, baseUrl, 1, "slice1");
|
||||
createCollection(oneInstanceCollection2, collectionClients, baseUrl, 2, "slice2");
|
||||
createCollection(oneInstanceCollection2, collectionClients, baseUrl, 3, "slice2");
|
||||
createCollection(oneInstanceCollection2, collectionClients, baseUrl, 4, "slice1");
|
||||
createSolrCore(oneInstanceCollection2, collectionClients, baseUrl, 1, "slice1");
|
||||
createSolrCore(oneInstanceCollection2, collectionClients, baseUrl, 2, "slice2");
|
||||
createSolrCore(oneInstanceCollection2, collectionClients, baseUrl, 3, "slice2");
|
||||
createSolrCore(oneInstanceCollection2, collectionClients, baseUrl, 4, "slice1");
|
||||
|
||||
while (pending != null && pending.size() > 0) {
|
||||
|
||||
|
@ -764,7 +765,7 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
|
|||
|
||||
assertAllActive(oneInstanceCollection2, solrj.getZkStateReader());
|
||||
|
||||
printLayout();
|
||||
//printLayout();
|
||||
|
||||
// TODO: enable when we don't falsely get slice1...
|
||||
// solrj.getZkStateReader().getLeaderUrl(oneInstanceCollection2, "slice1", 30000);
|
||||
|
@ -803,6 +804,27 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
|
|||
assertNotNull(slices);
|
||||
String roles = slices.get("slice1").getReplicasMap().values().iterator().next().getStr(ZkStateReader.ROLES_PROP);
|
||||
assertEquals("none", roles);
|
||||
|
||||
|
||||
ZkCoreNodeProps props = new ZkCoreNodeProps(solrj.getZkStateReader().getClusterState().getLeader(oneInstanceCollection2, "slice1"));
|
||||
|
||||
// now test that unloading a core gets us a new leader
|
||||
HttpSolrServer server = new HttpSolrServer(baseUrl);
|
||||
Unload unloadCmd = new Unload(true);
|
||||
unloadCmd.setCoreName(props.getCoreName());
|
||||
|
||||
String leader = props.getCoreUrl();
|
||||
|
||||
server.request(unloadCmd);
|
||||
|
||||
int tries = 50;
|
||||
while (leader.equals(zkStateReader.getLeaderUrl(oneInstanceCollection2, "slice1", 10000))) {
|
||||
Thread.sleep(100);
|
||||
if (tries-- == 0) {
|
||||
fail("Leader never changed");
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private void testSearchByCollectionName() throws SolrServerException {
|
||||
|
@ -875,10 +897,10 @@ public class BasicDistributedZkTest extends AbstractFullDistribZkTestBase {
|
|||
|
||||
private void createCollection(String collection,
|
||||
List<SolrServer> collectionClients, String baseUrl, int num) {
|
||||
createCollection(collection, collectionClients, baseUrl, num, null);
|
||||
createSolrCore(collection, collectionClients, baseUrl, num, null);
|
||||
}
|
||||
|
||||
private void createCollection(final String collection,
|
||||
private void createSolrCore(final String collection,
|
||||
List<SolrServer> collectionClients, final String baseUrl, final int num,
|
||||
final String shardId) {
|
||||
Callable call = new Callable() {
|
||||
|
|
|
@ -40,7 +40,6 @@ import org.apache.zookeeper.KeeperException;
|
|||
import org.apache.zookeeper.KeeperException.NoNodeException;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
@Slow
|
||||
|
@ -114,7 +113,7 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
|
|||
elector, "shard1", "collection1", Integer.toString(nodeNumber),
|
||||
props, zkStateReader);
|
||||
elector.setup(context);
|
||||
seq = elector.joinElection(context);
|
||||
seq = elector.joinElection(context, false);
|
||||
electionDone = true;
|
||||
seqToThread.put(seq, this);
|
||||
}
|
||||
|
@ -175,7 +174,7 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
|
|||
ElectionContext context = new ShardLeaderElectionContextBase(elector,
|
||||
"shard2", "collection1", "dummynode1", props, zkStateReader);
|
||||
elector.setup(context);
|
||||
elector.joinElection(context);
|
||||
elector.joinElection(context, false);
|
||||
assertEquals("http://127.0.0.1/solr/",
|
||||
getLeaderUrl("collection1", "shard2"));
|
||||
}
|
||||
|
@ -188,7 +187,7 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
|
|||
ElectionContext firstContext = new ShardLeaderElectionContextBase(first,
|
||||
"slice1", "collection2", "dummynode1", props, zkStateReader);
|
||||
first.setup(firstContext);
|
||||
first.joinElection(firstContext);
|
||||
first.joinElection(firstContext, false);
|
||||
|
||||
Thread.sleep(1000);
|
||||
assertEquals("original leader was not registered", "http://127.0.0.1/solr/1/", getLeaderUrl("collection2", "slice1"));
|
||||
|
@ -199,7 +198,7 @@ public class LeaderElectionTest extends SolrTestCaseJ4 {
|
|||
ElectionContext context = new ShardLeaderElectionContextBase(second,
|
||||
"slice1", "collection2", "dummynode1", props, zkStateReader);
|
||||
second.setup(context);
|
||||
second.joinElection(context);
|
||||
second.joinElection(context, false);
|
||||
Thread.sleep(1000);
|
||||
assertEquals("original leader should have stayed leader", "http://127.0.0.1/solr/1/", getLeaderUrl("collection2", "slice1"));
|
||||
firstContext.cancelElection();
|
||||
|
|
|
@ -139,7 +139,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
|||
ShardLeaderElectionContextBase ctx = new ShardLeaderElectionContextBase(
|
||||
elector, shardId, collection, nodeName + "_" + coreName, props,
|
||||
zkStateReader);
|
||||
elector.joinElection(ctx);
|
||||
elector.joinElection(ctx, false);
|
||||
return shardId;
|
||||
}
|
||||
Thread.sleep(500);
|
||||
|
@ -876,7 +876,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
|
|||
new HttpShardHandlerFactory().getShardHandler(), "/admin/cores", reader);
|
||||
ElectionContext ec = new OverseerElectionContext(zkClient, overseer, address.replaceAll("/", "_"));
|
||||
overseerElector.setup(ec);
|
||||
overseerElector.joinElection(ec);
|
||||
overseerElector.joinElection(ec, false);
|
||||
return zkClient;
|
||||
}
|
||||
|
||||
|
|
|
@ -37,14 +37,14 @@ public class TestCodecSupport extends SolrTestCaseJ4 {
|
|||
Map<String, SchemaField> fields = h.getCore().getSchema().getFields();
|
||||
SchemaField schemaField = fields.get("string_pulsing_f");
|
||||
PerFieldPostingsFormat format = (PerFieldPostingsFormat) codec.postingsFormat();
|
||||
assertEquals("Pulsing40", format.getPostingsFormatForField(schemaField.getName()).getName());
|
||||
assertEquals("Pulsing41", format.getPostingsFormatForField(schemaField.getName()).getName());
|
||||
schemaField = fields.get("string_simpletext_f");
|
||||
assertEquals("SimpleText",
|
||||
format.getPostingsFormatForField(schemaField.getName()).getName());
|
||||
schemaField = fields.get("string_standard_f");
|
||||
assertEquals("Lucene40", format.getPostingsFormatForField(schemaField.getName()).getName());
|
||||
assertEquals("Lucene41", format.getPostingsFormatForField(schemaField.getName()).getName());
|
||||
schemaField = fields.get("string_f");
|
||||
assertEquals("Lucene40", format.getPostingsFormatForField(schemaField.getName()).getName());
|
||||
assertEquals("Lucene41", format.getPostingsFormatForField(schemaField.getName()).getName());
|
||||
}
|
||||
|
||||
public void testDynamicFields() {
|
||||
|
@ -53,10 +53,10 @@ public class TestCodecSupport extends SolrTestCaseJ4 {
|
|||
|
||||
assertEquals("SimpleText", format.getPostingsFormatForField("foo_simple").getName());
|
||||
assertEquals("SimpleText", format.getPostingsFormatForField("bar_simple").getName());
|
||||
assertEquals("Pulsing40", format.getPostingsFormatForField("foo_pulsing").getName());
|
||||
assertEquals("Pulsing40", format.getPostingsFormatForField("bar_pulsing").getName());
|
||||
assertEquals("Lucene40", format.getPostingsFormatForField("foo_standard").getName());
|
||||
assertEquals("Lucene40", format.getPostingsFormatForField("bar_standard").getName());
|
||||
assertEquals("Pulsing41", format.getPostingsFormatForField("foo_pulsing").getName());
|
||||
assertEquals("Pulsing41", format.getPostingsFormatForField("bar_pulsing").getName());
|
||||
assertEquals("Lucene41", format.getPostingsFormatForField("foo_standard").getName());
|
||||
assertEquals("Lucene41", format.getPostingsFormatForField("bar_standard").getName());
|
||||
}
|
||||
|
||||
public void testUnknownField() {
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue